[SPARK-12069][SQL] Update documentation with Datasets

Author: Michael Armbrust <michael@databricks.com>

Closes #10060 from marmbrus/docs.
This commit is contained in:
Michael Armbrust 2015-12-08 15:58:35 -08:00
parent 9494521695
commit 3959489423
5 changed files with 237 additions and 104 deletions

View file

@ -71,7 +71,7 @@
<li><a href="programming-guide.html">Spark Programming Guide</a></li>
<li class="divider"></li>
<li><a href="streaming-programming-guide.html">Spark Streaming</a></li>
<li><a href="sql-programming-guide.html">DataFrames and SQL</a></li>
<li><a href="sql-programming-guide.html">DataFrames, Datasets and SQL</a></li>
<li><a href="mllib-guide.html">MLlib (Machine Learning)</a></li>
<li><a href="graphx-programming-guide.html">GraphX (Graph Processing)</a></li>
<li><a href="bagel-programming-guide.html">Bagel (Pregel on Spark)</a></li>

View file

@ -87,7 +87,7 @@ options for deployment:
in all supported languages (Scala, Java, Python, R)
* Modules built on Spark:
* [Spark Streaming](streaming-programming-guide.html): processing real-time data streams
* [Spark SQL and DataFrames](sql-programming-guide.html): support for structured data and relational queries
* [Spark SQL, Datasets, and DataFrames](sql-programming-guide.html): support for structured data and relational queries
* [MLlib](mllib-guide.html): built-in machine learning library
* [GraphX](graphx-programming-guide.html): Spark's new API for graph processing

View file

