| |
| <!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>Spark SQL and DataFrames - Spark 2.3.1 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/bootstrap-responsive.min.css"> |
| <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"> |
| |
| |
| <!-- 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="http://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 --> |
| |
| <div class="navbar navbar-fixed-top" id="topbar"> |
| <div class="navbar-inner"> |
| <div class="container"> |
| <div class="brand"><a href="index.html"> |
| <img src="img/spark-logo-hd.png" style="height:50px;"/></a><span class="version">2.3.1</span> |
| </div> |
| <ul class="nav"> |
| <!--TODO(andyk): Add class="active" attribute to li some how.--> |
| <li><a href="index.html">Overview</a></li> |
| |
| <li class="dropdown"> |
| <a href="#" class="dropdown-toggle" data-toggle="dropdown">Programming Guides<b class="caret"></b></a> |
| <ul class="dropdown-menu"> |
| <li><a href="quick-start.html">Quick Start</a></li> |
| <li><a href="rdd-programming-guide.html">RDDs, Accumulators, Broadcasts Vars</a></li> |
| <li><a href="sql-programming-guide.html">SQL, DataFrames, and Datasets</a></li> |
| <li><a href="structured-streaming-programming-guide.html">Structured Streaming</a></li> |
| <li><a href="streaming-programming-guide.html">Spark Streaming (DStreams)</a></li> |
| <li><a href="ml-guide.html">MLlib (Machine Learning)</a></li> |
| <li><a href="graphx-programming-guide.html">GraphX (Graph Processing)</a></li> |
| <li><a href="sparkr.html">SparkR (R on Spark)</a></li> |
| </ul> |
| </li> |
| |
| <li class="dropdown"> |
| <a href="#" class="dropdown-toggle" data-toggle="dropdown">API Docs<b class="caret"></b></a> |
| <ul class="dropdown-menu"> |
| <li><a href="api/scala/index.html#org.apache.spark.package">Scala</a></li> |
| <li><a href="api/java/index.html">Java</a></li> |
| <li><a href="api/python/index.html">Python</a></li> |
| <li><a href="api/R/index.html">R</a></li> |
| <li><a href="api/sql/index.html">SQL, Built-in Functions</a></li> |
| </ul> |
| </li> |
| |
| <li class="dropdown"> |
| <a href="#" class="dropdown-toggle" data-toggle="dropdown">Deploying<b class="caret"></b></a> |
| <ul class="dropdown-menu"> |
| <li><a href="cluster-overview.html">Overview</a></li> |
| <li><a href="submitting-applications.html">Submitting Applications</a></li> |
| <li class="divider"></li> |
| <li><a href="spark-standalone.html">Spark Standalone</a></li> |
| <li><a href="running-on-mesos.html">Mesos</a></li> |
| <li><a href="running-on-yarn.html">YARN</a></li> |
| <li><a href="running-on-kubernetes.html">Kubernetes</a></li> |
| </ul> |
| </li> |
| |
| <li class="dropdown"> |
| <a href="api.html" class="dropdown-toggle" data-toggle="dropdown">More<b class="caret"></b></a> |
| <ul class="dropdown-menu"> |
| <li><a href="configuration.html">Configuration</a></li> |
| <li><a href="monitoring.html">Monitoring</a></li> |
| <li><a href="tuning.html">Tuning Guide</a></li> |
| <li><a href="job-scheduling.html">Job Scheduling</a></li> |
| <li><a href="security.html">Security</a></li> |
| <li><a href="hardware-provisioning.html">Hardware Provisioning</a></li> |
| <li class="divider"></li> |
| <li><a href="building-spark.html">Building Spark</a></li> |
| <li><a href="http://spark.apache.org/contributing.html">Contributing to Spark</a></li> |
| <li><a href="http://spark.apache.org/third-party-projects.html">Third Party Projects</a></li> |
| </ul> |
| </li> |
| </ul> |
| <!--<p class="navbar-text pull-right"><span class="version-text">v2.3.1</span></p>--> |
| </div> |
| </div> |
| </div> |
| |
| <div class="container-wrapper"> |
| |
| |
| <div class="content" id="content"> |
| |
| <h1 class="title">Spark SQL, DataFrames and Datasets Guide</h1> |
| |
| |
| <ul id="markdown-toc"> |
| <li><a href="#overview" id="markdown-toc-overview">Overview</a> <ul> |
| <li><a href="#sql" id="markdown-toc-sql">SQL</a></li> |
| <li><a href="#datasets-and-dataframes" id="markdown-toc-datasets-and-dataframes">Datasets and DataFrames</a></li> |
| </ul> |
| </li> |
| <li><a href="#getting-started" id="markdown-toc-getting-started">Getting Started</a> <ul> |
| <li><a href="#starting-point-sparksession" id="markdown-toc-starting-point-sparksession">Starting Point: SparkSession</a></li> |
| <li><a href="#creating-dataframes" id="markdown-toc-creating-dataframes">Creating DataFrames</a></li> |
| <li><a href="#untyped-dataset-operations-aka-dataframe-operations" id="markdown-toc-untyped-dataset-operations-aka-dataframe-operations">Untyped Dataset Operations (aka DataFrame Operations)</a></li> |
| <li><a href="#running-sql-queries-programmatically" id="markdown-toc-running-sql-queries-programmatically">Running SQL Queries Programmatically</a></li> |
| <li><a href="#global-temporary-view" id="markdown-toc-global-temporary-view">Global Temporary View</a></li> |
| <li><a href="#creating-datasets" id="markdown-toc-creating-datasets">Creating Datasets</a></li> |
| <li><a href="#interoperating-with-rdds" id="markdown-toc-interoperating-with-rdds">Interoperating with RDDs</a> <ul> |
| <li><a href="#inferring-the-schema-using-reflection" id="markdown-toc-inferring-the-schema-using-reflection">Inferring the Schema Using Reflection</a></li> |
| <li><a href="#programmatically-specifying-the-schema" id="markdown-toc-programmatically-specifying-the-schema">Programmatically Specifying the Schema</a></li> |
| </ul> |
| </li> |
| <li><a href="#aggregations" id="markdown-toc-aggregations">Aggregations</a> <ul> |
| <li><a href="#untyped-user-defined-aggregate-functions" id="markdown-toc-untyped-user-defined-aggregate-functions">Untyped User-Defined Aggregate Functions</a></li> |
| <li><a href="#type-safe-user-defined-aggregate-functions" id="markdown-toc-type-safe-user-defined-aggregate-functions">Type-Safe User-Defined Aggregate Functions</a></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><a href="#data-sources" id="markdown-toc-data-sources">Data Sources</a> <ul> |
| <li><a href="#generic-loadsave-functions" id="markdown-toc-generic-loadsave-functions">Generic Load/Save Functions</a> <ul> |
| <li><a href="#manually-specifying-options" id="markdown-toc-manually-specifying-options">Manually Specifying Options</a></li> |
| <li><a href="#run-sql-on-files-directly" id="markdown-toc-run-sql-on-files-directly">Run SQL on files directly</a></li> |
| <li><a href="#save-modes" id="markdown-toc-save-modes">Save Modes</a></li> |
| <li><a href="#saving-to-persistent-tables" id="markdown-toc-saving-to-persistent-tables">Saving to Persistent Tables</a></li> |
| <li><a href="#bucketing-sorting-and-partitioning" id="markdown-toc-bucketing-sorting-and-partitioning">Bucketing, Sorting and Partitioning</a></li> |
| </ul> |
| </li> |
| <li><a href="#parquet-files" id="markdown-toc-parquet-files">Parquet Files</a> <ul> |
| <li><a href="#loading-data-programmatically" id="markdown-toc-loading-data-programmatically">Loading Data Programmatically</a></li> |
| <li><a href="#partition-discovery" id="markdown-toc-partition-discovery">Partition Discovery</a></li> |
| <li><a href="#schema-merging" id="markdown-toc-schema-merging">Schema Merging</a></li> |
| <li><a href="#hive-metastore-parquet-table-conversion" id="markdown-toc-hive-metastore-parquet-table-conversion">Hive metastore Parquet table conversion</a> <ul> |
| <li><a href="#hiveparquet-schema-reconciliation" id="markdown-toc-hiveparquet-schema-reconciliation">Hive/Parquet Schema Reconciliation</a></li> |
| <li><a href="#metadata-refreshing" id="markdown-toc-metadata-refreshing">Metadata Refreshing</a></li> |
| </ul> |
| </li> |
| <li><a href="#configuration" id="markdown-toc-configuration">Configuration</a></li> |
| </ul> |
| </li> |
| <li><a href="#orc-files" id="markdown-toc-orc-files">ORC Files</a></li> |
| <li><a href="#json-datasets" id="markdown-toc-json-datasets">JSON Datasets</a></li> |
| <li><a href="#hive-tables" id="markdown-toc-hive-tables">Hive Tables</a> <ul> |
| <li><a href="#specifying-storage-format-for-hive-tables" id="markdown-toc-specifying-storage-format-for-hive-tables">Specifying storage format for Hive tables</a></li> |
| <li><a href="#interacting-with-different-versions-of-hive-metastore" id="markdown-toc-interacting-with-different-versions-of-hive-metastore">Interacting with Different Versions of Hive Metastore</a></li> |
| </ul> |
| </li> |
| <li><a href="#jdbc-to-other-databases" id="markdown-toc-jdbc-to-other-databases">JDBC To Other Databases</a></li> |
| <li><a href="#troubleshooting" id="markdown-toc-troubleshooting">Troubleshooting</a></li> |
| </ul> |
| </li> |
| <li><a href="#performance-tuning" id="markdown-toc-performance-tuning">Performance Tuning</a> <ul> |
| <li><a href="#caching-data-in-memory" id="markdown-toc-caching-data-in-memory">Caching Data In Memory</a></li> |
| <li><a href="#other-configuration-options" id="markdown-toc-other-configuration-options">Other Configuration Options</a></li> |
| <li><a href="#broadcast-hint-for-sql-queries" id="markdown-toc-broadcast-hint-for-sql-queries">Broadcast Hint for SQL Queries</a></li> |
| </ul> |
| </li> |
| <li><a href="#distributed-sql-engine" id="markdown-toc-distributed-sql-engine">Distributed SQL Engine</a> <ul> |
| <li><a href="#running-the-thrift-jdbcodbc-server" id="markdown-toc-running-the-thrift-jdbcodbc-server">Running the Thrift JDBC/ODBC server</a></li> |
| <li><a href="#running-the-spark-sql-cli" id="markdown-toc-running-the-spark-sql-cli">Running the Spark SQL CLI</a></li> |
| </ul> |
| </li> |
| <li><a href="#pyspark-usage-guide-for-pandas-with-apache-arrow" id="markdown-toc-pyspark-usage-guide-for-pandas-with-apache-arrow">PySpark Usage Guide for Pandas with Apache Arrow</a> <ul> |
| <li><a href="#apache-arrow-in-spark" id="markdown-toc-apache-arrow-in-spark">Apache Arrow in Spark</a> <ul> |
| <li><a href="#ensure-pyarrow-installed" id="markdown-toc-ensure-pyarrow-installed">Ensure PyArrow Installed</a></li> |
| </ul> |
| </li> |
| <li><a href="#enabling-for-conversion-tofrom-pandas" id="markdown-toc-enabling-for-conversion-tofrom-pandas">Enabling for Conversion to/from Pandas</a></li> |
| <li><a href="#pandas-udfs-aka-vectorized-udfs" id="markdown-toc-pandas-udfs-aka-vectorized-udfs">Pandas UDFs (a.k.a. Vectorized UDFs)</a> <ul> |
| <li><a href="#scalar" id="markdown-toc-scalar">Scalar</a></li> |
| <li><a href="#grouped-map" id="markdown-toc-grouped-map">Grouped Map</a></li> |
| </ul> |
| </li> |
| <li><a href="#usage-notes" id="markdown-toc-usage-notes">Usage Notes</a> <ul> |
| <li><a href="#supported-sql-types" id="markdown-toc-supported-sql-types">Supported SQL Types</a></li> |
| <li><a href="#setting-arrow-batch-size" id="markdown-toc-setting-arrow-batch-size">Setting Arrow Batch Size</a></li> |
| <li><a href="#timestamp-with-time-zone-semantics" id="markdown-toc-timestamp-with-time-zone-semantics">Timestamp with Time Zone Semantics</a></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><a href="#migration-guide" id="markdown-toc-migration-guide">Migration Guide</a> <ul> |
| <li><a href="#upgrading-from-spark-sql-230-to-231-and-above" id="markdown-toc-upgrading-from-spark-sql-230-to-231-and-above">Upgrading From Spark SQL 2.3.0 to 2.3.1 and above</a></li> |
| <li><a href="#upgrading-from-spark-sql-22-to-23" id="markdown-toc-upgrading-from-spark-sql-22-to-23">Upgrading From Spark SQL 2.2 to 2.3</a></li> |
| <li><a href="#upgrading-from-spark-sql-21-to-22" id="markdown-toc-upgrading-from-spark-sql-21-to-22">Upgrading From Spark SQL 2.1 to 2.2</a></li> |
| <li><a href="#upgrading-from-spark-sql-20-to-21" id="markdown-toc-upgrading-from-spark-sql-20-to-21">Upgrading From Spark SQL 2.0 to 2.1</a></li> |
| <li><a href="#upgrading-from-spark-sql-16-to-20" id="markdown-toc-upgrading-from-spark-sql-16-to-20">Upgrading From Spark SQL 1.6 to 2.0</a></li> |
| <li><a href="#upgrading-from-spark-sql-15-to-16" id="markdown-toc-upgrading-from-spark-sql-15-to-16">Upgrading From Spark SQL 1.5 to 1.6</a></li> |
| <li><a href="#upgrading-from-spark-sql-14-to-15" id="markdown-toc-upgrading-from-spark-sql-14-to-15">Upgrading From Spark SQL 1.4 to 1.5</a></li> |
| <li><a href="#upgrading-from-spark-sql-13-to-14" id="markdown-toc-upgrading-from-spark-sql-13-to-14">Upgrading from Spark SQL 1.3 to 1.4</a> <ul> |
| <li><a href="#dataframe-data-readerwriter-interface" id="markdown-toc-dataframe-data-readerwriter-interface">DataFrame data reader/writer interface</a></li> |
| <li><a href="#dataframegroupby-retains-grouping-columns" id="markdown-toc-dataframegroupby-retains-grouping-columns">DataFrame.groupBy retains grouping columns</a></li> |
| <li><a href="#behavior-change-on-dataframewithcolumn" id="markdown-toc-behavior-change-on-dataframewithcolumn">Behavior change on DataFrame.withColumn</a></li> |
| </ul> |
| </li> |
| <li><a href="#upgrading-from-spark-sql-10-12-to-13" id="markdown-toc-upgrading-from-spark-sql-10-12-to-13">Upgrading from Spark SQL 1.0-1.2 to 1.3</a> <ul> |
| <li><a href="#rename-of-schemardd-to-dataframe" id="markdown-toc-rename-of-schemardd-to-dataframe">Rename of SchemaRDD to DataFrame</a></li> |
| <li><a href="#unification-of-the-java-and-scala-apis" id="markdown-toc-unification-of-the-java-and-scala-apis">Unification of the Java and Scala APIs</a></li> |
| <li><a href="#isolation-of-implicit-conversions-and-removal-of-dsl-package-scala-only" id="markdown-toc-isolation-of-implicit-conversions-and-removal-of-dsl-package-scala-only">Isolation of Implicit Conversions and Removal of dsl Package (Scala-only)</a></li> |
| <li><a href="#removal-of-the-type-aliases-in-orgapachesparksql-for-datatype-scala-only" id="markdown-toc-removal-of-the-type-aliases-in-orgapachesparksql-for-datatype-scala-only">Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only)</a></li> |
| <li><a href="#udf-registration-moved-to-sqlcontextudf-java--scala" id="markdown-toc-udf-registration-moved-to-sqlcontextudf-java--scala">UDF Registration Moved to <code>sqlContext.udf</code> (Java & Scala)</a></li> |
| <li><a href="#python-datatypes-no-longer-singletons" id="markdown-toc-python-datatypes-no-longer-singletons">Python DataTypes No Longer Singletons</a></li> |
| </ul> |
| </li> |
| <li><a href="#compatibility-with-apache-hive" id="markdown-toc-compatibility-with-apache-hive">Compatibility with Apache Hive</a> <ul> |
| <li><a href="#deploying-in-existing-hive-warehouses" id="markdown-toc-deploying-in-existing-hive-warehouses">Deploying in Existing Hive Warehouses</a></li> |
| <li><a href="#supported-hive-features" id="markdown-toc-supported-hive-features">Supported Hive Features</a></li> |
| <li><a href="#unsupported-hive-functionality" id="markdown-toc-unsupported-hive-functionality">Unsupported Hive Functionality</a></li> |
| <li><a href="#incompatible-hive-udf" id="markdown-toc-incompatible-hive-udf">Incompatible Hive UDF</a></li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| <li><a href="#reference" id="markdown-toc-reference">Reference</a> <ul> |
| <li><a href="#data-types" id="markdown-toc-data-types">Data Types</a></li> |
| <li><a href="#nan-semantics" id="markdown-toc-nan-semantics">NaN Semantics</a></li> |
| </ul> |
| </li> |
| </ul> |
| |
| <h1 id="overview">Overview</h1> |
| |
| <p>Spark SQL is a Spark module for structured data processing. Unlike the basic Spark RDD API, the interfaces provided |
| by Spark SQL provide Spark with more information about the structure of both the data and the computation being performed. Internally, |
| Spark SQL uses this extra information to perform extra optimizations. There are several ways to |
| interact with Spark SQL including SQL and the Dataset API. When computing a result |
| the same execution engine is used, independent of which API/language you are using to express the |
| computation. This unification means that developers can easily switch back and forth between |
| different APIs based on which provides the most natural way to express a given transformation.</p> |
| |
| <p>All of the examples on this page use sample data included in the Spark distribution and can be run in |
| the <code>spark-shell</code>, <code>pyspark</code> shell, or <code>sparkR</code> shell.</p> |
| |
| <h2 id="sql">SQL</h2> |
| |
| <p>One use of Spark SQL is to execute SQL queries. |
| Spark SQL can also be used to read data from an existing Hive installation. For more on how to |
| configure this feature, please refer to the <a href="#hive-tables">Hive Tables</a> section. When running |
| SQL from within another programming language the results will be returned as a <a href="#datasets-and-dataframes">Dataset/DataFrame</a>. |
| You can also interact with the SQL interface using the <a href="#running-the-spark-sql-cli">command-line</a> |
| or over <a href="#running-the-thrift-jdbcodbc-server">JDBC/ODBC</a>.</p> |
| |
| <h2 id="datasets-and-dataframes">Datasets and DataFrames</h2> |
| |
| <p>A Dataset is a distributed collection of data. |
| Dataset is a new interface added in Spark 1.6 that provides the benefits of RDDs (strong |
| typing, ability to use powerful lambda functions) with the benefits of Spark SQL’s optimized |
| execution engine. A Dataset can be <a href="#creating-datasets">constructed</a> from JVM objects and then |
| manipulated using functional transformations (<code>map</code>, <code>flatMap</code>, <code>filter</code>, etc.). |
| The Dataset API is available in <a href="api/scala/index.html#org.apache.spark.sql.Dataset">Scala</a> and |
| <a href="api/java/index.html?org/apache/spark/sql/Dataset.html">Java</a>. Python does not have the support for the Dataset API. But due to Python’s dynamic nature, |
| many of the benefits of the Dataset API are already available (i.e. you can access the field of a row by name naturally |
| <code>row.columnName</code>). The case for R is similar.</p> |
| |
| <p>A DataFrame is a <em>Dataset</em> organized into named columns. It is conceptually |
| equivalent to a table in a relational database or a data frame in R/Python, but with richer |
| optimizations under the hood. DataFrames can be constructed from a wide array of <a href="#data-sources">sources</a> such |
| as: structured data files, tables in Hive, external databases, or existing RDDs. |
| The DataFrame API is available in Scala, |
| Java, <a href="api/python/pyspark.sql.html#pyspark.sql.DataFrame">Python</a>, and <a href="api/R/index.html">R</a>. |
| In Scala and Java, a DataFrame is represented by a Dataset of <code>Row</code>s. |
| In <a href="api/scala/index.html#org.apache.spark.sql.Dataset">the Scala API</a>, <code>DataFrame</code> is simply a type alias of <code>Dataset[Row]</code>. |
| While, in <a href="api/java/index.html?org/apache/spark/sql/Dataset.html">Java API</a>, users need to use <code>Dataset<Row></code> to represent a <code>DataFrame</code>.</p> |
| |
| <p>Throughout this document, we will often refer to Scala/Java Datasets of <code>Row</code>s as DataFrames.</p> |
| |
| <h1 id="getting-started">Getting Started</h1> |
| |
| <h2 id="starting-point-sparksession">Starting Point: SparkSession</h2> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| |
| <p>The entry point into all functionality in Spark is the <a href="api/scala/index.html#org.apache.spark.sql.SparkSession"><code>SparkSession</code></a> class. To create a basic <code>SparkSession</code>, just use <code>SparkSession.builder()</code>:</p> |
| |
| <div class="highlight"><pre><span></span><span class="k">import</span> <span class="nn">org.apache.spark.sql.SparkSession</span> |
| |
| <span class="k">val</span> <span class="n">spark</span> <span class="k">=</span> <span class="nc">SparkSession</span> |
| <span class="o">.</span><span class="n">builder</span><span class="o">()</span> |
| <span class="o">.</span><span class="n">appName</span><span class="o">(</span><span class="s">"Spark SQL basic example"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">config</span><span class="o">(</span><span class="s">"spark.some.config.option"</span><span class="o">,</span> <span class="s">"some-value"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">getOrCreate</span><span class="o">()</span> |
| |
| <span class="c1">// For implicit conversions like converting RDDs to DataFrames</span> |
| <span class="k">import</span> <span class="nn">spark.implicits._</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| |
| <p>The entry point into all functionality in Spark is the <a href="api/java/index.html#org.apache.spark.sql.SparkSession"><code>SparkSession</code></a> class. To create a basic <code>SparkSession</code>, just use <code>SparkSession.builder()</code>:</p> |
| |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">org.apache.spark.sql.SparkSession</span><span class="o">;</span> |
| |
| <span class="n">SparkSession</span> <span class="n">spark</span> <span class="o">=</span> <span class="n">SparkSession</span> |
| <span class="o">.</span><span class="na">builder</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">appName</span><span class="o">(</span><span class="s">"Java Spark SQL basic example"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">config</span><span class="o">(</span><span class="s">"spark.some.config.option"</span><span class="o">,</span> <span class="s">"some-value"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">getOrCreate</span><span class="o">();</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| |
| <p>The entry point into all functionality in Spark is the <a href="api/python/pyspark.sql.html#pyspark.sql.SparkSession"><code>SparkSession</code></a> class. To create a basic <code>SparkSession</code>, just use <code>SparkSession.builder</code>:</p> |
| |
| <div class="highlight"><pre><span></span><span class="kn">from</span> <span class="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">SparkSession</span> |
| |
| <span class="n">spark</span> <span class="o">=</span> <span class="n">SparkSession</span> \ |
| <span class="o">.</span><span class="n">builder</span> \ |
| <span class="o">.</span><span class="n">appName</span><span class="p">(</span><span class="s2">"Python Spark SQL basic example"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">config</span><span class="p">(</span><span class="s2">"spark.some.config.option"</span><span class="p">,</span> <span class="s2">"some-value"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">getOrCreate</span><span class="p">()</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/basic.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="r"> |
| |
| <p>The entry point into all functionality in Spark is the <a href="api/R/sparkR.session.html"><code>SparkSession</code></a> class. To initialize a basic <code>SparkSession</code>, just call <code>sparkR.session()</code>:</p> |
| |
| <div class="highlight"><pre><span></span>sparkR.session<span class="p">(</span>appName <span class="o">=</span> <span class="s">"R Spark SQL basic example"</span><span class="p">,</span> sparkConfig <span class="o">=</span> <span class="kt">list</span><span class="p">(</span>spark.some.config.option <span class="o">=</span> <span class="s">"some-value"</span><span class="p">))</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| |
| <p>Note that when invoked for the first time, <code>sparkR.session()</code> initializes a global <code>SparkSession</code> singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the <code>SparkSession</code> once, then SparkR functions like <code>read.df</code> will be able to access this global instance implicitly, and users don’t need to pass the <code>SparkSession</code> instance around.</p> |
| </div> |
| </div> |
| |
| <p><code>SparkSession</code> in Spark 2.0 provides builtin support for Hive features including the ability to |
| write queries using HiveQL, access to Hive UDFs, and the ability to read data from Hive tables. |
| To use these features, you do not need to have an existing Hive setup.</p> |
| |
| <h2 id="creating-dataframes">Creating DataFrames</h2> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| <p>With a <code>SparkSession</code>, applications can create DataFrames from an <a href="#interoperating-with-rdds">existing <code>RDD</code></a>, |
| from a Hive table, or from <a href="#data-sources">Spark data sources</a>.</p> |
| |
| <p>As an example, the following creates a DataFrame based on the content of a JSON file:</p> |
| |
| <div class="highlight"><pre><span></span><span class="k">val</span> <span class="n">df</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">json</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.json"</span><span class="o">)</span> |
| |
| <span class="c1">// Displays the content of the DataFrame to stdout</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// | age| name|</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// |null|Michael|</span> |
| <span class="c1">// | 30| Andy|</span> |
| <span class="c1">// | 19| Justin|</span> |
| <span class="c1">// +----+-------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <p>With a <code>SparkSession</code>, applications can create DataFrames from an <a href="#interoperating-with-rdds">existing <code>RDD</code></a>, |
| from a Hive table, or from <a href="#data-sources">Spark data sources</a>.</p> |
| |
| <p>As an example, the following creates a DataFrame based on the content of a JSON file:</p> |
| |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">org.apache.spark.sql.Dataset</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Row</span><span class="o">;</span> |
| |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">df</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">json</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.json"</span><span class="o">);</span> |
| |
| <span class="c1">// Displays the content of the DataFrame to stdout</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// | age| name|</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// |null|Michael|</span> |
| <span class="c1">// | 30| Andy|</span> |
| <span class="c1">// | 19| Justin|</span> |
| <span class="c1">// +----+-------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| <p>With a <code>SparkSession</code>, applications can create DataFrames from an <a href="#interoperating-with-rdds">existing <code>RDD</code></a>, |
| from a Hive table, or from <a href="#data-sources">Spark data sources</a>.</p> |
| |
| <p>As an example, the following creates a DataFrame based on the content of a JSON file:</p> |
| |
| <div class="highlight"><pre><span></span><span class="c1"># spark is an existing SparkSession</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">json</span><span class="p">(</span><span class="s2">"examples/src/main/resources/people.json"</span><span class="p">)</span> |
| <span class="c1"># Displays the content of the DataFrame to stdout</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +----+-------+</span> |
| <span class="c1"># | age| name|</span> |
| <span class="c1"># +----+-------+</span> |
| <span class="c1"># |null|Michael|</span> |
| <span class="c1"># | 30| Andy|</span> |
| <span class="c1"># | 19| Justin|</span> |
| <span class="c1"># +----+-------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/basic.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="r"> |
| <p>With a <code>SparkSession</code>, applications can create DataFrames from a local R data.frame, |
| from a Hive table, or from <a href="#data-sources">Spark data sources</a>.</p> |
| |
| <p>As an example, the following creates a DataFrame based on the content of a JSON file:</p> |
| |
| <div class="highlight"><pre><span></span>df <span class="o"><-</span> read.json<span class="p">(</span><span class="s">"examples/src/main/resources/people.json"</span><span class="p">)</span> |
| |
| <span class="c1"># Displays the content of the DataFrame</span> |
| <span class="kp">head</span><span class="p">(</span>df<span class="p">)</span> |
| <span class="c1">## age name</span> |
| <span class="c1">## 1 NA Michael</span> |
| <span class="c1">## 2 30 Andy</span> |
| <span class="c1">## 3 19 Justin</span> |
| |
| <span class="c1"># Another method to print the first few rows and optionally truncate the printing of long values</span> |
| showDF<span class="p">(</span>df<span class="p">)</span> |
| <span class="c1">## +----+-------+</span> |
| <span class="c1">## | age| name|</span> |
| <span class="c1">## +----+-------+</span> |
| <span class="c1">## |null|Michael|</span> |
| <span class="c1">## | 30| Andy|</span> |
| <span class="c1">## | 19| Justin|</span> |
| <span class="c1">## +----+-------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| |
| </div> |
| </div> |
| |
| <h2 id="untyped-dataset-operations-aka-dataframe-operations">Untyped Dataset Operations (aka DataFrame Operations)</h2> |
| |
| <p>DataFrames provide a domain-specific language for structured data manipulation in <a href="api/scala/index.html#org.apache.spark.sql.Dataset">Scala</a>, <a href="api/java/index.html?org/apache/spark/sql/Dataset.html">Java</a>, <a href="api/python/pyspark.sql.html#pyspark.sql.DataFrame">Python</a> and <a href="api/R/SparkDataFrame.html">R</a>.</p> |
| |
| <p>As mentioned above, in Spark 2.0, DataFrames are just Dataset of <code>Row</code>s in Scala and Java API. These operations are also referred as “untyped transformations” in contrast to “typed transformations” come with strongly typed Scala/Java Datasets.</p> |
| |
| <p>Here we include some basic examples of structured data processing using Datasets:</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="c1">// This import is needed to use the $-notation</span> |
| <span class="k">import</span> <span class="nn">spark.implicits._</span> |
| <span class="c1">// Print the schema in a tree format</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">printSchema</span><span class="o">()</span> |
| <span class="c1">// root</span> |
| <span class="c1">// |-- age: long (nullable = true)</span> |
| <span class="c1">// |-- name: string (nullable = true)</span> |
| |
| <span class="c1">// Select only the "name" column</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">select</span><span class="o">(</span><span class="s">"name"</span><span class="o">).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +-------+</span> |
| <span class="c1">// | name|</span> |
| <span class="c1">// +-------+</span> |
| <span class="c1">// |Michael|</span> |
| <span class="c1">// | Andy|</span> |
| <span class="c1">// | Justin|</span> |
| <span class="c1">// +-------+</span> |
| |
| <span class="c1">// Select everybody, but increment the age by 1</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">select</span><span class="o">(</span><span class="n">$</span><span class="s">"name"</span><span class="o">,</span> <span class="n">$</span><span class="s">"age"</span> <span class="o">+</span> <span class="mi">1</span><span class="o">).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +-------+---------+</span> |
| <span class="c1">// | name|(age + 1)|</span> |
| <span class="c1">// +-------+---------+</span> |
| <span class="c1">// |Michael| null|</span> |
| <span class="c1">// | Andy| 31|</span> |
| <span class="c1">// | Justin| 20|</span> |
| <span class="c1">// +-------+---------+</span> |
| |
| <span class="c1">// Select people older than 21</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">filter</span><span class="o">(</span><span class="n">$</span><span class="s">"age"</span> <span class="o">></span> <span class="mi">21</span><span class="o">).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +---+----+</span> |
| <span class="c1">// |age|name|</span> |
| <span class="c1">// +---+----+</span> |
| <span class="c1">// | 30|Andy|</span> |
| <span class="c1">// +---+----+</span> |
| |
| <span class="c1">// Count people by age</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">groupBy</span><span class="o">(</span><span class="s">"age"</span><span class="o">).</span><span class="n">count</span><span class="o">().</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +----+-----+</span> |
| <span class="c1">// | age|count|</span> |
| <span class="c1">// +----+-----+</span> |
| <span class="c1">// | 19| 1|</span> |
| <span class="c1">// |null| 1|</span> |
| <span class="c1">// | 30| 1|</span> |
| <span class="c1">// +----+-----+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala" in the Spark repo.</small></div> |
| |
| <p>For a complete list of the types of operations that can be performed on a Dataset refer to the <a href="api/scala/index.html#org.apache.spark.sql.Dataset">API Documentation</a>.</p> |
| |
| <p>In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the <a href="api/scala/index.html#org.apache.spark.sql.functions$">DataFrame Function Reference</a>.</p> |
| </div> |
| |
| <div data-lang="java"> |
| |
| <div class="highlight"><pre><span></span><span class="c1">// col("...") is preferable to df.col("...")</span> |
| <span class="kn">import static</span> <span class="nn">org.apache.spark.sql.functions.col</span><span class="o">;</span> |
| |
| <span class="c1">// Print the schema in a tree format</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">printSchema</span><span class="o">();</span> |
| <span class="c1">// root</span> |
| <span class="c1">// |-- age: long (nullable = true)</span> |
| <span class="c1">// |-- name: string (nullable = true)</span> |
| |
| <span class="c1">// Select only the "name" column</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">select</span><span class="o">(</span><span class="s">"name"</span><span class="o">).</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +-------+</span> |
| <span class="c1">// | name|</span> |
| <span class="c1">// +-------+</span> |
| <span class="c1">// |Michael|</span> |
| <span class="c1">// | Andy|</span> |
| <span class="c1">// | Justin|</span> |
| <span class="c1">// +-------+</span> |
| |
| <span class="c1">// Select everybody, but increment the age by 1</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">select</span><span class="o">(</span><span class="n">col</span><span class="o">(</span><span class="s">"name"</span><span class="o">),</span> <span class="n">col</span><span class="o">(</span><span class="s">"age"</span><span class="o">).</span><span class="na">plus</span><span class="o">(</span><span class="mi">1</span><span class="o">)).</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +-------+---------+</span> |
| <span class="c1">// | name|(age + 1)|</span> |
| <span class="c1">// +-------+---------+</span> |
| <span class="c1">// |Michael| null|</span> |
| <span class="c1">// | Andy| 31|</span> |
| <span class="c1">// | Justin| 20|</span> |
| <span class="c1">// +-------+---------+</span> |
| |
| <span class="c1">// Select people older than 21</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">filter</span><span class="o">(</span><span class="n">col</span><span class="o">(</span><span class="s">"age"</span><span class="o">).</span><span class="na">gt</span><span class="o">(</span><span class="mi">21</span><span class="o">)).</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +---+----+</span> |
| <span class="c1">// |age|name|</span> |
| <span class="c1">// +---+----+</span> |
| <span class="c1">// | 30|Andy|</span> |
| <span class="c1">// +---+----+</span> |
| |
| <span class="c1">// Count people by age</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span><span class="s">"age"</span><span class="o">).</span><span class="na">count</span><span class="o">().</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +----+-----+</span> |
| <span class="c1">// | age|count|</span> |
| <span class="c1">// +----+-----+</span> |
| <span class="c1">// | 19| 1|</span> |
| <span class="c1">// |null| 1|</span> |
| <span class="c1">// | 30| 1|</span> |
| <span class="c1">// +----+-----+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java" in the Spark repo.</small></div> |
| |
| <p>For a complete list of the types of operations that can be performed on a Dataset refer to the <a href="api/java/org/apache/spark/sql/Dataset.html">API Documentation</a>.</p> |
| |
| <p>In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the <a href="api/java/org/apache/spark/sql/functions.html">DataFrame Function Reference</a>.</p> |
| </div> |
| |
| <div data-lang="python"> |
| <p>In Python it’s possible to access a DataFrame’s columns either by attribute |
| (<code>df.age</code>) or by indexing (<code>df['age']</code>). While the former is convenient for |
| interactive data exploration, users are highly encouraged to use the |
| latter form, which is future proof and won’t break with column names that |
| are also attributes on the DataFrame class.</p> |
| |
| <div class="highlight"><pre><span></span><span class="c1"># spark, df are from the previous example</span> |
| <span class="c1"># Print the schema in a tree format</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">printSchema</span><span class="p">()</span> |
| <span class="c1"># root</span> |
| <span class="c1"># |-- age: long (nullable = true)</span> |
| <span class="c1"># |-- name: string (nullable = true)</span> |
| |
| <span class="c1"># Select only the "name" column</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="s2">"name"</span><span class="p">)</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +-------+</span> |
| <span class="c1"># | name|</span> |
| <span class="c1"># +-------+</span> |
| <span class="c1"># |Michael|</span> |
| <span class="c1"># | Andy|</span> |
| <span class="c1"># | Justin|</span> |
| <span class="c1"># +-------+</span> |
| |
| <span class="c1"># Select everybody, but increment the age by 1</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">df</span><span class="p">[</span><span class="s1">'name'</span><span class="p">],</span> <span class="n">df</span><span class="p">[</span><span class="s1">'age'</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">show</span><span class="p">()</span> |
| <span class="c1"># +-------+---------+</span> |
| <span class="c1"># | name|(age + 1)|</span> |
| <span class="c1"># +-------+---------+</span> |
| <span class="c1"># |Michael| null|</span> |
| <span class="c1"># | Andy| 31|</span> |
| <span class="c1"># | Justin| 20|</span> |
| <span class="c1"># +-------+---------+</span> |
| |
| <span class="c1"># Select people older than 21</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">filter</span><span class="p">(</span><span class="n">df</span><span class="p">[</span><span class="s1">'age'</span><span class="p">]</span> <span class="o">></span> <span class="mi">21</span><span class="p">)</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +---+----+</span> |
| <span class="c1"># |age|name|</span> |
| <span class="c1"># +---+----+</span> |
| <span class="c1"># | 30|Andy|</span> |
| <span class="c1"># +---+----+</span> |
| |
| <span class="c1"># Count people by age</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">groupBy</span><span class="p">(</span><span class="s2">"age"</span><span class="p">)</span><span class="o">.</span><span class="n">count</span><span class="p">()</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +----+-----+</span> |
| <span class="c1"># | age|count|</span> |
| <span class="c1"># +----+-----+</span> |
| <span class="c1"># | 19| 1|</span> |
| <span class="c1"># |null| 1|</span> |
| <span class="c1"># | 30| 1|</span> |
| <span class="c1"># +----+-----+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/basic.py" in the Spark repo.</small></div> |
| <p>For a complete list of the types of operations that can be performed on a DataFrame refer to the <a href="api/python/pyspark.sql.html#pyspark.sql.DataFrame">API Documentation</a>.</p> |
| |
| <p>In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the <a href="api/python/pyspark.sql.html#module-pyspark.sql.functions">DataFrame Function Reference</a>.</p> |
| |
| </div> |
| |
| <div data-lang="r"> |
| |
| <div class="highlight"><pre><span></span><span class="c1"># Create the DataFrame</span> |
| df <span class="o"><-</span> read.json<span class="p">(</span><span class="s">"examples/src/main/resources/people.json"</span><span class="p">)</span> |
| |
| <span class="c1"># Show the content of the DataFrame</span> |
| <span class="kp">head</span><span class="p">(</span>df<span class="p">)</span> |
| <span class="c1">## age name</span> |
| <span class="c1">## 1 NA Michael</span> |
| <span class="c1">## 2 30 Andy</span> |
| <span class="c1">## 3 19 Justin</span> |
| |
| |
| <span class="c1"># Print the schema in a tree format</span> |
| printSchema<span class="p">(</span>df<span class="p">)</span> |
| <span class="c1">## root</span> |
| <span class="c1">## |-- age: long (nullable = true)</span> |
| <span class="c1">## |-- name: string (nullable = true)</span> |
| |
| <span class="c1"># Select only the "name" column</span> |
| <span class="kp">head</span><span class="p">(</span>select<span class="p">(</span>df<span class="p">,</span> <span class="s">"name"</span><span class="p">))</span> |
| <span class="c1">## name</span> |
| <span class="c1">## 1 Michael</span> |
| <span class="c1">## 2 Andy</span> |
| <span class="c1">## 3 Justin</span> |
| |
| <span class="c1"># Select everybody, but increment the age by 1</span> |
| <span class="kp">head</span><span class="p">(</span>select<span class="p">(</span>df<span class="p">,</span> df<span class="o">$</span>name<span class="p">,</span> df<span class="o">$</span>age <span class="o">+</span> <span class="m">1</span><span class="p">))</span> |
| <span class="c1">## name (age + 1.0)</span> |
| <span class="c1">## 1 Michael NA</span> |
| <span class="c1">## 2 Andy 31</span> |
| <span class="c1">## 3 Justin 20</span> |
| |
| <span class="c1"># Select people older than 21</span> |
| <span class="kp">head</span><span class="p">(</span>where<span class="p">(</span>df<span class="p">,</span> df<span class="o">$</span>age <span class="o">></span> <span class="m">21</span><span class="p">))</span> |
| <span class="c1">## age name</span> |
| <span class="c1">## 1 30 Andy</span> |
| |
| <span class="c1"># Count people by age</span> |
| <span class="kp">head</span><span class="p">(</span>count<span class="p">(</span>groupBy<span class="p">(</span>df<span class="p">,</span> <span class="s">"age"</span><span class="p">)))</span> |
| <span class="c1">## age count</span> |
| <span class="c1">## 1 19 1</span> |
| <span class="c1">## 2 NA 1</span> |
| <span class="c1">## 3 30 1</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| |
| <p>For a complete list of the types of operations that can be performed on a DataFrame refer to the <a href="api/R/index.html">API Documentation</a>.</p> |
| |
| <p>In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the <a href="api/R/SparkDataFrame.html">DataFrame Function Reference</a>.</p> |
| |
| </div> |
| |
| </div> |
| |
| <h2 id="running-sql-queries-programmatically">Running SQL Queries Programmatically</h2> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| <p>The <code>sql</code> function on a <code>SparkSession</code> enables applications to run SQL queries programmatically and returns the result as a <code>DataFrame</code>.</p> |
| |
| <div class="highlight"><pre><span></span><span class="c1">// Register the DataFrame as a SQL temporary view</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">createOrReplaceTempView</span><span class="o">(</span><span class="s">"people"</span><span class="o">)</span> |
| |
| <span class="k">val</span> <span class="n">sqlDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="o">(</span><span class="s">"SELECT * FROM people"</span><span class="o">)</span> |
| <span class="n">sqlDF</span><span class="o">.</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// | age| name|</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// |null|Michael|</span> |
| <span class="c1">// | 30| Andy|</span> |
| <span class="c1">// | 19| Justin|</span> |
| <span class="c1">// +----+-------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <p>The <code>sql</code> function on a <code>SparkSession</code> enables applications to run SQL queries programmatically and returns the result as a <code>Dataset<Row></code>.</p> |
| |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">org.apache.spark.sql.Dataset</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Row</span><span class="o">;</span> |
| |
| <span class="c1">// Register the DataFrame as a SQL temporary view</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">createOrReplaceTempView</span><span class="o">(</span><span class="s">"people"</span><span class="o">);</span> |
| |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">sqlDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT * FROM people"</span><span class="o">);</span> |
| <span class="n">sqlDF</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// | age| name|</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// |null|Michael|</span> |
| <span class="c1">// | 30| Andy|</span> |
| <span class="c1">// | 19| Justin|</span> |
| <span class="c1">// +----+-------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| <p>The <code>sql</code> function on a <code>SparkSession</code> enables applications to run SQL queries programmatically and returns the result as a <code>DataFrame</code>.</p> |
| |
| <div class="highlight"><pre><span></span><span class="c1"># Register the DataFrame as a SQL temporary view</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">createOrReplaceTempView</span><span class="p">(</span><span class="s2">"people"</span><span class="p">)</span> |
| |
| <span class="n">sqlDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT * FROM people"</span><span class="p">)</span> |
| <span class="n">sqlDF</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +----+-------+</span> |
| <span class="c1"># | age| name|</span> |
| <span class="c1"># +----+-------+</span> |
| <span class="c1"># |null|Michael|</span> |
| <span class="c1"># | 30| Andy|</span> |
| <span class="c1"># | 19| Justin|</span> |
| <span class="c1"># +----+-------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/basic.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="r"> |
| <p>The <code>sql</code> function enables applications to run SQL queries programmatically and returns the result as a <code>SparkDataFrame</code>.</p> |
| |
| <div class="highlight"><pre><span></span>df <span class="o"><-</span> sql<span class="p">(</span><span class="s">"SELECT * FROM table"</span><span class="p">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| |
| </div> |
| </div> |
| |
| <h2 id="global-temporary-view">Global Temporary View</h2> |
| |
| <p>Temporary views in Spark SQL are session-scoped and will disappear if the session that creates it |
| terminates. If you want to have a temporary view that is shared among all sessions and keep alive |
| until the Spark application terminates, you can create a global temporary view. Global temporary |
| view is tied to a system preserved database <code>global_temp</code>, and we must use the qualified name to |
| refer it, e.g. <code>SELECT * FROM global_temp.view1</code>.</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="c1">// Register the DataFrame as a global temporary view</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">createGlobalTempView</span><span class="o">(</span><span class="s">"people"</span><span class="o">)</span> |
| |
| <span class="c1">// Global temporary view is tied to a system preserved database `global_temp`</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="o">(</span><span class="s">"SELECT * FROM global_temp.people"</span><span class="o">).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// | age| name|</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// |null|Michael|</span> |
| <span class="c1">// | 30| Andy|</span> |
| <span class="c1">// | 19| Justin|</span> |
| <span class="c1">// +----+-------+</span> |
| |
| <span class="c1">// Global temporary view is cross-session</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">newSession</span><span class="o">().</span><span class="n">sql</span><span class="o">(</span><span class="s">"SELECT * FROM global_temp.people"</span><span class="o">).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// | age| name|</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// |null|Michael|</span> |
| <span class="c1">// | 30| Andy|</span> |
| <span class="c1">// | 19| Justin|</span> |
| <span class="c1">// +----+-------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="c1">// Register the DataFrame as a global temporary view</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">createGlobalTempView</span><span class="o">(</span><span class="s">"people"</span><span class="o">);</span> |
| |
| <span class="c1">// Global temporary view is tied to a system preserved database `global_temp`</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT * FROM global_temp.people"</span><span class="o">).</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// | age| name|</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// |null|Michael|</span> |
| <span class="c1">// | 30| Andy|</span> |
| <span class="c1">// | 19| Justin|</span> |
| <span class="c1">// +----+-------+</span> |
| |
| <span class="c1">// Global temporary view is cross-session</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">newSession</span><span class="o">().</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT * FROM global_temp.people"</span><span class="o">).</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// | age| name|</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// |null|Michael|</span> |
| <span class="c1">// | 30| Andy|</span> |
| <span class="c1">// | 19| Justin|</span> |
| <span class="c1">// +----+-------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| <div class="highlight"><pre><span></span><span class="c1"># Register the DataFrame as a global temporary view</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">createGlobalTempView</span><span class="p">(</span><span class="s2">"people"</span><span class="p">)</span> |
| |
| <span class="c1"># Global temporary view is tied to a system preserved database `global_temp`</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT * FROM global_temp.people"</span><span class="p">)</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +----+-------+</span> |
| <span class="c1"># | age| name|</span> |
| <span class="c1"># +----+-------+</span> |
| <span class="c1"># |null|Michael|</span> |
| <span class="c1"># | 30| Andy|</span> |
| <span class="c1"># | 19| Justin|</span> |
| <span class="c1"># +----+-------+</span> |
| |
| <span class="c1"># Global temporary view is cross-session</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">newSession</span><span class="p">()</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT * FROM global_temp.people"</span><span class="p">)</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +----+-------+</span> |
| <span class="c1"># | age| name|</span> |
| <span class="c1"># +----+-------+</span> |
| <span class="c1"># |null|Michael|</span> |
| <span class="c1"># | 30| Andy|</span> |
| <span class="c1"># | 19| Justin|</span> |
| <span class="c1"># +----+-------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/basic.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="sql"> |
| |
| <figure class="highlight"><pre><code class="language-sql" data-lang="sql"><span></span><span class="k">CREATE</span> <span class="k">GLOBAL</span> <span class="k">TEMPORARY</span> <span class="k">VIEW</span> <span class="n">temp_view</span> <span class="k">AS</span> <span class="k">SELECT</span> <span class="n">a</span> <span class="o">+</span> <span class="mi">1</span><span class="p">,</span> <span class="n">b</span> <span class="o">*</span> <span class="mi">2</span> <span class="k">FROM</span> <span class="n">tbl</span> |
| |
| <span class="k">SELECT</span> <span class="o">*</span> <span class="k">FROM</span> <span class="n">global_temp</span><span class="p">.</span><span class="n">temp_view</span></code></pre></figure> |
| |
| </div> |
| </div> |
| |
| <h2 id="creating-datasets">Creating Datasets</h2> |
| |
| <p>Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use |
| a specialized <a href="api/scala/index.html#org.apache.spark.sql.Encoder">Encoder</a> to serialize the objects |
| for processing or transmitting over the network. While both encoders and standard serialization are |
| responsible for turning an object into bytes, encoders are code generated dynamically and use a format |
| that allows Spark to perform many operations like filtering, sorting and hashing without deserializing |
| the bytes back into an object.</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="k">case</span> <span class="k">class</span> <span class="nc">Person</span><span class="o">(</span><span class="n">name</span><span class="k">:</span> <span class="kt">String</span><span class="o">,</span> <span class="n">age</span><span class="k">:</span> <span class="kt">Long</span><span class="o">)</span> |
| |
| <span class="c1">// Encoders are created for case classes</span> |
| <span class="k">val</span> <span class="n">caseClassDS</span> <span class="k">=</span> <span class="nc">Seq</span><span class="o">(</span><span class="nc">Person</span><span class="o">(</span><span class="s">"Andy"</span><span class="o">,</span> <span class="mi">32</span><span class="o">)).</span><span class="n">toDS</span><span class="o">()</span> |
| <span class="n">caseClassDS</span><span class="o">.</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +----+---+</span> |
| <span class="c1">// |name|age|</span> |
| <span class="c1">// +----+---+</span> |
| <span class="c1">// |Andy| 32|</span> |
| <span class="c1">// +----+---+</span> |
| |
| <span class="c1">// Encoders for most common types are automatically provided by importing spark.implicits._</span> |
| <span class="k">val</span> <span class="n">primitiveDS</span> <span class="k">=</span> <span class="nc">Seq</span><span class="o">(</span><span class="mi">1</span><span class="o">,</span> <span class="mi">2</span><span class="o">,</span> <span class="mi">3</span><span class="o">).</span><span class="n">toDS</span><span class="o">()</span> |
| <span class="n">primitiveDS</span><span class="o">.</span><span class="n">map</span><span class="o">(</span><span class="k">_</span> <span class="o">+</span> <span class="mi">1</span><span class="o">).</span><span class="n">collect</span><span class="o">()</span> <span class="c1">// Returns: Array(2, 3, 4)</span> |
| |
| <span class="c1">// DataFrames can be converted to a Dataset by providing a class. Mapping will be done by name</span> |
| <span class="k">val</span> <span class="n">path</span> <span class="k">=</span> <span class="s">"examples/src/main/resources/people.json"</span> |
| <span class="k">val</span> <span class="n">peopleDS</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">json</span><span class="o">(</span><span class="n">path</span><span class="o">).</span><span class="n">as</span><span class="o">[</span><span class="kt">Person</span><span class="o">]</span> |
| <span class="n">peopleDS</span><span class="o">.</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// | age| name|</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// |null|Michael|</span> |
| <span class="c1">// | 30| Andy|</span> |
| <span class="c1">// | 19| Justin|</span> |
| <span class="c1">// +----+-------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.util.Arrays</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">java.util.Collections</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">java.io.Serializable</span><span class="o">;</span> |
| |
| <span class="kn">import</span> <span class="nn">org.apache.spark.api.java.function.MapFunction</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Dataset</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Row</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Encoder</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Encoders</span><span class="o">;</span> |
| |
| <span class="kd">public</span> <span class="kd">static</span> <span class="kd">class</span> <span class="nc">Person</span> <span class="kd">implements</span> <span class="n">Serializable</span> <span class="o">{</span> |
| <span class="kd">private</span> <span class="n">String</span> <span class="n">name</span><span class="o">;</span> |
| <span class="kd">private</span> <span class="kt">int</span> <span class="n">age</span><span class="o">;</span> |
| |
| <span class="kd">public</span> <span class="n">String</span> <span class="nf">getName</span><span class="o">()</span> <span class="o">{</span> |
| <span class="k">return</span> <span class="n">name</span><span class="o">;</span> |
| <span class="o">}</span> |
| |
| <span class="kd">public</span> <span class="kt">void</span> <span class="nf">setName</span><span class="o">(</span><span class="n">String</span> <span class="n">name</span><span class="o">)</span> <span class="o">{</span> |
| <span class="k">this</span><span class="o">.</span><span class="na">name</span> <span class="o">=</span> <span class="n">name</span><span class="o">;</span> |
| <span class="o">}</span> |
| |
| <span class="kd">public</span> <span class="kt">int</span> <span class="nf">getAge</span><span class="o">()</span> <span class="o">{</span> |
| <span class="k">return</span> <span class="n">age</span><span class="o">;</span> |
| <span class="o">}</span> |
| |
| <span class="kd">public</span> <span class="kt">void</span> <span class="nf">setAge</span><span class="o">(</span><span class="kt">int</span> <span class="n">age</span><span class="o">)</span> <span class="o">{</span> |
| <span class="k">this</span><span class="o">.</span><span class="na">age</span> <span class="o">=</span> <span class="n">age</span><span class="o">;</span> |
| <span class="o">}</span> |
| <span class="o">}</span> |
| |
| <span class="c1">// Create an instance of a Bean class</span> |
| <span class="n">Person</span> <span class="n">person</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Person</span><span class="o">();</span> |
| <span class="n">person</span><span class="o">.</span><span class="na">setName</span><span class="o">(</span><span class="s">"Andy"</span><span class="o">);</span> |
| <span class="n">person</span><span class="o">.</span><span class="na">setAge</span><span class="o">(</span><span class="mi">32</span><span class="o">);</span> |
| |
| <span class="c1">// Encoders are created for Java beans</span> |
| <span class="n">Encoder</span><span class="o"><</span><span class="n">Person</span><span class="o">></span> <span class="n">personEncoder</span> <span class="o">=</span> <span class="n">Encoders</span><span class="o">.</span><span class="na">bean</span><span class="o">(</span><span class="n">Person</span><span class="o">.</span><span class="na">class</span><span class="o">);</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Person</span><span class="o">></span> <span class="n">javaBeanDS</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">createDataset</span><span class="o">(</span> |
| <span class="n">Collections</span><span class="o">.</span><span class="na">singletonList</span><span class="o">(</span><span class="n">person</span><span class="o">),</span> |
| <span class="n">personEncoder</span> |
| <span class="o">);</span> |
| <span class="n">javaBeanDS</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +---+----+</span> |
| <span class="c1">// |age|name|</span> |
| <span class="c1">// +---+----+</span> |
| <span class="c1">// | 32|Andy|</span> |
| <span class="c1">// +---+----+</span> |
| |
| <span class="c1">// Encoders for most common types are provided in class Encoders</span> |
| <span class="n">Encoder</span><span class="o"><</span><span class="n">Integer</span><span class="o">></span> <span class="n">integerEncoder</span> <span class="o">=</span> <span class="n">Encoders</span><span class="o">.</span><span class="na">INT</span><span class="o">();</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Integer</span><span class="o">></span> <span class="n">primitiveDS</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">createDataset</span><span class="o">(</span><span class="n">Arrays</span><span class="o">.</span><span class="na">asList</span><span class="o">(</span><span class="mi">1</span><span class="o">,</span> <span class="mi">2</span><span class="o">,</span> <span class="mi">3</span><span class="o">),</span> <span class="n">integerEncoder</span><span class="o">);</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Integer</span><span class="o">></span> <span class="n">transformedDS</span> <span class="o">=</span> <span class="n">primitiveDS</span><span class="o">.</span><span class="na">map</span><span class="o">(</span> |
| <span class="o">(</span><span class="n">MapFunction</span><span class="o"><</span><span class="n">Integer</span><span class="o">,</span> <span class="n">Integer</span><span class="o">>)</span> <span class="n">value</span> <span class="o">-></span> <span class="n">value</span> <span class="o">+</span> <span class="mi">1</span><span class="o">,</span> |
| <span class="n">integerEncoder</span><span class="o">);</span> |
| <span class="n">transformedDS</span><span class="o">.</span><span class="na">collect</span><span class="o">();</span> <span class="c1">// Returns [2, 3, 4]</span> |
| |
| <span class="c1">// DataFrames can be converted to a Dataset by providing a class. Mapping based on name</span> |
| <span class="n">String</span> <span class="n">path</span> <span class="o">=</span> <span class="s">"examples/src/main/resources/people.json"</span><span class="o">;</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Person</span><span class="o">></span> <span class="n">peopleDS</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">json</span><span class="o">(</span><span class="n">path</span><span class="o">).</span><span class="na">as</span><span class="o">(</span><span class="n">personEncoder</span><span class="o">);</span> |
| <span class="n">peopleDS</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// | age| name|</span> |
| <span class="c1">// +----+-------+</span> |
| <span class="c1">// |null|Michael|</span> |
| <span class="c1">// | 30| Andy|</span> |
| <span class="c1">// | 19| Justin|</span> |
| <span class="c1">// +----+-------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java" in the Spark repo.</small></div> |
| </div> |
| </div> |
| |
| <h2 id="interoperating-with-rdds">Interoperating with RDDs</h2> |
| |
| <p>Spark SQL supports two different methods for converting existing RDDs into Datasets. The first |
| method uses reflection to infer the schema of an RDD that contains specific types of objects. This |
| reflection based approach leads to more concise code and works well when you already know the schema |
| while writing your Spark application.</p> |
| |
| <p>The second method for creating Datasets is through a programmatic interface that allows you to |
| construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows |
| you to construct Datasets when the columns and their types are not known until runtime.</p> |
| |
| <h3 id="inferring-the-schema-using-reflection">Inferring the Schema Using Reflection</h3> |
| <div class="codetabs"> |
| |
| <div data-lang="scala"> |
| |
| <p>The Scala interface for Spark SQL supports automatically converting an RDD containing case classes |
| to a DataFrame. The case class |
| defines the schema of the table. The names of the arguments to the case class are read using |
| reflection and become the names of the columns. Case classes can also be nested or contain complex |
| types such as <code>Seq</code>s or <code>Array</code>s. This RDD can be implicitly converted to a DataFrame and then be |
| registered as a table. Tables can be used in subsequent SQL statements.</p> |
| |
| <div class="highlight"><pre><span></span><span class="c1">// For implicit conversions from RDDs to DataFrames</span> |
| <span class="k">import</span> <span class="nn">spark.implicits._</span> |
| |
| <span class="c1">// Create an RDD of Person objects from a text file, convert it to a Dataframe</span> |
| <span class="k">val</span> <span class="n">peopleDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sparkContext</span> |
| <span class="o">.</span><span class="n">textFile</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.txt"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">map</span><span class="o">(</span><span class="k">_</span><span class="o">.</span><span class="n">split</span><span class="o">(</span><span class="s">","</span><span class="o">))</span> |
| <span class="o">.</span><span class="n">map</span><span class="o">(</span><span class="n">attributes</span> <span class="k">=></span> <span class="nc">Person</span><span class="o">(</span><span class="n">attributes</span><span class="o">(</span><span class="mi">0</span><span class="o">),</span> <span class="n">attributes</span><span class="o">(</span><span class="mi">1</span><span class="o">).</span><span class="n">trim</span><span class="o">.</span><span class="n">toInt</span><span class="o">))</span> |
| <span class="o">.</span><span class="n">toDF</span><span class="o">()</span> |
| <span class="c1">// Register the DataFrame as a temporary view</span> |
| <span class="n">peopleDF</span><span class="o">.</span><span class="n">createOrReplaceTempView</span><span class="o">(</span><span class="s">"people"</span><span class="o">)</span> |
| |
| <span class="c1">// SQL statements can be run by using the sql methods provided by Spark</span> |
| <span class="k">val</span> <span class="n">teenagersDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="o">(</span><span class="s">"SELECT name, age FROM people WHERE age BETWEEN 13 AND 19"</span><span class="o">)</span> |
| |
| <span class="c1">// The columns of a row in the result can be accessed by field index</span> |
| <span class="n">teenagersDF</span><span class="o">.</span><span class="n">map</span><span class="o">(</span><span class="n">teenager</span> <span class="k">=></span> <span class="s">"Name: "</span> <span class="o">+</span> <span class="n">teenager</span><span class="o">(</span><span class="mi">0</span><span class="o">)).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +------------+</span> |
| <span class="c1">// | value|</span> |
| <span class="c1">// +------------+</span> |
| <span class="c1">// |Name: Justin|</span> |
| <span class="c1">// +------------+</span> |
| |
| <span class="c1">// or by field name</span> |
| <span class="n">teenagersDF</span><span class="o">.</span><span class="n">map</span><span class="o">(</span><span class="n">teenager</span> <span class="k">=></span> <span class="s">"Name: "</span> <span class="o">+</span> <span class="n">teenager</span><span class="o">.</span><span class="n">getAs</span><span class="o">[</span><span class="kt">String</span><span class="o">](</span><span class="s">"name"</span><span class="o">)).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +------------+</span> |
| <span class="c1">// | value|</span> |
| <span class="c1">// +------------+</span> |
| <span class="c1">// |Name: Justin|</span> |
| <span class="c1">// +------------+</span> |
| |
| <span class="c1">// No pre-defined encoders for Dataset[Map[K,V]], define explicitly</span> |
| <span class="k">implicit</span> <span class="k">val</span> <span class="n">mapEncoder</span> <span class="k">=</span> <span class="n">org</span><span class="o">.</span><span class="n">apache</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="o">.</span><span class="nc">Encoders</span><span class="o">.</span><span class="n">kryo</span><span class="o">[</span><span class="kt">Map</span><span class="o">[</span><span class="kt">String</span>, <span class="kt">Any</span><span class="o">]]</span> |
| <span class="c1">// Primitive types and case classes can be also defined as</span> |
| <span class="c1">// implicit val stringIntMapEncoder: Encoder[Map[String, Any]] = ExpressionEncoder()</span> |
| |
| <span class="c1">// row.getValuesMap[T] retrieves multiple columns at once into a Map[String, T]</span> |
| <span class="n">teenagersDF</span><span class="o">.</span><span class="n">map</span><span class="o">(</span><span class="n">teenager</span> <span class="k">=></span> <span class="n">teenager</span><span class="o">.</span><span class="n">getValuesMap</span><span class="o">[</span><span class="kt">Any</span><span class="o">](</span><span class="nc">List</span><span class="o">(</span><span class="s">"name"</span><span class="o">,</span> <span class="s">"age"</span><span class="o">))).</span><span class="n">collect</span><span class="o">()</span> |
| <span class="c1">// Array(Map("name" -> "Justin", "age" -> 19))</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| |
| <p>Spark SQL supports automatically converting an RDD of |
| <a href="http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly">JavaBeans</a> into a DataFrame. |
| The <code>BeanInfo</code>, obtained using reflection, defines the schema of the table. Currently, Spark SQL |
| does not support JavaBeans that contain <code>Map</code> field(s). Nested JavaBeans and <code>List</code> or <code>Array</code> |
| fields are supported though. You can create a JavaBean by creating a class that implements |
| Serializable and has getters and setters for all of its fields.</p> |
| |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">org.apache.spark.api.java.JavaRDD</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.api.java.function.Function</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.api.java.function.MapFunction</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Dataset</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Row</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Encoder</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Encoders</span><span class="o">;</span> |
| |
| <span class="c1">// Create an RDD of Person objects from a text file</span> |
| <span class="n">JavaRDD</span><span class="o"><</span><span class="n">Person</span><span class="o">></span> <span class="n">peopleRDD</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">textFile</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.txt"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">javaRDD</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">map</span><span class="o">(</span><span class="n">line</span> <span class="o">-></span> <span class="o">{</span> |
| <span class="n">String</span><span class="o">[]</span> <span class="n">parts</span> <span class="o">=</span> <span class="n">line</span><span class="o">.</span><span class="na">split</span><span class="o">(</span><span class="s">","</span><span class="o">);</span> |
| <span class="n">Person</span> <span class="n">person</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Person</span><span class="o">();</span> |
| <span class="n">person</span><span class="o">.</span><span class="na">setName</span><span class="o">(</span><span class="n">parts</span><span class="o">[</span><span class="mi">0</span><span class="o">]);</span> |
| <span class="n">person</span><span class="o">.</span><span class="na">setAge</span><span class="o">(</span><span class="n">Integer</span><span class="o">.</span><span class="na">parseInt</span><span class="o">(</span><span class="n">parts</span><span class="o">[</span><span class="mi">1</span><span class="o">].</span><span class="na">trim</span><span class="o">()));</span> |
| <span class="k">return</span> <span class="n">person</span><span class="o">;</span> |
| <span class="o">});</span> |
| |
| <span class="c1">// Apply a schema to an RDD of JavaBeans to get a DataFrame</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">peopleDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">createDataFrame</span><span class="o">(</span><span class="n">peopleRDD</span><span class="o">,</span> <span class="n">Person</span><span class="o">.</span><span class="na">class</span><span class="o">);</span> |
| <span class="c1">// Register the DataFrame as a temporary view</span> |
| <span class="n">peopleDF</span><span class="o">.</span><span class="na">createOrReplaceTempView</span><span class="o">(</span><span class="s">"people"</span><span class="o">);</span> |
| |
| <span class="c1">// SQL statements can be run by using the sql methods provided by spark</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">teenagersDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT name FROM people WHERE age BETWEEN 13 AND 19"</span><span class="o">);</span> |
| |
| <span class="c1">// The columns of a row in the result can be accessed by field index</span> |
| <span class="n">Encoder</span><span class="o"><</span><span class="n">String</span><span class="o">></span> <span class="n">stringEncoder</span> <span class="o">=</span> <span class="n">Encoders</span><span class="o">.</span><span class="na">STRING</span><span class="o">();</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">String</span><span class="o">></span> <span class="n">teenagerNamesByIndexDF</span> <span class="o">=</span> <span class="n">teenagersDF</span><span class="o">.</span><span class="na">map</span><span class="o">(</span> |
| <span class="o">(</span><span class="n">MapFunction</span><span class="o"><</span><span class="n">Row</span><span class="o">,</span> <span class="n">String</span><span class="o">>)</span> <span class="n">row</span> <span class="o">-></span> <span class="s">"Name: "</span> <span class="o">+</span> <span class="n">row</span><span class="o">.</span><span class="na">getString</span><span class="o">(</span><span class="mi">0</span><span class="o">),</span> |
| <span class="n">stringEncoder</span><span class="o">);</span> |
| <span class="n">teenagerNamesByIndexDF</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +------------+</span> |
| <span class="c1">// | value|</span> |
| <span class="c1">// +------------+</span> |
| <span class="c1">// |Name: Justin|</span> |
| <span class="c1">// +------------+</span> |
| |
| <span class="c1">// or by field name</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">String</span><span class="o">></span> <span class="n">teenagerNamesByFieldDF</span> <span class="o">=</span> <span class="n">teenagersDF</span><span class="o">.</span><span class="na">map</span><span class="o">(</span> |
| <span class="o">(</span><span class="n">MapFunction</span><span class="o"><</span><span class="n">Row</span><span class="o">,</span> <span class="n">String</span><span class="o">>)</span> <span class="n">row</span> <span class="o">-></span> <span class="s">"Name: "</span> <span class="o">+</span> <span class="n">row</span><span class="o">.<</span><span class="n">String</span><span class="o">></span><span class="n">getAs</span><span class="o">(</span><span class="s">"name"</span><span class="o">),</span> |
| <span class="n">stringEncoder</span><span class="o">);</span> |
| <span class="n">teenagerNamesByFieldDF</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +------------+</span> |
| <span class="c1">// | value|</span> |
| <span class="c1">// +------------+</span> |
| <span class="c1">// |Name: Justin|</span> |
| <span class="c1">// +------------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| |
| <p>Spark SQL can convert an RDD of Row objects to a DataFrame, inferring the datatypes. Rows are constructed by passing a list of |
| key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table, |
| and the types are inferred by sampling the whole dataset, similar to the inference that is performed on JSON files.</p> |
| |
| <div class="highlight"><pre><span></span><span class="kn">from</span> <span class="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">Row</span> |
| |
| <span class="n">sc</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sparkContext</span> |
| |
| <span class="c1"># Load a text file and convert each line to a Row.</span> |
| <span class="n">lines</span> <span class="o">=</span> <span class="n">sc</span><span class="o">.</span><span class="n">textFile</span><span class="p">(</span><span class="s2">"examples/src/main/resources/people.txt"</span><span class="p">)</span> |
| <span class="n">parts</span> <span class="o">=</span> <span class="n">lines</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">l</span><span class="p">:</span> <span class="n">l</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s2">","</span><span class="p">))</span> |
| <span class="n">people</span> <span class="o">=</span> <span class="n">parts</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">p</span><span class="p">:</span> <span class="n">Row</span><span class="p">(</span><span class="n">name</span><span class="o">=</span><span class="n">p</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="n">age</span><span class="o">=</span><span class="nb">int</span><span class="p">(</span><span class="n">p</span><span class="p">[</span><span class="mi">1</span><span class="p">])))</span> |
| |
| <span class="c1"># Infer the schema, and register the DataFrame as a table.</span> |
| <span class="n">schemaPeople</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">createDataFrame</span><span class="p">(</span><span class="n">people</span><span class="p">)</span> |
| <span class="n">schemaPeople</span><span class="o">.</span><span class="n">createOrReplaceTempView</span><span class="p">(</span><span class="s2">"people"</span><span class="p">)</span> |
| |
| <span class="c1"># SQL can be run over DataFrames that have been registered as a table.</span> |
| <span class="n">teenagers</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT name FROM people WHERE age >= 13 AND age <= 19"</span><span class="p">)</span> |
| |
| <span class="c1"># The results of SQL queries are Dataframe objects.</span> |
| <span class="c1"># rdd returns the content as an :class:`pyspark.RDD` of :class:`Row`.</span> |
| <span class="n">teenNames</span> <span class="o">=</span> <span class="n">teenagers</span><span class="o">.</span><span class="n">rdd</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">p</span><span class="p">:</span> <span class="s2">"Name: "</span> <span class="o">+</span> <span class="n">p</span><span class="o">.</span><span class="n">name</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">name</span> <span class="ow">in</span> <span class="n">teenNames</span><span class="p">:</span> |
| <span class="k">print</span><span class="p">(</span><span class="n">name</span><span class="p">)</span> |
| <span class="c1"># Name: Justin</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/basic.py" in the Spark repo.</small></div> |
| </div> |
| |
| </div> |
| |
| <h3 id="programmatically-specifying-the-schema">Programmatically Specifying the Schema</h3> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="scala"> |
| |
| <p>When case classes cannot be defined ahead of time (for example, |
| the structure of records is encoded in a string, or a text dataset will be parsed |
| and fields will be projected differently for different users), |
| a <code>DataFrame</code> can be created programmatically with three steps.</p> |
| |
| <ol> |
| <li>Create an RDD of <code>Row</code>s from the original RDD;</li> |
| <li>Create the schema represented by a <code>StructType</code> matching the structure of |
| <code>Row</code>s in the RDD created in Step 1.</li> |
| <li>Apply the schema to the RDD of <code>Row</code>s via <code>createDataFrame</code> method provided |
| by <code>SparkSession</code>.</li> |
| </ol> |
| |
| <p>For example:</p> |
| |
| <div class="highlight"><pre><span></span><span class="k">import</span> <span class="nn">org.apache.spark.sql.types._</span> |
| |
| <span class="c1">// Create an RDD</span> |
| <span class="k">val</span> <span class="n">peopleRDD</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sparkContext</span><span class="o">.</span><span class="n">textFile</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.txt"</span><span class="o">)</span> |
| |
| <span class="c1">// The schema is encoded in a string</span> |
| <span class="k">val</span> <span class="n">schemaString</span> <span class="k">=</span> <span class="s">"name age"</span> |
| |
| <span class="c1">// Generate the schema based on the string of schema</span> |
| <span class="k">val</span> <span class="n">fields</span> <span class="k">=</span> <span class="n">schemaString</span><span class="o">.</span><span class="n">split</span><span class="o">(</span><span class="s">" "</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">map</span><span class="o">(</span><span class="n">fieldName</span> <span class="k">=></span> <span class="nc">StructField</span><span class="o">(</span><span class="n">fieldName</span><span class="o">,</span> <span class="nc">StringType</span><span class="o">,</span> <span class="n">nullable</span> <span class="k">=</span> <span class="kc">true</span><span class="o">))</span> |
| <span class="k">val</span> <span class="n">schema</span> <span class="k">=</span> <span class="nc">StructType</span><span class="o">(</span><span class="n">fields</span><span class="o">)</span> |
| |
| <span class="c1">// Convert records of the RDD (people) to Rows</span> |
| <span class="k">val</span> <span class="n">rowRDD</span> <span class="k">=</span> <span class="n">peopleRDD</span> |
| <span class="o">.</span><span class="n">map</span><span class="o">(</span><span class="k">_</span><span class="o">.</span><span class="n">split</span><span class="o">(</span><span class="s">","</span><span class="o">))</span> |
| <span class="o">.</span><span class="n">map</span><span class="o">(</span><span class="n">attributes</span> <span class="k">=></span> <span class="nc">Row</span><span class="o">(</span><span class="n">attributes</span><span class="o">(</span><span class="mi">0</span><span class="o">),</span> <span class="n">attributes</span><span class="o">(</span><span class="mi">1</span><span class="o">).</span><span class="n">trim</span><span class="o">))</span> |
| |
| <span class="c1">// Apply the schema to the RDD</span> |
| <span class="k">val</span> <span class="n">peopleDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">createDataFrame</span><span class="o">(</span><span class="n">rowRDD</span><span class="o">,</span> <span class="n">schema</span><span class="o">)</span> |
| |
| <span class="c1">// Creates a temporary view using the DataFrame</span> |
| <span class="n">peopleDF</span><span class="o">.</span><span class="n">createOrReplaceTempView</span><span class="o">(</span><span class="s">"people"</span><span class="o">)</span> |
| |
| <span class="c1">// SQL can be run over a temporary view created using DataFrames</span> |
| <span class="k">val</span> <span class="n">results</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="o">(</span><span class="s">"SELECT name FROM people"</span><span class="o">)</span> |
| |
| <span class="c1">// The results of SQL queries are DataFrames and support all the normal RDD operations</span> |
| <span class="c1">// The columns of a row in the result can be accessed by field index or by field name</span> |
| <span class="n">results</span><span class="o">.</span><span class="n">map</span><span class="o">(</span><span class="n">attributes</span> <span class="k">=></span> <span class="s">"Name: "</span> <span class="o">+</span> <span class="n">attributes</span><span class="o">(</span><span class="mi">0</span><span class="o">)).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +-------------+</span> |
| <span class="c1">// | value|</span> |
| <span class="c1">// +-------------+</span> |
| <span class="c1">// |Name: Michael|</span> |
| <span class="c1">// | Name: Andy|</span> |
| <span class="c1">// | Name: Justin|</span> |
| <span class="c1">// +-------------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| |
| <p>When JavaBean classes cannot be defined ahead of time (for example, |
| the structure of records is encoded in a string, or a text dataset will be parsed and |
| fields will be projected differently for different users), |
| a <code>Dataset<Row></code> can be created programmatically with three steps.</p> |
| |
| <ol> |
| <li>Create an RDD of <code>Row</code>s from the original RDD;</li> |
| <li>Create the schema represented by a <code>StructType</code> matching the structure of |
| <code>Row</code>s in the RDD created in Step 1.</li> |
| <li>Apply the schema to the RDD of <code>Row</code>s via <code>createDataFrame</code> method provided |
| by <code>SparkSession</code>.</li> |
| </ol> |
| |
| <p>For example:</p> |
| |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.util.ArrayList</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">java.util.List</span><span class="o">;</span> |
| |
| <span class="kn">import</span> <span class="nn">org.apache.spark.api.java.JavaRDD</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.api.java.function.Function</span><span class="o">;</span> |
| |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Dataset</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Row</span><span class="o">;</span> |
| |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.types.DataTypes</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.types.StructField</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.types.StructType</span><span class="o">;</span> |
| |
| <span class="c1">// Create an RDD</span> |
| <span class="n">JavaRDD</span><span class="o"><</span><span class="n">String</span><span class="o">></span> <span class="n">peopleRDD</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">sparkContext</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">textFile</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.txt"</span><span class="o">,</span> <span class="mi">1</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">toJavaRDD</span><span class="o">();</span> |
| |
| <span class="c1">// The schema is encoded in a string</span> |
| <span class="n">String</span> <span class="n">schemaString</span> <span class="o">=</span> <span class="s">"name age"</span><span class="o">;</span> |
| |
| <span class="c1">// Generate the schema based on the string of schema</span> |
| <span class="n">List</span><span class="o"><</span><span class="n">StructField</span><span class="o">></span> <span class="n">fields</span> <span class="o">=</span> <span class="k">new</span> <span class="n">ArrayList</span><span class="o"><>();</span> |
| <span class="k">for</span> <span class="o">(</span><span class="n">String</span> <span class="n">fieldName</span> <span class="o">:</span> <span class="n">schemaString</span><span class="o">.</span><span class="na">split</span><span class="o">(</span><span class="s">" "</span><span class="o">))</span> <span class="o">{</span> |
| <span class="n">StructField</span> <span class="n">field</span> <span class="o">=</span> <span class="n">DataTypes</span><span class="o">.</span><span class="na">createStructField</span><span class="o">(</span><span class="n">fieldName</span><span class="o">,</span> <span class="n">DataTypes</span><span class="o">.</span><span class="na">StringType</span><span class="o">,</span> <span class="kc">true</span><span class="o">);</span> |
| <span class="n">fields</span><span class="o">.</span><span class="na">add</span><span class="o">(</span><span class="n">field</span><span class="o">);</span> |
| <span class="o">}</span> |
| <span class="n">StructType</span> <span class="n">schema</span> <span class="o">=</span> <span class="n">DataTypes</span><span class="o">.</span><span class="na">createStructType</span><span class="o">(</span><span class="n">fields</span><span class="o">);</span> |
| |
| <span class="c1">// Convert records of the RDD (people) to Rows</span> |
| <span class="n">JavaRDD</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">rowRDD</span> <span class="o">=</span> <span class="n">peopleRDD</span><span class="o">.</span><span class="na">map</span><span class="o">((</span><span class="n">Function</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Row</span><span class="o">>)</span> <span class="n">record</span> <span class="o">-></span> <span class="o">{</span> |
| <span class="n">String</span><span class="o">[]</span> <span class="n">attributes</span> <span class="o">=</span> <span class="n">record</span><span class="o">.</span><span class="na">split</span><span class="o">(</span><span class="s">","</span><span class="o">);</span> |
| <span class="k">return</span> <span class="n">RowFactory</span><span class="o">.</span><span class="na">create</span><span class="o">(</span><span class="n">attributes</span><span class="o">[</span><span class="mi">0</span><span class="o">],</span> <span class="n">attributes</span><span class="o">[</span><span class="mi">1</span><span class="o">].</span><span class="na">trim</span><span class="o">());</span> |
| <span class="o">});</span> |
| |
| <span class="c1">// Apply the schema to the RDD</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">peopleDataFrame</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">createDataFrame</span><span class="o">(</span><span class="n">rowRDD</span><span class="o">,</span> <span class="n">schema</span><span class="o">);</span> |
| |
| <span class="c1">// Creates a temporary view using the DataFrame</span> |
| <span class="n">peopleDataFrame</span><span class="o">.</span><span class="na">createOrReplaceTempView</span><span class="o">(</span><span class="s">"people"</span><span class="o">);</span> |
| |
| <span class="c1">// SQL can be run over a temporary view created using DataFrames</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">results</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT name FROM people"</span><span class="o">);</span> |
| |
| <span class="c1">// The results of SQL queries are DataFrames and support all the normal RDD operations</span> |
| <span class="c1">// The columns of a row in the result can be accessed by field index or by field name</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">String</span><span class="o">></span> <span class="n">namesDS</span> <span class="o">=</span> <span class="n">results</span><span class="o">.</span><span class="na">map</span><span class="o">(</span> |
| <span class="o">(</span><span class="n">MapFunction</span><span class="o"><</span><span class="n">Row</span><span class="o">,</span> <span class="n">String</span><span class="o">>)</span> <span class="n">row</span> <span class="o">-></span> <span class="s">"Name: "</span> <span class="o">+</span> <span class="n">row</span><span class="o">.</span><span class="na">getString</span><span class="o">(</span><span class="mi">0</span><span class="o">),</span> |
| <span class="n">Encoders</span><span class="o">.</span><span class="na">STRING</span><span class="o">());</span> |
| <span class="n">namesDS</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +-------------+</span> |
| <span class="c1">// | value|</span> |
| <span class="c1">// +-------------+</span> |
| <span class="c1">// |Name: Michael|</span> |
| <span class="c1">// | Name: Andy|</span> |
| <span class="c1">// | Name: Justin|</span> |
| <span class="c1">// +-------------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| |
| <p>When a dictionary of kwargs cannot be defined ahead of time (for example, |
| the structure of records is encoded in a string, or a text dataset will be parsed and |
| fields will be projected differently for different users), |
| a <code>DataFrame</code> can be created programmatically with three steps.</p> |
| |
| <ol> |
| <li>Create an RDD of tuples or lists from the original RDD;</li> |
| <li>Create the schema represented by a <code>StructType</code> matching the structure of |
| tuples or lists in the RDD created in the step 1.</li> |
| <li>Apply the schema to the RDD via <code>createDataFrame</code> method provided by <code>SparkSession</code>.</li> |
| </ol> |
| |
| <p>For example:</p> |
| |
| <div class="highlight"><pre><span></span><span class="c1"># Import data types</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql.types</span> <span class="kn">import</span> <span class="o">*</span> |
| |
| <span class="n">sc</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sparkContext</span> |
| |
| <span class="c1"># Load a text file and convert each line to a Row.</span> |
| <span class="n">lines</span> <span class="o">=</span> <span class="n">sc</span><span class="o">.</span><span class="n">textFile</span><span class="p">(</span><span class="s2">"examples/src/main/resources/people.txt"</span><span class="p">)</span> |
| <span class="n">parts</span> <span class="o">=</span> <span class="n">lines</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">l</span><span class="p">:</span> <span class="n">l</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s2">","</span><span class="p">))</span> |
| <span class="c1"># Each line is converted to a tuple.</span> |
| <span class="n">people</span> <span class="o">=</span> <span class="n">parts</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">p</span><span class="p">:</span> <span class="p">(</span><span class="n">p</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="n">p</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span><span class="o">.</span><span class="n">strip</span><span class="p">()))</span> |
| |
| <span class="c1"># The schema is encoded in a string.</span> |
| <span class="n">schemaString</span> <span class="o">=</span> <span class="s2">"name age"</span> |
| |
| <span class="n">fields</span> <span class="o">=</span> <span class="p">[</span><span class="n">StructField</span><span class="p">(</span><span class="n">field_name</span><span class="p">,</span> <span class="n">StringType</span><span class="p">(),</span> <span class="bp">True</span><span class="p">)</span> <span class="k">for</span> <span class="n">field_name</span> <span class="ow">in</span> <span class="n">schemaString</span><span class="o">.</span><span class="n">split</span><span class="p">()]</span> |
| <span class="n">schema</span> <span class="o">=</span> <span class="n">StructType</span><span class="p">(</span><span class="n">fields</span><span class="p">)</span> |
| |
| <span class="c1"># Apply the schema to the RDD.</span> |
| <span class="n">schemaPeople</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">createDataFrame</span><span class="p">(</span><span class="n">people</span><span class="p">,</span> <span class="n">schema</span><span class="p">)</span> |
| |
| <span class="c1"># Creates a temporary view using the DataFrame</span> |
| <span class="n">schemaPeople</span><span class="o">.</span><span class="n">createOrReplaceTempView</span><span class="p">(</span><span class="s2">"people"</span><span class="p">)</span> |
| |
| <span class="c1"># SQL can be run over DataFrames that have been registered as a table.</span> |
| <span class="n">results</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT name FROM people"</span><span class="p">)</span> |
| |
| <span class="n">results</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +-------+</span> |
| <span class="c1"># | name|</span> |
| <span class="c1"># +-------+</span> |
| <span class="c1"># |Michael|</span> |
| <span class="c1"># | Andy|</span> |
| <span class="c1"># | Justin|</span> |
| <span class="c1"># +-------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/basic.py" in the Spark repo.</small></div> |
| </div> |
| |
| </div> |
| |
| <h2 id="aggregations">Aggregations</h2> |
| |
| <p>The <a href="api/scala/index.html#org.apache.spark.sql.functions$">built-in DataFrames functions</a> provide common |
| aggregations such as <code>count()</code>, <code>countDistinct()</code>, <code>avg()</code>, <code>max()</code>, <code>min()</code>, etc. |
| While those functions are designed for DataFrames, Spark SQL also has type-safe versions for some of them in |
| <a href="api/scala/index.html#org.apache.spark.sql.expressions.scalalang.typed$">Scala</a> and |
| <a href="api/java/org/apache/spark/sql/expressions/javalang/typed.html">Java</a> to work with strongly typed Datasets. |
| Moreover, users are not limited to the predefined aggregate functions and can create their own.</p> |
| |
| <h3 id="untyped-user-defined-aggregate-functions">Untyped User-Defined Aggregate Functions</h3> |
| <p>Users have to extend the <a href="api/scala/index.html#org.apache.spark.sql.expressions.UserDefinedAggregateFunction">UserDefinedAggregateFunction</a> |
| abstract class to implement a custom untyped aggregate function. For example, a user-defined average |
| can look like:</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="k">import</span> <span class="nn">org.apache.spark.sql.</span><span class="o">{</span><span class="nc">Row</span><span class="o">,</span> <span class="nc">SparkSession</span><span class="o">}</span> |
| <span class="k">import</span> <span class="nn">org.apache.spark.sql.expressions.MutableAggregationBuffer</span> |
| <span class="k">import</span> <span class="nn">org.apache.spark.sql.expressions.UserDefinedAggregateFunction</span> |
| <span class="k">import</span> <span class="nn">org.apache.spark.sql.types._</span> |
| |
| <span class="k">object</span> <span class="nc">MyAverage</span> <span class="k">extends</span> <span class="nc">UserDefinedAggregateFunction</span> <span class="o">{</span> |
| <span class="c1">// Data types of input arguments of this aggregate function</span> |
| <span class="k">def</span> <span class="n">inputSchema</span><span class="k">:</span> <span class="kt">StructType</span> <span class="o">=</span> <span class="nc">StructType</span><span class="o">(</span><span class="nc">StructField</span><span class="o">(</span><span class="s">"inputColumn"</span><span class="o">,</span> <span class="nc">LongType</span><span class="o">)</span> <span class="o">::</span> <span class="nc">Nil</span><span class="o">)</span> |
| <span class="c1">// Data types of values in the aggregation buffer</span> |
| <span class="k">def</span> <span class="n">bufferSchema</span><span class="k">:</span> <span class="kt">StructType</span> <span class="o">=</span> <span class="o">{</span> |
| <span class="nc">StructType</span><span class="o">(</span><span class="nc">StructField</span><span class="o">(</span><span class="s">"sum"</span><span class="o">,</span> <span class="nc">LongType</span><span class="o">)</span> <span class="o">::</span> <span class="nc">StructField</span><span class="o">(</span><span class="s">"count"</span><span class="o">,</span> <span class="nc">LongType</span><span class="o">)</span> <span class="o">::</span> <span class="nc">Nil</span><span class="o">)</span> |
| <span class="o">}</span> |
| <span class="c1">// The data type of the returned value</span> |
| <span class="k">def</span> <span class="n">dataType</span><span class="k">:</span> <span class="kt">DataType</span> <span class="o">=</span> <span class="nc">DoubleType</span> |
| <span class="c1">// Whether this function always returns the same output on the identical input</span> |
| <span class="k">def</span> <span class="n">deterministic</span><span class="k">:</span> <span class="kt">Boolean</span> <span class="o">=</span> <span class="kc">true</span> |
| <span class="c1">// Initializes the given aggregation buffer. The buffer itself is a `Row` that in addition to</span> |
| <span class="c1">// standard methods like retrieving a value at an index (e.g., get(), getBoolean()), provides</span> |
| <span class="c1">// the opportunity to update its values. Note that arrays and maps inside the buffer are still</span> |
| <span class="c1">// immutable.</span> |
| <span class="k">def</span> <span class="n">initialize</span><span class="o">(</span><span class="n">buffer</span><span class="k">:</span> <span class="kt">MutableAggregationBuffer</span><span class="o">)</span><span class="k">:</span> <span class="kt">Unit</span> <span class="o">=</span> <span class="o">{</span> |
| <span class="n">buffer</span><span class="o">(</span><span class="mi">0</span><span class="o">)</span> <span class="k">=</span> <span class="mi">0L</span> |
| <span class="n">buffer</span><span class="o">(</span><span class="mi">1</span><span class="o">)</span> <span class="k">=</span> <span class="mi">0L</span> |
| <span class="o">}</span> |
| <span class="c1">// Updates the given aggregation buffer `buffer` with new input data from `input`</span> |
| <span class="k">def</span> <span class="n">update</span><span class="o">(</span><span class="n">buffer</span><span class="k">:</span> <span class="kt">MutableAggregationBuffer</span><span class="o">,</span> <span class="n">input</span><span class="k">:</span> <span class="kt">Row</span><span class="o">)</span><span class="k">:</span> <span class="kt">Unit</span> <span class="o">=</span> <span class="o">{</span> |
| <span class="k">if</span> <span class="o">(!</span><span class="n">input</span><span class="o">.</span><span class="n">isNullAt</span><span class="o">(</span><span class="mi">0</span><span class="o">))</span> <span class="o">{</span> |
| <span class="n">buffer</span><span class="o">(</span><span class="mi">0</span><span class="o">)</span> <span class="k">=</span> <span class="n">buffer</span><span class="o">.</span><span class="n">getLong</span><span class="o">(</span><span class="mi">0</span><span class="o">)</span> <span class="o">+</span> <span class="n">input</span><span class="o">.</span><span class="n">getLong</span><span class="o">(</span><span class="mi">0</span><span class="o">)</span> |
| <span class="n">buffer</span><span class="o">(</span><span class="mi">1</span><span class="o">)</span> <span class="k">=</span> <span class="n">buffer</span><span class="o">.</span><span class="n">getLong</span><span class="o">(</span><span class="mi">1</span><span class="o">)</span> <span class="o">+</span> <span class="mi">1</span> |
| <span class="o">}</span> |
| <span class="o">}</span> |
| <span class="c1">// Merges two aggregation buffers and stores the updated buffer values back to `buffer1`</span> |
| <span class="k">def</span> <span class="n">merge</span><span class="o">(</span><span class="n">buffer1</span><span class="k">:</span> <span class="kt">MutableAggregationBuffer</span><span class="o">,</span> <span class="n">buffer2</span><span class="k">:</span> <span class="kt">Row</span><span class="o">)</span><span class="k">:</span> <span class="kt">Unit</span> <span class="o">=</span> <span class="o">{</span> |
| <span class="n">buffer1</span><span class="o">(</span><span class="mi">0</span><span class="o">)</span> <span class="k">=</span> <span class="n">buffer1</span><span class="o">.</span><span class="n">getLong</span><span class="o">(</span><span class="mi">0</span><span class="o">)</span> <span class="o">+</span> <span class="n">buffer2</span><span class="o">.</span><span class="n">getLong</span><span class="o">(</span><span class="mi">0</span><span class="o">)</span> |
| <span class="n">buffer1</span><span class="o">(</span><span class="mi">1</span><span class="o">)</span> <span class="k">=</span> <span class="n">buffer1</span><span class="o">.</span><span class="n">getLong</span><span class="o">(</span><span class="mi">1</span><span class="o">)</span> <span class="o">+</span> <span class="n">buffer2</span><span class="o">.</span><span class="n">getLong</span><span class="o">(</span><span class="mi">1</span><span class="o">)</span> |
| <span class="o">}</span> |
| <span class="c1">// Calculates the final result</span> |
| <span class="k">def</span> <span class="n">evaluate</span><span class="o">(</span><span class="n">buffer</span><span class="k">:</span> <span class="kt">Row</span><span class="o">)</span><span class="k">:</span> <span class="kt">Double</span> <span class="o">=</span> <span class="n">buffer</span><span class="o">.</span><span class="n">getLong</span><span class="o">(</span><span class="mi">0</span><span class="o">).</span><span class="n">toDouble</span> <span class="o">/</span> <span class="n">buffer</span><span class="o">.</span><span class="n">getLong</span><span class="o">(</span><span class="mi">1</span><span class="o">)</span> |
| <span class="o">}</span> |
| |
| <span class="c1">// Register the function to access it</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">udf</span><span class="o">.</span><span class="n">register</span><span class="o">(</span><span class="s">"myAverage"</span><span class="o">,</span> <span class="nc">MyAverage</span><span class="o">)</span> |
| |
| <span class="k">val</span> <span class="n">df</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">json</span><span class="o">(</span><span class="s">"examples/src/main/resources/employees.json"</span><span class="o">)</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">createOrReplaceTempView</span><span class="o">(</span><span class="s">"employees"</span><span class="o">)</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +-------+------+</span> |
| <span class="c1">// | name|salary|</span> |
| <span class="c1">// +-------+------+</span> |
| <span class="c1">// |Michael| 3000|</span> |
| <span class="c1">// | Andy| 4500|</span> |
| <span class="c1">// | Justin| 3500|</span> |
| <span class="c1">// | Berta| 4000|</span> |
| <span class="c1">// +-------+------+</span> |
| |
| <span class="k">val</span> <span class="n">result</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="o">(</span><span class="s">"SELECT myAverage(salary) as average_salary FROM employees"</span><span class="o">)</span> |
| <span class="n">result</span><span class="o">.</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +--------------+</span> |
| <span class="c1">// |average_salary|</span> |
| <span class="c1">// +--------------+</span> |
| <span class="c1">// | 3750.0|</span> |
| <span class="c1">// +--------------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala" in the Spark repo.</small></div> |
| </div> |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.util.ArrayList</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">java.util.List</span><span class="o">;</span> |
| |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Dataset</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Row</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.SparkSession</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.expressions.MutableAggregationBuffer</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.expressions.UserDefinedAggregateFunction</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.types.DataType</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.types.DataTypes</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.types.StructField</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.types.StructType</span><span class="o">;</span> |
| |
| <span class="kd">public</span> <span class="kd">static</span> <span class="kd">class</span> <span class="nc">MyAverage</span> <span class="kd">extends</span> <span class="n">UserDefinedAggregateFunction</span> <span class="o">{</span> |
| |
| <span class="kd">private</span> <span class="n">StructType</span> <span class="n">inputSchema</span><span class="o">;</span> |
| <span class="kd">private</span> <span class="n">StructType</span> <span class="n">bufferSchema</span><span class="o">;</span> |
| |
| <span class="kd">public</span> <span class="nf">MyAverage</span><span class="o">()</span> <span class="o">{</span> |
| <span class="n">List</span><span class="o"><</span><span class="n">StructField</span><span class="o">></span> <span class="n">inputFields</span> <span class="o">=</span> <span class="k">new</span> <span class="n">ArrayList</span><span class="o"><>();</span> |
| <span class="n">inputFields</span><span class="o">.</span><span class="na">add</span><span class="o">(</span><span class="n">DataTypes</span><span class="o">.</span><span class="na">createStructField</span><span class="o">(</span><span class="s">"inputColumn"</span><span class="o">,</span> <span class="n">DataTypes</span><span class="o">.</span><span class="na">LongType</span><span class="o">,</span> <span class="kc">true</span><span class="o">));</span> |
| <span class="n">inputSchema</span> <span class="o">=</span> <span class="n">DataTypes</span><span class="o">.</span><span class="na">createStructType</span><span class="o">(</span><span class="n">inputFields</span><span class="o">);</span> |
| |
| <span class="n">List</span><span class="o"><</span><span class="n">StructField</span><span class="o">></span> <span class="n">bufferFields</span> <span class="o">=</span> <span class="k">new</span> <span class="n">ArrayList</span><span class="o"><>();</span> |
| <span class="n">bufferFields</span><span class="o">.</span><span class="na">add</span><span class="o">(</span><span class="n">DataTypes</span><span class="o">.</span><span class="na">createStructField</span><span class="o">(</span><span class="s">"sum"</span><span class="o">,</span> <span class="n">DataTypes</span><span class="o">.</span><span class="na">LongType</span><span class="o">,</span> <span class="kc">true</span><span class="o">));</span> |
| <span class="n">bufferFields</span><span class="o">.</span><span class="na">add</span><span class="o">(</span><span class="n">DataTypes</span><span class="o">.</span><span class="na">createStructField</span><span class="o">(</span><span class="s">"count"</span><span class="o">,</span> <span class="n">DataTypes</span><span class="o">.</span><span class="na">LongType</span><span class="o">,</span> <span class="kc">true</span><span class="o">));</span> |
| <span class="n">bufferSchema</span> <span class="o">=</span> <span class="n">DataTypes</span><span class="o">.</span><span class="na">createStructType</span><span class="o">(</span><span class="n">bufferFields</span><span class="o">);</span> |
| <span class="o">}</span> |
| <span class="c1">// Data types of input arguments of this aggregate function</span> |
| <span class="kd">public</span> <span class="n">StructType</span> <span class="nf">inputSchema</span><span class="o">()</span> <span class="o">{</span> |
| <span class="k">return</span> <span class="n">inputSchema</span><span class="o">;</span> |
| <span class="o">}</span> |
| <span class="c1">// Data types of values in the aggregation buffer</span> |
| <span class="kd">public</span> <span class="n">StructType</span> <span class="nf">bufferSchema</span><span class="o">()</span> <span class="o">{</span> |
| <span class="k">return</span> <span class="n">bufferSchema</span><span class="o">;</span> |
| <span class="o">}</span> |
| <span class="c1">// The data type of the returned value</span> |
| <span class="kd">public</span> <span class="n">DataType</span> <span class="nf">dataType</span><span class="o">()</span> <span class="o">{</span> |
| <span class="k">return</span> <span class="n">DataTypes</span><span class="o">.</span><span class="na">DoubleType</span><span class="o">;</span> |
| <span class="o">}</span> |
| <span class="c1">// Whether this function always returns the same output on the identical input</span> |
| <span class="kd">public</span> <span class="kt">boolean</span> <span class="nf">deterministic</span><span class="o">()</span> <span class="o">{</span> |
| <span class="k">return</span> <span class="kc">true</span><span class="o">;</span> |
| <span class="o">}</span> |
| <span class="c1">// Initializes the given aggregation buffer. The buffer itself is a `Row` that in addition to</span> |
| <span class="c1">// standard methods like retrieving a value at an index (e.g., get(), getBoolean()), provides</span> |
| <span class="c1">// the opportunity to update its values. Note that arrays and maps inside the buffer are still</span> |
| <span class="c1">// immutable.</span> |
| <span class="kd">public</span> <span class="kt">void</span> <span class="nf">initialize</span><span class="o">(</span><span class="n">MutableAggregationBuffer</span> <span class="n">buffer</span><span class="o">)</span> <span class="o">{</span> |
| <span class="n">buffer</span><span class="o">.</span><span class="na">update</span><span class="o">(</span><span class="mi">0</span><span class="o">,</span> <span class="mi">0</span><span class="n">L</span><span class="o">);</span> |
| <span class="n">buffer</span><span class="o">.</span><span class="na">update</span><span class="o">(</span><span class="mi">1</span><span class="o">,</span> <span class="mi">0</span><span class="n">L</span><span class="o">);</span> |
| <span class="o">}</span> |
| <span class="c1">// Updates the given aggregation buffer `buffer` with new input data from `input`</span> |
| <span class="kd">public</span> <span class="kt">void</span> <span class="nf">update</span><span class="o">(</span><span class="n">MutableAggregationBuffer</span> <span class="n">buffer</span><span class="o">,</span> <span class="n">Row</span> <span class="n">input</span><span class="o">)</span> <span class="o">{</span> |
| <span class="k">if</span> <span class="o">(!</span><span class="n">input</span><span class="o">.</span><span class="na">isNullAt</span><span class="o">(</span><span class="mi">0</span><span class="o">))</span> <span class="o">{</span> |
| <span class="kt">long</span> <span class="n">updatedSum</span> <span class="o">=</span> <span class="n">buffer</span><span class="o">.</span><span class="na">getLong</span><span class="o">(</span><span class="mi">0</span><span class="o">)</span> <span class="o">+</span> <span class="n">input</span><span class="o">.</span><span class="na">getLong</span><span class="o">(</span><span class="mi">0</span><span class="o">);</span> |
| <span class="kt">long</span> <span class="n">updatedCount</span> <span class="o">=</span> <span class="n">buffer</span><span class="o">.</span><span class="na">getLong</span><span class="o">(</span><span class="mi">1</span><span class="o">)</span> <span class="o">+</span> <span class="mi">1</span><span class="o">;</span> |
| <span class="n">buffer</span><span class="o">.</span><span class="na">update</span><span class="o">(</span><span class="mi">0</span><span class="o">,</span> <span class="n">updatedSum</span><span class="o">);</span> |
| <span class="n">buffer</span><span class="o">.</span><span class="na">update</span><span class="o">(</span><span class="mi">1</span><span class="o">,</span> <span class="n">updatedCount</span><span class="o">);</span> |
| <span class="o">}</span> |
| <span class="o">}</span> |
| <span class="c1">// Merges two aggregation buffers and stores the updated buffer values back to `buffer1`</span> |
| <span class="kd">public</span> <span class="kt">void</span> <span class="nf">merge</span><span class="o">(</span><span class="n">MutableAggregationBuffer</span> <span class="n">buffer1</span><span class="o">,</span> <span class="n">Row</span> <span class="n">buffer2</span><span class="o">)</span> <span class="o">{</span> |
| <span class="kt">long</span> <span class="n">mergedSum</span> <span class="o">=</span> <span class="n">buffer1</span><span class="o">.</span><span class="na">getLong</span><span class="o">(</span><span class="mi">0</span><span class="o">)</span> <span class="o">+</span> <span class="n">buffer2</span><span class="o">.</span><span class="na">getLong</span><span class="o">(</span><span class="mi">0</span><span class="o">);</span> |
| <span class="kt">long</span> <span class="n">mergedCount</span> <span class="o">=</span> <span class="n">buffer1</span><span class="o">.</span><span class="na">getLong</span><span class="o">(</span><span class="mi">1</span><span class="o">)</span> <span class="o">+</span> <span class="n">buffer2</span><span class="o">.</span><span class="na">getLong</span><span class="o">(</span><span class="mi">1</span><span class="o">);</span> |
| <span class="n">buffer1</span><span class="o">.</span><span class="na">update</span><span class="o">(</span><span class="mi">0</span><span class="o">,</span> <span class="n">mergedSum</span><span class="o">);</span> |
| <span class="n">buffer1</span><span class="o">.</span><span class="na">update</span><span class="o">(</span><span class="mi">1</span><span class="o">,</span> <span class="n">mergedCount</span><span class="o">);</span> |
| <span class="o">}</span> |
| <span class="c1">// Calculates the final result</span> |
| <span class="kd">public</span> <span class="n">Double</span> <span class="nf">evaluate</span><span class="o">(</span><span class="n">Row</span> <span class="n">buffer</span><span class="o">)</span> <span class="o">{</span> |
| <span class="k">return</span> <span class="o">((</span><span class="kt">double</span><span class="o">)</span> <span class="n">buffer</span><span class="o">.</span><span class="na">getLong</span><span class="o">(</span><span class="mi">0</span><span class="o">))</span> <span class="o">/</span> <span class="n">buffer</span><span class="o">.</span><span class="na">getLong</span><span class="o">(</span><span class="mi">1</span><span class="o">);</span> |
| <span class="o">}</span> |
| <span class="o">}</span> |
| |
| <span class="c1">// Register the function to access it</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">udf</span><span class="o">().</span><span class="na">register</span><span class="o">(</span><span class="s">"myAverage"</span><span class="o">,</span> <span class="k">new</span> <span class="n">MyAverage</span><span class="o">());</span> |
| |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">df</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">json</span><span class="o">(</span><span class="s">"examples/src/main/resources/employees.json"</span><span class="o">);</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">createOrReplaceTempView</span><span class="o">(</span><span class="s">"employees"</span><span class="o">);</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +-------+------+</span> |
| <span class="c1">// | name|salary|</span> |
| <span class="c1">// +-------+------+</span> |
| <span class="c1">// |Michael| 3000|</span> |
| <span class="c1">// | Andy| 4500|</span> |
| <span class="c1">// | Justin| 3500|</span> |
| <span class="c1">// | Berta| 4000|</span> |
| <span class="c1">// +-------+------+</span> |
| |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">result</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT myAverage(salary) as average_salary FROM employees"</span><span class="o">);</span> |
| <span class="n">result</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +--------------+</span> |
| <span class="c1">// |average_salary|</span> |
| <span class="c1">// +--------------+</span> |
| <span class="c1">// | 3750.0|</span> |
| <span class="c1">// +--------------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java" in the Spark repo.</small></div> |
| </div> |
| </div> |
| |
| <h3 id="type-safe-user-defined-aggregate-functions">Type-Safe User-Defined Aggregate Functions</h3> |
| |
| <p>User-defined aggregations for strongly typed Datasets revolve around the <a href="api/scala/index.html#org.apache.spark.sql.expressions.Aggregator">Aggregator</a> abstract class. |
| For example, a type-safe user-defined average can look like:</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="k">import</span> <span class="nn">org.apache.spark.sql.</span><span class="o">{</span><span class="nc">Encoder</span><span class="o">,</span> <span class="nc">Encoders</span><span class="o">,</span> <span class="nc">SparkSession</span><span class="o">}</span> |
| <span class="k">import</span> <span class="nn">org.apache.spark.sql.expressions.Aggregator</span> |
| |
| <span class="k">case</span> <span class="k">class</span> <span class="nc">Employee</span><span class="o">(</span><span class="n">name</span><span class="k">:</span> <span class="kt">String</span><span class="o">,</span> <span class="n">salary</span><span class="k">:</span> <span class="kt">Long</span><span class="o">)</span> |
| <span class="k">case</span> <span class="k">class</span> <span class="nc">Average</span><span class="o">(</span><span class="k">var</span> <span class="n">sum</span><span class="k">:</span> <span class="kt">Long</span><span class="o">,</span> <span class="k">var</span> <span class="n">count</span><span class="k">:</span> <span class="kt">Long</span><span class="o">)</span> |
| |
| <span class="k">object</span> <span class="nc">MyAverage</span> <span class="k">extends</span> <span class="nc">Aggregator</span><span class="o">[</span><span class="kt">Employee</span>, <span class="kt">Average</span>, <span class="kt">Double</span><span class="o">]</span> <span class="o">{</span> |
| <span class="c1">// A zero value for this aggregation. Should satisfy the property that any b + zero = b</span> |
| <span class="k">def</span> <span class="n">zero</span><span class="k">:</span> <span class="kt">Average</span> <span class="o">=</span> <span class="nc">Average</span><span class="o">(</span><span class="mi">0L</span><span class="o">,</span> <span class="mi">0L</span><span class="o">)</span> |
| <span class="c1">// Combine two values to produce a new value. For performance, the function may modify `buffer`</span> |
| <span class="c1">// and return it instead of constructing a new object</span> |
| <span class="k">def</span> <span class="n">reduce</span><span class="o">(</span><span class="n">buffer</span><span class="k">:</span> <span class="kt">Average</span><span class="o">,</span> <span class="n">employee</span><span class="k">:</span> <span class="kt">Employee</span><span class="o">)</span><span class="k">:</span> <span class="kt">Average</span> <span class="o">=</span> <span class="o">{</span> |
| <span class="n">buffer</span><span class="o">.</span><span class="n">sum</span> <span class="o">+=</span> <span class="n">employee</span><span class="o">.</span><span class="n">salary</span> |
| <span class="n">buffer</span><span class="o">.</span><span class="n">count</span> <span class="o">+=</span> <span class="mi">1</span> |
| <span class="n">buffer</span> |
| <span class="o">}</span> |
| <span class="c1">// Merge two intermediate values</span> |
| <span class="k">def</span> <span class="n">merge</span><span class="o">(</span><span class="n">b1</span><span class="k">:</span> <span class="kt">Average</span><span class="o">,</span> <span class="n">b2</span><span class="k">:</span> <span class="kt">Average</span><span class="o">)</span><span class="k">:</span> <span class="kt">Average</span> <span class="o">=</span> <span class="o">{</span> |
| <span class="n">b1</span><span class="o">.</span><span class="n">sum</span> <span class="o">+=</span> <span class="n">b2</span><span class="o">.</span><span class="n">sum</span> |
| <span class="n">b1</span><span class="o">.</span><span class="n">count</span> <span class="o">+=</span> <span class="n">b2</span><span class="o">.</span><span class="n">count</span> |
| <span class="n">b1</span> |
| <span class="o">}</span> |
| <span class="c1">// Transform the output of the reduction</span> |
| <span class="k">def</span> <span class="n">finish</span><span class="o">(</span><span class="n">reduction</span><span class="k">:</span> <span class="kt">Average</span><span class="o">)</span><span class="k">:</span> <span class="kt">Double</span> <span class="o">=</span> <span class="n">reduction</span><span class="o">.</span><span class="n">sum</span><span class="o">.</span><span class="n">toDouble</span> <span class="o">/</span> <span class="n">reduction</span><span class="o">.</span><span class="n">count</span> |
| <span class="c1">// Specifies the Encoder for the intermediate value type</span> |
| <span class="k">def</span> <span class="n">bufferEncoder</span><span class="k">:</span> <span class="kt">Encoder</span><span class="o">[</span><span class="kt">Average</span><span class="o">]</span> <span class="k">=</span> <span class="nc">Encoders</span><span class="o">.</span><span class="n">product</span> |
| <span class="c1">// Specifies the Encoder for the final output value type</span> |
| <span class="k">def</span> <span class="n">outputEncoder</span><span class="k">:</span> <span class="kt">Encoder</span><span class="o">[</span><span class="kt">Double</span><span class="o">]</span> <span class="k">=</span> <span class="nc">Encoders</span><span class="o">.</span><span class="n">scalaDouble</span> |
| <span class="o">}</span> |
| |
| <span class="k">val</span> <span class="n">ds</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">json</span><span class="o">(</span><span class="s">"examples/src/main/resources/employees.json"</span><span class="o">).</span><span class="n">as</span><span class="o">[</span><span class="kt">Employee</span><span class="o">]</span> |
| <span class="n">ds</span><span class="o">.</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +-------+------+</span> |
| <span class="c1">// | name|salary|</span> |
| <span class="c1">// +-------+------+</span> |
| <span class="c1">// |Michael| 3000|</span> |
| <span class="c1">// | Andy| 4500|</span> |
| <span class="c1">// | Justin| 3500|</span> |
| <span class="c1">// | Berta| 4000|</span> |
| <span class="c1">// +-------+------+</span> |
| |
| <span class="c1">// Convert the function to a `TypedColumn` and give it a name</span> |
| <span class="k">val</span> <span class="n">averageSalary</span> <span class="k">=</span> <span class="nc">MyAverage</span><span class="o">.</span><span class="n">toColumn</span><span class="o">.</span><span class="n">name</span><span class="o">(</span><span class="s">"average_salary"</span><span class="o">)</span> |
| <span class="k">val</span> <span class="n">result</span> <span class="k">=</span> <span class="n">ds</span><span class="o">.</span><span class="n">select</span><span class="o">(</span><span class="n">averageSalary</span><span class="o">)</span> |
| <span class="n">result</span><span class="o">.</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +--------------+</span> |
| <span class="c1">// |average_salary|</span> |
| <span class="c1">// +--------------+</span> |
| <span class="c1">// | 3750.0|</span> |
| <span class="c1">// +--------------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala" in the Spark repo.</small></div> |
| </div> |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.io.Serializable</span><span class="o">;</span> |
| |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Dataset</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Encoder</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Encoders</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.SparkSession</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.TypedColumn</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.expressions.Aggregator</span><span class="o">;</span> |
| |
| <span class="kd">public</span> <span class="kd">static</span> <span class="kd">class</span> <span class="nc">Employee</span> <span class="kd">implements</span> <span class="n">Serializable</span> <span class="o">{</span> |
| <span class="kd">private</span> <span class="n">String</span> <span class="n">name</span><span class="o">;</span> |
| <span class="kd">private</span> <span class="kt">long</span> <span class="n">salary</span><span class="o">;</span> |
| |
| <span class="c1">// Constructors, getters, setters...</span> |
| |
| <span class="o">}</span> |
| |
| <span class="kd">public</span> <span class="kd">static</span> <span class="kd">class</span> <span class="nc">Average</span> <span class="kd">implements</span> <span class="n">Serializable</span> <span class="o">{</span> |
| <span class="kd">private</span> <span class="kt">long</span> <span class="n">sum</span><span class="o">;</span> |
| <span class="kd">private</span> <span class="kt">long</span> <span class="n">count</span><span class="o">;</span> |
| |
| <span class="c1">// Constructors, getters, setters...</span> |
| |
| <span class="o">}</span> |
| |
| <span class="kd">public</span> <span class="kd">static</span> <span class="kd">class</span> <span class="nc">MyAverage</span> <span class="kd">extends</span> <span class="n">Aggregator</span><span class="o"><</span><span class="n">Employee</span><span class="o">,</span> <span class="n">Average</span><span class="o">,</span> <span class="n">Double</span><span class="o">></span> <span class="o">{</span> |
| <span class="c1">// A zero value for this aggregation. Should satisfy the property that any b + zero = b</span> |
| <span class="kd">public</span> <span class="n">Average</span> <span class="nf">zero</span><span class="o">()</span> <span class="o">{</span> |
| <span class="k">return</span> <span class="k">new</span> <span class="n">Average</span><span class="o">(</span><span class="mi">0</span><span class="n">L</span><span class="o">,</span> <span class="mi">0</span><span class="n">L</span><span class="o">);</span> |
| <span class="o">}</span> |
| <span class="c1">// Combine two values to produce a new value. For performance, the function may modify `buffer`</span> |
| <span class="c1">// and return it instead of constructing a new object</span> |
| <span class="kd">public</span> <span class="n">Average</span> <span class="nf">reduce</span><span class="o">(</span><span class="n">Average</span> <span class="n">buffer</span><span class="o">,</span> <span class="n">Employee</span> <span class="n">employee</span><span class="o">)</span> <span class="o">{</span> |
| <span class="kt">long</span> <span class="n">newSum</span> <span class="o">=</span> <span class="n">buffer</span><span class="o">.</span><span class="na">getSum</span><span class="o">()</span> <span class="o">+</span> <span class="n">employee</span><span class="o">.</span><span class="na">getSalary</span><span class="o">();</span> |
| <span class="kt">long</span> <span class="n">newCount</span> <span class="o">=</span> <span class="n">buffer</span><span class="o">.</span><span class="na">getCount</span><span class="o">()</span> <span class="o">+</span> <span class="mi">1</span><span class="o">;</span> |
| <span class="n">buffer</span><span class="o">.</span><span class="na">setSum</span><span class="o">(</span><span class="n">newSum</span><span class="o">);</span> |
| <span class="n">buffer</span><span class="o">.</span><span class="na">setCount</span><span class="o">(</span><span class="n">newCount</span><span class="o">);</span> |
| <span class="k">return</span> <span class="n">buffer</span><span class="o">;</span> |
| <span class="o">}</span> |
| <span class="c1">// Merge two intermediate values</span> |
| <span class="kd">public</span> <span class="n">Average</span> <span class="nf">merge</span><span class="o">(</span><span class="n">Average</span> <span class="n">b1</span><span class="o">,</span> <span class="n">Average</span> <span class="n">b2</span><span class="o">)</span> <span class="o">{</span> |
| <span class="kt">long</span> <span class="n">mergedSum</span> <span class="o">=</span> <span class="n">b1</span><span class="o">.</span><span class="na">getSum</span><span class="o">()</span> <span class="o">+</span> <span class="n">b2</span><span class="o">.</span><span class="na">getSum</span><span class="o">();</span> |
| <span class="kt">long</span> <span class="n">mergedCount</span> <span class="o">=</span> <span class="n">b1</span><span class="o">.</span><span class="na">getCount</span><span class="o">()</span> <span class="o">+</span> <span class="n">b2</span><span class="o">.</span><span class="na">getCount</span><span class="o">();</span> |
| <span class="n">b1</span><span class="o">.</span><span class="na">setSum</span><span class="o">(</span><span class="n">mergedSum</span><span class="o">);</span> |
| <span class="n">b1</span><span class="o">.</span><span class="na">setCount</span><span class="o">(</span><span class="n">mergedCount</span><span class="o">);</span> |
| <span class="k">return</span> <span class="n">b1</span><span class="o">;</span> |
| <span class="o">}</span> |
| <span class="c1">// Transform the output of the reduction</span> |
| <span class="kd">public</span> <span class="n">Double</span> <span class="nf">finish</span><span class="o">(</span><span class="n">Average</span> <span class="n">reduction</span><span class="o">)</span> <span class="o">{</span> |
| <span class="k">return</span> <span class="o">((</span><span class="kt">double</span><span class="o">)</span> <span class="n">reduction</span><span class="o">.</span><span class="na">getSum</span><span class="o">())</span> <span class="o">/</span> <span class="n">reduction</span><span class="o">.</span><span class="na">getCount</span><span class="o">();</span> |
| <span class="o">}</span> |
| <span class="c1">// Specifies the Encoder for the intermediate value type</span> |
| <span class="kd">public</span> <span class="n">Encoder</span><span class="o"><</span><span class="n">Average</span><span class="o">></span> <span class="nf">bufferEncoder</span><span class="o">()</span> <span class="o">{</span> |
| <span class="k">return</span> <span class="n">Encoders</span><span class="o">.</span><span class="na">bean</span><span class="o">(</span><span class="n">Average</span><span class="o">.</span><span class="na">class</span><span class="o">);</span> |
| <span class="o">}</span> |
| <span class="c1">// Specifies the Encoder for the final output value type</span> |
| <span class="kd">public</span> <span class="n">Encoder</span><span class="o"><</span><span class="n">Double</span><span class="o">></span> <span class="nf">outputEncoder</span><span class="o">()</span> <span class="o">{</span> |
| <span class="k">return</span> <span class="n">Encoders</span><span class="o">.</span><span class="na">DOUBLE</span><span class="o">();</span> |
| <span class="o">}</span> |
| <span class="o">}</span> |
| |
| <span class="n">Encoder</span><span class="o"><</span><span class="n">Employee</span><span class="o">></span> <span class="n">employeeEncoder</span> <span class="o">=</span> <span class="n">Encoders</span><span class="o">.</span><span class="na">bean</span><span class="o">(</span><span class="n">Employee</span><span class="o">.</span><span class="na">class</span><span class="o">);</span> |
| <span class="n">String</span> <span class="n">path</span> <span class="o">=</span> <span class="s">"examples/src/main/resources/employees.json"</span><span class="o">;</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Employee</span><span class="o">></span> <span class="n">ds</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">json</span><span class="o">(</span><span class="n">path</span><span class="o">).</span><span class="na">as</span><span class="o">(</span><span class="n">employeeEncoder</span><span class="o">);</span> |
| <span class="n">ds</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +-------+------+</span> |
| <span class="c1">// | name|salary|</span> |
| <span class="c1">// +-------+------+</span> |
| <span class="c1">// |Michael| 3000|</span> |
| <span class="c1">// | Andy| 4500|</span> |
| <span class="c1">// | Justin| 3500|</span> |
| <span class="c1">// | Berta| 4000|</span> |
| <span class="c1">// +-------+------+</span> |
| |
| <span class="n">MyAverage</span> <span class="n">myAverage</span> <span class="o">=</span> <span class="k">new</span> <span class="n">MyAverage</span><span class="o">();</span> |
| <span class="c1">// Convert the function to a `TypedColumn` and give it a name</span> |
| <span class="n">TypedColumn</span><span class="o"><</span><span class="n">Employee</span><span class="o">,</span> <span class="n">Double</span><span class="o">></span> <span class="n">averageSalary</span> <span class="o">=</span> <span class="n">myAverage</span><span class="o">.</span><span class="na">toColumn</span><span class="o">().</span><span class="na">name</span><span class="o">(</span><span class="s">"average_salary"</span><span class="o">);</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Double</span><span class="o">></span> <span class="n">result</span> <span class="o">=</span> <span class="n">ds</span><span class="o">.</span><span class="na">select</span><span class="o">(</span><span class="n">averageSalary</span><span class="o">);</span> |
| <span class="n">result</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +--------------+</span> |
| <span class="c1">// |average_salary|</span> |
| <span class="c1">// +--------------+</span> |
| <span class="c1">// | 3750.0|</span> |
| <span class="c1">// +--------------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java" in the Spark repo.</small></div> |
| </div> |
| </div> |
| |
| <h1 id="data-sources">Data Sources</h1> |
| |
| <p>Spark SQL supports operating on a variety of data sources through the DataFrame interface. |
| A DataFrame can be operated on using relational transformations and can also be used to create a temporary view. |
| Registering a DataFrame as a temporary view allows you to run SQL queries over its data. This section |
| describes the general methods for loading and saving data using the Spark Data Sources and then |
| goes into specific options that are available for the built-in data sources.</p> |
| |
| <h2 id="generic-loadsave-functions">Generic Load/Save Functions</h2> |
| |
| <p>In the simplest form, the default data source (<code>parquet</code> unless otherwise configured by |
| <code>spark.sql.sources.default</code>) will be used for all operations.</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="k">val</span> <span class="n">usersDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">load</span><span class="o">(</span><span class="s">"examples/src/main/resources/users.parquet"</span><span class="o">)</span> |
| <span class="n">usersDF</span><span class="o">.</span><span class="n">select</span><span class="o">(</span><span class="s">"name"</span><span class="o">,</span> <span class="s">"favorite_color"</span><span class="o">).</span><span class="n">write</span><span class="o">.</span><span class="n">save</span><span class="o">(</span><span class="s">"namesAndFavColors.parquet"</span><span class="o">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">usersDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">load</span><span class="o">(</span><span class="s">"examples/src/main/resources/users.parquet"</span><span class="o">);</span> |
| <span class="n">usersDF</span><span class="o">.</span><span class="na">select</span><span class="o">(</span><span class="s">"name"</span><span class="o">,</span> <span class="s">"favorite_color"</span><span class="o">).</span><span class="na">write</span><span class="o">().</span><span class="na">save</span><span class="o">(</span><span class="s">"namesAndFavColors.parquet"</span><span class="o">);</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| |
| <div class="highlight"><pre><span></span><span class="n">df</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">load</span><span class="p">(</span><span class="s2">"examples/src/main/resources/users.parquet"</span><span class="p">)</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="s2">"name"</span><span class="p">,</span> <span class="s2">"favorite_color"</span><span class="p">)</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">save</span><span class="p">(</span><span class="s2">"namesAndFavColors.parquet"</span><span class="p">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/datasource.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="r"> |
| |
| <div class="highlight"><pre><span></span>df <span class="o"><-</span> read.df<span class="p">(</span><span class="s">"examples/src/main/resources/users.parquet"</span><span class="p">)</span> |
| write.df<span class="p">(</span>select<span class="p">(</span>df<span class="p">,</span> <span class="s">"name"</span><span class="p">,</span> <span class="s">"favorite_color"</span><span class="p">),</span> <span class="s">"namesAndFavColors.parquet"</span><span class="p">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| |
| </div> |
| </div> |
| |
| <h3 id="manually-specifying-options">Manually Specifying Options</h3> |
| |
| <p>You can also manually specify the data source that will be used along with any extra options |
| that you would like to pass to the data source. Data sources are specified by their fully qualified |
| name (i.e., <code>org.apache.spark.sql.parquet</code>), but for built-in sources you can also use their short |
| names (<code>json</code>, <code>parquet</code>, <code>jdbc</code>, <code>orc</code>, <code>libsvm</code>, <code>csv</code>, <code>text</code>). DataFrames loaded from any data |
| source type can be converted into other types using this syntax.</p> |
| |
| <p>To load a JSON file you can use:</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="k">val</span> <span class="n">peopleDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">format</span><span class="o">(</span><span class="s">"json"</span><span class="o">).</span><span class="n">load</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.json"</span><span class="o">)</span> |
| <span class="n">peopleDF</span><span class="o">.</span><span class="n">select</span><span class="o">(</span><span class="s">"name"</span><span class="o">,</span> <span class="s">"age"</span><span class="o">).</span><span class="n">write</span><span class="o">.</span><span class="n">format</span><span class="o">(</span><span class="s">"parquet"</span><span class="o">).</span><span class="n">save</span><span class="o">(</span><span class="s">"namesAndAges.parquet"</span><span class="o">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">peopleDF</span> <span class="o">=</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">format</span><span class="o">(</span><span class="s">"json"</span><span class="o">).</span><span class="na">load</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.json"</span><span class="o">);</span> |
| <span class="n">peopleDF</span><span class="o">.</span><span class="na">select</span><span class="o">(</span><span class="s">"name"</span><span class="o">,</span> <span class="s">"age"</span><span class="o">).</span><span class="na">write</span><span class="o">().</span><span class="na">format</span><span class="o">(</span><span class="s">"parquet"</span><span class="o">).</span><span class="na">save</span><span class="o">(</span><span class="s">"namesAndAges.parquet"</span><span class="o">);</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| <div class="highlight"><pre><span></span><span class="n">df</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">load</span><span class="p">(</span><span class="s2">"examples/src/main/resources/people.json"</span><span class="p">,</span> <span class="n">format</span><span class="o">=</span><span class="s2">"json"</span><span class="p">)</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="s2">"name"</span><span class="p">,</span> <span class="s2">"age"</span><span class="p">)</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">save</span><span class="p">(</span><span class="s2">"namesAndAges.parquet"</span><span class="p">,</span> <span class="n">format</span><span class="o">=</span><span class="s2">"parquet"</span><span class="p">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/datasource.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="r"> |
| <div class="highlight"><pre><span></span>df <span class="o"><-</span> read.df<span class="p">(</span><span class="s">"examples/src/main/resources/people.json"</span><span class="p">,</span> <span class="s">"json"</span><span class="p">)</span> |
| namesAndAges <span class="o"><-</span> select<span class="p">(</span>df<span class="p">,</span> <span class="s">"name"</span><span class="p">,</span> <span class="s">"age"</span><span class="p">)</span> |
| write.df<span class="p">(</span>namesAndAges<span class="p">,</span> <span class="s">"namesAndAges.parquet"</span><span class="p">,</span> <span class="s">"parquet"</span><span class="p">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| </div> |
| </div> |
| |
| <p>To load a CSV file you can use:</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="k">val</span> <span class="n">peopleDFCsv</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">format</span><span class="o">(</span><span class="s">"csv"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">option</span><span class="o">(</span><span class="s">"sep"</span><span class="o">,</span> <span class="s">";"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">option</span><span class="o">(</span><span class="s">"inferSchema"</span><span class="o">,</span> <span class="s">"true"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">option</span><span class="o">(</span><span class="s">"header"</span><span class="o">,</span> <span class="s">"true"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">load</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.csv"</span><span class="o">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">peopleDFCsv</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">format</span><span class="o">(</span><span class="s">"csv"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"sep"</span><span class="o">,</span> <span class="s">";"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"inferSchema"</span><span class="o">,</span> <span class="s">"true"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"header"</span><span class="o">,</span> <span class="s">"true"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">load</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.csv"</span><span class="o">);</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| <div class="highlight"><pre><span></span><span class="n">df</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">load</span><span class="p">(</span><span class="s2">"examples/src/main/resources/people.csv"</span><span class="p">,</span> |
| <span class="n">format</span><span class="o">=</span><span class="s2">"csv"</span><span class="p">,</span> <span class="n">sep</span><span class="o">=</span><span class="s2">":"</span><span class="p">,</span> <span class="n">inferSchema</span><span class="o">=</span><span class="s2">"true"</span><span class="p">,</span> <span class="n">header</span><span class="o">=</span><span class="s2">"true"</span><span class="p">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/datasource.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="r"> |
| <div class="highlight"><pre><span></span>df <span class="o"><-</span> read.df<span class="p">(</span><span class="s">"examples/src/main/resources/people.csv"</span><span class="p">,</span> <span class="s">"csv"</span><span class="p">)</span> |
| namesAndAges <span class="o"><-</span> select<span class="p">(</span>df<span class="p">,</span> <span class="s">"name"</span><span class="p">,</span> <span class="s">"age"</span><span class="p">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| |
| </div> |
| </div> |
| |
| <h3 id="run-sql-on-files-directly">Run SQL on files directly</h3> |
| |
| <p>Instead of using read API to load a file into DataFrame and query it, you can also query that |
| file directly with SQL.</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="k">val</span> <span class="n">sqlDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="o">(</span><span class="s">"SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"</span><span class="o">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">sqlDF</span> <span class="o">=</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"</span><span class="o">);</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| <div class="highlight"><pre><span></span><span class="n">df</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"</span><span class="p">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/datasource.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="r"> |
| <div class="highlight"><pre><span></span>df <span class="o"><-</span> sql<span class="p">(</span><span class="s">"SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"</span><span class="p">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| |
| </div> |
| </div> |
| |
| <h3 id="save-modes">Save Modes</h3> |
| |
| <p>Save operations can optionally take a <code>SaveMode</code>, that specifies how to handle existing data if |
| present. It is important to realize that these save modes do not utilize any locking and are not |
| atomic. Additionally, when performing an <code>Overwrite</code>, the data will be deleted before writing out the |
| new data.</p> |
| |
| <table class="table"> |
| <tr><th>Scala/Java</th><th>Any Language</th><th>Meaning</th></tr> |
| <tr> |
| <td><code>SaveMode.ErrorIfExists</code> (default)</td> |
| <td><code>"error" or "errorifexists"</code> (default)</td> |
| <td> |
| When saving a DataFrame to a data source, if data already exists, |
| an exception is expected to be thrown. |
| </td> |
| </tr> |
| <tr> |
| <td><code>SaveMode.Append</code></td> |
| <td><code>"append"</code></td> |
| <td> |
| When saving a DataFrame to a data source, if data/table already exists, |
| contents of the DataFrame are expected to be appended to existing data. |
| </td> |
| </tr> |
| <tr> |
| <td><code>SaveMode.Overwrite</code></td> |
| <td><code>"overwrite"</code></td> |
| <td> |
| Overwrite mode means that when saving a DataFrame to a data source, |
| if data/table already exists, existing data is expected to be overwritten by the contents of |
| the DataFrame. |
| </td> |
| </tr> |
| <tr> |
| <td><code>SaveMode.Ignore</code></td> |
| <td><code>"ignore"</code></td> |
| <td> |
| Ignore mode means that when saving a DataFrame to a data source, if data already exists, |
| the save operation is expected to not save the contents of the DataFrame and to not |
| change the existing data. This is similar to a <code>CREATE TABLE IF NOT EXISTS</code> in SQL. |
| </td> |
| </tr> |
| </table> |
| |
| <h3 id="saving-to-persistent-tables">Saving to Persistent Tables</h3> |
| |
| <p><code>DataFrames</code> can also be saved as persistent tables into Hive metastore using the <code>saveAsTable</code> |
| command. Notice that an existing Hive deployment is not necessary to use this feature. Spark will create a |
| default local Hive metastore (using Derby) for you. Unlike the <code>createOrReplaceTempView</code> command, |
| <code>saveAsTable</code> will materialize the contents of the DataFrame and create a pointer to the data in the |
| Hive metastore. Persistent tables will still exist even after your Spark program has restarted, as |
| long as you maintain your connection to the same metastore. A DataFrame for a persistent table can |
| be created by calling the <code>table</code> method on a <code>SparkSession</code> with the name of the table.</p> |
| |
| <p>For file-based data source, e.g. text, parquet, json, etc. you can specify a custom table path via the |
| <code>path</code> option, e.g. <code>df.write.option("path", "/some/path").saveAsTable("t")</code>. When the table is dropped, |
| the custom table path will not be removed and the table data is still there. If no custom table path is |
| specified, Spark will write data to a default table path under the warehouse directory. When the table is |
| dropped, the default table path will be removed too.</p> |
| |
| <p>Starting from Spark 2.1, persistent datasource tables have per-partition metadata stored in the Hive metastore. This brings several benefits:</p> |
| |
| <ul> |
| <li>Since the metastore can return only necessary partitions for a query, discovering all the partitions on the first query to the table is no longer needed.</li> |
| <li>Hive DDLs such as <code>ALTER TABLE PARTITION ... SET LOCATION</code> are now available for tables created with the Datasource API.</li> |
| </ul> |
| |
| <p>Note that partition information is not gathered by default when creating external datasource tables (those with a <code>path</code> option). To sync the partition information in the metastore, you can invoke <code>MSCK REPAIR TABLE</code>.</p> |
| |
| <h3 id="bucketing-sorting-and-partitioning">Bucketing, Sorting and Partitioning</h3> |
| |
| <p>For file-based data source, it is also possible to bucket and sort or partition the output. |
| Bucketing and sorting are applicable only to persistent tables:</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="n">peopleDF</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">bucketBy</span><span class="o">(</span><span class="mi">42</span><span class="o">,</span> <span class="s">"name"</span><span class="o">).</span><span class="n">sortBy</span><span class="o">(</span><span class="s">"age"</span><span class="o">).</span><span class="n">saveAsTable</span><span class="o">(</span><span class="s">"people_bucketed"</span><span class="o">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="n">peopleDF</span><span class="o">.</span><span class="na">write</span><span class="o">().</span><span class="na">bucketBy</span><span class="o">(</span><span class="mi">42</span><span class="o">,</span> <span class="s">"name"</span><span class="o">).</span><span class="na">sortBy</span><span class="o">(</span><span class="s">"age"</span><span class="o">).</span><span class="na">saveAsTable</span><span class="o">(</span><span class="s">"people_bucketed"</span><span class="o">);</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| <div class="highlight"><pre><span></span><span class="n">df</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">bucketBy</span><span class="p">(</span><span class="mi">42</span><span class="p">,</span> <span class="s2">"name"</span><span class="p">)</span><span class="o">.</span><span class="n">sortBy</span><span class="p">(</span><span class="s2">"age"</span><span class="p">)</span><span class="o">.</span><span class="n">saveAsTable</span><span class="p">(</span><span class="s2">"people_bucketed"</span><span class="p">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/datasource.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="sql"> |
| |
| <figure class="highlight"><pre><code class="language-sql" data-lang="sql"><span></span><span class="k">CREATE</span> <span class="k">TABLE</span> <span class="n">users_bucketed_by_name</span><span class="p">(</span> |
| <span class="n">name</span> <span class="n">STRING</span><span class="p">,</span> |
| <span class="n">favorite_color</span> <span class="n">STRING</span><span class="p">,</span> |
| <span class="n">favorite_numbers</span> <span class="nb">array</span><span class="o"><</span><span class="nb">integer</span><span class="o">></span> |
| <span class="p">)</span> <span class="k">USING</span> <span class="n">parquet</span> |
| <span class="n">CLUSTERED</span> <span class="k">BY</span><span class="p">(</span><span class="n">name</span><span class="p">)</span> <span class="k">INTO</span> <span class="mi">42</span> <span class="n">BUCKETS</span><span class="p">;</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>while partitioning can be used with both <code>save</code> and <code>saveAsTable</code> when using the Dataset APIs.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="n">usersDF</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">partitionBy</span><span class="o">(</span><span class="s">"favorite_color"</span><span class="o">).</span><span class="n">format</span><span class="o">(</span><span class="s">"parquet"</span><span class="o">).</span><span class="n">save</span><span class="o">(</span><span class="s">"namesPartByColor.parquet"</span><span class="o">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="n">usersDF</span> |
| <span class="o">.</span><span class="na">write</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">partitionBy</span><span class="o">(</span><span class="s">"favorite_color"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">format</span><span class="o">(</span><span class="s">"parquet"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">save</span><span class="o">(</span><span class="s">"namesPartByColor.parquet"</span><span class="o">);</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| <div class="highlight"><pre><span></span><span class="n">df</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">partitionBy</span><span class="p">(</span><span class="s2">"favorite_color"</span><span class="p">)</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="s2">"parquet"</span><span class="p">)</span><span class="o">.</span><span class="n">save</span><span class="p">(</span><span class="s2">"namesPartByColor.parquet"</span><span class="p">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/datasource.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="sql"> |
| |
| <figure class="highlight"><pre><code class="language-sql" data-lang="sql"><span></span><span class="k">CREATE</span> <span class="k">TABLE</span> <span class="n">users_by_favorite_color</span><span class="p">(</span> |
| <span class="n">name</span> <span class="n">STRING</span><span class="p">,</span> |
| <span class="n">favorite_color</span> <span class="n">STRING</span><span class="p">,</span> |
| <span class="n">favorite_numbers</span> <span class="nb">array</span><span class="o"><</span><span class="nb">integer</span><span class="o">></span> |
| <span class="p">)</span> <span class="k">USING</span> <span class="n">csv</span> <span class="n">PARTITIONED</span> <span class="k">BY</span><span class="p">(</span><span class="n">favorite_color</span><span class="p">);</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>It is possible to use both partitioning and bucketing for a single table:</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="n">usersDF</span> |
| <span class="o">.</span><span class="n">write</span> |
| <span class="o">.</span><span class="n">partitionBy</span><span class="o">(</span><span class="s">"favorite_color"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">bucketBy</span><span class="o">(</span><span class="mi">42</span><span class="o">,</span> <span class="s">"name"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">saveAsTable</span><span class="o">(</span><span class="s">"users_partitioned_bucketed"</span><span class="o">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="n">peopleDF</span> |
| <span class="o">.</span><span class="na">write</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">partitionBy</span><span class="o">(</span><span class="s">"favorite_color"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">bucketBy</span><span class="o">(</span><span class="mi">42</span><span class="o">,</span> <span class="s">"name"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">saveAsTable</span><span class="o">(</span><span class="s">"people_partitioned_bucketed"</span><span class="o">);</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| <div class="highlight"><pre><span></span><span class="n">df</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">parquet</span><span class="p">(</span><span class="s2">"examples/src/main/resources/users.parquet"</span><span class="p">)</span> |
| <span class="p">(</span><span class="n">df</span> |
| <span class="o">.</span><span class="n">write</span> |
| <span class="o">.</span><span class="n">partitionBy</span><span class="p">(</span><span class="s2">"favorite_color"</span><span class="p">)</span> |
| <span class="o">.</span><span class="n">bucketBy</span><span class="p">(</span><span class="mi">42</span><span class="p">,</span> <span class="s2">"name"</span><span class="p">)</span> |
| <span class="o">.</span><span class="n">saveAsTable</span><span class="p">(</span><span class="s2">"people_partitioned_bucketed"</span><span class="p">))</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/datasource.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="sql"> |
| |
| <figure class="highlight"><pre><code class="language-sql" data-lang="sql"><span></span><span class="k">CREATE</span> <span class="k">TABLE</span> <span class="n">users_bucketed_and_partitioned</span><span class="p">(</span> |
| <span class="n">name</span> <span class="n">STRING</span><span class="p">,</span> |
| <span class="n">favorite_color</span> <span class="n">STRING</span><span class="p">,</span> |
| <span class="n">favorite_numbers</span> <span class="nb">array</span><span class="o"><</span><span class="nb">integer</span><span class="o">></span> |
| <span class="p">)</span> <span class="k">USING</span> <span class="n">parquet</span> |
| <span class="n">PARTITIONED</span> <span class="k">BY</span> <span class="p">(</span><span class="n">favorite_color</span><span class="p">)</span> |
| <span class="n">CLUSTERED</span> <span class="k">BY</span><span class="p">(</span><span class="n">name</span><span class="p">)</span> <span class="n">SORTED</span> <span class="k">BY</span> <span class="p">(</span><span class="n">favorite_numbers</span><span class="p">)</span> <span class="k">INTO</span> <span class="mi">42</span> <span class="n">BUCKETS</span><span class="p">;</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p><code>partitionBy</code> creates a directory structure as described in the <a href="#partition-discovery">Partition Discovery</a> section. |
| Thus, it has limited applicability to columns with high cardinality. In contrast |
| <code>bucketBy</code> distributes |
| data across a fixed number of buckets and can be used when a number of unique values is unbounded.</p> |
| |
| <h2 id="parquet-files">Parquet Files</h2> |
| |
| <p><a href="http://parquet.io">Parquet</a> is a columnar format that is supported by many other data processing systems. |
| Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema |
| of the original data. When writing Parquet files, all columns are automatically converted to be nullable for |
| compatibility reasons.</p> |
| |
| <h3 id="loading-data-programmatically">Loading Data Programmatically</h3> |
| |
| <p>Using the data from the above example:</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="c1">// Encoders for most common types are automatically provided by importing spark.implicits._</span> |
| <span class="k">import</span> <span class="nn">spark.implicits._</span> |
| |
| <span class="k">val</span> <span class="n">peopleDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">json</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.json"</span><span class="o">)</span> |
| |
| <span class="c1">// DataFrames can be saved as Parquet files, maintaining the schema information</span> |
| <span class="n">peopleDF</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">parquet</span><span class="o">(</span><span class="s">"people.parquet"</span><span class="o">)</span> |
| |
| <span class="c1">// Read in the parquet file created above</span> |
| <span class="c1">// Parquet files are self-describing so the schema is preserved</span> |
| <span class="c1">// The result of loading a Parquet file is also a DataFrame</span> |
| <span class="k">val</span> <span class="n">parquetFileDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">parquet</span><span class="o">(</span><span class="s">"people.parquet"</span><span class="o">)</span> |
| |
| <span class="c1">// Parquet files can also be used to create a temporary view and then used in SQL statements</span> |
| <span class="n">parquetFileDF</span><span class="o">.</span><span class="n">createOrReplaceTempView</span><span class="o">(</span><span class="s">"parquetFile"</span><span class="o">)</span> |
| <span class="k">val</span> <span class="n">namesDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="o">(</span><span class="s">"SELECT name FROM parquetFile WHERE age BETWEEN 13 AND 19"</span><span class="o">)</span> |
| <span class="n">namesDF</span><span class="o">.</span><span class="n">map</span><span class="o">(</span><span class="n">attributes</span> <span class="k">=></span> <span class="s">"Name: "</span> <span class="o">+</span> <span class="n">attributes</span><span class="o">(</span><span class="mi">0</span><span class="o">)).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +------------+</span> |
| <span class="c1">// | value|</span> |
| <span class="c1">// +------------+</span> |
| <span class="c1">// |Name: Justin|</span> |
| <span class="c1">// +------------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">org.apache.spark.api.java.function.MapFunction</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Encoders</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Dataset</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Row</span><span class="o">;</span> |
| |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">peopleDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">json</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.json"</span><span class="o">);</span> |
| |
| <span class="c1">// DataFrames can be saved as Parquet files, maintaining the schema information</span> |
| <span class="n">peopleDF</span><span class="o">.</span><span class="na">write</span><span class="o">().</span><span class="na">parquet</span><span class="o">(</span><span class="s">"people.parquet"</span><span class="o">);</span> |
| |
| <span class="c1">// Read in the Parquet file created above.</span> |
| <span class="c1">// Parquet files are self-describing so the schema is preserved</span> |
| <span class="c1">// The result of loading a parquet file is also a DataFrame</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">parquetFileDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">parquet</span><span class="o">(</span><span class="s">"people.parquet"</span><span class="o">);</span> |
| |
| <span class="c1">// Parquet files can also be used to create a temporary view and then used in SQL statements</span> |
| <span class="n">parquetFileDF</span><span class="o">.</span><span class="na">createOrReplaceTempView</span><span class="o">(</span><span class="s">"parquetFile"</span><span class="o">);</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">namesDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT name FROM parquetFile WHERE age BETWEEN 13 AND 19"</span><span class="o">);</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">String</span><span class="o">></span> <span class="n">namesDS</span> <span class="o">=</span> <span class="n">namesDF</span><span class="o">.</span><span class="na">map</span><span class="o">(</span> |
| <span class="o">(</span><span class="n">MapFunction</span><span class="o"><</span><span class="n">Row</span><span class="o">,</span> <span class="n">String</span><span class="o">>)</span> <span class="n">row</span> <span class="o">-></span> <span class="s">"Name: "</span> <span class="o">+</span> <span class="n">row</span><span class="o">.</span><span class="na">getString</span><span class="o">(</span><span class="mi">0</span><span class="o">),</span> |
| <span class="n">Encoders</span><span class="o">.</span><span class="na">STRING</span><span class="o">());</span> |
| <span class="n">namesDS</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +------------+</span> |
| <span class="c1">// | value|</span> |
| <span class="c1">// +------------+</span> |
| <span class="c1">// |Name: Justin|</span> |
| <span class="c1">// +------------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| |
| <div class="highlight"><pre><span></span><span class="n">peopleDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">json</span><span class="p">(</span><span class="s2">"examples/src/main/resources/people.json"</span><span class="p">)</span> |
| |
| <span class="c1"># DataFrames can be saved as Parquet files, maintaining the schema information.</span> |
| <span class="n">peopleDF</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">parquet</span><span class="p">(</span><span class="s2">"people.parquet"</span><span class="p">)</span> |
| |
| <span class="c1"># Read in the Parquet file created above.</span> |
| <span class="c1"># Parquet files are self-describing so the schema is preserved.</span> |
| <span class="c1"># The result of loading a parquet file is also a DataFrame.</span> |
| <span class="n">parquetFile</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">parquet</span><span class="p">(</span><span class="s2">"people.parquet"</span><span class="p">)</span> |
| |
| <span class="c1"># Parquet files can also be used to create a temporary view and then used in SQL statements.</span> |
| <span class="n">parquetFile</span><span class="o">.</span><span class="n">createOrReplaceTempView</span><span class="p">(</span><span class="s2">"parquetFile"</span><span class="p">)</span> |
| <span class="n">teenagers</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"</span><span class="p">)</span> |
| <span class="n">teenagers</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +------+</span> |
| <span class="c1"># | name|</span> |
| <span class="c1"># +------+</span> |
| <span class="c1"># |Justin|</span> |
| <span class="c1"># +------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/datasource.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="r"> |
| |
| <div class="highlight"><pre><span></span>df <span class="o"><-</span> read.df<span class="p">(</span><span class="s">"examples/src/main/resources/people.json"</span><span class="p">,</span> <span class="s">"json"</span><span class="p">)</span> |
| |
| <span class="c1"># SparkDataFrame can be saved as Parquet files, maintaining the schema information.</span> |
| write.parquet<span class="p">(</span>df<span class="p">,</span> <span class="s">"people.parquet"</span><span class="p">)</span> |
| |
| <span class="c1"># Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved.</span> |
| <span class="c1"># The result of loading a parquet file is also a DataFrame.</span> |
| parquetFile <span class="o"><-</span> read.parquet<span class="p">(</span><span class="s">"people.parquet"</span><span class="p">)</span> |
| |
| <span class="c1"># Parquet files can also be used to create a temporary view and then used in SQL statements.</span> |
| createOrReplaceTempView<span class="p">(</span>parquetFile<span class="p">,</span> <span class="s">"parquetFile"</span><span class="p">)</span> |
| teenagers <span class="o"><-</span> sql<span class="p">(</span><span class="s">"SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"</span><span class="p">)</span> |
| <span class="kp">head</span><span class="p">(</span>teenagers<span class="p">)</span> |
| <span class="c1">## name</span> |
| <span class="c1">## 1 Justin</span> |
| |
| <span class="c1"># We can also run custom R-UDFs on Spark DataFrames. Here we prefix all the names with "Name:"</span> |
| schema <span class="o"><-</span> structType<span class="p">(</span>structField<span class="p">(</span><span class="s">"name"</span><span class="p">,</span> <span class="s">"string"</span><span class="p">))</span> |
| teenNames <span class="o"><-</span> dapply<span class="p">(</span>df<span class="p">,</span> <span class="kr">function</span><span class="p">(</span>p<span class="p">)</span> <span class="p">{</span> <span class="kp">cbind</span><span class="p">(</span><span class="kp">paste</span><span class="p">(</span><span class="s">"Name:"</span><span class="p">,</span> p<span class="o">$</span>name<span class="p">))</span> <span class="p">},</span> schema<span class="p">)</span> |
| <span class="kr">for</span> <span class="p">(</span>teenName <span class="kr">in</span> collect<span class="p">(</span>teenNames<span class="p">)</span><span class="o">$</span>name<span class="p">)</span> <span class="p">{</span> |
| <span class="kp">cat</span><span class="p">(</span>teenName<span class="p">,</span> <span class="s">"\n"</span><span class="p">)</span> |
| <span class="p">}</span> |
| <span class="c1">## Name: Michael</span> |
| <span class="c1">## Name: Andy</span> |
| <span class="c1">## Name: Justin</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| |
| </div> |
| |
| <div data-lang="sql"> |
| |
| <figure class="highlight"><pre><code class="language-sql" data-lang="sql"><span></span><span class="k">CREATE</span> <span class="k">TEMPORARY</span> <span class="k">VIEW</span> <span class="n">parquetTable</span> |
| <span class="k">USING</span> <span class="n">org</span><span class="p">.</span><span class="n">apache</span><span class="p">.</span><span class="n">spark</span><span class="p">.</span><span class="k">sql</span><span class="p">.</span><span class="n">parquet</span> |
| <span class="k">OPTIONS</span> <span class="p">(</span> |
| <span class="n">path</span> <span class="ss">"examples/src/main/resources/people.parquet"</span> |
| <span class="p">)</span> |
| |
| <span class="k">SELECT</span> <span class="o">*</span> <span class="k">FROM</span> <span class="n">parquetTable</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h3 id="partition-discovery">Partition Discovery</h3> |
| |
| <p>Table partitioning is a common optimization approach used in systems like Hive. In a partitioned |
| table, data are usually stored in different directories, with partitioning column values encoded in |
| the path of each partition directory. All built-in file sources (including Text/CSV/JSON/ORC/Parquet) |
| are able to discover and infer partitioning information automatically. |
| For example, we can store all our previously used |
| population data into a partitioned table using the following directory structure, with two extra |
| columns, <code>gender</code> and <code>country</code> as partitioning columns:</p> |
| |
| <figure class="highlight"><pre><code class="language-text" data-lang="text"><span></span>path |
| └── to |
| └── table |
| ├── gender=male |
| │  ├── ... |
| │  │ |
| │  ├── country=US |
| │  │  └── data.parquet |
| │  ├── country=CN |
| │  │  └── data.parquet |
| │  └── ... |
| └── gender=female |
|   ├── ... |
|   │ |
|   ├── country=US |
|   │  └── data.parquet |
|   ├── country=CN |
|   │  └── data.parquet |
|   └── ...</code></pre></figure> |
| |
| <p>By passing <code>path/to/table</code> to either <code>SparkSession.read.parquet</code> or <code>SparkSession.read.load</code>, Spark SQL |
| will automatically extract the partitioning information from the paths. |
| Now the schema of the returned DataFrame becomes:</p> |
| |
| <figure class="highlight"><pre><code class="language-text" data-lang="text"><span></span>root |
| |-- name: string (nullable = true) |
| |-- age: long (nullable = true) |
| |-- gender: string (nullable = true) |
| |-- country: string (nullable = true)</code></pre></figure> |
| |
| <p>Notice that the data types of the partitioning columns are automatically inferred. Currently, |
| numeric data types, date, timestamp and string type are supported. Sometimes users may not want |
| to automatically infer the data types of the partitioning columns. For these use cases, the |
| automatic type inference can be configured by |
| <code>spark.sql.sources.partitionColumnTypeInference.enabled</code>, which is default to <code>true</code>. When type |
| inference is disabled, string type will be used for the partitioning columns.</p> |
| |
| <p>Starting from Spark 1.6.0, partition discovery only finds partitions under the given paths |
| by default. For the above example, if users pass <code>path/to/table/gender=male</code> to either |
| <code>SparkSession.read.parquet</code> or <code>SparkSession.read.load</code>, <code>gender</code> will not be considered as a |
| partitioning column. If users need to specify the base path that partition discovery |
| should start with, they can set <code>basePath</code> in the data source options. For example, |
| when <code>path/to/table/gender=male</code> is the path of the data and |
| users set <code>basePath</code> to <code>path/to/table/</code>, <code>gender</code> will be a partitioning column.</p> |
| |
| <h3 id="schema-merging">Schema Merging</h3> |
| |
| <p>Like ProtocolBuffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with |
| a simple schema, and gradually add more columns to the schema as needed. In this way, users may end |
| up with multiple Parquet files with different but mutually compatible schemas. The Parquet data |
| source is now able to automatically detect this case and merge schemas of all these files.</p> |
| |
| <p>Since schema merging is a relatively expensive operation, and is not a necessity in most cases, we |
| turned it off by default starting from 1.5.0. You may enable it by</p> |
| |
| <ol> |
| <li>setting data source option <code>mergeSchema</code> to <code>true</code> when reading Parquet files (as shown in the |
| examples below), or</li> |
| <li>setting the global SQL option <code>spark.sql.parquet.mergeSchema</code> to <code>true</code>.</li> |
| </ol> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="c1">// This is used to implicitly convert an RDD to a DataFrame.</span> |
| <span class="k">import</span> <span class="nn">spark.implicits._</span> |
| |
| <span class="c1">// Create a simple DataFrame, store into a partition directory</span> |
| <span class="k">val</span> <span class="n">squaresDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sparkContext</span><span class="o">.</span><span class="n">makeRDD</span><span class="o">(</span><span class="mi">1</span> <span class="n">to</span> <span class="mi">5</span><span class="o">).</span><span class="n">map</span><span class="o">(</span><span class="n">i</span> <span class="k">=></span> <span class="o">(</span><span class="n">i</span><span class="o">,</span> <span class="n">i</span> <span class="o">*</span> <span class="n">i</span><span class="o">)).</span><span class="n">toDF</span><span class="o">(</span><span class="s">"value"</span><span class="o">,</span> <span class="s">"square"</span><span class="o">)</span> |
| <span class="n">squaresDF</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">parquet</span><span class="o">(</span><span class="s">"data/test_table/key=1"</span><span class="o">)</span> |
| |
| <span class="c1">// Create another DataFrame in a new partition directory,</span> |
| <span class="c1">// adding a new column and dropping an existing column</span> |
| <span class="k">val</span> <span class="n">cubesDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sparkContext</span><span class="o">.</span><span class="n">makeRDD</span><span class="o">(</span><span class="mi">6</span> <span class="n">to</span> <span class="mi">10</span><span class="o">).</span><span class="n">map</span><span class="o">(</span><span class="n">i</span> <span class="k">=></span> <span class="o">(</span><span class="n">i</span><span class="o">,</span> <span class="n">i</span> <span class="o">*</span> <span class="n">i</span> <span class="o">*</span> <span class="n">i</span><span class="o">)).</span><span class="n">toDF</span><span class="o">(</span><span class="s">"value"</span><span class="o">,</span> <span class="s">"cube"</span><span class="o">)</span> |
| <span class="n">cubesDF</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">parquet</span><span class="o">(</span><span class="s">"data/test_table/key=2"</span><span class="o">)</span> |
| |
| <span class="c1">// Read the partitioned table</span> |
| <span class="k">val</span> <span class="n">mergedDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">option</span><span class="o">(</span><span class="s">"mergeSchema"</span><span class="o">,</span> <span class="s">"true"</span><span class="o">).</span><span class="n">parquet</span><span class="o">(</span><span class="s">"data/test_table"</span><span class="o">)</span> |
| <span class="n">mergedDF</span><span class="o">.</span><span class="n">printSchema</span><span class="o">()</span> |
| |
| <span class="c1">// The final schema consists of all 3 columns in the Parquet files together</span> |
| <span class="c1">// with the partitioning column appeared in the partition directory paths</span> |
| <span class="c1">// root</span> |
| <span class="c1">// |-- value: int (nullable = true)</span> |
| <span class="c1">// |-- square: int (nullable = true)</span> |
| <span class="c1">// |-- cube: int (nullable = true)</span> |
| <span class="c1">// |-- key: int (nullable = true)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.io.Serializable</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">java.util.ArrayList</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">java.util.Arrays</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">java.util.List</span><span class="o">;</span> |
| |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Dataset</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Row</span><span class="o">;</span> |
| |
| <span class="kd">public</span> <span class="kd">static</span> <span class="kd">class</span> <span class="nc">Square</span> <span class="kd">implements</span> <span class="n">Serializable</span> <span class="o">{</span> |
| <span class="kd">private</span> <span class="kt">int</span> <span class="n">value</span><span class="o">;</span> |
| <span class="kd">private</span> <span class="kt">int</span> <span class="n">square</span><span class="o">;</span> |
| |
| <span class="c1">// Getters and setters...</span> |
| |
| <span class="o">}</span> |
| |
| <span class="kd">public</span> <span class="kd">static</span> <span class="kd">class</span> <span class="nc">Cube</span> <span class="kd">implements</span> <span class="n">Serializable</span> <span class="o">{</span> |
| <span class="kd">private</span> <span class="kt">int</span> <span class="n">value</span><span class="o">;</span> |
| <span class="kd">private</span> <span class="kt">int</span> <span class="n">cube</span><span class="o">;</span> |
| |
| <span class="c1">// Getters and setters...</span> |
| |
| <span class="o">}</span> |
| |
| <span class="n">List</span><span class="o"><</span><span class="n">Square</span><span class="o">></span> <span class="n">squares</span> <span class="o">=</span> <span class="k">new</span> <span class="n">ArrayList</span><span class="o"><>();</span> |
| <span class="k">for</span> <span class="o">(</span><span class="kt">int</span> <span class="n">value</span> <span class="o">=</span> <span class="mi">1</span><span class="o">;</span> <span class="n">value</span> <span class="o"><=</span> <span class="mi">5</span><span class="o">;</span> <span class="n">value</span><span class="o">++)</span> <span class="o">{</span> |
| <span class="n">Square</span> <span class="n">square</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Square</span><span class="o">();</span> |
| <span class="n">square</span><span class="o">.</span><span class="na">setValue</span><span class="o">(</span><span class="n">value</span><span class="o">);</span> |
| <span class="n">square</span><span class="o">.</span><span class="na">setSquare</span><span class="o">(</span><span class="n">value</span> <span class="o">*</span> <span class="n">value</span><span class="o">);</span> |
| <span class="n">squares</span><span class="o">.</span><span class="na">add</span><span class="o">(</span><span class="n">square</span><span class="o">);</span> |
| <span class="o">}</span> |
| |
| <span class="c1">// Create a simple DataFrame, store into a partition directory</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">squaresDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">createDataFrame</span><span class="o">(</span><span class="n">squares</span><span class="o">,</span> <span class="n">Square</span><span class="o">.</span><span class="na">class</span><span class="o">);</span> |
| <span class="n">squaresDF</span><span class="o">.</span><span class="na">write</span><span class="o">().</span><span class="na">parquet</span><span class="o">(</span><span class="s">"data/test_table/key=1"</span><span class="o">);</span> |
| |
| <span class="n">List</span><span class="o"><</span><span class="n">Cube</span><span class="o">></span> <span class="n">cubes</span> <span class="o">=</span> <span class="k">new</span> <span class="n">ArrayList</span><span class="o"><>();</span> |
| <span class="k">for</span> <span class="o">(</span><span class="kt">int</span> <span class="n">value</span> <span class="o">=</span> <span class="mi">6</span><span class="o">;</span> <span class="n">value</span> <span class="o"><=</span> <span class="mi">10</span><span class="o">;</span> <span class="n">value</span><span class="o">++)</span> <span class="o">{</span> |
| <span class="n">Cube</span> <span class="n">cube</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Cube</span><span class="o">();</span> |
| <span class="n">cube</span><span class="o">.</span><span class="na">setValue</span><span class="o">(</span><span class="n">value</span><span class="o">);</span> |
| <span class="n">cube</span><span class="o">.</span><span class="na">setCube</span><span class="o">(</span><span class="n">value</span> <span class="o">*</span> <span class="n">value</span> <span class="o">*</span> <span class="n">value</span><span class="o">);</span> |
| <span class="n">cubes</span><span class="o">.</span><span class="na">add</span><span class="o">(</span><span class="n">cube</span><span class="o">);</span> |
| <span class="o">}</span> |
| |
| <span class="c1">// Create another DataFrame in a new partition directory,</span> |
| <span class="c1">// adding a new column and dropping an existing column</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">cubesDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">createDataFrame</span><span class="o">(</span><span class="n">cubes</span><span class="o">,</span> <span class="n">Cube</span><span class="o">.</span><span class="na">class</span><span class="o">);</span> |
| <span class="n">cubesDF</span><span class="o">.</span><span class="na">write</span><span class="o">().</span><span class="na">parquet</span><span class="o">(</span><span class="s">"data/test_table/key=2"</span><span class="o">);</span> |
| |
| <span class="c1">// Read the partitioned table</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">mergedDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">option</span><span class="o">(</span><span class="s">"mergeSchema"</span><span class="o">,</span> <span class="kc">true</span><span class="o">).</span><span class="na">parquet</span><span class="o">(</span><span class="s">"data/test_table"</span><span class="o">);</span> |
| <span class="n">mergedDF</span><span class="o">.</span><span class="na">printSchema</span><span class="o">();</span> |
| |
| <span class="c1">// The final schema consists of all 3 columns in the Parquet files together</span> |
| <span class="c1">// with the partitioning column appeared in the partition directory paths</span> |
| <span class="c1">// root</span> |
| <span class="c1">// |-- value: int (nullable = true)</span> |
| <span class="c1">// |-- square: int (nullable = true)</span> |
| <span class="c1">// |-- cube: int (nullable = true)</span> |
| <span class="c1">// |-- key: int (nullable = true)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| |
| <div class="highlight"><pre><span></span><span class="kn">from</span> <span class="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">Row</span> |
| |
| <span class="c1"># spark is from the previous example.</span> |
| <span class="c1"># Create a simple DataFrame, stored into a partition directory</span> |
| <span class="n">sc</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sparkContext</span> |
| |
| <span class="n">squaresDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">createDataFrame</span><span class="p">(</span><span class="n">sc</span><span class="o">.</span><span class="n">parallelize</span><span class="p">(</span><span class="nb">range</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="mi">6</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">i</span><span class="p">:</span> <span class="n">Row</span><span class="p">(</span><span class="n">single</span><span class="o">=</span><span class="n">i</span><span class="p">,</span> <span class="n">double</span><span class="o">=</span><span class="n">i</span> <span class="o">**</span> <span class="mi">2</span><span class="p">)))</span> |
| <span class="n">squaresDF</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">parquet</span><span class="p">(</span><span class="s2">"data/test_table/key=1"</span><span class="p">)</span> |
| |
| <span class="c1"># Create another DataFrame in a new partition directory,</span> |
| <span class="c1"># adding a new column and dropping an existing column</span> |
| <span class="n">cubesDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">createDataFrame</span><span class="p">(</span><span class="n">sc</span><span class="o">.</span><span class="n">parallelize</span><span class="p">(</span><span class="nb">range</span><span class="p">(</span><span class="mi">6</span><span class="p">,</span> <span class="mi">11</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">i</span><span class="p">:</span> <span class="n">Row</span><span class="p">(</span><span class="n">single</span><span class="o">=</span><span class="n">i</span><span class="p">,</span> <span class="n">triple</span><span class="o">=</span><span class="n">i</span> <span class="o">**</span> <span class="mi">3</span><span class="p">)))</span> |
| <span class="n">cubesDF</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">parquet</span><span class="p">(</span><span class="s2">"data/test_table/key=2"</span><span class="p">)</span> |
| |
| <span class="c1"># Read the partitioned table</span> |
| <span class="n">mergedDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"mergeSchema"</span><span class="p">,</span> <span class="s2">"true"</span><span class="p">)</span><span class="o">.</span><span class="n">parquet</span><span class="p">(</span><span class="s2">"data/test_table"</span><span class="p">)</span> |
| <span class="n">mergedDF</span><span class="o">.</span><span class="n">printSchema</span><span class="p">()</span> |
| |
| <span class="c1"># The final schema consists of all 3 columns in the Parquet files together</span> |
| <span class="c1"># with the partitioning column appeared in the partition directory paths.</span> |
| <span class="c1"># root</span> |
| <span class="c1"># |-- double: long (nullable = true)</span> |
| <span class="c1"># |-- single: long (nullable = true)</span> |
| <span class="c1"># |-- triple: long (nullable = true)</span> |
| <span class="c1"># |-- key: integer (nullable = true)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/datasource.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="r"> |
| |
| <div class="highlight"><pre><span></span>df1 <span class="o"><-</span> createDataFrame<span class="p">(</span><span class="kt">data.frame</span><span class="p">(</span>single<span class="o">=</span><span class="kt">c</span><span class="p">(</span><span class="m">12</span><span class="p">,</span> <span class="m">29</span><span class="p">),</span> <span class="kt">double</span><span class="o">=</span><span class="kt">c</span><span class="p">(</span><span class="m">19</span><span class="p">,</span> <span class="m">23</span><span class="p">)))</span> |
| df2 <span class="o"><-</span> createDataFrame<span class="p">(</span><span class="kt">data.frame</span><span class="p">(</span><span class="kt">double</span><span class="o">=</span><span class="kt">c</span><span class="p">(</span><span class="m">19</span><span class="p">,</span> <span class="m">23</span><span class="p">),</span> triple<span class="o">=</span><span class="kt">c</span><span class="p">(</span><span class="m">23</span><span class="p">,</span> <span class="m">18</span><span class="p">)))</span> |
| |
| <span class="c1"># Create a simple DataFrame, stored into a partition directory</span> |
| write.df<span class="p">(</span>df1<span class="p">,</span> <span class="s">"data/test_table/key=1"</span><span class="p">,</span> <span class="s">"parquet"</span><span class="p">,</span> <span class="s">"overwrite"</span><span class="p">)</span> |
| |
| <span class="c1"># Create another DataFrame in a new partition directory,</span> |
| <span class="c1"># adding a new column and dropping an existing column</span> |
| write.df<span class="p">(</span>df2<span class="p">,</span> <span class="s">"data/test_table/key=2"</span><span class="p">,</span> <span class="s">"parquet"</span><span class="p">,</span> <span class="s">"overwrite"</span><span class="p">)</span> |
| |
| <span class="c1"># Read the partitioned table</span> |
| df3 <span class="o"><-</span> read.df<span class="p">(</span><span class="s">"data/test_table"</span><span class="p">,</span> <span class="s">"parquet"</span><span class="p">,</span> mergeSchema <span class="o">=</span> <span class="s">"true"</span><span class="p">)</span> |
| printSchema<span class="p">(</span>df3<span class="p">)</span> |
| <span class="c1"># The final schema consists of all 3 columns in the Parquet files together</span> |
| <span class="c1"># with the partitioning column appeared in the partition directory paths</span> |
| <span class="c1">## root</span> |
| <span class="c1">## |-- single: double (nullable = true)</span> |
| <span class="c1">## |-- double: double (nullable = true)</span> |
| <span class="c1">## |-- triple: double (nullable = true)</span> |
| <span class="c1">## |-- key: integer (nullable = true)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| |
| </div> |
| |
| </div> |
| |
| <h3 id="hive-metastore-parquet-table-conversion">Hive metastore Parquet table conversion</h3> |
| |
| <p>When reading from and writing to Hive metastore Parquet tables, Spark SQL will try to use its own |
| Parquet support instead of Hive SerDe for better performance. This behavior is controlled by the |
| <code>spark.sql.hive.convertMetastoreParquet</code> configuration, and is turned on by default.</p> |
| |
| <h4 id="hiveparquet-schema-reconciliation">Hive/Parquet Schema Reconciliation</h4> |
| |
| <p>There are two key differences between Hive and Parquet from the perspective of table schema |
| processing.</p> |
| |
| <ol> |
| <li>Hive is case insensitive, while Parquet is not</li> |
| <li>Hive considers all columns nullable, while nullability in Parquet is significant</li> |
| </ol> |
| |
| <p>Due to this reason, we must reconcile Hive metastore schema with Parquet schema when converting a |
| Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation rules are:</p> |
| |
| <ol> |
| <li> |
| <p>Fields that have the same name in both schema must have the same data type regardless of |
| nullability. The reconciled field should have the data type of the Parquet side, so that |
| nullability is respected.</p> |
| </li> |
| <li> |
| <p>The reconciled schema contains exactly those fields defined in Hive metastore schema.</p> |
| |
| <ul> |
| <li>Any fields that only appear in the Parquet schema are dropped in the reconciled schema.</li> |
| <li>Any fields that only appear in the Hive metastore schema are added as nullable field in the |
| reconciled schema.</li> |
| </ul> |
| </li> |
| </ol> |
| |
| <h4 id="metadata-refreshing">Metadata Refreshing</h4> |
| |
| <p>Spark SQL caches Parquet metadata for better performance. When Hive metastore Parquet table |
| conversion is enabled, metadata of those converted tables are also cached. If these tables are |
| updated by Hive or other external tools, you need to refresh them manually to ensure consistent |
| metadata.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span></span><span class="c1">// spark is an existing SparkSession</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">catalog</span><span class="o">.</span><span class="n">refreshTable</span><span class="o">(</span><span class="s">"my_table"</span><span class="o">)</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span></span><span class="c1">// spark is an existing SparkSession</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">catalog</span><span class="o">().</span><span class="na">refreshTable</span><span class="o">(</span><span class="s">"my_table"</span><span class="o">);</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span></span><span class="c1"># spark is an existing SparkSession</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">catalog</span><span class="o">.</span><span class="n">refreshTable</span><span class="p">(</span><span class="s2">"my_table"</span><span class="p">)</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="r"> |
| |
| <figure class="highlight"><pre><code class="language-r" data-lang="r"><span></span>refreshTable<span class="p">(</span><span class="s">"my_table"</span><span class="p">)</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="sql"> |
| |
| <figure class="highlight"><pre><code class="language-sql" data-lang="sql"><span></span><span class="n">REFRESH</span> <span class="k">TABLE</span> <span class="n">my_table</span><span class="p">;</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h3 id="configuration">Configuration</h3> |
| |
| <p>Configuration of Parquet can be done using the <code>setConf</code> method on <code>SparkSession</code> or by running |
| <code>SET key=value</code> commands using SQL.</p> |
| |
| <table class="table"> |
| <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr> |
| <tr> |
| <td><code>spark.sql.parquet.binaryAsString</code></td> |
| <td>false</td> |
| <td> |
| Some other Parquet-producing systems, in particular Impala, Hive, and older versions of Spark SQL, do |
| not differentiate between binary data and strings when writing out the Parquet schema. This |
| flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems. |
| </td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.parquet.int96AsTimestamp</code></td> |
| <td>true</td> |
| <td> |
| Some Parquet-producing systems, in particular Impala and Hive, store Timestamp into INT96. This |
| flag tells Spark SQL to interpret INT96 data as a timestamp to provide compatibility with these systems. |
| </td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.parquet.compression.codec</code></td> |
| <td>snappy</td> |
| <td> |
| Sets the compression codec used when writing Parquet files. If either `compression` or |
| `parquet.compression` is specified in the table-specific options/properties, the precedence would be |
| `compression`, `parquet.compression`, `spark.sql.parquet.compression.codec`. Acceptable values include: |
| none, uncompressed, snappy, gzip, lzo. |
| </td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.parquet.filterPushdown</code></td> |
| <td>true</td> |
| <td>Enables Parquet filter push-down optimization when set to true.</td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.hive.convertMetastoreParquet</code></td> |
| <td>true</td> |
| <td> |
| When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of the built in |
| support. |
| </td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.parquet.mergeSchema</code></td> |
| <td>false</td> |
| <td> |
| <p> |
| When true, the Parquet data source merges schemas collected from all data files, otherwise the |
| schema is picked from the summary file or a random data file if no summary file is available. |
| </p> |
| </td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.optimizer.metadataOnly</code></td> |
| <td>true</td> |
| <td> |
| <p> |
| When true, enable the metadata-only query optimization that use the table's metadata to |
| produce the partition columns instead of table scans. It applies when all the columns scanned |
| are partition columns and the query has an aggregate operator that satisfies distinct |
| semantics. |
| </p> |
| </td> |
| </tr> |
| </table> |
| |
| <h2 id="orc-files">ORC Files</h2> |
| |
| <p>Since Spark 2.3, Spark supports a vectorized ORC reader with a new ORC file format for ORC files. |
| To do that, the following configurations are newly added. The vectorized reader is used for the |
| native ORC tables (e.g., the ones created using the clause <code>USING ORC</code>) when <code>spark.sql.orc.impl</code> |
| is set to <code>native</code> and <code>spark.sql.orc.enableVectorizedReader</code> is set to <code>true</code>. For the Hive ORC |
| serde tables (e.g., the ones created using the clause <code>USING HIVE OPTIONS (fileFormat 'ORC')</code>), |
| the vectorized reader is used when <code>spark.sql.hive.convertMetastoreOrc</code> is also set to <code>true</code>.</p> |
| |
| <table class="table"> |
| <tr><th><b>Property Name</b></th><th><b>Default</b></th><th><b>Meaning</b></th></tr> |
| <tr> |
| <td><code>spark.sql.orc.impl</code></td> |
| <td><code>hive</code></td> |
| <td>The name of ORC implementation. It can be one of <code>native</code> and <code>hive</code>. <code>native</code> means the native ORC support that is built on Apache ORC 1.4.1. `hive` means the ORC library in Hive 1.2.1.</td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.orc.enableVectorizedReader</code></td> |
| <td><code>true</code></td> |
| <td>Enables vectorized orc decoding in <code>native</code> implementation. If <code>false</code>, a new non-vectorized ORC reader is used in <code>native</code> implementation. For <code>hive</code> implementation, this is ignored.</td> |
| </tr> |
| </table> |
| |
| <h2 id="json-datasets">JSON Datasets</h2> |
| <div class="codetabs"> |
| |
| <div data-lang="scala"> |
| <p>Spark SQL can automatically infer the schema of a JSON dataset and load it as a <code>Dataset[Row]</code>. |
| This conversion can be done using <code>SparkSession.read.json()</code> on either a <code>Dataset[String]</code>, |
| or a JSON file.</p> |
| |
| <p>Note that the file that is offered as <em>a json file</em> is not a typical JSON file. Each |
| line 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>.</p> |
| |
| <p>For a regular multi-line JSON file, set the <code>multiLine</code> option to <code>true</code>.</p> |
| |
| <div class="highlight"><pre><span></span><span class="c1">// Primitive types (Int, String, etc) and Product types (case classes) encoders are</span> |
| <span class="c1">// supported by importing this when creating a Dataset.</span> |
| <span class="k">import</span> <span class="nn">spark.implicits._</span> |
| |
| <span class="c1">// A JSON dataset is pointed to by path.</span> |
| <span class="c1">// The path can be either a single text file or a directory storing text files</span> |
| <span class="k">val</span> <span class="n">path</span> <span class="k">=</span> <span class="s">"examples/src/main/resources/people.json"</span> |
| <span class="k">val</span> <span class="n">peopleDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">json</span><span class="o">(</span><span class="n">path</span><span class="o">)</span> |
| |
| <span class="c1">// The inferred schema can be visualized using the printSchema() method</span> |
| <span class="n">peopleDF</span><span class="o">.</span><span class="n">printSchema</span><span class="o">()</span> |
| <span class="c1">// root</span> |
| <span class="c1">// |-- age: long (nullable = true)</span> |
| <span class="c1">// |-- name: string (nullable = true)</span> |
| |
| <span class="c1">// Creates a temporary view using the DataFrame</span> |
| <span class="n">peopleDF</span><span class="o">.</span><span class="n">createOrReplaceTempView</span><span class="o">(</span><span class="s">"people"</span><span class="o">)</span> |
| |
| <span class="c1">// SQL statements can be run by using the sql methods provided by spark</span> |
| <span class="k">val</span> <span class="n">teenagerNamesDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="o">(</span><span class="s">"SELECT name FROM people WHERE age BETWEEN 13 AND 19"</span><span class="o">)</span> |
| <span class="n">teenagerNamesDF</span><span class="o">.</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +------+</span> |
| <span class="c1">// | name|</span> |
| <span class="c1">// +------+</span> |
| <span class="c1">// |Justin|</span> |
| <span class="c1">// +------+</span> |
| |
| <span class="c1">// Alternatively, a DataFrame can be created for a JSON dataset represented by</span> |
| <span class="c1">// a Dataset[String] storing one JSON object per string</span> |
| <span class="k">val</span> <span class="n">otherPeopleDataset</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">createDataset</span><span class="o">(</span> |
| <span class="s">"""{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}"""</span> <span class="o">::</span> <span class="nc">Nil</span><span class="o">)</span> |
| <span class="k">val</span> <span class="n">otherPeople</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">json</span><span class="o">(</span><span class="n">otherPeopleDataset</span><span class="o">)</span> |
| <span class="n">otherPeople</span><span class="o">.</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +---------------+----+</span> |
| <span class="c1">// | address|name|</span> |
| <span class="c1">// +---------------+----+</span> |
| <span class="c1">// |[Columbus,Ohio]| Yin|</span> |
| <span class="c1">// +---------------+----+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <p>Spark SQL can automatically infer the schema of a JSON dataset and load it as a <code>Dataset<Row></code>. |
| This conversion can be done using <code>SparkSession.read().json()</code> on either a <code>Dataset<String></code>, |
| or a JSON file.</p> |
| |
| <p>Note that the file that is offered as <em>a json file</em> is not a typical JSON file. Each |
| line 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>.</p> |
| |
| <p>For a regular multi-line JSON file, set the <code>multiLine</code> option to <code>true</code>.</p> |
| |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">org.apache.spark.sql.Dataset</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Row</span><span class="o">;</span> |
| |
| <span class="c1">// A JSON dataset is pointed to by path.</span> |
| <span class="c1">// The path can be either a single text file or a directory storing text files</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">people</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">json</span><span class="o">(</span><span class="s">"examples/src/main/resources/people.json"</span><span class="o">);</span> |
| |
| <span class="c1">// The inferred schema can be visualized using the printSchema() method</span> |
| <span class="n">people</span><span class="o">.</span><span class="na">printSchema</span><span class="o">();</span> |
| <span class="c1">// root</span> |
| <span class="c1">// |-- age: long (nullable = true)</span> |
| <span class="c1">// |-- name: string (nullable = true)</span> |
| |
| <span class="c1">// Creates a temporary view using the DataFrame</span> |
| <span class="n">people</span><span class="o">.</span><span class="na">createOrReplaceTempView</span><span class="o">(</span><span class="s">"people"</span><span class="o">);</span> |
| |
| <span class="c1">// SQL statements can be run by using the sql methods provided by spark</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">namesDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT name FROM people WHERE age BETWEEN 13 AND 19"</span><span class="o">);</span> |
| <span class="n">namesDF</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +------+</span> |
| <span class="c1">// | name|</span> |
| <span class="c1">// +------+</span> |
| <span class="c1">// |Justin|</span> |
| <span class="c1">// +------+</span> |
| |
| <span class="c1">// Alternatively, a DataFrame can be created for a JSON dataset represented by</span> |
| <span class="c1">// a Dataset<String> storing one JSON object per string.</span> |
| <span class="n">List</span><span class="o"><</span><span class="n">String</span><span class="o">></span> <span class="n">jsonData</span> <span class="o">=</span> <span class="n">Arrays</span><span class="o">.</span><span class="na">asList</span><span class="o">(</span> |
| <span class="s">"{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"</span><span class="o">);</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">String</span><span class="o">></span> <span class="n">anotherPeopleDataset</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">createDataset</span><span class="o">(</span><span class="n">jsonData</span><span class="o">,</span> <span class="n">Encoders</span><span class="o">.</span><span class="na">STRING</span><span class="o">());</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">anotherPeople</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">().</span><span class="na">json</span><span class="o">(</span><span class="n">anotherPeopleDataset</span><span class="o">);</span> |
| <span class="n">anotherPeople</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +---------------+----+</span> |
| <span class="c1">// | address|name|</span> |
| <span class="c1">// +---------------+----+</span> |
| <span class="c1">// |[Columbus,Ohio]| Yin|</span> |
| <span class="c1">// +---------------+----+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| <p>Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. |
| This conversion can be done using <code>SparkSession.read.json</code> on a JSON file.</p> |
| |
| <p>Note that the file that is offered as <em>a json file</em> is not a typical JSON file. Each |
| line 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>.</p> |
| |
| <p>For a regular multi-line JSON file, set the <code>multiLine</code> parameter to <code>True</code>.</p> |
| |
| <div class="highlight"><pre><span></span><span class="c1"># spark is from the previous example.</span> |
| <span class="n">sc</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sparkContext</span> |
| |
| <span class="c1"># A JSON dataset is pointed to by path.</span> |
| <span class="c1"># The path can be either a single text file or a directory storing text files</span> |
| <span class="n">path</span> <span class="o">=</span> <span class="s2">"examples/src/main/resources/people.json"</span> |
| <span class="n">peopleDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">json</span><span class="p">(</span><span class="n">path</span><span class="p">)</span> |
| |
| <span class="c1"># The inferred schema can be visualized using the printSchema() method</span> |
| <span class="n">peopleDF</span><span class="o">.</span><span class="n">printSchema</span><span class="p">()</span> |
| <span class="c1"># root</span> |
| <span class="c1"># |-- age: long (nullable = true)</span> |
| <span class="c1"># |-- name: string (nullable = true)</span> |
| |
| <span class="c1"># Creates a temporary view using the DataFrame</span> |
| <span class="n">peopleDF</span><span class="o">.</span><span class="n">createOrReplaceTempView</span><span class="p">(</span><span class="s2">"people"</span><span class="p">)</span> |
| |
| <span class="c1"># SQL statements can be run by using the sql methods provided by spark</span> |
| <span class="n">teenagerNamesDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT name FROM people WHERE age BETWEEN 13 AND 19"</span><span class="p">)</span> |
| <span class="n">teenagerNamesDF</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +------+</span> |
| <span class="c1"># | name|</span> |
| <span class="c1"># +------+</span> |
| <span class="c1"># |Justin|</span> |
| <span class="c1"># +------+</span> |
| |
| <span class="c1"># Alternatively, a DataFrame can be created for a JSON dataset represented by</span> |
| <span class="c1"># an RDD[String] storing one JSON object per string</span> |
| <span class="n">jsonStrings</span> <span class="o">=</span> <span class="p">[</span><span class="s1">'{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}'</span><span class="p">]</span> |
| <span class="n">otherPeopleRDD</span> <span class="o">=</span> <span class="n">sc</span><span class="o">.</span><span class="n">parallelize</span><span class="p">(</span><span class="n">jsonStrings</span><span class="p">)</span> |
| <span class="n">otherPeople</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span><span class="o">.</span><span class="n">json</span><span class="p">(</span><span class="n">otherPeopleRDD</span><span class="p">)</span> |
| <span class="n">otherPeople</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +---------------+----+</span> |
| <span class="c1"># | address|name|</span> |
| <span class="c1"># +---------------+----+</span> |
| <span class="c1"># |[Columbus,Ohio]| Yin|</span> |
| <span class="c1"># +---------------+----+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/datasource.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="r"> |
| <p>Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. using |
| the <code>read.json()</code> function, which loads data from a directory of JSON files where each line of the |
| files is a JSON object.</p> |
| |
| <p>Note that the file that is offered as <em>a json file</em> is not a typical JSON file. Each |
| line 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>.</p> |
| |
| <p>For a regular multi-line JSON file, set a named parameter <code>multiLine</code> to <code>TRUE</code>.</p> |
| |
| <div class="highlight"><pre><span></span><span class="c1"># A JSON dataset is pointed to by path.</span> |
| <span class="c1"># The path can be either a single text file or a directory storing text files.</span> |
| path <span class="o"><-</span> <span class="s">"examples/src/main/resources/people.json"</span> |
| <span class="c1"># Create a DataFrame from the file(s) pointed to by path</span> |
| people <span class="o"><-</span> read.json<span class="p">(</span>path<span class="p">)</span> |
| |
| <span class="c1"># The inferred schema can be visualized using the printSchema() method.</span> |
| printSchema<span class="p">(</span>people<span class="p">)</span> |
| <span class="c1">## root</span> |
| <span class="c1">## |-- age: long (nullable = true)</span> |
| <span class="c1">## |-- name: string (nullable = true)</span> |
| |
| <span class="c1"># Register this DataFrame as a table.</span> |
| createOrReplaceTempView<span class="p">(</span>people<span class="p">,</span> <span class="s">"people"</span><span class="p">)</span> |
| |
| <span class="c1"># SQL statements can be run by using the sql methods.</span> |
| teenagers <span class="o"><-</span> sql<span class="p">(</span><span class="s">"SELECT name FROM people WHERE age >= 13 AND age <= 19"</span><span class="p">)</span> |
| <span class="kp">head</span><span class="p">(</span>teenagers<span class="p">)</span> |
| <span class="c1">## name</span> |
| <span class="c1">## 1 Justin</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| |
| </div> |
| |
| <div data-lang="sql"> |
| |
| <figure class="highlight"><pre><code class="language-sql" data-lang="sql"><span></span><span class="k">CREATE</span> <span class="k">TEMPORARY</span> <span class="k">VIEW</span> <span class="n">jsonTable</span> |
| <span class="k">USING</span> <span class="n">org</span><span class="p">.</span><span class="n">apache</span><span class="p">.</span><span class="n">spark</span><span class="p">.</span><span class="k">sql</span><span class="p">.</span><span class="n">json</span> |
| <span class="k">OPTIONS</span> <span class="p">(</span> |
| <span class="n">path</span> <span class="ss">"examples/src/main/resources/people.json"</span> |
| <span class="p">)</span> |
| |
| <span class="k">SELECT</span> <span class="o">*</span> <span class="k">FROM</span> <span class="n">jsonTable</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h2 id="hive-tables">Hive Tables</h2> |
| |
| <p>Spark SQL also supports reading and writing data stored in <a href="http://hive.apache.org/">Apache Hive</a>. |
| However, since Hive has a large number of dependencies, these dependencies are not included in the |
| default Spark distribution. If Hive dependencies can be found on the classpath, Spark will load them |
| automatically. Note that these Hive dependencies must also be present on all of the worker nodes, as |
| they will need access to the Hive serialization and deserialization libraries (SerDes) in order to |
| access data stored in Hive.</p> |
| |
| <p>Configuration of Hive is done by placing your <code>hive-site.xml</code>, <code>core-site.xml</code> (for security configuration), |
| and <code>hdfs-site.xml</code> (for HDFS configuration) file in <code>conf/</code>.</p> |
| |
| <p>When working with Hive, one must instantiate <code>SparkSession</code> with Hive support, including |
| connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. |
| Users who do not have an existing Hive deployment can still enable Hive support. When not configured |
| by the <code>hive-site.xml</code>, the context automatically creates <code>metastore_db</code> in the current directory and |
| creates a directory configured by <code>spark.sql.warehouse.dir</code>, which defaults to the directory |
| <code>spark-warehouse</code> in the current directory that the Spark application is started. Note that |
| the <code>hive.metastore.warehouse.dir</code> property in <code>hive-site.xml</code> is deprecated since Spark 2.0.0. |
| Instead, use <code>spark.sql.warehouse.dir</code> to specify the default location of database in warehouse. |
| You may need to grant write privilege to the user who starts the Spark application.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="k">import</span> <span class="nn">java.io.File</span> |
| |
| <span class="k">import</span> <span class="nn">org.apache.spark.sql.</span><span class="o">{</span><span class="nc">Row</span><span class="o">,</span> <span class="nc">SaveMode</span><span class="o">,</span> <span class="nc">SparkSession</span><span class="o">}</span> |
| |
| <span class="k">case</span> <span class="k">class</span> <span class="nc">Record</span><span class="o">(</span><span class="n">key</span><span class="k">:</span> <span class="kt">Int</span><span class="o">,</span> <span class="n">value</span><span class="k">:</span> <span class="kt">String</span><span class="o">)</span> |
| |
| <span class="c1">// warehouseLocation points to the default location for managed databases and tables</span> |
| <span class="k">val</span> <span class="n">warehouseLocation</span> <span class="k">=</span> <span class="k">new</span> <span class="nc">File</span><span class="o">(</span><span class="s">"spark-warehouse"</span><span class="o">).</span><span class="n">getAbsolutePath</span> |
| |
| <span class="k">val</span> <span class="n">spark</span> <span class="k">=</span> <span class="nc">SparkSession</span> |
| <span class="o">.</span><span class="n">builder</span><span class="o">()</span> |
| <span class="o">.</span><span class="n">appName</span><span class="o">(</span><span class="s">"Spark Hive Example"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">config</span><span class="o">(</span><span class="s">"spark.sql.warehouse.dir"</span><span class="o">,</span> <span class="n">warehouseLocation</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">enableHiveSupport</span><span class="o">()</span> |
| <span class="o">.</span><span class="n">getOrCreate</span><span class="o">()</span> |
| |
| <span class="k">import</span> <span class="nn">spark.implicits._</span> |
| <span class="k">import</span> <span class="nn">spark.sql</span> |
| |
| <span class="n">sql</span><span class="o">(</span><span class="s">"CREATE TABLE IF NOT EXISTS src (key INT, value STRING) USING hive"</span><span class="o">)</span> |
| <span class="n">sql</span><span class="o">(</span><span class="s">"LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"</span><span class="o">)</span> |
| |
| <span class="c1">// Queries are expressed in HiveQL</span> |
| <span class="n">sql</span><span class="o">(</span><span class="s">"SELECT * FROM src"</span><span class="o">).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +---+-------+</span> |
| <span class="c1">// |key| value|</span> |
| <span class="c1">// +---+-------+</span> |
| <span class="c1">// |238|val_238|</span> |
| <span class="c1">// | 86| val_86|</span> |
| <span class="c1">// |311|val_311|</span> |
| <span class="c1">// ...</span> |
| |
| <span class="c1">// Aggregation queries are also supported.</span> |
| <span class="n">sql</span><span class="o">(</span><span class="s">"SELECT COUNT(*) FROM src"</span><span class="o">).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +--------+</span> |
| <span class="c1">// |count(1)|</span> |
| <span class="c1">// +--------+</span> |
| <span class="c1">// | 500 |</span> |
| <span class="c1">// +--------+</span> |
| |
| <span class="c1">// The results of SQL queries are themselves DataFrames and support all normal functions.</span> |
| <span class="k">val</span> <span class="n">sqlDF</span> <span class="k">=</span> <span class="n">sql</span><span class="o">(</span><span class="s">"SELECT key, value FROM src WHERE key < 10 ORDER BY key"</span><span class="o">)</span> |
| |
| <span class="c1">// The items in DataFrames are of type Row, which allows you to access each column by ordinal.</span> |
| <span class="k">val</span> <span class="n">stringsDS</span> <span class="k">=</span> <span class="n">sqlDF</span><span class="o">.</span><span class="n">map</span> <span class="o">{</span> |
| <span class="k">case</span> <span class="nc">Row</span><span class="o">(</span><span class="n">key</span><span class="k">:</span> <span class="kt">Int</span><span class="o">,</span> <span class="n">value</span><span class="k">:</span> <span class="kt">String</span><span class="o">)</span> <span class="k">=></span> <span class="s">s"Key: </span><span class="si">$key</span><span class="s">, Value: </span><span class="si">$value</span><span class="s">"</span> |
| <span class="o">}</span> |
| <span class="n">stringsDS</span><span class="o">.</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +--------------------+</span> |
| <span class="c1">// | value|</span> |
| <span class="c1">// +--------------------+</span> |
| <span class="c1">// |Key: 0, Value: val_0|</span> |
| <span class="c1">// |Key: 0, Value: val_0|</span> |
| <span class="c1">// |Key: 0, Value: val_0|</span> |
| <span class="c1">// ...</span> |
| |
| <span class="c1">// You can also use DataFrames to create temporary views within a SparkSession.</span> |
| <span class="k">val</span> <span class="n">recordsDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">createDataFrame</span><span class="o">((</span><span class="mi">1</span> <span class="n">to</span> <span class="mi">100</span><span class="o">).</span><span class="n">map</span><span class="o">(</span><span class="n">i</span> <span class="k">=></span> <span class="nc">Record</span><span class="o">(</span><span class="n">i</span><span class="o">,</span> <span class="s">s"val_</span><span class="si">$i</span><span class="s">"</span><span class="o">)))</span> |
| <span class="n">recordsDF</span><span class="o">.</span><span class="n">createOrReplaceTempView</span><span class="o">(</span><span class="s">"records"</span><span class="o">)</span> |
| |
| <span class="c1">// Queries can then join DataFrame data with data stored in Hive.</span> |
| <span class="n">sql</span><span class="o">(</span><span class="s">"SELECT * FROM records r JOIN src s ON r.key = s.key"</span><span class="o">).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +---+------+---+------+</span> |
| <span class="c1">// |key| value|key| value|</span> |
| <span class="c1">// +---+------+---+------+</span> |
| <span class="c1">// | 2| val_2| 2| val_2|</span> |
| <span class="c1">// | 4| val_4| 4| val_4|</span> |
| <span class="c1">// | 5| val_5| 5| val_5|</span> |
| <span class="c1">// ...</span> |
| |
| <span class="c1">// Create a Hive managed Parquet table, with HQL syntax instead of the Spark SQL native syntax</span> |
| <span class="c1">// `USING hive`</span> |
| <span class="n">sql</span><span class="o">(</span><span class="s">"CREATE TABLE hive_records(key int, value string) STORED AS PARQUET"</span><span class="o">)</span> |
| <span class="c1">// Save DataFrame to the Hive managed table</span> |
| <span class="k">val</span> <span class="n">df</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">table</span><span class="o">(</span><span class="s">"src"</span><span class="o">)</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">mode</span><span class="o">(</span><span class="nc">SaveMode</span><span class="o">.</span><span class="nc">Overwrite</span><span class="o">).</span><span class="n">saveAsTable</span><span class="o">(</span><span class="s">"hive_records"</span><span class="o">)</span> |
| <span class="c1">// After insertion, the Hive managed table has data now</span> |
| <span class="n">sql</span><span class="o">(</span><span class="s">"SELECT * FROM hive_records"</span><span class="o">).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +---+-------+</span> |
| <span class="c1">// |key| value|</span> |
| <span class="c1">// +---+-------+</span> |
| <span class="c1">// |238|val_238|</span> |
| <span class="c1">// | 86| val_86|</span> |
| <span class="c1">// |311|val_311|</span> |
| <span class="c1">// ...</span> |
| |
| <span class="c1">// Prepare a Parquet data directory</span> |
| <span class="k">val</span> <span class="n">dataDir</span> <span class="k">=</span> <span class="s">"/tmp/parquet_data"</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">range</span><span class="o">(</span><span class="mi">10</span><span class="o">).</span><span class="n">write</span><span class="o">.</span><span class="n">parquet</span><span class="o">(</span><span class="n">dataDir</span><span class="o">)</span> |
| <span class="c1">// Create a Hive external Parquet table</span> |
| <span class="n">sql</span><span class="o">(</span><span class="s">s"CREATE EXTERNAL TABLE hive_ints(key int) STORED AS PARQUET LOCATION '</span><span class="si">$dataDir</span><span class="s">'"</span><span class="o">)</span> |
| <span class="c1">// The Hive external table should already have data</span> |
| <span class="n">sql</span><span class="o">(</span><span class="s">"SELECT * FROM hive_ints"</span><span class="o">).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +---+</span> |
| <span class="c1">// |key|</span> |
| <span class="c1">// +---+</span> |
| <span class="c1">// | 0|</span> |
| <span class="c1">// | 1|</span> |
| <span class="c1">// | 2|</span> |
| <span class="c1">// ...</span> |
| |
| <span class="c1">// Turn on flag for Hive Dynamic Partitioning</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">sqlContext</span><span class="o">.</span><span class="n">setConf</span><span class="o">(</span><span class="s">"hive.exec.dynamic.partition"</span><span class="o">,</span> <span class="s">"true"</span><span class="o">)</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">sqlContext</span><span class="o">.</span><span class="n">setConf</span><span class="o">(</span><span class="s">"hive.exec.dynamic.partition.mode"</span><span class="o">,</span> <span class="s">"nonstrict"</span><span class="o">)</span> |
| <span class="c1">// Create a Hive partitioned table using DataFrame API</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">partitionBy</span><span class="o">(</span><span class="s">"key"</span><span class="o">).</span><span class="n">format</span><span class="o">(</span><span class="s">"hive"</span><span class="o">).</span><span class="n">saveAsTable</span><span class="o">(</span><span class="s">"hive_part_tbl"</span><span class="o">)</span> |
| <span class="c1">// Partitioned column `key` will be moved to the end of the schema.</span> |
| <span class="n">sql</span><span class="o">(</span><span class="s">"SELECT * FROM hive_part_tbl"</span><span class="o">).</span><span class="n">show</span><span class="o">()</span> |
| <span class="c1">// +-------+---+</span> |
| <span class="c1">// | value|key|</span> |
| <span class="c1">// +-------+---+</span> |
| <span class="c1">// |val_238|238|</span> |
| <span class="c1">// | val_86| 86|</span> |
| <span class="c1">// |val_311|311|</span> |
| <span class="c1">// ...</span> |
| |
| <span class="n">spark</span><span class="o">.</span><span class="n">stop</span><span class="o">()</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">java.io.File</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">java.io.Serializable</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">java.util.ArrayList</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">java.util.List</span><span class="o">;</span> |
| |
| <span class="kn">import</span> <span class="nn">org.apache.spark.api.java.function.MapFunction</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Dataset</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Encoders</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.Row</span><span class="o">;</span> |
| <span class="kn">import</span> <span class="nn">org.apache.spark.sql.SparkSession</span><span class="o">;</span> |
| |
| <span class="kd">public</span> <span class="kd">static</span> <span class="kd">class</span> <span class="nc">Record</span> <span class="kd">implements</span> <span class="n">Serializable</span> <span class="o">{</span> |
| <span class="kd">private</span> <span class="kt">int</span> <span class="n">key</span><span class="o">;</span> |
| <span class="kd">private</span> <span class="n">String</span> <span class="n">value</span><span class="o">;</span> |
| |
| <span class="kd">public</span> <span class="kt">int</span> <span class="nf">getKey</span><span class="o">()</span> <span class="o">{</span> |
| <span class="k">return</span> <span class="n">key</span><span class="o">;</span> |
| <span class="o">}</span> |
| |
| <span class="kd">public</span> <span class="kt">void</span> <span class="nf">setKey</span><span class="o">(</span><span class="kt">int</span> <span class="n">key</span><span class="o">)</span> <span class="o">{</span> |
| <span class="k">this</span><span class="o">.</span><span class="na">key</span> <span class="o">=</span> <span class="n">key</span><span class="o">;</span> |
| <span class="o">}</span> |
| |
| <span class="kd">public</span> <span class="n">String</span> <span class="nf">getValue</span><span class="o">()</span> <span class="o">{</span> |
| <span class="k">return</span> <span class="n">value</span><span class="o">;</span> |
| <span class="o">}</span> |
| |
| <span class="kd">public</span> <span class="kt">void</span> <span class="nf">setValue</span><span class="o">(</span><span class="n">String</span> <span class="n">value</span><span class="o">)</span> <span class="o">{</span> |
| <span class="k">this</span><span class="o">.</span><span class="na">value</span> <span class="o">=</span> <span class="n">value</span><span class="o">;</span> |
| <span class="o">}</span> |
| <span class="o">}</span> |
| |
| <span class="c1">// warehouseLocation points to the default location for managed databases and tables</span> |
| <span class="n">String</span> <span class="n">warehouseLocation</span> <span class="o">=</span> <span class="k">new</span> <span class="n">File</span><span class="o">(</span><span class="s">"spark-warehouse"</span><span class="o">).</span><span class="na">getAbsolutePath</span><span class="o">();</span> |
| <span class="n">SparkSession</span> <span class="n">spark</span> <span class="o">=</span> <span class="n">SparkSession</span> |
| <span class="o">.</span><span class="na">builder</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">appName</span><span class="o">(</span><span class="s">"Java Spark Hive Example"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">config</span><span class="o">(</span><span class="s">"spark.sql.warehouse.dir"</span><span class="o">,</span> <span class="n">warehouseLocation</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">enableHiveSupport</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">getOrCreate</span><span class="o">();</span> |
| |
| <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"CREATE TABLE IF NOT EXISTS src (key INT, value STRING) USING hive"</span><span class="o">);</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"</span><span class="o">);</span> |
| |
| <span class="c1">// Queries are expressed in HiveQL</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT * FROM src"</span><span class="o">).</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +---+-------+</span> |
| <span class="c1">// |key| value|</span> |
| <span class="c1">// +---+-------+</span> |
| <span class="c1">// |238|val_238|</span> |
| <span class="c1">// | 86| val_86|</span> |
| <span class="c1">// |311|val_311|</span> |
| <span class="c1">// ...</span> |
| |
| <span class="c1">// Aggregation queries are also supported.</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT COUNT(*) FROM src"</span><span class="o">).</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +--------+</span> |
| <span class="c1">// |count(1)|</span> |
| <span class="c1">// +--------+</span> |
| <span class="c1">// | 500 |</span> |
| <span class="c1">// +--------+</span> |
| |
| <span class="c1">// The results of SQL queries are themselves DataFrames and support all normal functions.</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">sqlDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT key, value FROM src WHERE key < 10 ORDER BY key"</span><span class="o">);</span> |
| |
| <span class="c1">// The items in DataFrames are of type Row, which lets you to access each column by ordinal.</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">String</span><span class="o">></span> <span class="n">stringsDS</span> <span class="o">=</span> <span class="n">sqlDF</span><span class="o">.</span><span class="na">map</span><span class="o">(</span> |
| <span class="o">(</span><span class="n">MapFunction</span><span class="o"><</span><span class="n">Row</span><span class="o">,</span> <span class="n">String</span><span class="o">>)</span> <span class="n">row</span> <span class="o">-></span> <span class="s">"Key: "</span> <span class="o">+</span> <span class="n">row</span><span class="o">.</span><span class="na">get</span><span class="o">(</span><span class="mi">0</span><span class="o">)</span> <span class="o">+</span> <span class="s">", Value: "</span> <span class="o">+</span> <span class="n">row</span><span class="o">.</span><span class="na">get</span><span class="o">(</span><span class="mi">1</span><span class="o">),</span> |
| <span class="n">Encoders</span><span class="o">.</span><span class="na">STRING</span><span class="o">());</span> |
| <span class="n">stringsDS</span><span class="o">.</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +--------------------+</span> |
| <span class="c1">// | value|</span> |
| <span class="c1">// +--------------------+</span> |
| <span class="c1">// |Key: 0, Value: val_0|</span> |
| <span class="c1">// |Key: 0, Value: val_0|</span> |
| <span class="c1">// |Key: 0, Value: val_0|</span> |
| <span class="c1">// ...</span> |
| |
| <span class="c1">// You can also use DataFrames to create temporary views within a SparkSession.</span> |
| <span class="n">List</span><span class="o"><</span><span class="n">Record</span><span class="o">></span> <span class="n">records</span> <span class="o">=</span> <span class="k">new</span> <span class="n">ArrayList</span><span class="o"><>();</span> |
| <span class="k">for</span> <span class="o">(</span><span class="kt">int</span> <span class="n">key</span> <span class="o">=</span> <span class="mi">1</span><span class="o">;</span> <span class="n">key</span> <span class="o"><</span> <span class="mi">100</span><span class="o">;</span> <span class="n">key</span><span class="o">++)</span> <span class="o">{</span> |
| <span class="n">Record</span> <span class="n">record</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Record</span><span class="o">();</span> |
| <span class="n">record</span><span class="o">.</span><span class="na">setKey</span><span class="o">(</span><span class="n">key</span><span class="o">);</span> |
| <span class="n">record</span><span class="o">.</span><span class="na">setValue</span><span class="o">(</span><span class="s">"val_"</span> <span class="o">+</span> <span class="n">key</span><span class="o">);</span> |
| <span class="n">records</span><span class="o">.</span><span class="na">add</span><span class="o">(</span><span class="n">record</span><span class="o">);</span> |
| <span class="o">}</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">recordsDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">createDataFrame</span><span class="o">(</span><span class="n">records</span><span class="o">,</span> <span class="n">Record</span><span class="o">.</span><span class="na">class</span><span class="o">);</span> |
| <span class="n">recordsDF</span><span class="o">.</span><span class="na">createOrReplaceTempView</span><span class="o">(</span><span class="s">"records"</span><span class="o">);</span> |
| |
| <span class="c1">// Queries can then join DataFrames data with data stored in Hive.</span> |
| <span class="n">spark</span><span class="o">.</span><span class="na">sql</span><span class="o">(</span><span class="s">"SELECT * FROM records r JOIN src s ON r.key = s.key"</span><span class="o">).</span><span class="na">show</span><span class="o">();</span> |
| <span class="c1">// +---+------+---+------+</span> |
| <span class="c1">// |key| value|key| value|</span> |
| <span class="c1">// +---+------+---+------+</span> |
| <span class="c1">// | 2| val_2| 2| val_2|</span> |
| <span class="c1">// | 2| val_2| 2| val_2|</span> |
| <span class="c1">// | 4| val_4| 4| val_4|</span> |
| <span class="c1">// ...</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| <div class="highlight"><pre><span></span><span class="kn">from</span> <span class="nn">os.path</span> <span class="kn">import</span> <span class="n">expanduser</span><span class="p">,</span> <span class="n">join</span><span class="p">,</span> <span class="n">abspath</span> |
| |
| <span class="kn">from</span> <span class="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">SparkSession</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">Row</span> |
| |
| <span class="c1"># warehouse_location points to the default location for managed databases and tables</span> |
| <span class="n">warehouse_location</span> <span class="o">=</span> <span class="n">abspath</span><span class="p">(</span><span class="s1">'spark-warehouse'</span><span class="p">)</span> |
| |
| <span class="n">spark</span> <span class="o">=</span> <span class="n">SparkSession</span> \ |
| <span class="o">.</span><span class="n">builder</span> \ |
| <span class="o">.</span><span class="n">appName</span><span class="p">(</span><span class="s2">"Python Spark SQL Hive integration example"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">config</span><span class="p">(</span><span class="s2">"spark.sql.warehouse.dir"</span><span class="p">,</span> <span class="n">warehouse_location</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">enableHiveSupport</span><span class="p">()</span> \ |
| <span class="o">.</span><span class="n">getOrCreate</span><span class="p">()</span> |
| |
| <span class="c1"># spark is an existing SparkSession</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">"CREATE TABLE IF NOT EXISTS src (key INT, value STRING) USING hive"</span><span class="p">)</span> |
| <span class="n">spark</span><span class="o">.</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="c1"># Queries are expressed in HiveQL</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT * FROM src"</span><span class="p">)</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +---+-------+</span> |
| <span class="c1"># |key| value|</span> |
| <span class="c1"># +---+-------+</span> |
| <span class="c1"># |238|val_238|</span> |
| <span class="c1"># | 86| val_86|</span> |
| <span class="c1"># |311|val_311|</span> |
| <span class="c1"># ...</span> |
| |
| <span class="c1"># Aggregation queries are also supported.</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT COUNT(*) FROM src"</span><span class="p">)</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +--------+</span> |
| <span class="c1"># |count(1)|</span> |
| <span class="c1"># +--------+</span> |
| <span class="c1"># | 500 |</span> |
| <span class="c1"># +--------+</span> |
| |
| <span class="c1"># The results of SQL queries are themselves DataFrames and support all normal functions.</span> |
| <span class="n">sqlDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT key, value FROM src WHERE key < 10 ORDER BY key"</span><span class="p">)</span> |
| |
| <span class="c1"># The items in DataFrames are of type Row, which allows you to access each column by ordinal.</span> |
| <span class="n">stringsDS</span> <span class="o">=</span> <span class="n">sqlDF</span><span class="o">.</span><span class="n">rdd</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">row</span><span class="p">:</span> <span class="s2">"Key: </span><span class="si">%d</span><span class="s2">, Value: </span><span class="si">%s</span><span class="s2">"</span> <span class="o">%</span> <span class="p">(</span><span class="n">row</span><span class="o">.</span><span class="n">key</span><span class="p">,</span> <span class="n">row</span><span class="o">.</span><span class="n">value</span><span class="p">))</span> |
| <span class="k">for</span> <span class="n">record</span> <span class="ow">in</span> <span class="n">stringsDS</span><span class="o">.</span><span class="n">collect</span><span class="p">():</span> |
| <span class="k">print</span><span class="p">(</span><span class="n">record</span><span class="p">)</span> |
| <span class="c1"># Key: 0, Value: val_0</span> |
| <span class="c1"># Key: 0, Value: val_0</span> |
| <span class="c1"># Key: 0, Value: val_0</span> |
| <span class="c1"># ...</span> |
| |
| <span class="c1"># You can also use DataFrames to create temporary views within a SparkSession.</span> |
| <span class="n">Record</span> <span class="o">=</span> <span class="n">Row</span><span class="p">(</span><span class="s2">"key"</span><span class="p">,</span> <span class="s2">"value"</span><span class="p">)</span> |
| <span class="n">recordsDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">createDataFrame</span><span class="p">([</span><span class="n">Record</span><span class="p">(</span><span class="n">i</span><span class="p">,</span> <span class="s2">"val_"</span> <span class="o">+</span> <span class="nb">str</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="mi">1</span><span class="p">,</span> <span class="mi">101</span><span class="p">)])</span> |
| <span class="n">recordsDF</span><span class="o">.</span><span class="n">createOrReplaceTempView</span><span class="p">(</span><span class="s2">"records"</span><span class="p">)</span> |
| |
| <span class="c1"># Queries can then join DataFrame data with data stored in Hive.</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT * FROM records r JOIN src s ON r.key = s.key"</span><span class="p">)</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +---+------+---+------+</span> |
| <span class="c1"># |key| value|key| value|</span> |
| <span class="c1"># +---+------+---+------+</span> |
| <span class="c1"># | 2| val_2| 2| val_2|</span> |
| <span class="c1"># | 4| val_4| 4| val_4|</span> |
| <span class="c1"># | 5| val_5| 5| val_5|</span> |
| <span class="c1"># ...</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/hive.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="r"> |
| |
| <p>When working with Hive one must instantiate <code>SparkSession</code> with Hive support. This |
| adds support for finding tables in the MetaStore and writing queries using HiveQL.</p> |
| |
| <div class="highlight"><pre><span></span><span class="c1"># enableHiveSupport defaults to TRUE</span> |
| sparkR.session<span class="p">(</span>enableHiveSupport <span class="o">=</span> <span class="kc">TRUE</span><span class="p">)</span> |
| sql<span class="p">(</span><span class="s">"CREATE TABLE IF NOT EXISTS src (key INT, value STRING) USING hive"</span><span class="p">)</span> |
| sql<span class="p">(</span><span class="s">"LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"</span><span class="p">)</span> |
| |
| <span class="c1"># Queries can be expressed in HiveQL.</span> |
| results <span class="o"><-</span> collect<span class="p">(</span>sql<span class="p">(</span><span class="s">"FROM src SELECT key, value"</span><span class="p">))</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| |
| </div> |
| </div> |
| |
| <h3 id="specifying-storage-format-for-hive-tables">Specifying storage format for Hive tables</h3> |
| |
| <p>When you create a Hive table, you need to define how this table should read/write data from/to file system, |
| i.e. the “input format” and “output format”. You also need to define how this table should deserialize the data |
| to rows, or serialize rows to data, i.e. the “serde”. The following options can be used to specify the storage |
| format(“serde”, “input format”, “output format”), e.g. <code>CREATE TABLE src(id int) USING hive OPTIONS(fileFormat 'parquet')</code>. |
| By default, we will read the table files as plain text. Note that, Hive storage handler is not supported yet when |
| creating table, you can create a table using storage handler at Hive side, and use Spark SQL to read it.</p> |
| |
| <table class="table"> |
| <tr><th>Property Name</th><th>Meaning</th></tr> |
| <tr> |
| <td><code>fileFormat</code></td> |
| <td> |
| A fileFormat is kind of a package of storage format specifications, including "serde", "input format" and |
| "output format". Currently we support 6 fileFormats: 'sequencefile', 'rcfile', 'orc', 'parquet', 'textfile' and 'avro'. |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>inputFormat, outputFormat</code></td> |
| <td> |
| These 2 options specify the name of a corresponding `InputFormat` and `OutputFormat` class as a string literal, |
| e.g. `org.apache.hadoop.hive.ql.io.orc.OrcInputFormat`. These 2 options must be appeared in pair, and you can not |
| specify them if you already specified the `fileFormat` option. |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>serde</code></td> |
| <td> |
| This option specifies the name of a serde class. When the `fileFormat` option is specified, do not specify this option |
| if the given `fileFormat` already include the information of serde. Currently "sequencefile", "textfile" and "rcfile" |
| don't include the serde information and you can use this option with these 3 fileFormats. |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>fieldDelim, escapeDelim, collectionDelim, mapkeyDelim, lineDelim</code></td> |
| <td> |
| These options can only be used with "textfile" fileFormat. They define how to read delimited files into rows. |
| </td> |
| </tr> |
| </table> |
| |
| <p>All other properties defined with <code>OPTIONS</code> will be regarded as Hive serde properties.</p> |
| |
| <h3 id="interacting-with-different-versions-of-hive-metastore">Interacting with Different Versions of Hive Metastore</h3> |
| |
| <p>One of the most important pieces of Spark SQL’s Hive support is interaction with Hive metastore, |
| which enables Spark SQL to access metadata of Hive tables. Starting from Spark 1.4.0, a single binary |
| build of Spark SQL can be used to query different versions of Hive metastores, using the configuration described below. |
| Note that independent of the version of Hive that is being used to talk to the metastore, internally Spark SQL |
| will compile against Hive 1.2.1 and use those classes for internal execution (serdes, UDFs, UDAFs, etc).</p> |
| |
| <p>The following options can be used to configure the version of Hive that is used to retrieve metadata:</p> |
| |
| <table class="table"> |
| <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr> |
| <tr> |
| <td><code>spark.sql.hive.metastore.version</code></td> |
| <td><code>1.2.1</code></td> |
| <td> |
| Version of the Hive metastore. Available |
| options are <code>0.12.0</code> through <code>1.2.1</code>. |
| </td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.hive.metastore.jars</code></td> |
| <td><code>builtin</code></td> |
| <td> |
| Location of the jars that should be used to instantiate the HiveMetastoreClient. This |
| property can be one of three options: |
| <ol> |
| <li><code>builtin</code></li> |
| Use Hive 1.2.1, which is bundled with the Spark assembly when <code>-Phive</code> is |
| enabled. When this option is chosen, <code>spark.sql.hive.metastore.version</code> must be |
| either <code>1.2.1</code> or not defined. |
| <li><code>maven</code></li> |
| Use Hive jars of specified version downloaded from Maven repositories. This configuration |
| is not generally recommended for production deployments. |
| <li>A classpath in the standard format for the JVM. This classpath must include all of Hive |
| and its dependencies, including the correct version of Hadoop. These jars only need to be |
| present on the driver, but if you are running in yarn cluster mode then you must ensure |
| they are packaged with your application.</li> |
| </ol> |
| </td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.hive.metastore.sharedPrefixes</code></td> |
| <td><code>com.mysql.jdbc,<br />org.postgresql,<br />com.microsoft.sqlserver,<br />oracle.jdbc</code></td> |
| <td> |
| <p> |
| A comma separated list of class prefixes that should be loaded using the classloader that is |
| shared between Spark SQL and a specific version of Hive. An example of classes that should |
| be shared is JDBC drivers that are needed to talk to the metastore. Other classes that need |
| to be shared are those that interact with classes that are already shared. For example, |
| custom appenders that are used by log4j. |
| </p> |
| </td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.hive.metastore.barrierPrefixes</code></td> |
| <td><code>(empty)</code></td> |
| <td> |
| <p> |
| A comma separated list of class prefixes that should explicitly be reloaded for each version |
| of Hive that Spark SQL is communicating with. For example, Hive UDFs that are declared in a |
| prefix that typically would be shared (i.e. <code>org.apache.spark.*</code>). |
| </p> |
| </td> |
| </tr> |
| </table> |
| |
| <h2 id="jdbc-to-other-databases">JDBC To Other Databases</h2> |
| |
| <p>Spark SQL also includes a data source that can read data from other databases using JDBC. This |
| functionality should be preferred over using <a href="api/scala/index.html#org.apache.spark.rdd.JdbcRDD">JdbcRDD</a>. |
| This is because the results are returned |
| as a DataFrame and they can easily be processed in Spark SQL or joined with other data sources. |
| The JDBC data source is also easier to use from Java or Python as it does not require the user to |
| provide a ClassTag. |
| (Note that this is different than the Spark SQL JDBC server, which allows other applications to |
| run queries using Spark SQL).</p> |
| |
| <p>To get started you will need to include the JDBC driver for your particular database on the |
| spark classpath. For example, to connect to postgres from the Spark Shell you would run the |
| following command:</p> |
| |
| <figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span></span>bin/spark-shell --driver-class-path postgresql-9.4.1207.jar --jars postgresql-9.4.1207.jar</code></pre></figure> |
| |
| <p>Tables from the remote database can be loaded as a DataFrame or Spark SQL temporary view using |
| the Data Sources API. Users can specify the JDBC connection properties in the data source options. |
| <code>user</code> and <code>password</code> are normally provided as connection properties for |
| logging into the data sources. In addition to the connection properties, Spark also supports |
| the following case-insensitive options:</p> |
| |
| <table class="table"> |
| <tr><th>Property Name</th><th>Meaning</th></tr> |
| <tr> |
| <td><code>url</code></td> |
| <td> |
| The JDBC URL to connect to. The source-specific connection properties may be specified in the URL. e.g., <code>jdbc:postgresql://localhost/test?user=fred&password=secret</code> |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>dbtable</code></td> |
| <td> |
| The JDBC table that should be read. Note that anything that is valid in a <code>FROM</code> clause of |
| a SQL query can be used. For example, instead of a full table you could also use a |
| subquery in parentheses. |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>driver</code></td> |
| <td> |
| The class name of the JDBC driver to use to connect to this URL. |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>partitionColumn, lowerBound, upperBound</code></td> |
| <td> |
| These options must all be specified if any of them is specified. In addition, |
| <code>numPartitions</code> must be specified. They describe how to partition the table when |
| reading in parallel from multiple workers. |
| <code>partitionColumn</code> must be a numeric column from the table in question. Notice |
| that <code>lowerBound</code> and <code>upperBound</code> are just used to decide the |
| partition stride, not for filtering the rows in table. So all rows in the table will be |
| partitioned and returned. This option applies only to reading. |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>numPartitions</code></td> |
| <td> |
| The maximum number of partitions that can be used for parallelism in table reading and |
| writing. This also determines the maximum number of concurrent JDBC connections. |
| If the number of partitions to write exceeds this limit, we decrease it to this limit by |
| calling <code>coalesce(numPartitions)</code> before writing. |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>fetchsize</code></td> |
| <td> |
| The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows). This option applies only to reading. |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>batchsize</code></td> |
| <td> |
| The JDBC batch size, which determines how many rows to insert per round trip. This can help performance on JDBC drivers. This option applies only to writing. It defaults to <code>1000</code>. |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>isolationLevel</code></td> |
| <td> |
| The transaction isolation level, which applies to current connection. It can be one of <code>NONE</code>, <code>READ_COMMITTED</code>, <code>READ_UNCOMMITTED</code>, <code>REPEATABLE_READ</code>, or <code>SERIALIZABLE</code>, corresponding to standard transaction isolation levels defined by JDBC's Connection object, with default of <code>READ_UNCOMMITTED</code>. This option applies only to writing. Please refer the documentation in <code>java.sql.Connection</code>. |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>sessionInitStatement</code></td> |
| <td> |
| After each database session is opened to the remote DB and before starting to read data, this option executes a custom SQL statement (or a PL/SQL block). Use this to implement session initialization code. Example: <code>option("sessionInitStatement", """BEGIN execute immediate 'alter session set "_serial_direct_read"=true'; END;""")</code> |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>truncate</code></td> |
| <td> |
| This is a JDBC writer related option. When <code>SaveMode.Overwrite</code> is enabled, this option causes Spark to truncate an existing table instead of dropping and recreating it. This can be more efficient, and prevents the table metadata (e.g., indices) from being removed. However, it will not work in some cases, such as when the new data has a different schema. It defaults to <code>false</code>. This option applies only to writing. |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>createTableOptions</code></td> |
| <td> |
| This is a JDBC writer related option. If specified, this option allows setting of database-specific table and partition options when creating a table (e.g., <code>CREATE TABLE t (name string) ENGINE=InnoDB.</code>). This option applies only to writing. |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>createTableColumnTypes</code></td> |
| <td> |
| The database column data types to use instead of the defaults, when creating the table. Data type information should be specified in the same format as CREATE TABLE columns syntax (e.g: <code>"name CHAR(64), comments VARCHAR(1024)")</code>. The specified types should be valid spark sql data types. This option applies only to writing. |
| </td> |
| </tr> |
| |
| <tr> |
| <td><code>customSchema</code></td> |
| <td> |
| The custom schema to use for reading data from JDBC connectors. For example, <code>"id DECIMAL(38, 0), name STRING"</code>. You can also specify partial fields, and the others use the default type mapping. For example, <code>"id DECIMAL(38, 0)"</code>. The column names should be identical to the corresponding column names of JDBC table. Users can specify the corresponding data types of Spark SQL instead of using the defaults. This option applies only to reading. |
| </td> |
| </tr> |
| </table> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="scala"> |
| <div class="highlight"><pre><span></span><span class="c1">// Note: JDBC loading and saving can be achieved via either the load/save or jdbc methods</span> |
| <span class="c1">// Loading data from a JDBC source</span> |
| <span class="k">val</span> <span class="n">jdbcDF</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span> |
| <span class="o">.</span><span class="n">format</span><span class="o">(</span><span class="s">"jdbc"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">option</span><span class="o">(</span><span class="s">"url"</span><span class="o">,</span> <span class="s">"jdbc:postgresql:dbserver"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">option</span><span class="o">(</span><span class="s">"dbtable"</span><span class="o">,</span> <span class="s">"schema.tablename"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">option</span><span class="o">(</span><span class="s">"user"</span><span class="o">,</span> <span class="s">"username"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">option</span><span class="o">(</span><span class="s">"password"</span><span class="o">,</span> <span class="s">"password"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">load</span><span class="o">()</span> |
| |
| <span class="k">val</span> <span class="n">connectionProperties</span> <span class="k">=</span> <span class="k">new</span> <span class="nc">Properties</span><span class="o">()</span> |
| <span class="n">connectionProperties</span><span class="o">.</span><span class="n">put</span><span class="o">(</span><span class="s">"user"</span><span class="o">,</span> <span class="s">"username"</span><span class="o">)</span> |
| <span class="n">connectionProperties</span><span class="o">.</span><span class="n">put</span><span class="o">(</span><span class="s">"password"</span><span class="o">,</span> <span class="s">"password"</span><span class="o">)</span> |
| <span class="k">val</span> <span class="n">jdbcDF2</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span> |
| <span class="o">.</span><span class="n">jdbc</span><span class="o">(</span><span class="s">"jdbc:postgresql:dbserver"</span><span class="o">,</span> <span class="s">"schema.tablename"</span><span class="o">,</span> <span class="n">connectionProperties</span><span class="o">)</span> |
| <span class="c1">// Specifying the custom data types of the read schema</span> |
| <span class="n">connectionProperties</span><span class="o">.</span><span class="n">put</span><span class="o">(</span><span class="s">"customSchema"</span><span class="o">,</span> <span class="s">"id DECIMAL(38, 0), name STRING"</span><span class="o">)</span> |
| <span class="k">val</span> <span class="n">jdbcDF3</span> <span class="k">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span> |
| <span class="o">.</span><span class="n">jdbc</span><span class="o">(</span><span class="s">"jdbc:postgresql:dbserver"</span><span class="o">,</span> <span class="s">"schema.tablename"</span><span class="o">,</span> <span class="n">connectionProperties</span><span class="o">)</span> |
| |
| <span class="c1">// Saving data to a JDBC source</span> |
| <span class="n">jdbcDF</span><span class="o">.</span><span class="n">write</span> |
| <span class="o">.</span><span class="n">format</span><span class="o">(</span><span class="s">"jdbc"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">option</span><span class="o">(</span><span class="s">"url"</span><span class="o">,</span> <span class="s">"jdbc:postgresql:dbserver"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">option</span><span class="o">(</span><span class="s">"dbtable"</span><span class="o">,</span> <span class="s">"schema.tablename"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">option</span><span class="o">(</span><span class="s">"user"</span><span class="o">,</span> <span class="s">"username"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">option</span><span class="o">(</span><span class="s">"password"</span><span class="o">,</span> <span class="s">"password"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">save</span><span class="o">()</span> |
| |
| <span class="n">jdbcDF2</span><span class="o">.</span><span class="n">write</span> |
| <span class="o">.</span><span class="n">jdbc</span><span class="o">(</span><span class="s">"jdbc:postgresql:dbserver"</span><span class="o">,</span> <span class="s">"schema.tablename"</span><span class="o">,</span> <span class="n">connectionProperties</span><span class="o">)</span> |
| |
| <span class="c1">// Specifying create table column data types on write</span> |
| <span class="n">jdbcDF</span><span class="o">.</span><span class="n">write</span> |
| <span class="o">.</span><span class="n">option</span><span class="o">(</span><span class="s">"createTableColumnTypes"</span><span class="o">,</span> <span class="s">"name CHAR(64), comments VARCHAR(1024)"</span><span class="o">)</span> |
| <span class="o">.</span><span class="n">jdbc</span><span class="o">(</span><span class="s">"jdbc:postgresql:dbserver"</span><span class="o">,</span> <span class="s">"schema.tablename"</span><span class="o">,</span> <span class="n">connectionProperties</span><span class="o">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="java"> |
| <div class="highlight"><pre><span></span><span class="c1">// Note: JDBC loading and saving can be achieved via either the load/save or jdbc methods</span> |
| <span class="c1">// Loading data from a JDBC source</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">jdbcDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">format</span><span class="o">(</span><span class="s">"jdbc"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"url"</span><span class="o">,</span> <span class="s">"jdbc:postgresql:dbserver"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"dbtable"</span><span class="o">,</span> <span class="s">"schema.tablename"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"user"</span><span class="o">,</span> <span class="s">"username"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"password"</span><span class="o">,</span> <span class="s">"password"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">load</span><span class="o">();</span> |
| |
| <span class="n">Properties</span> <span class="n">connectionProperties</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Properties</span><span class="o">();</span> |
| <span class="n">connectionProperties</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="s">"user"</span><span class="o">,</span> <span class="s">"username"</span><span class="o">);</span> |
| <span class="n">connectionProperties</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="s">"password"</span><span class="o">,</span> <span class="s">"password"</span><span class="o">);</span> |
| <span class="n">Dataset</span><span class="o"><</span><span class="n">Row</span><span class="o">></span> <span class="n">jdbcDF2</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="na">read</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">jdbc</span><span class="o">(</span><span class="s">"jdbc:postgresql:dbserver"</span><span class="o">,</span> <span class="s">"schema.tablename"</span><span class="o">,</span> <span class="n">connectionProperties</span><span class="o">);</span> |
| |
| <span class="c1">// Saving data to a JDBC source</span> |
| <span class="n">jdbcDF</span><span class="o">.</span><span class="na">write</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">format</span><span class="o">(</span><span class="s">"jdbc"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"url"</span><span class="o">,</span> <span class="s">"jdbc:postgresql:dbserver"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"dbtable"</span><span class="o">,</span> <span class="s">"schema.tablename"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"user"</span><span class="o">,</span> <span class="s">"username"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"password"</span><span class="o">,</span> <span class="s">"password"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">save</span><span class="o">();</span> |
| |
| <span class="n">jdbcDF2</span><span class="o">.</span><span class="na">write</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">jdbc</span><span class="o">(</span><span class="s">"jdbc:postgresql:dbserver"</span><span class="o">,</span> <span class="s">"schema.tablename"</span><span class="o">,</span> <span class="n">connectionProperties</span><span class="o">);</span> |
| |
| <span class="c1">// Specifying create table column data types on write</span> |
| <span class="n">jdbcDF</span><span class="o">.</span><span class="na">write</span><span class="o">()</span> |
| <span class="o">.</span><span class="na">option</span><span class="o">(</span><span class="s">"createTableColumnTypes"</span><span class="o">,</span> <span class="s">"name CHAR(64), comments VARCHAR(1024)"</span><span class="o">)</span> |
| <span class="o">.</span><span class="na">jdbc</span><span class="o">(</span><span class="s">"jdbc:postgresql:dbserver"</span><span class="o">,</span> <span class="s">"schema.tablename"</span><span class="o">,</span> <span class="n">connectionProperties</span><span class="o">);</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="python"> |
| <div class="highlight"><pre><span></span><span class="c1"># Note: JDBC loading and saving can be achieved via either the load/save or jdbc methods</span> |
| <span class="c1"># Loading data from a JDBC source</span> |
| <span class="n">jdbcDF</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span> \ |
| <span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="s2">"jdbc"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"url"</span><span class="p">,</span> <span class="s2">"jdbc:postgresql:dbserver"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"dbtable"</span><span class="p">,</span> <span class="s2">"schema.tablename"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"user"</span><span class="p">,</span> <span class="s2">"username"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"password"</span><span class="p">,</span> <span class="s2">"password"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">load</span><span class="p">()</span> |
| |
| <span class="n">jdbcDF2</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span> \ |
| <span class="o">.</span><span class="n">jdbc</span><span class="p">(</span><span class="s2">"jdbc:postgresql:dbserver"</span><span class="p">,</span> <span class="s2">"schema.tablename"</span><span class="p">,</span> |
| <span class="n">properties</span><span class="o">=</span><span class="p">{</span><span class="s2">"user"</span><span class="p">:</span> <span class="s2">"username"</span><span class="p">,</span> <span class="s2">"password"</span><span class="p">:</span> <span class="s2">"password"</span><span class="p">})</span> |
| |
| <span class="c1"># Specifying dataframe column data types on read</span> |
| <span class="n">jdbcDF3</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">read</span> \ |
| <span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="s2">"jdbc"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"url"</span><span class="p">,</span> <span class="s2">"jdbc:postgresql:dbserver"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"dbtable"</span><span class="p">,</span> <span class="s2">"schema.tablename"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"user"</span><span class="p">,</span> <span class="s2">"username"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"password"</span><span class="p">,</span> <span class="s2">"password"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"customSchema"</span><span class="p">,</span> <span class="s2">"id DECIMAL(38, 0), name STRING"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">load</span><span class="p">()</span> |
| |
| <span class="c1"># Saving data to a JDBC source</span> |
| <span class="n">jdbcDF</span><span class="o">.</span><span class="n">write</span> \ |
| <span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="s2">"jdbc"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"url"</span><span class="p">,</span> <span class="s2">"jdbc:postgresql:dbserver"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"dbtable"</span><span class="p">,</span> <span class="s2">"schema.tablename"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"user"</span><span class="p">,</span> <span class="s2">"username"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"password"</span><span class="p">,</span> <span class="s2">"password"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">save</span><span class="p">()</span> |
| |
| <span class="n">jdbcDF2</span><span class="o">.</span><span class="n">write</span> \ |
| <span class="o">.</span><span class="n">jdbc</span><span class="p">(</span><span class="s2">"jdbc:postgresql:dbserver"</span><span class="p">,</span> <span class="s2">"schema.tablename"</span><span class="p">,</span> |
| <span class="n">properties</span><span class="o">=</span><span class="p">{</span><span class="s2">"user"</span><span class="p">:</span> <span class="s2">"username"</span><span class="p">,</span> <span class="s2">"password"</span><span class="p">:</span> <span class="s2">"password"</span><span class="p">})</span> |
| |
| <span class="c1"># Specifying create table column data types on write</span> |
| <span class="n">jdbcDF</span><span class="o">.</span><span class="n">write</span> \ |
| <span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">"createTableColumnTypes"</span><span class="p">,</span> <span class="s2">"name CHAR(64), comments VARCHAR(1024)"</span><span class="p">)</span> \ |
| <span class="o">.</span><span class="n">jdbc</span><span class="p">(</span><span class="s2">"jdbc:postgresql:dbserver"</span><span class="p">,</span> <span class="s2">"schema.tablename"</span><span class="p">,</span> |
| <span class="n">properties</span><span class="o">=</span><span class="p">{</span><span class="s2">"user"</span><span class="p">:</span> <span class="s2">"username"</span><span class="p">,</span> <span class="s2">"password"</span><span class="p">:</span> <span class="s2">"password"</span><span class="p">})</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/datasource.py" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="r"> |
| <div class="highlight"><pre><span></span><span class="c1"># Loading data from a JDBC source</span> |
| df <span class="o"><-</span> read.jdbc<span class="p">(</span><span class="s">"jdbc:postgresql:dbserver"</span><span class="p">,</span> <span class="s">"schema.tablename"</span><span class="p">,</span> user <span class="o">=</span> <span class="s">"username"</span><span class="p">,</span> password <span class="o">=</span> <span class="s">"password"</span><span class="p">)</span> |
| |
| <span class="c1"># Saving data to a JDBC source</span> |
| write.jdbc<span class="p">(</span>df<span class="p">,</span> <span class="s">"jdbc:postgresql:dbserver"</span><span class="p">,</span> <span class="s">"schema.tablename"</span><span class="p">,</span> user <span class="o">=</span> <span class="s">"username"</span><span class="p">,</span> password <span class="o">=</span> <span class="s">"password"</span><span class="p">)</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/r/RSparkSQLExample.R" in the Spark repo.</small></div> |
| </div> |
| |
| <div data-lang="sql"> |
| |
| <figure class="highlight"><pre><code class="language-sql" data-lang="sql"><span></span><span class="k">CREATE</span> <span class="k">TEMPORARY</span> <span class="k">VIEW</span> <span class="n">jdbcTable</span> |
| <span class="k">USING</span> <span class="n">org</span><span class="p">.</span><span class="n">apache</span><span class="p">.</span><span class="n">spark</span><span class="p">.</span><span class="k">sql</span><span class="p">.</span><span class="n">jdbc</span> |
| <span class="k">OPTIONS</span> <span class="p">(</span> |
| <span class="n">url</span> <span class="ss">"jdbc:postgresql:dbserver"</span><span class="p">,</span> |
| <span class="n">dbtable</span> <span class="ss">"schema.tablename"</span><span class="p">,</span> |
| <span class="k">user</span> <span class="s1">'username'</span><span class="p">,</span> |
| <span class="n">password</span> <span class="s1">'password'</span> |
| <span class="p">)</span> |
| |
| <span class="k">INSERT</span> <span class="k">INTO</span> <span class="k">TABLE</span> <span class="n">jdbcTable</span> |
| <span class="k">SELECT</span> <span class="o">*</span> <span class="k">FROM</span> <span class="n">resultTable</span></code></pre></figure> |
| |
| </div> |
| </div> |
| |
| <h2 id="troubleshooting">Troubleshooting</h2> |
| |
| <ul> |
| <li>The JDBC driver class must be visible to the primordial class loader on the client session and on all executors. This is because Java’s DriverManager class does a security check that results in it ignoring all drivers not visible to the primordial class loader when one goes to open a connection. One convenient way to do this is to modify compute_classpath.sh on all worker nodes to include your driver JARs.</li> |
| <li>Some databases, such as H2, convert all names to upper case. You’ll need to use upper case to refer to those names in Spark SQL.</li> |
| </ul> |
| |
| <h1 id="performance-tuning">Performance Tuning</h1> |
| |
| <p>For some workloads it is possible to improve performance by either caching data in memory, or by |
| turning on some experimental options.</p> |
| |
| <h2 id="caching-data-in-memory">Caching Data In Memory</h2> |
| |
| <p>Spark SQL can cache tables using an in-memory columnar format by calling <code>spark.catalog.cacheTable("tableName")</code> or <code>dataFrame.cache()</code>. |
| Then Spark SQL will scan only required columns and will automatically tune compression to minimize |
| memory usage and GC pressure. You can call <code>spark.catalog.uncacheTable("tableName")</code> to remove the table from memory.</p> |
| |
| <p>Configuration of in-memory caching can be done using the <code>setConf</code> method on <code>SparkSession</code> or by running |
| <code>SET key=value</code> commands using SQL.</p> |
| |
| <table class="table"> |
| <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr> |
| <tr> |
| <td><code>spark.sql.inMemoryColumnarStorage.compressed</code></td> |
| <td>true</td> |
| <td> |
| When set to true Spark SQL will automatically select a compression codec for each column based |
| on statistics of the data. |
| </td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.inMemoryColumnarStorage.batchSize</code></td> |
| <td>10000</td> |
| <td> |
| Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization |
| and compression, but risk OOMs when caching data. |
| </td> |
| </tr> |
| |
| </table> |
| |
| <h2 id="other-configuration-options">Other Configuration Options</h2> |
| |
| <p>The following options can also be used to tune the performance of query execution. It is possible |
| that these options will be deprecated in future release as more optimizations are performed automatically.</p> |
| |
| <table class="table"> |
| <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr> |
| <tr> |
| <td><code>spark.sql.files.maxPartitionBytes</code></td> |
| <td>134217728 (128 MB)</td> |
| <td> |
| The maximum number of bytes to pack into a single partition when reading files. |
| </td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.files.openCostInBytes</code></td> |
| <td>4194304 (4 MB)</td> |
| <td> |
| The estimated cost to open a file, measured by the number of bytes could be scanned in the same |
| time. This is used when putting multiple files into a partition. It is better to over estimated, |
| then the partitions with small files will be faster than partitions with bigger files (which is |
| scheduled first). |
| </td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.broadcastTimeout</code></td> |
| <td>300</td> |
| <td> |
| <p> |
| Timeout in seconds for the broadcast wait time in broadcast joins |
| </p> |
| </td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.autoBroadcastJoinThreshold</code></td> |
| <td>10485760 (10 MB)</td> |
| <td> |
| Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when |
| performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently |
| statistics are only supported for Hive Metastore tables where the command |
| <code>ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan</code> has been run. |
| </td> |
| </tr> |
| <tr> |
| <td><code>spark.sql.shuffle.partitions</code></td> |
| <td>200</td> |
| <td> |
| Configures the number of partitions to use when shuffling data for joins or aggregations. |
| </td> |
| </tr> |
| </table> |
| |
| <h2 id="broadcast-hint-for-sql-queries">Broadcast Hint for SQL Queries</h2> |
| |
| <p>The <code>BROADCAST</code> hint guides Spark to broadcast each specified table when joining them with another table or view. |
| When Spark deciding the join methods, the broadcast hash join (i.e., BHJ) is preferred, |
| even if the statistics is above the configuration <code>spark.sql.autoBroadcastJoinThreshold</code>. |
| When both sides of a join are specified, Spark broadcasts the one having the lower statistics. |
| Note Spark does not guarantee BHJ is always chosen, since not all cases (e.g. full outer join) |
| support BHJ. When the broadcast nested loop join is selected, we still respect the hint.</p> |
| |
| <div class="codetabs"> |
| |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span></span><span class="k">import</span> <span class="nn">org.apache.spark.sql.functions.broadcast</span> |
| <span class="n">broadcast</span><span class="o">(</span><span class="n">spark</span><span class="o">.</span><span class="n">table</span><span class="o">(</span><span class="s">"src"</span><span class="o">)).</span><span class="n">join</span><span class="o">(</span><span class="n">spark</span><span class="o">.</span><span class="n">table</span><span class="o">(</span><span class="s">"records"</span><span class="o">),</span> <span class="s">"key"</span><span class="o">).</span><span class="n">show</span><span class="o">()</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span></span><span class="kn">import static</span> <span class="nn">org.apache.spark.sql.functions.broadcast</span><span class="o">;</span> |
| <span class="n">broadcast</span><span class="o">(</span><span class="n">spark</span><span class="o">.</span><span class="na">table</span><span class="o">(</span><span class="s">"src"</span><span class="o">)).</span><span class="na">join</span><span class="o">(</span><span class="n">spark</span><span class="o">.</span><span class="na">table</span><span class="o">(</span><span class="s">"records"</span><span class="o">),</span> <span class="s">"key"</span><span class="o">).</span><span class="na">show</span><span class="o">();</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span></span><span class="kn">from</span> <span class="nn">pyspark.sql.functions</span> <span class="kn">import</span> <span class="n">broadcast</span> |
| <span class="n">broadcast</span><span class="p">(</span><span class="n">spark</span><span class="o">.</span><span class="n">table</span><span class="p">(</span><span class="s2">"src"</span><span class="p">))</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">spark</span><span class="o">.</span><span class="n">table</span><span class="p">(</span><span class="s2">"records"</span><span class="p">),</span> <span class="s2">"key"</span><span class="p">)</span><span class="o">.</span><span class="n">show</span><span class="p">()</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="r"> |
| |
| <figure class="highlight"><pre><code class="language-r" data-lang="r"><span></span>src <span class="o"><-</span> sql<span class="p">(</span><span class="s">"SELECT * FROM src"</span><span class="p">)</span> |
| records <span class="o"><-</span> sql<span class="p">(</span><span class="s">"SELECT * FROM records"</span><span class="p">)</span> |
| <span class="kp">head</span><span class="p">(</span>join<span class="p">(</span>broadcast<span class="p">(</span>src<span class="p">),</span> records<span class="p">,</span> src<span class="o">$</span>key <span class="o">==</span> records<span class="o">$</span>key<span class="p">))</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="sql"> |
| |
| <figure class="highlight"><pre><code class="language-sql" data-lang="sql"><span></span><span class="c1">-- We accept BROADCAST, BROADCASTJOIN and MAPJOIN for broadcast hint</span> |
| <span class="k">SELECT</span> <span class="cm">/*+ BROADCAST(r) */</span> <span class="o">*</span> <span class="k">FROM</span> <span class="n">records</span> <span class="n">r</span> <span class="k">JOIN</span> <span class="n">src</span> <span class="n">s</span> <span class="k">ON</span> <span class="n">r</span><span class="p">.</span><span class="k">key</span> <span class="o">=</span> <span class="n">s</span><span class="p">.</span><span class="k">key</span></code></pre></figure> |
| |
| </div> |
| </div> |
| |
| <h1 id="distributed-sql-engine">Distributed SQL Engine</h1> |
| |
| <p>Spark SQL can also act as a distributed query engine using its JDBC/ODBC or command-line interface. |
| In this mode, end-users or applications can interact with Spark SQL directly to run SQL queries, |
| without the need to write any code.</p> |
| |
| <h2 id="running-the-thrift-jdbcodbc-server">Running the Thrift JDBC/ODBC server</h2> |
| |
| <p>The Thrift JDBC/ODBC server implemented here corresponds to the <a href="https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2"><code>HiveServer2</code></a> |
| in Hive 1.2.1 You can test the JDBC server with the beeline script that comes with either Spark or Hive 1.2.1.</p> |
| |
| <p>To start the JDBC/ODBC server, run the following in the Spark directory:</p> |
| |
| <pre><code>./sbin/start-thriftserver.sh |
| </code></pre> |
| |
| <p>This script accepts all <code>bin/spark-submit</code> command line options, plus a <code>--hiveconf</code> option to |
| specify Hive properties. You may run <code>./sbin/start-thriftserver.sh --help</code> for a complete list of |
| all available options. By default, the server listens on localhost:10000. You may override this |
| behaviour via either environment variables, i.e.:</p> |
| |
| <figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span></span><span class="nb">export</span> <span class="nv">HIVE_SERVER2_THRIFT_PORT</span><span class="o">=</span><listening-port> |
| <span class="nb">export</span> <span class="nv">HIVE_SERVER2_THRIFT_BIND_HOST</span><span class="o">=</span><listening-host> |
| ./sbin/start-thriftserver.sh <span class="se">\</span> |
| --master <master-uri> <span class="se">\</span> |
| ...</code></pre></figure> |
| |
| <p>or system properties:</p> |
| |
| <figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span></span>./sbin/start-thriftserver.sh <span class="se">\</span> |
| --hiveconf hive.server2.thrift.port<span class="o">=</span><listening-port> <span class="se">\</span> |
| --hiveconf hive.server2.thrift.bind.host<span class="o">=</span><listening-host> <span class="se">\</span> |
| --master <master-uri> |
| ...</code></pre></figure> |
| |
| <p>Now you can use beeline to test the Thrift JDBC/ODBC server:</p> |
| |
| <pre><code>./bin/beeline |
| </code></pre> |
| |
| <p>Connect to the JDBC/ODBC server in beeline with:</p> |
| |
| <pre><code>beeline> !connect jdbc:hive2://localhost:10000 |
| </code></pre> |
| |
| <p>Beeline will ask you for a username and password. In non-secure mode, simply enter the username on |
| your machine and a blank password. For secure mode, please follow the instructions given in the |
| <a href="https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients">beeline documentation</a>.</p> |
| |
| <p>Configuration of Hive is done by placing your <code>hive-site.xml</code>, <code>core-site.xml</code> and <code>hdfs-site.xml</code> files in <code>conf/</code>.</p> |
| |
| <p>You may also use the beeline script that comes with Hive.</p> |
| |
| <p>Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. |
| Use the following setting to enable HTTP mode as system property or in <code>hive-site.xml</code> file in <code>conf/</code>:</p> |
| |
| <pre><code>hive.server2.transport.mode - Set this to value: http |
| hive.server2.thrift.http.port - HTTP port number to listen on; default is 10001 |
| hive.server2.http.endpoint - HTTP endpoint; default is cliservice |
| </code></pre> |
| |
| <p>To test, use beeline to connect to the JDBC/ODBC server in http mode with:</p> |
| |
| <pre><code>beeline> !connect jdbc:hive2://<host>:<port>/<database>?hive.server2.transport.mode=http;hive.server2.thrift.http.path=<http_endpoint> |
| </code></pre> |
| |
| <h2 id="running-the-spark-sql-cli">Running the Spark SQL CLI</h2> |
| |
| <p>The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute |
| queries input from the command line. Note that the Spark SQL CLI cannot talk to the Thrift JDBC server.</p> |
| |
| <p>To start the Spark SQL CLI, run the following in the Spark directory:</p> |
| |
| <pre><code>./bin/spark-sql |
| </code></pre> |
| |
| <p>Configuration of Hive is done by placing your <code>hive-site.xml</code>, <code>core-site.xml</code> and <code>hdfs-site.xml</code> files in <code>conf/</code>. |
| You may run <code>./bin/spark-sql --help</code> for a complete list of all available |
| options.</p> |
| |
| <h1 id="pyspark-usage-guide-for-pandas-with-apache-arrow">PySpark Usage Guide for Pandas with Apache Arrow</h1> |
| |
| <h2 id="apache-arrow-in-spark">Apache Arrow in Spark</h2> |
| |
| <p>Apache Arrow is an in-memory columnar data format that is used in Spark to efficiently transfer |
| data between JVM and Python processes. This currently is most beneficial to Python users that |
| work with Pandas/NumPy data. Its usage is not automatic and might require some minor |
| changes to configuration or code to take full advantage and ensure compatibility. This guide will |
| give a high-level description of how to use Arrow in Spark and highlight any differences when |
| working with Arrow-enabled data.</p> |
| |
| <h3 id="ensure-pyarrow-installed">Ensure PyArrow Installed</h3> |
| |
| <p>If you install PySpark using pip, then PyArrow can be brought in as an extra dependency of the |
| SQL module with the command <code>pip install pyspark[sql]</code>. Otherwise, you must ensure that PyArrow |
| is installed and available on all cluster nodes. The current supported version is 0.8.0. |
| You can install using pip or conda from the conda-forge channel. See PyArrow |
| <a href="https://arrow.apache.org/docs/python/install.html">installation</a> for details.</p> |
| |
| <h2 id="enabling-for-conversion-tofrom-pandas">Enabling for Conversion to/from Pandas</h2> |
| |
| <p>Arrow is available as an optimization when converting a Spark DataFrame to a Pandas DataFrame |
| using the call <code>toPandas()</code> and when creating a Spark DataFrame from a Pandas DataFrame with |
| <code>createDataFrame(pandas_df)</code>. To use Arrow when executing these calls, users need to first set |
| the Spark configuration ‘spark.sql.execution.arrow.enabled’ to ‘true’. This is disabled by default.</p> |
| |
| <div class="codetabs"> |
| <div data-lang="python"> |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">numpy</span> <span class="kn">as</span> <span class="nn">np</span> |
| <span class="kn">import</span> <span class="nn">pandas</span> <span class="kn">as</span> <span class="nn">pd</span> |
| |
| <span class="c1"># Enable Arrow-based columnar data transfers</span> |
| <span class="n">spark</span><span class="o">.</span><span class="n">conf</span><span class="o">.</span><span class="n">set</span><span class="p">(</span><span class="s2">"spark.sql.execution.arrow.enabled"</span><span class="p">,</span> <span class="s2">"true"</span><span class="p">)</span> |
| |
| <span class="c1"># Generate a Pandas DataFrame</span> |
| <span class="n">pdf</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">random</span><span class="o">.</span><span class="n">rand</span><span class="p">(</span><span class="mi">100</span><span class="p">,</span> <span class="mi">3</span><span class="p">))</span> |
| |
| <span class="c1"># Create a Spark DataFrame from a Pandas DataFrame using Arrow</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">createDataFrame</span><span class="p">(</span><span class="n">pdf</span><span class="p">)</span> |
| |
| <span class="c1"># Convert the Spark DataFrame back to a Pandas DataFrame using Arrow</span> |
| <span class="n">result_pdf</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="s2">"*"</span><span class="p">)</span><span class="o">.</span><span class="n">toPandas</span><span class="p">()</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/arrow.py" in the Spark repo.</small></div> |
| </div> |
| </div> |
| |
| <p>Using the above optimizations with Arrow will produce the same results as when Arrow is not |
| enabled. Note that even with Arrow, <code>toPandas()</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. Not all Spark |
| data types are currently supported and an error can be raised if a column has an unsupported type, |
| see <a href="#supported-sql-types">Supported SQL Types</a>. If an error occurs during <code>createDataFrame()</code>, |
| Spark will fall back to create the DataFrame without Arrow.</p> |
| |
| <h2 id="pandas-udfs-aka-vectorized-udfs">Pandas UDFs (a.k.a. Vectorized UDFs)</h2> |
| |
| <p>Pandas UDFs are user defined functions that are executed by Spark using Arrow to transfer data and |
| Pandas to work with the data. A Pandas UDF is defined using the keyword <code>pandas_udf</code> as a decorator |
| or to wrap the function, no additional configuration is required. Currently, there are two types of |
| Pandas UDF: Scalar and Grouped Map.</p> |
| |
| <h3 id="scalar">Scalar</h3> |
| |
| <p>Scalar Pandas UDFs are used for vectorizing scalar operations. They can be used with functions such |
| as <code>select</code> and <code>withColumn</code>. The Python function should take <code>pandas.Series</code> as inputs and return |
| a <code>pandas.Series</code> of the same length. Internally, Spark will execute a Pandas UDF by splitting |
| columns into batches and calling the function for each batch as a subset of the data, then |
| concatenating the results together.</p> |
| |
| <p>The following example shows how to create a scalar Pandas UDF that computes the product of 2 columns.</p> |
| |
| <div class="codetabs"> |
| <div data-lang="python"> |
| <div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">pandas</span> <span class="kn">as</span> <span class="nn">pd</span> |
| |
| <span class="kn">from</span> <span class="nn">pyspark.sql.functions</span> <span class="kn">import</span> <span class="n">col</span><span class="p">,</span> <span class="n">pandas_udf</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql.types</span> <span class="kn">import</span> <span class="n">LongType</span> |
| |
| <span class="c1"># Declare the function and create the UDF</span> |
| <span class="k">def</span> <span class="nf">multiply_func</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">a</span> <span class="o">*</span> <span class="n">b</span> |
| |
| <span class="n">multiply</span> <span class="o">=</span> <span class="n">pandas_udf</span><span class="p">(</span><span class="n">multiply_func</span><span class="p">,</span> <span class="n">returnType</span><span class="o">=</span><span class="n">LongType</span><span class="p">())</span> |
| |
| <span class="c1"># The function for a pandas_udf should be able to execute with local Pandas data</span> |
| <span class="n">x</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">([</span><span class="mi">1</span><span class="p">,</span> <span class="mi">2</span><span class="p">,</span> <span class="mi">3</span><span class="p">])</span> |
| <span class="k">print</span><span class="p">(</span><span class="n">multiply_func</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="c1"># 0 1</span> |
| <span class="c1"># 1 4</span> |
| <span class="c1"># 2 9</span> |
| <span class="c1"># dtype: int64</span> |
| |
| <span class="c1"># Create a Spark DataFrame, 'spark' is an existing SparkSession</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">createDataFrame</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="p">[</span><span class="s2">"x"</span><span class="p">]))</span> |
| |
| <span class="c1"># Execute function as a Spark vectorized UDF</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">multiply</span><span class="p">(</span><span class="n">col</span><span class="p">(</span><span class="s2">"x"</span><span class="p">),</span> <span class="n">col</span><span class="p">(</span><span class="s2">"x"</span><span class="p">)))</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +-------------------+</span> |
| <span class="c1"># |multiply_func(x, x)|</span> |
| <span class="c1"># +-------------------+</span> |
| <span class="c1"># | 1|</span> |
| <span class="c1"># | 4|</span> |
| <span class="c1"># | 9|</span> |
| <span class="c1"># +-------------------+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/arrow.py" in the Spark repo.</small></div> |
| </div> |
| </div> |
| |
| <h3 id="grouped-map">Grouped Map</h3> |
| <p>Grouped map Pandas UDFs are used with <code>groupBy().apply()</code> which implements the “split-apply-combine” pattern. |
| Split-apply-combine consists of three steps:</p> |
| <ul> |
| <li>Split the data into groups by using <code>DataFrame.groupBy</code>.</li> |
| <li>Apply a function on each group. The input and output of the function are both <code>pandas.DataFrame</code>. The |
| input data contains all the rows and columns for each group.</li> |
| <li>Combine the results into a new <code>DataFrame</code>.</li> |
| </ul> |
| |
| <p>To use <code>groupBy().apply()</code>, the user needs to define the following:</p> |
| <ul> |
| <li>A Python function that defines the computation for each group.</li> |
| <li>A <code>StructType</code> object or a string that defines the schema of the output <code>DataFrame</code>.</li> |
| </ul> |
| |
| <p>The output schema will be applied to the columns of the returned <code>pandas.DataFrame</code> in order by position, |
| not by name. This means that the columns in the <code>pandas.DataFrame</code> must be indexed so that their |
| position matches the corresponding field in the schema.</p> |
| |
| <p>Note that when creating a new <code>pandas.DataFrame</code> using a dictionary, the actual position of the column |
| can differ from the order that it was placed in the dictionary. It is recommended in this case to |
| explicitly define the column order using the <code>columns</code> keyword, e.g. |
| <code>pandas.DataFrame({'id': ids, 'a': data}, columns=['id', 'a'])</code>, or alternatively use an <code>OrderedDict</code>.</p> |
| |
| <p>Note that all data for a group will be loaded into memory before the function is applied. This can |
| lead to out of memory exceptons, especially if the group sizes are skewed. The configuration for |
| <a href="#setting-arrow-batch-size">maxRecordsPerBatch</a> is not applied on groups and it is up to the user |
| to ensure that the grouped data will fit into the available memory.</p> |
| |
| <p>The following example shows how to use <code>groupby().apply()</code> to subtract the mean from each value in the group.</p> |
| |
| <div class="codetabs"> |
| <div data-lang="python"> |
| <div class="highlight"><pre><span></span><span class="kn">from</span> <span class="nn">pyspark.sql.functions</span> <span class="kn">import</span> <span class="n">pandas_udf</span><span class="p">,</span> <span class="n">PandasUDFType</span> |
| |
| <span class="n">df</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">createDataFrame</span><span class="p">(</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="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="mf">2.0</span><span class="p">),</span> <span class="p">(</span><span class="mi">2</span><span class="p">,</span> <span class="mf">3.0</span><span class="p">),</span> <span class="p">(</span><span class="mi">2</span><span class="p">,</span> <span class="mf">5.0</span><span class="p">),</span> <span class="p">(</span><span class="mi">2</span><span class="p">,</span> <span class="mf">10.0</span><span class="p">)],</span> |
| <span class="p">(</span><span class="s2">"id"</span><span class="p">,</span> <span class="s2">"v"</span><span class="p">))</span> |
| |
| <span class="nd">@pandas_udf</span><span class="p">(</span><span class="s2">"id long, v double"</span><span class="p">,</span> <span class="n">PandasUDFType</span><span class="o">.</span><span class="n">GROUPED_MAP</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">substract_mean</span><span class="p">(</span><span class="n">pdf</span><span class="p">):</span> |
| <span class="c1"># pdf is a pandas.DataFrame</span> |
| <span class="n">v</span> <span class="o">=</span> <span class="n">pdf</span><span class="o">.</span><span class="n">v</span> |
| <span class="k">return</span> <span class="n">pdf</span><span class="o">.</span><span class="n">assign</span><span class="p">(</span><span class="n">v</span><span class="o">=</span><span class="n">v</span> <span class="o">-</span> <span class="n">v</span><span class="o">.</span><span class="n">mean</span><span class="p">())</span> |
| |
| <span class="n">df</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="s2">"id"</span><span class="p">)</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">substract_mean</span><span class="p">)</span><span class="o">.</span><span class="n">show</span><span class="p">()</span> |
| <span class="c1"># +---+----+</span> |
| <span class="c1"># | id| v|</span> |
| <span class="c1"># +---+----+</span> |
| <span class="c1"># | 1|-0.5|</span> |
| <span class="c1"># | 1| 0.5|</span> |
| <span class="c1"># | 2|-3.0|</span> |
| <span class="c1"># | 2|-1.0|</span> |
| <span class="c1"># | 2| 4.0|</span> |
| <span class="c1"># +---+----+</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/python/sql/arrow.py" in the Spark repo.</small></div> |
| </div> |
| </div> |
| |
| <p>For detailed usage, please see <a href="api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf"><code>pyspark.sql.functions.pandas_udf</code></a> and |
| <a href="api/python/pyspark.sql.html#pyspark.sql.GroupedData.apply"><code>pyspark.sql.GroupedData.apply</code></a>.</p> |
| |
| <h2 id="usage-notes">Usage Notes</h2> |
| |
| <h3 id="supported-sql-types">Supported SQL Types</h3> |
| |
| <p>Currently, all Spark SQL data types are supported by Arrow-based conversion except <code>BinaryType</code>, <code>MapType</code>, |
| <code>ArrayType</code> of <code>TimestampType</code>, and nested <code>StructType</code>.</p> |
| |
| <h3 id="setting-arrow-batch-size">Setting Arrow Batch Size</h3> |
| |
| <p>Data partitions in Spark are converted into Arrow record batches, which can temporarily lead to |
| high memory usage in the JVM. To avoid possible out of memory exceptions, the size of the Arrow |
| record batches can be adjusted by setting the conf “spark.sql.execution.arrow.maxRecordsPerBatch” |
| to an integer that will determine the maximum number of rows for each batch. The default value is |
| 10,000 records per batch. If the number of columns is large, the value should be adjusted |
| accordingly. Using this limit, each data partition will be made into 1 or more record batches for |
| processing.</p> |
| |
| <h3 id="timestamp-with-time-zone-semantics">Timestamp with Time Zone Semantics</h3> |
| |
| <p>Spark internally stores timestamps as UTC values, and timestamp data that is brought in without |
| a specified time zone is converted as local time to UTC with microsecond resolution. When timestamp |
| data is exported or displayed in Spark, the session time zone is used to localize the timestamp |
| values. The session time zone is set with the configuration ‘spark.sql.session.timeZone’ and will |
| default to the JVM system local time zone if not set. Pandas uses a <code>datetime64</code> type with nanosecond |
| resolution, <code>datetime64[ns]</code>, with optional time zone on a per-column basis.</p> |
| |
| <p>When timestamp data is transferred from Spark to Pandas it will be converted to nanoseconds |
| and each column will be converted to the Spark session time zone then localized to that time |
| zone, which removes the time zone and displays values as local time. This will occur |
| when calling <code>toPandas()</code> or <code>pandas_udf</code> with timestamp columns.</p> |
| |
| <p>When timestamp data is transferred from Pandas to Spark, it will be converted to UTC microseconds. This |
| occurs when calling <code>createDataFrame</code> with a Pandas DataFrame or when returning a timestamp from a |
| <code>pandas_udf</code>. These conversions are done automatically to ensure Spark will have data in the |
| expected format, so it is not necessary to do any of these conversions yourself. Any nanosecond |
| values will be truncated.</p> |
| |
| <p>Note that a standard UDF (non-Pandas) will load timestamp data as Python datetime objects, which is |
| different than a Pandas timestamp. It is recommended to use Pandas time series functionality when |
| working with timestamps in <code>pandas_udf</code>s to get the best performance, see |
| <a href="https://pandas.pydata.org/pandas-docs/stable/timeseries.html">here</a> for details.</p> |
| |
| <h1 id="migration-guide">Migration Guide</h1> |
| |
| <h2 id="upgrading-from-spark-sql-230-to-231-and-above">Upgrading From Spark SQL 2.3.0 to 2.3.1 and above</h2> |
| |
| <ul> |
| <li>As of version 2.3.1 Arrow functionality, including <code>pandas_udf</code> and <code>toPandas()</code>/<code>createDataFrame()</code> with <code>spark.sql.execution.arrow.enabled</code> set to <code>True</code>, has been marked as experimental. These are still evolving and not currently recommended for use in production.</li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-22-to-23">Upgrading From Spark SQL 2.2 to 2.3</h2> |
| |
| <ul> |
| <li>Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named <code>_corrupt_record</code> by default). For example, <code>spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()</code> and <code>spark.read.schema(schema).json(file).select("_corrupt_record").show()</code>. Instead, you can cache or save the parsed results and then send the same query. For example, <code>val df = spark.read.schema(schema).json(file).cache()</code> and then <code>df.filter($"_corrupt_record".isNotNull).count()</code>.</li> |
| <li>The <code>percentile_approx</code> function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles.</li> |
| <li>Since Spark 2.3, the Join/Filter’s deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown.</li> |
| <li> |
| <p>Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below:</p> |
| |
| <table class="table"> |
| <tr> |
| <th> |
| <b>InputA \ InputB</b> |
| </th> |
| <th> |
| <b>NullType</b> |
| </th> |
| <th> |
| <b>IntegerType</b> |
| </th> |
| <th> |
| <b>LongType</b> |
| </th> |
| <th> |
| <b>DecimalType(38,0)*</b> |
| </th> |
| <th> |
| <b>DoubleType</b> |
| </th> |
| <th> |
| <b>DateType</b> |
| </th> |
| <th> |
| <b>TimestampType</b> |
| </th> |
| <th> |
| <b>StringType</b> |
| </th> |
| </tr> |
| <tr> |
| <td> |
| <b>NullType</b> |
| </td> |
| <td>NullType</td> |
| <td>IntegerType</td> |
| <td>LongType</td> |
| <td>DecimalType(38,0)</td> |
| <td>DoubleType</td> |
| <td>DateType</td> |
| <td>TimestampType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>IntegerType</b> |
| </td> |
| <td>IntegerType</td> |
| <td>IntegerType</td> |
| <td>LongType</td> |
| <td>DecimalType(38,0)</td> |
| <td>DoubleType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>LongType</b> |
| </td> |
| <td>LongType</td> |
| <td>LongType</td> |
| <td>LongType</td> |
| <td>DecimalType(38,0)</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>DecimalType(38,0)*</b> |
| </td> |
| <td>DecimalType(38,0)</td> |
| <td>DecimalType(38,0)</td> |
| <td>DecimalType(38,0)</td> |
| <td>DecimalType(38,0)</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>DoubleType</b> |
| </td> |
| <td>DoubleType</td> |
| <td>DoubleType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>DoubleType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>DateType</b> |
| </td> |
| <td>DateType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>DateType</td> |
| <td>TimestampType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>TimestampType</b> |
| </td> |
| <td>TimestampType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>TimestampType</td> |
| <td>TimestampType</td> |
| <td>StringType</td> |
| </tr> |
| <tr> |
| <td> |
| <b>StringType</b> |
| </td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| <td>StringType</td> |
| </tr> |
| </table> |
| |
| <p>Note that, for <b>DecimalType(38,0)*</b>, the table above intentionally does not cover all other combinations of scales and precisions because currently we only infer decimal type like <code>BigInteger</code>/<code>BigInt</code>. For example, 1.1 is inferred as double type.</p> |
| </li> |
| <li>In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas related functionalities, such as <code>toPandas</code>, <code>createDataFrame</code> from Pandas DataFrame, etc.</li> |
| <li>In PySpark, the behavior of timestamp values for Pandas related functionalities was changed to respect session timezone. If you want to use the old behavior, you need to set a configuration <code>spark.sql.execution.pandas.respectSessionTimeZone</code> to <code>False</code>. See <a href="https://issues.apache.org/jira/browse/SPARK-22395">SPARK-22395</a> for details.</li> |
| <li>In PySpark, <code>na.fill()</code> or <code>fillna</code> also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame.</li> |
| <li>Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section <a href="#broadcast-hint-for-sql-queries">Broadcast Hint</a> and <a href="https://issues.apache.org/jira/browse/SPARK-22489">SPARK-22489</a>.</li> |
| <li>Since Spark 2.3, when all inputs are binary, <code>functions.concat()</code> returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set <code>spark.sql.function.concatBinaryAsString</code> to <code>true</code>.</li> |
| <li> |
| <p>Since Spark 2.3, when all inputs are binary, SQL <code>elt()</code> returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set <code>spark.sql.function.eltOutputAsString</code> to <code>true</code>.</p> |
| </li> |
| <li>Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible (instead of returning NULL). This is compliant with SQL ANSI 2011 specification and Hive’s new behavior introduced in Hive 2.2 (HIVE-15331). This involves the following changes |
| <ul> |
| <li>The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. All the arithmetic operations are affected by the change, ie. addition (<code>+</code>), subtraction (<code>-</code>), multiplication (<code>*</code>), division (<code>/</code>), remainder (<code>%</code>) and positive module (<code>pmod</code>).</li> |
| <li>Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them.</li> |
| <li>The configuration <code>spark.sql.decimalOperations.allowPrecisionLoss</code> has been introduced. It defaults to <code>true</code>, which means the new behavior described here; if set to <code>false</code>, Spark uses previous rules, ie. it doesn’t adjust the needed scale to represent the values and it returns NULL if an exact representation of the value is not possible.</li> |
| </ul> |
| </li> |
| <li>In PySpark, <code>df.replace</code> does not allow to omit <code>value</code> when <code>to_replace</code> is not a dictionary. Previously, <code>value</code> could be omitted in the other cases and had <code>None</code> by default, which is counterintuitive and error prone.</li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-21-to-22">Upgrading From Spark SQL 2.1 to 2.2</h2> |
| |
| <ul> |
| <li> |
| <p>Spark 2.1.1 introduced a new configuration key: <code>spark.sql.hive.caseSensitiveInferenceMode</code>. It had a default setting of <code>NEVER_INFER</code>, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting’s default value to <code>INFER_AND_SAVE</code> to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the <code>INFER_AND_SAVE</code> configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set <code>spark.sql.hive.caseSensitiveInferenceMode</code> to <code>NEVER_INFER</code> to avoid the initial overhead of schema inference. Note that with the new default <code>INFER_AND_SAVE</code> setting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table’s first access.</p> |
| </li> |
| <li> |
| <p>Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty).</p> |
| </li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-20-to-21">Upgrading From Spark SQL 2.0 to 2.1</h2> |
| |
| <ul> |
| <li>Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as <code>ALTER TABLE PARTITION ... SET LOCATION</code> are now available for tables created with the Datasource API. |
| <ul> |
| <li>Legacy datasource tables can be migrated to this format via the <code>MSCK REPAIR TABLE</code> command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance.</li> |
| <li>To determine if a table has been migrated, look for the <code>PartitionProvider: Catalog</code> attribute when issuing <code>DESCRIBE FORMATTED</code> on the table.</li> |
| </ul> |
| </li> |
| <li>Changes to <code>INSERT OVERWRITE TABLE ... PARTITION ...</code> behavior for Datasource tables. |
| <ul> |
| <li>In prior Spark versions <code>INSERT OVERWRITE</code> overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten.</li> |
| <li>Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data.</li> |
| </ul> |
| </li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-16-to-20">Upgrading From Spark SQL 1.6 to 2.0</h2> |
| |
| <ul> |
| <li> |
| <p><code>SparkSession</code> is now the new entry point of Spark that replaces the old <code>SQLContext</code> and |
| <code>HiveContext</code>. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new <code>catalog</code> interface is accessible from <code>SparkSession</code> - existing API on databases and tables access such as <code>listTables</code>, <code>createExternalTable</code>, <code>dropTempView</code>, <code>cacheTable</code> are moved here.</p> |
| </li> |
| <li> |
| <p>Dataset API and DataFrame API are unified. In Scala, <code>DataFrame</code> becomes a type alias for |
| <code>Dataset[Row]</code>, while Java API users must replace <code>DataFrame</code> with <code>Dataset<Row></code>. Both the typed |
| transformations (e.g., <code>map</code>, <code>filter</code>, and <code>groupByKey</code>) and untyped transformations (e.g., |
| <code>select</code> and <code>groupBy</code>) are available on the Dataset class. Since compile-time type-safety in |
| Python and R is not a language feature, the concept of Dataset does not apply to these languages’ |
| APIs. Instead, <code>DataFrame</code> remains the primary programming abstraction, which is analogous to the |
| single-node data frame notion in these languages.</p> |
| </li> |
| <li>Dataset and DataFrame API <code>unionAll</code> has been deprecated and replaced by <code>union</code></li> |
| <li>Dataset and DataFrame API <code>explode</code> has been deprecated, alternatively, use <code>functions.explode()</code> with <code>select</code> or <code>flatMap</code></li> |
| <li> |
| <p>Dataset and DataFrame API <code>registerTempTable</code> has been deprecated and replaced by <code>createOrReplaceTempView</code></p> |
| </li> |
| <li>Changes to <code>CREATE TABLE ... LOCATION</code> behavior for Hive tables. |
| <ul> |
| <li>From Spark 2.0, <code>CREATE TABLE ... LOCATION</code> is equivalent to <code>CREATE EXTERNAL TABLE ... LOCATION</code> |
| in order to prevent accidental dropping the existing data in the user-provided locations. |
| That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. |
| Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. |
| Note that this is different from the Hive behavior.</li> |
| <li>As a result, <code>DROP TABLE</code> statements on those tables will not remove the data.</li> |
| </ul> |
| </li> |
| <li><code>spark.sql.parquet.cacheMetadata</code> is no longer used. |
| See <a href="https://issues.apache.org/jira/browse/SPARK-13664">SPARK-13664</a> for details.</li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-15-to-16">Upgrading From Spark SQL 1.5 to 1.6</h2> |
| |
| <ul> |
| <li>From Spark 1.6, by default the Thrift server runs in multi-session mode. Which means each JDBC/ODBC |
| connection owns a copy of their own SQL configuration and temporary function registry. Cached |
| tables are still shared though. If you prefer to run the Thrift server in the old single-session |
| mode, please set option <code>spark.sql.hive.thriftServer.singleSession</code> to <code>true</code>. You may either add |
| this option to <code>spark-defaults.conf</code>, or pass it to <code>start-thriftserver.sh</code> via <code>--conf</code>:</li> |
| </ul> |
| |
| <figure class="highlight"><pre><code class="language-bash" data-lang="bash"><span></span> ./sbin/start-thriftserver.sh <span class="se">\</span> |
| --conf spark.sql.hive.thriftServer.singleSession<span class="o">=</span><span class="nb">true</span> <span class="se">\</span> |
| ... |
| </code></pre></figure> |
| |
| <ul> |
| <li> |
| <p>Since 1.6.1, withColumn method in sparkR supports adding a new column to or replacing existing columns |
| of the same name of a DataFrame.</p> |
| </li> |
| <li> |
| <p>From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This |
| change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType |
| from numeric types. See <a href="https://issues.apache.org/jira/browse/SPARK-11724">SPARK-11724</a> for |
| details.</p> |
| </li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-14-to-15">Upgrading From Spark SQL 1.4 to 1.5</h2> |
| |
| <ul> |
| <li>Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with |
| code generation for expression evaluation. These features can both be disabled by setting |
| <code>spark.sql.tungsten.enabled</code> to <code>false</code>.</li> |
| <li>Parquet schema merging is no longer enabled by default. It can be re-enabled by setting |
| <code>spark.sql.parquet.mergeSchema</code> to <code>true</code>.</li> |
| <li>Resolution of strings to columns in python now supports using dots (<code>.</code>) to qualify the column or |
| access nested values. For example <code>df['table.column.nestedField']</code>. However, this means that if |
| your column name contains any dots you must now escape them using backticks (e.g., <code>table.`column.with.dots`.nested</code>).</li> |
| <li>In-memory columnar storage partition pruning is on by default. It can be disabled by setting |
| <code>spark.sql.inMemoryColumnarStorage.partitionPruning</code> to <code>false</code>.</li> |
| <li>Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum |
| precision of 38. When inferring schema from <code>BigDecimal</code> objects, a precision of (38, 18) is now |
| used. When no precision is specified in DDL then the default remains <code>Decimal(10, 0)</code>.</li> |
| <li>Timestamps are now stored at a precision of 1us, rather than 1ns</li> |
| <li>In the <code>sql</code> dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains |
| unchanged.</li> |
| <li>The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM).</li> |
| <li>JSON data source will not automatically load new files that are created by other applications |
| (i.e. files that are not inserted to the dataset through Spark SQL). |
| For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), |
| users can use <code>REFRESH TABLE</code> SQL command or <code>HiveContext</code>’s <code>refreshTable</code> method |
| to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate |
| the DataFrame and the new DataFrame will include new files.</li> |
| <li>DataFrame.withColumn method in pySpark supports adding a new column or replacing existing columns of the same name.</li> |
| </ul> |
| |
| <h2 id="upgrading-from-spark-sql-13-to-14">Upgrading from Spark SQL 1.3 to 1.4</h2> |
| |
| <h4 id="dataframe-data-readerwriter-interface">DataFrame data reader/writer interface</h4> |
| |
| <p>Based on user feedback, we created a new, more fluid API for reading data in (<code>SQLContext.read</code>) |
| and writing data out (<code>DataFrame.write</code>), |
| and deprecated the old APIs (e.g., <code>SQLContext.parquetFile</code>, <code>SQLContext.jsonFile</code>).</p> |
| |
| <p>See the API docs for <code>SQLContext.read</code> ( |
| <a href="api/scala/index.html#org.apache.spark.sql.SQLContext@read:DataFrameReader">Scala</a>, |
| <a href="api/java/org/apache/spark/sql/SQLContext.html#read()">Java</a>, |
| <a href="api/python/pyspark.sql.html#pyspark.sql.SQLContext.read">Python</a> |
| ) and <code>DataFrame.write</code> ( |
| <a href="api/scala/index.html#org.apache.spark.sql.DataFrame@write:DataFrameWriter">Scala</a>, |
| <a href="api/java/org/apache/spark/sql/DataFrame.html#write()">Java</a>, |
| <a href="api/python/pyspark.sql.html#pyspark.sql.DataFrame.write">Python</a> |
| ) more information.</p> |
| |
| <h4 id="dataframegroupby-retains-grouping-columns">DataFrame.groupBy retains grouping columns</h4> |
| |
| <p>Based on user feedback, we changed the default behavior of <code>DataFrame.groupBy().agg()</code> to retain the |
| grouping columns in the resulting <code>DataFrame</code>. To keep the behavior in 1.3, set <code>spark.sql.retainGroupColumns</code> to <code>false</code>.</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span></span><span class="c1">// In 1.3.x, in order for the grouping column "department" to show up,</span> |
| <span class="c1">// it must be included explicitly as part of the agg function call.</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">groupBy</span><span class="o">(</span><span class="s">"department"</span><span class="o">).</span><span class="n">agg</span><span class="o">(</span><span class="n">$</span><span class="s">"department"</span><span class="o">,</span> <span class="n">max</span><span class="o">(</span><span class="s">"age"</span><span class="o">),</span> <span class="n">sum</span><span class="o">(</span><span class="s">"expense"</span><span class="o">))</span> |
| |
| <span class="c1">// In 1.4+, grouping column "department" is included automatically.</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">groupBy</span><span class="o">(</span><span class="s">"department"</span><span class="o">).</span><span class="n">agg</span><span class="o">(</span><span class="n">max</span><span class="o">(</span><span class="s">"age"</span><span class="o">),</span> <span class="n">sum</span><span class="o">(</span><span class="s">"expense"</span><span class="o">))</span> |
| |
| <span class="c1">// Revert to 1.3 behavior (not retaining grouping column) by:</span> |
| <span class="n">sqlContext</span><span class="o">.</span><span class="n">setConf</span><span class="o">(</span><span class="s">"spark.sql.retainGroupColumns"</span><span class="o">,</span> <span class="s">"false"</span><span class="o">)</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span></span><span class="c1">// In 1.3.x, in order for the grouping column "department" to show up,</span> |
| <span class="c1">// it must be included explicitly as part of the agg function call.</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span><span class="s">"department"</span><span class="o">).</span><span class="na">agg</span><span class="o">(</span><span class="n">col</span><span class="o">(</span><span class="s">"department"</span><span class="o">),</span> <span class="n">max</span><span class="o">(</span><span class="s">"age"</span><span class="o">),</span> <span class="n">sum</span><span class="o">(</span><span class="s">"expense"</span><span class="o">));</span> |
| |
| <span class="c1">// In 1.4+, grouping column "department" is included automatically.</span> |
| <span class="n">df</span><span class="o">.</span><span class="na">groupBy</span><span class="o">(</span><span class="s">"department"</span><span class="o">).</span><span class="na">agg</span><span class="o">(</span><span class="n">max</span><span class="o">(</span><span class="s">"age"</span><span class="o">),</span> <span class="n">sum</span><span class="o">(</span><span class="s">"expense"</span><span class="o">));</span> |
| |
| <span class="c1">// Revert to 1.3 behavior (not retaining grouping column) by:</span> |
| <span class="n">sqlContext</span><span class="o">.</span><span class="na">setConf</span><span class="o">(</span><span class="s">"spark.sql.retainGroupColumns"</span><span class="o">,</span> <span class="s">"false"</span><span class="o">);</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="python"> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span></span><span class="kn">import</span> <span class="nn">pyspark.sql.functions</span> <span class="kn">as</span> <span class="nn">func</span> |
| |
| <span class="c1"># In 1.3.x, in order for the grouping column "department" to show up,</span> |
| <span class="c1"># it must be included explicitly as part of the agg function call.</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">groupBy</span><span class="p">(</span><span class="s2">"department"</span><span class="p">)</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">df</span><span class="p">[</span><span class="s2">"department"</span><span class="p">],</span> <span class="n">func</span><span class="o">.</span><span class="n">max</span><span class="p">(</span><span class="s2">"age"</span><span class="p">),</span> <span class="n">func</span><span class="o">.</span><span class="n">sum</span><span class="p">(</span><span class="s2">"expense"</span><span class="p">))</span> |
| |
| <span class="c1"># In 1.4+, grouping column "department" is included automatically.</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">groupBy</span><span class="p">(</span><span class="s2">"department"</span><span class="p">)</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">func</span><span class="o">.</span><span class="n">max</span><span class="p">(</span><span class="s2">"age"</span><span class="p">),</span> <span class="n">func</span><span class="o">.</span><span class="n">sum</span><span class="p">(</span><span class="s2">"expense"</span><span class="p">))</span> |
| |
| <span class="c1"># Revert to 1.3.x behavior (not retaining grouping column) by:</span> |
| <span class="n">sqlContext</span><span class="o">.</span><span class="n">setConf</span><span class="p">(</span><span class="s2">"spark.sql.retainGroupColumns"</span><span class="p">,</span> <span class="s2">"false"</span><span class="p">)</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <h4 id="behavior-change-on-dataframewithcolumn">Behavior change on DataFrame.withColumn</h4> |
| |
| <p>Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added |
| as a new column with its specified name in the result DataFrame even if there may be any existing |
| columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different |
| name from names of all existing columns or replacing existing columns of the same name.</p> |
| |
| <p>Note that this change is only for Scala API, not for PySpark and SparkR.</p> |
| |
| <h2 id="upgrading-from-spark-sql-10-12-to-13">Upgrading from Spark SQL 1.0-1.2 to 1.3</h2> |
| |
| <p>In Spark 1.3 we removed the “Alpha” label from Spark SQL and as part of this did a cleanup of the |
| available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other |
| releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked |
| as unstable (i.e., DeveloperAPI or Experimental).</p> |
| |
| <h4 id="rename-of-schemardd-to-dataframe">Rename of SchemaRDD to DataFrame</h4> |
| |
| <p>The largest change that users will notice when upgrading to Spark SQL 1.3 is that <code>SchemaRDD</code> has |
| been renamed to <code>DataFrame</code>. This is primarily because DataFrames no longer inherit from RDD |
| directly, but instead provide most of the functionality that RDDs provide though their own |
| implementation. DataFrames can still be converted to RDDs by calling the <code>.rdd</code> method.</p> |
| |
| <p>In Scala there is a type alias from <code>SchemaRDD</code> to <code>DataFrame</code> to provide source compatibility for |
| some use cases. It is still recommended that users update their code to use <code>DataFrame</code> instead. |
| Java and Python users will need to update their code.</p> |
| |
| <h4 id="unification-of-the-java-and-scala-apis">Unification of the Java and Scala APIs</h4> |
| |
| <p>Prior to Spark 1.3 there were separate Java compatible classes (<code>JavaSQLContext</code> and <code>JavaSchemaRDD</code>) |
| that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users |
| of either language should use <code>SQLContext</code> and <code>DataFrame</code>. In general these classes try to |
| use types that are usable from both languages (i.e. <code>Array</code> instead of language specific collections). |
| In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading |
| is used instead.</p> |
| |
| <p>Additionally the Java specific types API has been removed. Users of both Scala and Java should |
| use the classes present in <code>org.apache.spark.sql.types</code> to describe schema programmatically.</p> |
| |
| <h4 id="isolation-of-implicit-conversions-and-removal-of-dsl-package-scala-only">Isolation of Implicit Conversions and Removal of dsl Package (Scala-only)</h4> |
| |
| <p>Many of the code examples prior to Spark 1.3 started with <code>import sqlContext._</code>, which brought |
| all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit |
| conversions for converting <code>RDD</code>s into <code>DataFrame</code>s into an object inside of the <code>SQLContext</code>. |
| Users should now write <code>import sqlContext.implicits._</code>.</p> |
| |
| <p>Additionally, the implicit conversions now only augment RDDs that are composed of <code>Product</code>s (i.e., |
| case classes or tuples) with a method <code>toDF</code>, instead of applying automatically.</p> |
| |
| <p>When using function inside of the DSL (now replaced with the <code>DataFrame</code> API) users used to import |
| <code>org.apache.spark.sql.catalyst.dsl</code>. Instead the public dataframe functions API should be used: |
| <code>import org.apache.spark.sql.functions._</code>.</p> |
| |
| <h4 id="removal-of-the-type-aliases-in-orgapachesparksql-for-datatype-scala-only">Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only)</h4> |
| |
| <p>Spark 1.3 removes the type aliases that were present in the base sql package for <code>DataType</code>. Users |
| should instead import the classes in <code>org.apache.spark.sql.types</code></p> |
| |
| <h4 id="udf-registration-moved-to-sqlcontextudf-java--scala">UDF Registration Moved to <code>sqlContext.udf</code> (Java & Scala)</h4> |
| |
| <p>Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been |
| moved into the udf object in <code>SQLContext</code>.</p> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| |
| <figure class="highlight"><pre><code class="language-scala" data-lang="scala"><span></span><span class="n">sqlContext</span><span class="o">.</span><span class="n">udf</span><span class="o">.</span><span class="n">register</span><span class="o">(</span><span class="s">"strLen"</span><span class="o">,</span> <span class="o">(</span><span class="n">s</span><span class="k">:</span> <span class="kt">String</span><span class="o">)</span> <span class="k">=></span> <span class="n">s</span><span class="o">.</span><span class="n">length</span><span class="o">())</span></code></pre></figure> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <figure class="highlight"><pre><code class="language-java" data-lang="java"><span></span><span class="n">sqlContext</span><span class="o">.</span><span class="na">udf</span><span class="o">().</span><span class="na">register</span><span class="o">(</span><span class="s">"strLen"</span><span class="o">,</span> <span class="o">(</span><span class="n">String</span> <span class="n">s</span><span class="o">)</span> <span class="o">-></span> <span class="n">s</span><span class="o">.</span><span class="na">length</span><span class="o">(),</span> <span class="n">DataTypes</span><span class="o">.</span><span class="na">IntegerType</span><span class="o">);</span></code></pre></figure> |
| |
| </div> |
| |
| </div> |
| |
| <p>Python UDF registration is unchanged.</p> |
| |
| <h4 id="python-datatypes-no-longer-singletons">Python DataTypes No Longer Singletons</h4> |
| |
| <p>When using DataTypes in Python you will need to construct them (i.e. <code>StringType()</code>) instead of |
| referencing a singleton.</p> |
| |
| <h2 id="compatibility-with-apache-hive">Compatibility with Apache Hive</h2> |
| |
| <p>Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. |
| Currently Hive SerDes and UDFs are based on Hive 1.2.1, |
| and Spark SQL can be connected to different versions of Hive Metastore |
| (from 0.12.0 to 2.1.1. Also see <a href="#interacting-with-different-versions-of-hive-metastore">Interacting with Different Versions of Hive Metastore</a>).</p> |
| |
| <h4 id="deploying-in-existing-hive-warehouses">Deploying in Existing Hive Warehouses</h4> |
| |
| <p>The Spark SQL Thrift JDBC server is designed to be “out of the box” compatible with existing Hive |
| installations. You do not need to modify your existing Hive Metastore or change the data placement |
| or partitioning of your tables.</p> |
| |
| <h3 id="supported-hive-features">Supported Hive Features</h3> |
| |
| <p>Spark SQL supports the vast majority of Hive features, such as:</p> |
| |
| <ul> |
| <li>Hive query statements, including: |
| <ul> |
| <li><code>SELECT</code></li> |
| <li><code>GROUP BY</code></li> |
| <li><code>ORDER BY</code></li> |
| <li><code>CLUSTER BY</code></li> |
| <li><code>SORT BY</code></li> |
| </ul> |
| </li> |
| <li>All Hive operators, including: |
| <ul> |
| <li>Relational operators (<code>=</code>, <code>⇔</code>, <code>==</code>, <code><></code>, <code><</code>, <code>></code>, <code>>=</code>, <code><=</code>, etc)</li> |
| <li>Arithmetic operators (<code>+</code>, <code>-</code>, <code>*</code>, <code>/</code>, <code>%</code>, etc)</li> |
| <li>Logical operators (<code>AND</code>, <code>&&</code>, <code>OR</code>, <code>||</code>, etc)</li> |
| <li>Complex type constructors</li> |
| <li>Mathematical functions (<code>sign</code>, <code>ln</code>, <code>cos</code>, etc)</li> |
| <li>String functions (<code>instr</code>, <code>length</code>, <code>printf</code>, etc)</li> |
| </ul> |
| </li> |
| <li>User defined functions (UDF)</li> |
| <li>User defined aggregation functions (UDAF)</li> |
| <li>User defined serialization formats (SerDes)</li> |
| <li>Window functions</li> |
| <li>Joins |
| <ul> |
| <li><code>JOIN</code></li> |
| <li><code>{LEFT|RIGHT|FULL} OUTER JOIN</code></li> |
| <li><code>LEFT SEMI JOIN</code></li> |
| <li><code>CROSS JOIN</code></li> |
| </ul> |
| </li> |
| <li>Unions</li> |
| <li>Sub-queries |
| <ul> |
| <li><code>SELECT col FROM ( SELECT a + b AS col from t1) t2</code></li> |
| </ul> |
| </li> |
| <li>Sampling</li> |
| <li>Explain</li> |
| <li>Partitioned tables including dynamic partition insertion</li> |
| <li>View</li> |
| <li>All Hive DDL Functions, including: |
| <ul> |
| <li><code>CREATE TABLE</code></li> |
| <li><code>CREATE TABLE AS SELECT</code></li> |
| <li><code>ALTER TABLE</code></li> |
| </ul> |
| </li> |
| <li>Most Hive Data types, including: |
| <ul> |
| <li><code>TINYINT</code></li> |
| <li><code>SMALLINT</code></li> |
| <li><code>INT</code></li> |
| <li><code>BIGINT</code></li> |
| <li><code>BOOLEAN</code></li> |
| <li><code>FLOAT</code></li> |
| <li><code>DOUBLE</code></li> |
| <li><code>STRING</code></li> |
| <li><code>BINARY</code></li> |
| <li><code>TIMESTAMP</code></li> |
| <li><code>DATE</code></li> |
| <li><code>ARRAY<></code></li> |
| <li><code>MAP<></code></li> |
| <li><code>STRUCT<></code></li> |
| </ul> |
| </li> |
| </ul> |
| |
| <h3 id="unsupported-hive-functionality">Unsupported Hive Functionality</h3> |
| |
| <p>Below is a list of Hive features that we don’t support yet. Most of these features are rarely used |
| in Hive deployments.</p> |
| |
| <p><strong>Major Hive Features</strong></p> |
| |
| <ul> |
| <li>Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL |
| doesn’t support buckets yet.</li> |
| </ul> |
| |
| <p><strong>Esoteric Hive Features</strong></p> |
| |
| <ul> |
| <li><code>UNION</code> type</li> |
| <li>Unique join</li> |
| <li>Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at |
| the moment and only supports populating the sizeInBytes field of the hive metastore.</li> |
| </ul> |
| |
| <p><strong>Hive Input/Output Formats</strong></p> |
| |
| <ul> |
| <li>File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat.</li> |
| <li>Hadoop archive</li> |
| </ul> |
| |
| <p><strong>Hive Optimizations</strong></p> |
| |
| <p>A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are |
| less important due to Spark SQL’s in-memory computational model. Others are slotted for future |
| releases of Spark SQL.</p> |
| |
| <ul> |
| <li>Block level bitmap indexes and virtual columns (used to build indexes)</li> |
| <li>Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you |
| need to control the degree of parallelism post-shuffle using “<code>SET spark.sql.shuffle.partitions=[num_tasks];</code>”.</li> |
| <li>Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still |
| launches tasks to compute the result.</li> |
| <li>Skew data flag: Spark SQL does not follow the skew data flags in Hive.</li> |
| <li><code>STREAMTABLE</code> hint in join: Spark SQL does not follow the <code>STREAMTABLE</code> hint.</li> |
| <li>Merge multiple small files for query results: if the result output contains multiple small files, |
| Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS |
| metadata. Spark SQL does not support that.</li> |
| </ul> |
| |
| <p><strong>Hive UDF/UDTF/UDAF</strong></p> |
| |
| <p>Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs:</p> |
| |
| <ul> |
| <li><code>getRequiredJars</code> and <code>getRequiredFiles</code> (<code>UDF</code> and <code>GenericUDF</code>) are functions to automatically |
| include additional resources required by this UDF.</li> |
| <li><code>initialize(StructObjectInspector)</code> in <code>GenericUDTF</code> is not supported yet. Spark SQL currently uses |
| a deprecated interface <code>initialize(ObjectInspector[])</code> only.</li> |
| <li><code>configure</code> (<code>GenericUDF</code>, <code>GenericUDTF</code>, and <code>GenericUDAFEvaluator</code>) is a function to initialize |
| functions with <code>MapredContext</code>, which is inapplicable to Spark.</li> |
| <li><code>close</code> (<code>GenericUDF</code> and <code>GenericUDAFEvaluator</code>) is a function to release associated resources. |
| Spark SQL does not call this function when tasks finish.</li> |
| <li><code>reset</code> (<code>GenericUDAFEvaluator</code>) is a function to re-initialize aggregation for reusing the same aggregation. |
| Spark SQL currently does not support the reuse of aggregation.</li> |
| <li><code>getWindowingEvaluator</code> (<code>GenericUDAFEvaluator</code>) is a function to optimize aggregation by evaluating |
| an aggregate over a fixed window.</li> |
| </ul> |
| |
| <h3 id="incompatible-hive-udf">Incompatible Hive UDF</h3> |
| |
| <p>Below are the scenarios in which Hive and Spark generate different results:</p> |
| |
| <ul> |
| <li><code>SQRT(n)</code> If n < 0, Hive returns null, Spark SQL returns NaN.</li> |
| <li><code>ACOS(n)</code> If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN.</li> |
| <li><code>ASIN(n)</code> If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN.</li> |
| </ul> |
| |
| <h1 id="reference">Reference</h1> |
| |
| <h2 id="data-types">Data Types</h2> |
| |
| <p>Spark SQL and DataFrames support the following data types:</p> |
| |
| <ul> |
| <li>Numeric types |
| <ul> |
| <li><code>ByteType</code>: Represents 1-byte signed integer numbers. |
| The range of numbers is from <code>-128</code> to <code>127</code>.</li> |
| <li><code>ShortType</code>: Represents 2-byte signed integer numbers. |
| The range of numbers is from <code>-32768</code> to <code>32767</code>.</li> |
| <li><code>IntegerType</code>: Represents 4-byte signed integer numbers. |
| The range of numbers is from <code>-2147483648</code> to <code>2147483647</code>.</li> |
| <li><code>LongType</code>: Represents 8-byte signed integer numbers. |
| The range of numbers is from <code>-9223372036854775808</code> to <code>9223372036854775807</code>.</li> |
| <li><code>FloatType</code>: Represents 4-byte single-precision floating point numbers.</li> |
| <li><code>DoubleType</code>: Represents 8-byte double-precision floating point numbers.</li> |
| <li><code>DecimalType</code>: Represents arbitrary-precision signed decimal numbers. Backed internally by <code>java.math.BigDecimal</code>. A <code>BigDecimal</code> consists of an arbitrary precision integer unscaled value and a 32-bit integer scale.</li> |
| </ul> |
| </li> |
| <li>String type |
| <ul> |
| <li><code>StringType</code>: Represents character string values.</li> |
| </ul> |
| </li> |
| <li>Binary type |
| <ul> |
| <li><code>BinaryType</code>: Represents byte sequence values.</li> |
| </ul> |
| </li> |
| <li>Boolean type |
| <ul> |
| <li><code>BooleanType</code>: Represents boolean values.</li> |
| </ul> |
| </li> |
| <li>Datetime type |
| <ul> |
| <li><code>TimestampType</code>: Represents values comprising values of fields year, month, day, |
| hour, minute, and second.</li> |
| <li><code>DateType</code>: Represents values comprising values of fields year, month, day.</li> |
| </ul> |
| </li> |
| <li>Complex types |
| <ul> |
| <li><code>ArrayType(elementType, containsNull)</code>: Represents values comprising a sequence of |
| elements with the type of <code>elementType</code>. <code>containsNull</code> is used to indicate if |
| elements in a <code>ArrayType</code> value can have <code>null</code> values.</li> |
| <li><code>MapType(keyType, valueType, valueContainsNull)</code>: |
| Represents values comprising a set of key-value pairs. The data type of keys are |
| described by <code>keyType</code> and the data type of values are described by <code>valueType</code>. |
| For a <code>MapType</code> value, keys are not allowed to have <code>null</code> values. <code>valueContainsNull</code> |
| is used to indicate if values of a <code>MapType</code> value can have <code>null</code> values.</li> |
| <li><code>StructType(fields)</code>: Represents values with the structure described by |
| a sequence of <code>StructField</code>s (<code>fields</code>). |
| <ul> |
| <li><code>StructField(name, dataType, nullable)</code>: Represents a field in a <code>StructType</code>. |
| The name of a field is indicated by <code>name</code>. The data type of a field is indicated |
| by <code>dataType</code>. <code>nullable</code> is used to indicate if values of this fields can have |
| <code>null</code> values.</li> |
| </ul> |
| </li> |
| </ul> |
| </li> |
| </ul> |
| |
| <div class="codetabs"> |
| <div data-lang="scala"> |
| |
| <p>All data types of Spark SQL are located in the package <code>org.apache.spark.sql.types</code>. |
| You can access them by doing</p> |
| |
| <div class="highlight"><pre><span></span><span class="k">import</span> <span class="nn">org.apache.spark.sql.types._</span> |
| </pre></div> |
| <div><small>Find full example code at "examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala" in the Spark repo.</small></div> |
| |
| <table class="table"> |
| <tr> |
| <th style="width:20%">Data type</th> |
| <th style="width:40%">Value type in Scala</th> |
| <th>API to access or create a data type</th></tr> |
| <tr> |
| <td> <b>ByteType</b> </td> |
| <td> Byte </td> |
| <td> |
| ByteType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>ShortType</b> </td> |
| <td> Short </td> |
| <td> |
| ShortType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>IntegerType</b> </td> |
| <td> Int </td> |
| <td> |
| IntegerType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>LongType</b> </td> |
| <td> Long </td> |
| <td> |
| LongType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>FloatType</b> </td> |
| <td> Float </td> |
| <td> |
| FloatType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>DoubleType</b> </td> |
| <td> Double </td> |
| <td> |
| DoubleType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>DecimalType</b> </td> |
| <td> java.math.BigDecimal </td> |
| <td> |
| DecimalType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>StringType</b> </td> |
| <td> String </td> |
| <td> |
| StringType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>BinaryType</b> </td> |
| <td> Array[Byte] </td> |
| <td> |
| BinaryType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>BooleanType</b> </td> |
| <td> Boolean </td> |
| <td> |
| BooleanType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>TimestampType</b> </td> |
| <td> java.sql.Timestamp </td> |
| <td> |
| TimestampType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>DateType</b> </td> |
| <td> java.sql.Date </td> |
| <td> |
| DateType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>ArrayType</b> </td> |
| <td> scala.collection.Seq </td> |
| <td> |
| ArrayType(<i>elementType</i>, [<i>containsNull</i>])<br /> |
| <b>Note:</b> The default value of <i>containsNull</i> is <i>true</i>. |
| </td> |
| </tr> |
| <tr> |
| <td> <b>MapType</b> </td> |
| <td> scala.collection.Map </td> |
| <td> |
| MapType(<i>keyType</i>, <i>valueType</i>, [<i>valueContainsNull</i>])<br /> |
| <b>Note:</b> The default value of <i>valueContainsNull</i> is <i>true</i>. |
| </td> |
| </tr> |
| <tr> |
| <td> <b>StructType</b> </td> |
| <td> org.apache.spark.sql.Row </td> |
| <td> |
| StructType(<i>fields</i>)<br /> |
| <b>Note:</b> <i>fields</i> is a Seq of StructFields. Also, two fields with the same |
| name are not allowed. |
| </td> |
| </tr> |
| <tr> |
| <td> <b>StructField</b> </td> |
| <td> The value type in Scala of the data type of this field |
| (For example, Int for a StructField with the data type IntegerType) </td> |
| <td> |
| StructField(<i>name</i>, <i>dataType</i>, [<i>nullable</i>])<br /> |
| <b>Note:</b> The default value of <i>nullable</i> is <i>true</i>. |
| </td> |
| </tr> |
| </table> |
| |
| </div> |
| |
| <div data-lang="java"> |
| |
| <p>All data types of Spark SQL are located in the package of |
| <code>org.apache.spark.sql.types</code>. To access or create a data type, |
| please use factory methods provided in |
| <code>org.apache.spark.sql.types.DataTypes</code>.</p> |
| |
| <table class="table"> |
| <tr> |
| <th style="width:20%">Data type</th> |
| <th style="width:40%">Value type in Java</th> |
| <th>API to access or create a data type</th></tr> |
| <tr> |
| <td> <b>ByteType</b> </td> |
| <td> byte or Byte </td> |
| <td> |
| DataTypes.ByteType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>ShortType</b> </td> |
| <td> short or Short </td> |
| <td> |
| DataTypes.ShortType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>IntegerType</b> </td> |
| <td> int or Integer </td> |
| <td> |
| DataTypes.IntegerType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>LongType</b> </td> |
| <td> long or Long </td> |
| <td> |
| DataTypes.LongType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>FloatType</b> </td> |
| <td> float or Float </td> |
| <td> |
| DataTypes.FloatType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>DoubleType</b> </td> |
| <td> double or Double </td> |
| <td> |
| DataTypes.DoubleType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>DecimalType</b> </td> |
| <td> java.math.BigDecimal </td> |
| <td> |
| DataTypes.createDecimalType()<br /> |
| DataTypes.createDecimalType(<i>precision</i>, <i>scale</i>). |
| </td> |
| </tr> |
| <tr> |
| <td> <b>StringType</b> </td> |
| <td> String </td> |
| <td> |
| DataTypes.StringType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>BinaryType</b> </td> |
| <td> byte[] </td> |
| <td> |
| DataTypes.BinaryType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>BooleanType</b> </td> |
| <td> boolean or Boolean </td> |
| <td> |
| DataTypes.BooleanType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>TimestampType</b> </td> |
| <td> java.sql.Timestamp </td> |
| <td> |
| DataTypes.TimestampType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>DateType</b> </td> |
| <td> java.sql.Date </td> |
| <td> |
| DataTypes.DateType |
| </td> |
| </tr> |
| <tr> |
| <td> <b>ArrayType</b> </td> |
| <td> java.util.List </td> |
| <td> |
| DataTypes.createArrayType(<i>elementType</i>)<br /> |
| <b>Note:</b> The value of <i>containsNull</i> will be <i>true</i><br /> |
| DataTypes.createArrayType(<i>elementType</i>, <i>containsNull</i>). |
| </td> |
| </tr> |
| <tr> |
| <td> <b>MapType</b> </td> |
| <td> java.util.Map </td> |
| <td> |
| DataTypes.createMapType(<i>keyType</i>, <i>valueType</i>)<br /> |
| <b>Note:</b> The value of <i>valueContainsNull</i> will be <i>true</i>.<br /> |
| DataTypes.createMapType(<i>keyType</i>, <i>valueType</i>, <i>valueContainsNull</i>)<br /> |
| </td> |
| </tr> |
| <tr> |
| <td> <b>StructType</b> </td> |
| <td> org.apache.spark.sql.Row </td> |
| <td> |
| DataTypes.createStructType(<i>fields</i>)<br /> |
| <b>Note:</b> <i>fields</i> is a List or an array of StructFields. |
| Also, two fields with the same name are not allowed. |
| </td> |
| </tr> |
| <tr> |
| <td> <b>StructField</b> </td> |
| <td> The value type in Java of the data type of this field |
| (For example, int for a StructField with the data type IntegerType) </td> |
| <td> |
| DataTypes.createStructField(<i>name</i>, <i>dataType</i>, <i>nullable</i>) |
| </td> |
| </tr> |
| </table> |
| |
| </div> |
| |
| <div data-lang="python"> |
| |
| <p>All data types of Spark SQL are located in the package of <code>pyspark.sql.types</code>. |
| You can access them by doing</p> |
| |
| <figure class="highlight"><pre><code class="language-python" data-lang="python"><span></span><span class="kn">from</span> <span class="nn">pyspark.sql.types</span> <span class="kn">import</span> <span class="o">*</span></code></pre></figure> |
| |
| <table class="table"> |
| <tr> |
| <th style="width:20%">Data type</th> |
| <th style="width:40%">Value type in Python</th> |
| <th>API to access or create a data type</th></tr> |
| <tr> |
| <td> <b>ByteType</b> </td> |
| <td> |
| int or long <br /> |
| <b>Note:</b> Numbers will be converted to 1-byte signed integer numbers at runtime. |
| Please make sure that numbers are within the range of -128 to 127. |
| </td> |
| <td> |
| ByteType() |
| </td> |
| </tr> |
| <tr> |
| <td> <b>ShortType</b> </td> |
| <td> |
| int or long <br /> |
| <b>Note:</b> Numbers will be converted to 2-byte signed integer numbers at runtime. |
| Please make sure that numbers are within the range of -32768 to 32767. |
| </td> |
| <td> |
| ShortType() |
| </td> |
| </tr> |
| <tr> |
| <td> <b>IntegerType</b> </td> |
| <td> int or long </td> |
| <td> |
| IntegerType() |
| </td> |
| </tr> |
| <tr> |
| <td> <b>LongType</b> </td> |
| <td> |
| long <br /> |
| <b>Note:</b> Numbers will be converted to 8-byte signed integer numbers at runtime. |
| Please make sure that numbers are within the range of |
| -9223372036854775808 to 9223372036854775807. |
| Otherwise, please convert data to decimal.Decimal and use DecimalType. |
| </td> |
| <td> |
| LongType() |
| </td> |
| </tr> |
| <tr> |
| <td> <b>FloatType</b> </td> |
| <td> |
| float <br /> |
| <b>Note:</b> Numbers will be converted to 4-byte single-precision floating |
| point numbers at runtime. |
| </td> |
| <td> |
| FloatType() |
| </td> |
| </tr> |
| <tr> |
| <td> <b>DoubleType</b> </td> |
| <td> float </td> |
| <td> |
| DoubleType() |
| </td> |
| </tr> |
| <tr> |
| <td> <b>DecimalType</b> </td> |
| <td> decimal.Decimal </td> |
| <td> |
| DecimalType() |
| </td> |
| </tr> |
| <tr> |
| <td> <b>StringType</b> </td> |
| <td> string </td> |
| <td> |
| StringType() |
| </td> |
| </tr> |
| <tr> |
| <td> <b>BinaryType</b> </td> |
| <td> bytearray </td> |
| <td> |
| BinaryType() |
| </td> |
| </tr> |
| <tr> |
| <td> <b>BooleanType</b> </td> |
| <td> bool </td> |
| <td> |
| BooleanType() |
| </td> |
| </tr> |
| <tr> |
| <td> <b>TimestampType</b> </td> |
| <td> datetime.datetime </td> |
| <td> |
| TimestampType() |
| </td> |
| </tr> |
| <tr> |
| <td> <b>DateType</b> </td> |
| <td> datetime.date </td> |
| <td> |
| DateType() |
| </td> |
| </tr> |
| <tr> |
| <td> <b>ArrayType</b> </td> |
| <td> list, tuple, or array </td> |
| <td> |
| ArrayType(<i>elementType</i>, [<i>containsNull</i>])<br /> |
| <b>Note:</b> The default value of <i>containsNull</i> is <i>True</i>. |
| </td> |
| </tr> |
| <tr> |
| <td> <b>MapType</b> </td> |
| <td> dict </td> |
| <td> |
| MapType(<i>keyType</i>, <i>valueType</i>, [<i>valueContainsNull</i>])<br /> |
| <b>Note:</b> The default value of <i>valueContainsNull</i> is <i>True</i>. |
| </td> |
| </tr> |
| <tr> |
| <td> <b>StructType</b> </td> |
| <td> list or tuple </td> |
| <td> |
| StructType(<i>fields</i>)<br /> |
| <b>Note:</b> <i>fields</i> is a Seq of StructFields. Also, two fields with the same |
| name are not allowed. |
| </td> |
| </tr> |
| <tr> |
| <td> <b>StructField</b> </td> |
| <td> The value type in Python of the data type of this field |
| (For example, Int for a StructField with the data type IntegerType) </td> |
| <td> |
| StructField(<i>name</i>, <i>dataType</i>, [<i>nullable</i>])<br /> |
| <b>Note:</b> The default value of <i>nullable</i> is <i>True</i>. |
| </td> |
| </tr> |
| </table> |
| |
| </div> |
| |
| <div data-lang="r"> |
| |
| <table class="table"> |
| <tr> |
| <th style="width:20%">Data type</th> |
| <th style="width:40%">Value type in R</th> |
| <th>API to access or create a data type</th></tr> |
| <tr> |
| <td> <b>ByteType</b> </td> |
| <td> |
| integer <br /> |
| <b>Note:</b> Numbers will be converted to 1-byte signed integer numbers at runtime. |
| Please make sure that numbers are within the range of -128 to 127. |
| </td> |
| <td> |
| "byte" |
| </td> |
| </tr> |
| <tr> |
| <td> <b>ShortType</b> </td> |
| <td> |
| integer <br /> |
| <b>Note:</b> Numbers will be converted to 2-byte signed integer numbers at runtime. |
| Please make sure that numbers are within the range of -32768 to 32767. |
| </td> |
| <td> |
| "short" |
| </td> |
| </tr> |
| <tr> |
| <td> <b>IntegerType</b> </td> |
| <td> integer </td> |
| <td> |
| "integer" |
| </td> |
| </tr> |
| <tr> |
| <td> <b>LongType</b> </td> |
| <td> |
| integer <br /> |
| <b>Note:</b> Numbers will be converted to 8-byte signed integer numbers at runtime. |
| Please make sure that numbers are within the range of |
| -9223372036854775808 to 9223372036854775807. |
| Otherwise, please convert data to decimal.Decimal and use DecimalType. |
| </td> |
| <td> |
| "long" |
| </td> |
| </tr> |
| <tr> |
| <td> <b>FloatType</b> </td> |
| <td> |
| numeric <br /> |
| <b>Note:</b> Numbers will be converted to 4-byte single-precision floating |
| point numbers at runtime. |
| </td> |
| <td> |
| "float" |
| </td> |
| </tr> |
| <tr> |
| <td> <b>DoubleType</b> </td> |
| <td> numeric </td> |
| <td> |
| "double" |
| </td> |
| </tr> |
| <tr> |
| <td> <b>DecimalType</b> </td> |
| <td> Not supported </td> |
| <td> |
| Not supported |
| </td> |
| </tr> |
| <tr> |
| <td> <b>StringType</b> </td> |
| <td> character </td> |
| <td> |
| "string" |
| </td> |
| </tr> |
| <tr> |
| <td> <b>BinaryType</b> </td> |
| <td> raw </td> |
| <td> |
| "binary" |
| </td> |
| </tr> |
| <tr> |
| <td> <b>BooleanType</b> </td> |
| <td> logical </td> |
| <td> |
| "bool" |
| </td> |
| </tr> |
| <tr> |
| <td> <b>TimestampType</b> </td> |
| <td> POSIXct </td> |
| <td> |
| "timestamp" |
| </td> |
| </tr> |
| <tr> |
| <td> <b>DateType</b> </td> |
| <td> Date </td> |
| <td> |
| "date" |
| </td> |
| </tr> |
| <tr> |
| <td> <b>ArrayType</b> </td> |
| <td> vector or list </td> |
| <td> |
| list(type="array", elementType=<i>elementType</i>, containsNull=[<i>containsNull</i>])<br /> |
| <b>Note:</b> The default value of <i>containsNull</i> is <i>TRUE</i>. |
| </td> |
| </tr> |
| <tr> |
| <td> <b>MapType</b> </td> |
| <td> environment </td> |
| <td> |
| list(type="map", keyType=<i>keyType</i>, valueType=<i>valueType</i>, valueContainsNull=[<i>valueContainsNull</i>])<br /> |
| <b>Note:</b> The default value of <i>valueContainsNull</i> is <i>TRUE</i>. |
| </td> |
| </tr> |
| <tr> |
| <td> <b>StructType</b> </td> |
| <td> named list</td> |
| <td> |
| list(type="struct", fields=<i>fields</i>)<br /> |
| <b>Note:</b> <i>fields</i> is a Seq of StructFields. Also, two fields with the same |
| name are not allowed. |
| </td> |
| </tr> |
| <tr> |
| <td> <b>StructField</b> </td> |
| <td> The value type in R of the data type of this field |
| (For example, integer for a StructField with the data type IntegerType) </td> |
| <td> |
| list(name=<i>name</i>, type=<i>dataType</i>, nullable=[<i>nullable</i>])<br /> |
| <b>Note:</b> The default value of <i>nullable</i> is <i>TRUE</i>. |
| </td> |
| </tr> |
| </table> |
| |
| </div> |
| |
| </div> |
| |
| <h2 id="nan-semantics">NaN Semantics</h2> |
| |
| <p>There is specially handling for not-a-number (NaN) when dealing with <code>float</code> or <code>double</code> types that |
| does not exactly match standard floating point semantics. |
| Specifically:</p> |
| |
| <ul> |
| <li>NaN = NaN returns true.</li> |
| <li>In aggregations all NaN values are grouped together.</li> |
| <li>NaN is treated as a normal value in join keys.</li> |
| <li>NaN values go last when in ascending order, larger than any other numeric value.</li> |
| </ul> |
| |
| |
| </div> |
| |
| <!-- /container --> |
| </div> |
| |
| <script src="js/vendor/jquery-1.8.0.min.js"></script> |
| <script src="js/vendor/bootstrap.min.js"></script> |
| <script src="js/vendor/anchor.min.js"></script> |
| <script src="js/main.js"></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://') + |
| 'cdn.mathjax.org/mathjax/latest/MathJax.js?config=TeX-AMS-MML_HTMLorMML'; |
| d.getElementsByTagName('head')[0].appendChild(script); |
| }(document)); |
| </script> |
| </body> |
| </html> |