-
Sunitha Kambhampati authored
## What changes were proposed in this pull request? https://spark.apache.org/docs/latest/sql-programming-guide.html#caching-data-in-memory In the doc, the call spark.cacheTable(“tableName”) and spark.uncacheTable(“tableName”) actually needs to be spark.catalog.cacheTable and spark.catalog.uncacheTable ## How was this patch tested? Built the docs and verified the change shows up fine. Author: Sunitha Kambhampati <skambha@us.ibm.com> Closes #16919 from skambha/docChange.
Sunitha Kambhampati authored## What changes were proposed in this pull request? https://spark.apache.org/docs/latest/sql-programming-guide.html#caching-data-in-memory In the doc, the call spark.cacheTable(“tableName”) and spark.uncacheTable(“tableName”) actually needs to be spark.catalog.cacheTable and spark.catalog.uncacheTable ## How was this patch tested? Built the docs and verified the change shows up fine. Author: Sunitha Kambhampati <skambha@us.ibm.com> Closes #16919 from skambha/docChange.
layout: global
displayTitle: Spark SQL, DataFrames and Datasets Guide
title: Spark SQL and DataFrames
- This will become a table of contents (this text will be scraped). {:toc}
Overview
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.
All of the examples on this page use sample data included in the Spark distribution and can be run in
the spark-shell
, pyspark
shell, or sparkR
shell.
SQL
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 Hive Tables section. When running SQL from within another programming language the results will be returned as a Dataset/DataFrame. You can also interact with the SQL interface using the command-line or over JDBC/ODBC.
Datasets and DataFrames
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 constructed from JVM objects and then
manipulated using functional transformations (map
, flatMap
, filter
, etc.).
The Dataset API is available in Scala and
Java. 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
row.columnName
). The case for R is similar.
A DataFrame is a Dataset 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 sources such
as: structured data files, tables in Hive, external databases, or existing RDDs.
The DataFrame API is available in Scala,
Java, Python, and R.
In Scala and Java, a DataFrame is represented by a Dataset of Row
s.
In the Scala API, DataFrame
is simply a type alias of Dataset[Row]
.
While, in Java API, users need to use Dataset<Row>
to represent a DataFrame
.
Throughout this document, we will often refer to Scala/Java Datasets of Row
s as DataFrames.
Getting Started
Starting Point: SparkSession
The entry point into all functionality in Spark is the SparkSession
class. To create a basic SparkSession
, just use SparkSession.builder()
:
{% include_example init_session scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
The entry point into all functionality in Spark is the SparkSession
class. To create a basic SparkSession
, just use SparkSession.builder()
:
{% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
The entry point into all functionality in Spark is the SparkSession
class. To create a basic SparkSession
, just use SparkSession.builder
:
{% include_example init_session python/sql/basic.py %}
The entry point into all functionality in Spark is the SparkSession
class. To initialize a basic SparkSession
, just call sparkR.session()
:
{% include_example init_session r/RSparkSQLExample.R %}
Note that when invoked for the first time, sparkR.session()
initializes a global SparkSession
singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the SparkSession
once, then SparkR functions like read.df
will be able to access this global instance implicitly, and users don't need to pass the SparkSession
instance around.
SparkSession
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.
Creating DataFrames
RDD
,
from a Hive table, or from Spark data sources.
As an example, the following creates a DataFrame based on the content of a JSON file:
{% include_example create_df scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
RDD
,
from a Hive table, or from Spark data sources.
As an example, the following creates a DataFrame based on the content of a JSON file:
{% include_example create_df java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
RDD
,
from a Hive table, or from Spark data sources.
As an example, the following creates a DataFrame based on the content of a JSON file:
{% include_example create_df python/sql/basic.py %}
As an example, the following creates a DataFrame based on the content of a JSON file:
{% include_example create_df r/RSparkSQLExample.R %}
Untyped Dataset Operations (aka DataFrame Operations)
DataFrames provide a domain-specific language for structured data manipulation in Scala, Java, Python and R.
As mentioned above, in Spark 2.0, DataFrames are just Dataset of Row
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.
Here we include some basic examples of structured data processing using Datasets:
For a complete list of the types of operations that can be performed on a Dataset refer to the API Documentation.
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 DataFrame Function Reference.
{% include_example untyped_ops java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
For a complete list of the types of operations that can be performed on a Dataset refer to the API Documentation.
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 DataFrame Function Reference.
{% include_example untyped_ops python/sql/basic.py %} For a complete list of the types of operations that can be performed on a DataFrame refer to the API Documentation.
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 DataFrame Function Reference.
{% include_example untyped_ops r/RSparkSQLExample.R %}
For a complete list of the types of operations that can be performed on a DataFrame refer to the API Documentation.
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 DataFrame Function Reference.
Running SQL Queries Programmatically
{% include_example run_sql scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
{% include_example run_sql java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
{% include_example run_sql python/sql/basic.py %}
{% include_example run_sql r/RSparkSQLExample.R %}
Global Temporary View
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 global_temp
, and we must use the qualified name to
refer it, e.g. SELECT * FROM global_temp.view1
.
{% highlight sql %}
CREATE GLOBAL TEMPORARY VIEW temp_view AS SELECT a + 1, b * 2 FROM tbl
SELECT * FROM global_temp.temp_view
{% endhighlight %}
Creating Datasets
Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use a specialized Encoder 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.
Interoperating with RDDs
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.
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.
Inferring the Schema Using Reflection
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 Seq
s or Array
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.
{% include_example schema_inferring scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
Spark SQL supports automatically converting an RDD of
JavaBeans into a DataFrame.
The BeanInfo
, obtained using reflection, defines the schema of the table. Currently, Spark SQL
does not support JavaBeans that contain Map
field(s). Nested JavaBeans and List
or Array
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.
{% include_example schema_inferring java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
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.
{% include_example schema_inferring python/sql/basic.py %}
Programmatically Specifying the Schema
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 DataFrame
can be created programmatically with three steps.
- Create an RDD of
Row
s from the original RDD; - Create the schema represented by a
StructType
matching the structure ofRow
s in the RDD created in Step 1. - Apply the schema to the RDD of
Row
s viacreateDataFrame
method provided bySparkSession
.
For example:
{% include_example programmatic_schema scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
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 Dataset<Row>
can be created programmatically with three steps.
- Create an RDD of
Row
s from the original RDD; - Create the schema represented by a
StructType
matching the structure ofRow
s in the RDD created in Step 1. - Apply the schema to the RDD of
Row
s viacreateDataFrame
method provided bySparkSession
.
For example:
{% include_example programmatic_schema java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
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 DataFrame
can be created programmatically with three steps.
- Create an RDD of tuples or lists from the original RDD;
- Create the schema represented by a
StructType
matching the structure of tuples or lists in the RDD created in the step 1. - Apply the schema to the RDD via
createDataFrame
method provided bySparkSession
.
For example:
{% include_example programmatic_schema python/sql/basic.py %}
Aggregations
The built-in DataFrames functions provide common
aggregations such as count()
, countDistinct()
, avg()
, max()
, min()
, etc.
While those functions are designed for DataFrames, Spark SQL also has type-safe versions for some of them in
Scala and
Java to work with strongly typed Datasets.
Moreover, users are not limited to the predefined aggregate functions and can create their own.
Untyped User-Defined Aggregate Functions
Users have to extend the UserDefinedAggregateFunction abstract class to implement a custom untyped aggregate function. For example, a user-defined average can look like:
{% include_example untyped_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala%}
{% include_example untyped_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java%}
Type-Safe User-Defined Aggregate Functions
User-defined aggregations for strongly typed Datasets revolve around the Aggregator abstract class. For example, a type-safe user-defined average can look like:
{% include_example typed_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala%}
{% include_example typed_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java%}
Data Sources
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.
Generic Load/Save Functions
In the simplest form, the default data source (parquet
unless otherwise configured by
spark.sql.sources.default
) will be used for all operations.
{% include_example generic_load_save_functions python/sql/datasource.py %}
{% include_example generic_load_save_functions r/RSparkSQLExample.R %}
Manually Specifying Options
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., org.apache.spark.sql.parquet
), but for built-in sources you can also use their short
names (json
, parquet
, jdbc
, orc
, libsvm
, csv
, text
). DataFrames loaded from any data
source type can be converted into other types using this syntax.
Run SQL on files directly
Instead of using read API to load a file into DataFrame and query it, you can also query that file directly with SQL.
Save Modes
Save operations can optionally take a SaveMode
, 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 Overwrite
, the data will be deleted before writing out the
new data.
Scala/Java | Any Language | Meaning |
---|---|---|
SaveMode.ErrorIfExists (default) |
"error" (default) |
When saving a DataFrame to a data source, if data already exists, an exception is expected to be thrown. |
SaveMode.Append |
"append" |
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. |
SaveMode.Overwrite |
"overwrite" |
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. |
SaveMode.Ignore |
"ignore" |
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 CREATE TABLE IF NOT EXISTS in SQL.
|
Saving to Persistent Tables
DataFrames
can also be saved as persistent tables into Hive metastore using the saveAsTable
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 createOrReplaceTempView
command,
saveAsTable
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 table
method on a SparkSession
with the name of the table.
For file-based data source, e.g. text, parquet, json, etc. you can specify a custom table path via the
path
option, e.g. df.write.option("path", "/some/path").saveAsTable("t")
. 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
specifed, 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.
Starting from Spark 2.1, persistent datasource tables have per-partition metadata stored in the Hive metastore. This brings several benefits:
- 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.
- Hive DDLs such as
ALTER TABLE PARTITION ... SET LOCATION
are now available for tables created with the Datasource API.
Note that partition information is not gathered by default when creating external datasource tables (those with a path
option). To sync the partition information in the metastore, you can invoke MSCK REPAIR TABLE
.
Parquet Files
Parquet 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.
Loading Data Programmatically
Using the data from the above example:
{% include_example basic_parquet_example python/sql/datasource.py %}
{% include_example basic_parquet_example r/RSparkSQLExample.R %}
{% highlight sql %}
CREATE TEMPORARY VIEW parquetTable USING org.apache.spark.sql.parquet OPTIONS ( path "examples/src/main/resources/people.parquet" )
SELECT * FROM parquetTable
{% endhighlight %}
Partition Discovery
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. The Parquet data source is now 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, gender
and country
as partitioning columns:
{% highlight text %}
path └── to └── table ├── gender=male │ ├── ... │ │ │ ├── country=US │ │ └── data.parquet │ ├── country=CN │ │ └── data.parquet │ └── ... └── gender=female ├── ... │ ├── country=US │ └── data.parquet ├── country=CN │ └── data.parquet └── ...
{% endhighlight %}
By passing path/to/table
to either SparkSession.read.parquet
or SparkSession.read.load
, Spark SQL
will automatically extract the partitioning information from the paths.
Now the schema of the returned DataFrame becomes:
{% highlight text %}
root |-- name: string (nullable = true) |-- age: long (nullable = true) |-- gender: string (nullable = true) |-- country: string (nullable = true)
{% endhighlight %}
Notice that the data types of the partitioning columns are automatically inferred. Currently,
numeric data types 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 spark.sql.sources.partitionColumnTypeInference.enabled
, which is default to
true
. When type inference is disabled, string type will be used for the partitioning columns.
Starting from Spark 1.6.0, partition discovery only finds partitions under the given paths
by default. For the above example, if users pass path/to/table/gender=male
to either
SparkSession.read.parquet
or SparkSession.read.load
, gender
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 basePath
in the data source options. For example,
when path/to/table/gender=male
is the path of the data and
users set basePath
to path/to/table/
, gender
will be a partitioning column.
Schema Merging
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.
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
- setting data source option
mergeSchema
totrue
when reading Parquet files (as shown in the examples below), or - setting the global SQL option
spark.sql.parquet.mergeSchema
totrue
.
{% include_example schema_merging python/sql/datasource.py %}
{% include_example schema_merging r/RSparkSQLExample.R %}
Hive metastore Parquet table conversion
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
spark.sql.hive.convertMetastoreParquet
configuration, and is turned on by default.
Hive/Parquet Schema Reconciliation
There are two key differences between Hive and Parquet from the perspective of table schema processing.
- Hive is case insensitive, while Parquet is not
- Hive considers all columns nullable, while nullability in Parquet is significant