@ -1,6 +1,6 @@
---
layout: global
displayTitle: Spark SQL and DataFrame Guide
displayTitle: Spark SQL, DataFrames and Datasets Guide
title: Spark SQL and DataFrames
---
@ -9,18 +9,51 @@ title: Spark SQL and DataFrames
# Overview
Spark SQL is a Spark module for structured data processing. It provides a programming abstraction called DataFrames and can also act as distributed SQL query engine.
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, the DataFrames API and the Datasets 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 the
various APIs based on which provides the most natural way to express a given transformation.
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](#hive-tables) section.
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.
# DataFrames
## SQL
A DataFrame is a distributed collection of data organized into named columns. It is conceptually equivalent to a table in a relational database or a data frame in R/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.
One use of Spark SQL is to execute SQL queries written using either a basic SQL syntax or HiveQL.
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](#hive-tables) section. When running
SQL from within another programming language the results will be returned as a [DataFrame](#DataFrames).
You can also interact with the SQL interface using the [command-line](#running-the-spark-sql-cli)
or over [JDBC/ODBC](#running-the-thrift-jdbcodbc-server).
The DataFrame API is available in [Scala](api/scala/index.html#org.apache.spark.sql.DataFrame), [Java](api/java/index.html?org/apache/spark/sql/DataFrame.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame), and [R](api/R/index.html).
## DataFrames
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.
A DataFrame is a distributed collection of data organized into named columns. It is conceptually
equivalent to a table in a relational database or a data frame in R/Python, but with richer
optimizations under the hood. DataFrames can be constructed from a wide array of [sources](#data-sources) such
as: structured data files, tables in Hive, external databases, or existing RDDs.
The DataFrame API is available in [Scala](api/scala/index.html#org.apache.spark.sql.DataFrame),
[Java](api/java/index.html?org/apache/spark/sql/DataFrame.html),
[Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame), and [R](api/R/index.html).
## Datasets
A Dataset is a new experimental interface added in Spark 1.6 that tries to provide 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](#creating-datasets) from JVM objects and then manipulated
using functional transformations (map, flatMap, filter, etc.).
The unified Dataset API can be used both in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset) and
[Java](api/java/index.html?org/apache/spark/sql/Dataset.html). Python does not yet have support for
the Dataset API, but due to its dynamic nature many of the benefits are already available (i.e. you can
access the field of a row by name naturally `row.columnName`). Full python support will be added
in a future release.
# Getting Started
## Starting Point: SQLContext
@ -29,7 +62,7 @@ All of the examples on this page use sample data included in the Spark distribut
The entry point into all functionality in Spark SQL is the
[`SQLContext`](api/scala/index.html#org.apache.spark.sql.SQLContext) class, or one of its
descendants. To create a basic `SQLContext`, all you need is a SparkContext.
descendants. To create a basic `SQLContext`, all you need is a SparkContext.
{% highlight scala %}
val sc: SparkContext // An existing SparkContext.
@ -45,7 +78,7 @@ import sqlContext.implicits._
The entry point into all functionality in Spark SQL is the
[`SQLContext`](api/java/index.html#org.apache.spark.sql.SQLContext) class, or one of its
descendants. To create a basic `SQLContext`, all you need is a SparkContext.
descendants. To create a basic `SQLContext`, all you need is a SparkContext.
{% highlight java %}
JavaSparkContext sc = ...; // An existing JavaSparkContext.
@ -58,7 +91,7 @@ SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
The entry point into all relational functionality in Spark is the
[`SQLContext`](api/python/pyspark.sql.html#pyspark.sql.SQLContext) class, or one
of its decedents. To create a basic `SQLContext`, all you need is a SparkContext.
of its decedents. To create a basic `SQLContext`, all you need is a SparkContext.
{% highlight python %}
from pyspark.sql import SQLContext
@ -70,7 +103,7 @@ sqlContext = SQLContext(sc)
<div data-lang="r" markdown="1">
The entry point into all relational functionality in Spark is the
`SQLContext` class, or one of its decedents. To create a basic `SQLContext`, all you need is a SparkContext.
`SQLContext` class, or one of its decedents. To create a basic `SQLContext`, all you need is a SparkContext.
{% highlight r %}
sqlContext <- sparkRSQL.init(sc)
@ -82,18 +115,18 @@ sqlContext <- sparkRSQL.init(sc)
In addition to the basic `SQLContext`, you can also create a `HiveContext`, which provides a
superset of the functionality provided by the basic `SQLContext`. Additional features include
the ability to write queries using the more complete HiveQL parser, access to Hive UDFs, and the
ability to read data from Hive tables. To use a `HiveContext`, you do not need to have an
ability to read data from Hive tables. To use a `HiveContext`, you do not need to have an
existing Hive setup, and all of the data sources available to a `SQLContext` are still available.
`HiveContext` is only packaged separately to avoid including all of Hive's dependencies in the default
Spark build. If these dependencies are not a problem for your application then using `HiveContext`
is recommended for the 1.3 release of Spark. Future releases will focus on bringing `SQLContext` up
Spark build. If these dependencies are not a problem for your application then using `HiveContext`
is recommended for the 1.3 release of Spark. Future releases will focus on bringing `SQLContext` up
to feature parity with a `HiveContext`.
The specific variant of SQL that is used to parse queries can also be selected using the
`spark.sql.dialect` option. This parameter can be changed using either the `setConf` method on
a `SQLContext` or by using a `SET key=value` command in SQL. For a `SQLContext`, the only dialect
available is "sql" which uses a simple SQL parser provided by Spark SQL. In a `HiveContext`, the
default is "hiveql", though "sql" is also available. Since the HiveQL parser is much more complete,
`spark.sql.dialect` option. This parameter can be changed using either the `setConf` method on
a `SQLContext` or by using a `SET key=value` command in SQL. For a `SQLContext`, the only dialect
available is "sql" which uses a simple SQL parser provided by Spark SQL. In a `HiveContext`, the
default is "hiveql", though "sql" is also available. Since the HiveQL parser is much more complete,
this is recommended for most use cases.
@ -215,7 +248,7 @@ df.groupBy("age").count().show()
For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.DataFrame).
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](api/scala/index.html#org.apache.spark.sql.functions$).
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](api/scala/index.html#org.apache.spark.sql.functions$).
</div>
@ -270,7 +303,7 @@ df.groupBy("age").count().show();
For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/java/org/apache/spark/sql/DataFrame.html).
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](api/java/org/apache/spark/sql/functions.html).
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](api/java/org/apache/spark/sql/functions.html).
</div>
@ -331,7 +364,7 @@ df.groupBy("age").count().show()
For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/python/pyspark.sql.html#pyspark.sql.DataFrame).
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](api/python/pyspark.sql.html#module-pyspark.sql.functions).
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](api/python/pyspark.sql.html#module-pyspark.sql.functions).
</div>
@ -385,7 +418,7 @@ showDF(count(groupBy(df, "age")))
For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/R/index.html).
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](api/R/index.html).
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](api/R/index.html).
</div>
@ -398,14 +431,14 @@ The `sql` function on a `SQLContext` enables applications to run SQL queries pro
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% highlight scala %}
val sqlContext = ... // An existing SQLContext
val sqlContext = ... // An existing SQLContext
val df = sqlContext.sql("SELECT * FROM table")
{% endhighlight %}
</div>
<div data-lang="java" markdown="1">
{% highlight java %}
SQLContext sqlContext = ... // An existing SQLContext
SQLContext sqlContext = ... // An existing SQLContext
DataFrame df = sqlContext.sql("SELECT * FROM table")
{% endhighlight %}
</div>
@ -428,15 +461,54 @@ df <- sql(sqlContext, "SELECT * FROM table")
</div>
## Creating Datasets
Datasets are similar to RDDs, however, instead of using Java Serialization or Kryo they use
a specialized [Encoder](api/scala/index.html#org.apache.spark.sql.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.
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% highlight scala %}
// Encoders for most common types are automatically provided by importing sqlContext.implicits._
val ds = Seq(1, 2, 3).toDS()
ds.map(_ + 1).collect() // Returns: Array(2, 3, 4)
// Encoders are also created for case classes.
case class Person(name: String, age: Long)
val ds = Seq(Person("Andy", 32)).toDS()
// DataFrames can be converted to a Dataset by providing a class. Mapping will be done by name.
val path = "examples/src/main/resources/people.json"
val people = sqlContext.read.json(path).as[Person]
{% endhighlight %}
</div>
<div data-lang="java" markdown="1">
{% highlight java %}
JavaSparkContext sc = ...; // An existing JavaSparkContext.
SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
{% endhighlight %}
</div>
</div>
## Interoperating with RDDs
Spark SQL supports two different methods for converting existing RDDs into DataFrames. The first
method uses reflection to infer the schema of an RDD that contains specific types of objects. This
Spark SQL supports two different methods for converting existing RDDs into DataFrames. 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 DataFrames 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
construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows
you to construct DataFrames when the columns and their types are not known until runtime.
### Inferring the Schema Using Reflection
@ -445,11 +517,11 @@ you to construct DataFrames when the columns and their types are not known until
<div data-lang="scala" markdown="1">
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
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 Sequences or Arrays. This RDD can be implicitly converted to a DataFrame and then be
registered as a table. Tables can be used in subsequent SQL statements.
registered as a table. Tables can be used in subsequent SQL statements.
{% highlight scala %}
// sc is an existing SparkContext.
@ -486,9 +558,9 @@ teenagers.map(_.getValuesMap[Any](List("name", "age"))).collect().foreach(printl
<div data-lang="java" markdown="1">
Spark SQL supports automatically converting an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly)
into a DataFrame. The BeanInfo, obtained using reflection, defines the schema of the table.
into a DataFrame. The BeanInfo, obtained using reflection, defines the schema of the table.
Currently, Spark SQL does not support JavaBeans that contain
nested or contain complex types such as Lists or Arrays. You can create a JavaBean by creating a
nested or contain complex types such as Lists or Arrays. You can create a JavaBean by creating a
class that implements Serializable and has getters and setters for all of its fields.
{% highlight java %}
@ -559,9 +631,9 @@ List<String> teenagerNames = teenagers.javaRDD().map(new Function<Row, String>()
<div data-lang="python" markdown="1">
Spark SQL can convert an RDD of Row objects to a DataFrame, inferring the datatypes. Rows are constructed by passing a list of
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 looking at the first row. Since we currently only look at the first
and the types are inferred by looking at the first row. Since we currently only look at the first
row, it is important that there is no missing data in the first row of the RDD. In future versions we
plan to more completely infer the schema by looking at more data, similar to the inference that is
performed on JSON files.
@ -780,7 +852,7 @@ for name in names.collect():
Spark SQL supports operating on a variety of data sources through the `DataFrame` interface.
A DataFrame can be operated on as normal RDDs and can also be registered as a temporary table.
Registering a DataFrame as a table allows you to run SQL queries over its data. This section
Registering a DataFrame as a table 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.
@ -834,9 +906,9 @@ saveDF(select(df, "name", "age"), "namesAndAges.parquet")
### 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
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`). DataFrames of any type can be converted into other types
names (`json`, `parquet`, `jdbc`). DataFrames of any type can be converted into other types
using this syntax.
<div class="codetabs">
@ -923,8 +995,8 @@ df <- sql(sqlContext, "SELECT * FROM parquet.`examples/src/main/resources/users.
### 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 a `Overwrite`, the data will be deleted before writing out the
present. It is important to realize that these save modes do not utilize any locking and are not
atomic. Additionally, when performing a `Overwrite`, the data will be deleted before writing out the
new data.
<table class="table">
@ -960,7 +1032,7 @@ new data.
<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.
change the existing data. This is similar to a <code>CREATE TABLE IF NOT EXISTS</code> in SQL.
</td>
</tr>
</table>
@ -968,14 +1040,14 @@ new data.
### Saving to Persistent Tables
When working with a `HiveContext`, `DataFrames` can also be saved as persistent tables using the
`saveAsTable` command. Unlike the `registerTempTable` command, `saveAsTable` will materialize the
contents of the dataframe and create a pointer to the data in the HiveMetastore. Persistent tables
`saveAsTable` command. Unlike the `registerTempTable` command, `saveAsTable` will materialize the
contents of the dataframe and create a pointer to the data in the HiveMetastore. 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`
to the same metastore. A DataFrame for a persistent table can be created by calling the `table`
method on a `SQLContext` with the name of the table.
By default `saveAsTable` will create a "managed table", meaning that the location of the data will
be controlled by the metastore. Managed tables will also have their data deleted automatically
be controlled by the metastore. Managed tables will also have their data deleted automatically
when a table is dropped.
## Parquet Files
@ -1003,7 +1075,7 @@ val people: RDD[Person] = ... // An RDD of case class objects, from the previous
// The RDD is implicitly converted to a DataFrame by implicits, allowing it to be stored using Parquet.
people.write.parquet("people.parquet")
// Read in the parquet file created above. Parquet files are self-describing so the schema is preserved.
// Read in the parquet file created above. Parquet files are self-describing so the schema is preserved.
// The result of loading a Parquet file is also a DataFrame.
val parquetFile = sqlContext.read.parquet("people.parquet")
@ -1025,7 +1097,7 @@ DataFrame schemaPeople = ... // The DataFrame from the previous example.
// DataFrames can be saved as Parquet files, maintaining the schema information.
schemaPeople.write().parquet("people.parquet");
// Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved.
// Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved.
// The result of loading a parquet file is also a DataFrame.
DataFrame parquetFile = sqlContext.read().parquet("people.parquet");
@ -1051,7 +1123,7 @@ schemaPeople # The DataFrame from the previous example.
# DataFrames can be saved as Parquet files, maintaining the schema information.
schemaPeople.write.parquet("people.parquet")
# Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved.
# Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved.
# The result of loading a parquet file is also a DataFrame.
parquetFile = sqlContext.read.parquet("people.parquet")
@ -1075,7 +1147,7 @@ schemaPeople # The DataFrame from the previous example.
# DataFrames can be saved as Parquet files, maintaining the schema information.
saveAsParquetFile(schemaPeople, "people.parquet")
# Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved.
# Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved.
# The result of loading a parquet file is also a DataFrame.
parquetFile <- parquetFile(sqlContext, "people.parquet")
@ -1110,10 +1182,10 @@ SELECT * FROM parquetTable
### Partition Discovery
Table partitioning is a common optimization approach used in systems like Hive. In a partitioned
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
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:
@ -1155,7 +1227,7 @@ root
{% endhighlight %}
Notice that the data types of the partitioning columns are automatically inferred. Currently,
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
@ -1164,13 +1236,13 @@ can be configured by `spark.sql.sources.partitionColumnTypeInference.enabled`, w
### 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
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
turned it off by default starting from 1.5.0. You may enable it by
1. setting data source option `mergeSchema` to `true` when reading Parquet files (as shown in the
examples below), or
@ -1284,10 +1356,10 @@ processing.
1. Hive considers all columns nullable, while nullability in Parquet is significant
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:
Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation rules are:
1. 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. The reconciled field should have the data type of the Parquet side, so that
nullability is respected.
1. The reconciled schema contains exactly those fields defined in Hive metastore schema.
@ -1298,8 +1370,8 @@ Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation r
#### Metadata Refreshing
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
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.
@ -1362,7 +1434,7 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext`
<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
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>
@ -1400,7 +1472,7 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext`
<td>
<p>
The output committer class used by Parquet. The specified class needs to be a subclass of
<code>org.apache.hadoop.<br />mapreduce.OutputCommitter</code>. Typically, it's also a
<code>org.apache.hadoop.<br />mapreduce.OutputCommitter</code>. Typically, it's also a
subclass of <code>org.apache.parquet.hadoop.ParquetOutputCommitter</code>.
</p>
<p>
@ -1628,7 +1700,7 @@ YARN cluster. The convenient way to do this is adding them through the `--jars`
When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and
adds support for finding tables in the MetaStore and writing queries using HiveQL. Users who do
not have an existing Hive deployment can still create a `HiveContext`. When not configured by the
not have an existing Hive deployment can still create a `HiveContext`. When not configured by the
hive-site.xml, the context automatically creates `metastore_db` in the current directory and
creates `warehouse` directory indicated by HiveConf, which defaults to `/user/hive/warehouse`.
Note that you may need to grant write privilege on `/user/hive/warehouse` to the user who starts
@ -1738,10 +1810,10 @@ The following options can be used to configure the version of Hive that is used
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
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
<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 you application.</li>
</ol>
@ -1776,7 +1848,7 @@ The following options can be used to configure the version of Hive that is used
## JDBC To Other Databases
Spark SQL also includes a data source that can read data from other databases using JDBC. This
Spark SQL also includes a data source that can read data from other databases using JDBC. This
functionality should be preferred over using [JdbcRDD](api/scala/index.html#org.apache.spark.rdd.JdbcRDD).
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.
@ -1786,7 +1858,7 @@ provide a ClassTag.
run queries using Spark SQL).
To get started you will need to include the JDBC driver for you particular database on the
spark classpath. For example, to connect to postgres from the Spark Shell you would run the
spark classpath. For example, to connect to postgres from the Spark Shell you would run the
following command:
{% highlight bash %}
@ -1794,7 +1866,7 @@ SPARK_CLASSPATH=postgresql-9.3-1102-jdbc41.jar bin/spark-shell
{% endhighlight %}
Tables from the remote database can be loaded as a DataFrame or Spark SQL Temporary table using
the Data Sources API. The following options are supported:
the Data Sources API. The following options are supported:
<table class="table">
<tr><th>Property Name</th><th>Meaning</th></tr>
@ -1807,8 +1879,8 @@ the Data Sources API. The following options are supported:
<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
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>
@ -1816,7 +1888,7 @@ the Data Sources API. The following options are supported:
<tr>
<td><code>driver</code></td>
<td>
The class name of the JDBC driver needed to connect to this URL. This class will be loaded
The class name of the JDBC driver needed to connect to this URL. This class will be loaded
on the master and workers before running an JDBC commands to allow the driver to
register itself with the JDBC subsystem.
</td>
@ -1825,7 +1897,7 @@ the Data Sources API. The following options are supported:
<tr>
<td><code>partitionColumn, lowerBound, upperBound, numPartitions</code></td>
<td>
These options must all be specified if any of them is specified. They describe how to
These options must all be specified if any of them is 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
@ -1938,7 +2010,7 @@ Configuration of in-memory caching can be done using the `setConf` method on `SQ
<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
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>
@ -1947,7 +2019,7 @@ Configuration of in-memory caching can be done using the `setConf` method on `SQ
## Other Configuration Options
The following options can also be used to tune the performance of query execution. It is possible
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.
<table class="table">
@ -1957,7 +2029,7 @@ that these options will be deprecated in future release as more optimizations ar
<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
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 &lt;tableName&gt; COMPUTE STATISTICS noscan</code> has been run.
</td>
@ -1995,8 +2067,8 @@ To start the JDBC/ODBC server, run the following in the Spark directory:
./sbin/start-thriftserver.sh
This script accepts all `bin/spark-submit` command line options, plus a `--hiveconf` option to
specify Hive properties. You may run `./sbin/start-thriftserver.sh --help` for a complete list of
all available options. By default, the server listens on localhost:10000. You may override this
specify Hive properties. You may run `./sbin/start-thriftserver.sh --help` 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.:
{% highlight bash %}
@ -2062,10 +2134,10 @@ options.
## Upgrading From Spark SQL 1.5 to 1.6
- 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 `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add
- 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 `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add
this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` via `--conf`:
{% highlight bash %}
@ -2077,20 +2149,20 @@ options.
## Upgrading From Spark SQL 1.4 to 1.5
- 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 generation for expression evaluation. These features can both be disabled by setting
`spark.sql.tungsten.enabled` to `false`.
- Parquet schema merging is no longer enabled by default. It can be re-enabled by setting
- Parquet schema merging is no longer enabled by default. It can be re-enabled by setting
`spark.sql.parquet.mergeSchema` to `true`.
- Resolution of strings to columns in python now supports using dots (`.`) to qualify the column or
access nested values. For example `df['table.column.nestedField']`. However, this means that if
your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``).
access nested values. For example `df['table.column.nestedField']`. However, this means that if
your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``).
- In-memory columnar storage partition pruning is on by default. It can be disabled by setting
`spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`.
- Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum
precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now
precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now
used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`.
- Timestamps are now stored at a precision of 1us, rather than 1ns
- In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains
- In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains
unchanged.
- The canonical name of SQL/DataFrame functions are now lower case (e.g. sum vs SUM).
- It has been determined that using the DirectOutputCommitter when speculation is enabled is unsafe
@ -2183,38 +2255,38 @@ sqlContext.setConf("spark.sql.retainGroupColumns", "false")
## Upgrading from Spark SQL 1.0-1.2 to 1.3
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
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).
#### Rename of SchemaRDD to DataFrame
The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has
been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD
been renamed to `DataFrame`. 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 `.rdd` method.
implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method.
In Scala there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for
some use cases. It is still recommended that users update their code to use `DataFrame` instead.
some use cases. It is still recommended that users update their code to use `DataFrame` instead.
Java and Python users will need to update their code.
#### Unification of the Java and Scala APIs
Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`)
that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users
of either language should use `SQLContext` and `DataFrame`. In general theses classes try to
that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users
of either language should use `SQLContext` and `DataFrame`. In general theses classes try to
use types that are usable from both languages (i.e. `Array` 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.
Additionally the Java specific types API has been removed. Users of both Scala and Java should
Additionally the Java specific types API has been removed. Users of both Scala and Java should
use the classes present in `org.apache.spark.sql.types` to describe schema programmatically.
#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only)
Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought
all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit
all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit
conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`.
Users should now write `import sqlContext.implicits._`.
@ -2222,7 +2294,7 @@ Additionally, the implicit conversions now only augment RDDs that are composed o
case classes or tuples) with a method `toDF`, instead of applying automatically.
When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import
`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used:
`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used:
`import org.apache.spark.sql.functions._`.
#### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only)

View file

@ -19,20 +19,60 @@ package org.apache.spark.sql
import java.lang.reflect.Modifier
import scala.annotation.implicitNotFound
import scala.reflect.{ClassTag, classTag}
import org.apache.spark.annotation.Experimental
import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, encoderFor}
import org.apache.spark.sql.catalyst.expressions.{DecodeUsingSerializer, BoundReference, EncodeUsingSerializer}
import org.apache.spark.sql.types._
/**
* :: Experimental ::
* Used to convert a JVM object of type `T` to and from the internal Spark SQL representation.
*
* Encoders are not intended to be thread-safe and thus they are allow to avoid internal locking
* and reuse internal buffers to improve performance.
* == Scala ==
* Encoders are generally created automatically through implicits from a `SQLContext`.
*
* {{{
* import sqlContext.implicits._
*
* val ds = Seq(1, 2, 3).toDS() // implicitly provided (sqlContext.implicits.newIntEncoder)
* }}}
*
* == Java ==
* Encoders are specified by calling static methods on [[Encoders]].
*
* {{{
* List<String> data = Arrays.asList("abc", "abc", "xyz");
* Dataset<String> ds = context.createDataset(data, Encoders.STRING());
* }}}
*
* Encoders can be composed into tuples:
*
* {{{
* Encoder<Tuple2<Integer, String>> encoder2 = Encoders.tuple(Encoders.INT(), Encoders.STRING());
* List<Tuple2<Integer, String>> data2 = Arrays.asList(new scala.Tuple2(1, "a");
* Dataset<Tuple2<Integer, String>> ds2 = context.createDataset(data2, encoder2);
* }}}
*
* Or constructed from Java Beans:
*
* {{{
* Encoders.bean(MyClass.class);
* }}}
*
* == Implementation ==
* - Encoders are not required to be thread-safe and thus they do not need to use locks to guard
* against concurrent access if they reuse internal buffers to improve performance.
*
* @since 1.6.0
*/
@Experimental
@implicitNotFound("Unable to find encoder for type stored in a Dataset. Primitive types " +
"(Int, String, etc) and Product types (case classes) are supported by importing " +
"sqlContext.implicits._ Support for serializing other types will be added in future " +
"releases.")
trait Encoder[T] extends Serializable {
/** Returns the schema of encoding this type of object as a Row. */
@ -43,10 +83,12 @@ trait Encoder[T] extends Serializable {
}
/**
* Methods for creating encoders.
* :: Experimental ::
* Methods for creating an [[Encoder]].
*
* @since 1.6.0
*/
@Experimental
object Encoders {
/**

View file

@ -73,7 +73,26 @@ class TypedColumn[-T, U](
/**
* :: Experimental ::
* A column in a [[DataFrame]].
* A column that will be computed based on the data in a [[DataFrame]].
*
* A new column is constructed based on the input columns present in a dataframe:
*
* {{{
* df("columnName") // On a specific DataFrame.
* col("columnName") // A generic column no yet associcated with a DataFrame.
* col("columnName.field") // Extracting a struct field
* col("`a.column.with.dots`") // Escape `.` in column names.
* $"columnName" // Scala short hand for a named column.
* expr("a + 1") // A column that is constructed from a parsed SQL Expression.
* lit("1") // A column that produces a literal (constant) value.
* }}}
*
* [[Column]] objects can be composed to form complex expressions:
*
* {{{
* $"a" + 1
* $"a" === $"b"
* }}}
*
* @groupname java_expr_ops Java-specific expression operators
* @groupname expr_ops Expression operators