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.
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.
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.
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.
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 %}
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 %}
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 %}
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.
{% 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 %}
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.
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.
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 datase, similar to the inference that is performed on JSON files.
{% include_example schema_inferring python/sql/basic.py %}
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.
Row
s from the original RDD;StructType
matching the structure of Row
s in the RDD created in Step 1.Row
s via createDataFrame
method provided by SparkSession
.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.
Row
s from the original RDD;StructType
matching the structure of Row
s in the RDD created in Step 1.Row
s via createDataFrame
method provided by SparkSession
.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.
StructType
matching the structure of tuples or lists in the RDD created in the step 1.createDataFrame
method provided by SparkSession
.For example:
{% include_example programmatic_schema python/sql/basic.py %}
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.
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 %}
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.
Instead of using read API to load a file into DataFrame and query it, you can also query that file directly with SQL.
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.
DataFrames
can also be saved as persistent tables into Hive metastore using the saveAsTable
command. Notice 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.
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 when a table is dropped.
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.
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 %}
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.
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
mergeSchema
to true
when reading Parquet files (as shown in the examples below), orspark.sql.parquet.mergeSchema
to true
.{% include_example schema_merging python/sql/datasource.py %}
{% include_example schema_merging r/RSparkSQLExample.R %}
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.
There are two key differences between Hive and Parquet from the perspective of table schema processing.
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:
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.
The reconciled schema contains exactly those fields defined in Hive metastore schema.
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.
{% highlight scala %} // spark is an existing SparkSession spark.catalog.refreshTable(“my_table”) {% endhighlight %}
{% highlight java %} // spark is an existing SparkSession spark.catalog().refreshTable(“my_table”); {% endhighlight %}
{% highlight python %}
spark.catalog.refreshTable(“my_table”) {% endhighlight %}
{% highlight sql %} REFRESH TABLE my_table; {% endhighlight %}
Configuration of Parquet can be done using the setConf
method on SparkSession
or by running SET key=value
commands using SQL.
Note that the file that is offered as a json file is not a typical JSON file. Each line must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail.
{% include_example json_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
Note that the file that is offered as a json file is not a typical JSON file. Each line must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail.
{% include_example json_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
Note that the file that is offered as a json file is not a typical JSON file. Each line must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail.
{% include_example json_dataset python/sql/datasource.py %}
Note that the file that is offered as a json file is not a typical JSON file. Each line must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail.
{% include_example json_dataset r/RSparkSQLExample.R %}
{% highlight sql %}
CREATE TEMPORARY VIEW jsonTable USING org.apache.spark.sql.json OPTIONS ( path “examples/src/main/resources/people.json” )
SELECT * FROM jsonTable
{% endhighlight %}
Spark SQL also supports reading and writing data stored in Apache Hive. 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.
Configuration of Hive is done by placing your hive-site.xml
, core-site.xml
(for security configuration), and hdfs-site.xml
(for HDFS configuration) file in conf/
.
When working with Hive, one must instantiate SparkSession
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 hive-site.xml
, the context automatically creates metastore_db
in the current directory and creates a directory configured by spark.sql.warehouse.dir
, which defaults to the directory spark-warehouse
in the current directory that the Spark application is started. Note that the hive.metastore.warehouse.dir
property in hive-site.xml
is deprecated since Spark 2.0.0. Instead, use spark.sql.warehouse.dir
to specify the default location of database in warehouse. You may need to grant write privilege to the user who starts the Spark application.
When working with Hive one must instantiate SparkSession
with Hive support. This adds support for finding tables in the MetaStore and writing queries using HiveQL.
{% include_example spark_hive r/RSparkSQLExample.R %}
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).
The following options can be used to configure the version of Hive that is used to retrieve metadata:
Spark SQL also includes a data source that can read data from other databases using JDBC. This functionality should be preferred over using 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. 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).
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 following command:
{% highlight bash %} bin/spark-shell --driver-class-path postgresql-9.4.1207.jar --jars postgresql-9.4.1207.jar {% 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:
{% highlight sql %}
CREATE TEMPORARY VIEW jdbcTable USING org.apache.spark.sql.jdbc OPTIONS ( url “jdbc:postgresql:dbserver”, dbtable “schema.tablename” )
{% endhighlight %}
For some workloads it is possible to improve performance by either caching data in memory, or by turning on some experimental options.
Spark SQL can cache tables using an in-memory columnar format by calling spark.cacheTable("tableName")
or dataFrame.cache()
. Then Spark SQL will scan only required columns and will automatically tune compression to minimize memory usage and GC pressure. You can call spark.uncacheTable("tableName")
to remove the table from memory.
Configuration of in-memory caching can be done using the setConf
method on SparkSession
or by running SET key=value
commands using SQL.
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.
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.
The Thrift JDBC/ODBC server implemented here corresponds to the HiveServer2
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.
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 behaviour via either environment variables, i.e.:
{% highlight bash %} export HIVE_SERVER2_THRIFT_PORT= export HIVE_SERVER2_THRIFT_BIND_HOST= ./sbin/start-thriftserver.sh
--master
... {% endhighlight %}
or system properties:
{% highlight bash %} ./sbin/start-thriftserver.sh
--hiveconf hive.server2.thrift.port=
--hiveconf hive.server2.thrift.bind.host=
--master ... {% endhighlight %}
Now you can use beeline to test the Thrift JDBC/ODBC server:
./bin/beeline
Connect to the JDBC/ODBC server in beeline with:
beeline> !connect jdbc:hive2://localhost:10000
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 beeline documentation.
Configuration of Hive is done by placing your hive-site.xml
, core-site.xml
and hdfs-site.xml
files in conf/
.
You may also use the beeline script that comes with Hive.
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 hive-site.xml
file in conf/
:
hive.server2.transport.mode - Set this to value: http hive.server2.thrift.http.port - HTTP port number fo listen on; default is 10001 hive.server2.http.endpoint - HTTP endpoint; default is cliservice
To test, use beeline to connect to the JDBC/ODBC server in http mode with:
beeline> !connect jdbc:hive2://<host>:<port>/<database>?hive.server2.transport.mode=http;hive.server2.thrift.http.path=<http_endpoint>
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.
To start the Spark SQL CLI, run the following in the Spark directory:
./bin/spark-sql
Configuration of Hive is done by placing your hive-site.xml
, core-site.xml
and hdfs-site.xml
files in conf/
. You may run ./bin/spark-sql --help
for a complete list of all available options.
SparkSession
is now the new entry point of Spark that replaces the old SQLContext
and HiveContext
. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new catalog
interface is accessible from SparkSession
- existing API on databases and tables access such as listTables
, createExternalTable
, dropTempView
, cacheTable
are moved here.
Dataset API and DataFrame API are unified. In Scala, DataFrame
becomes a type alias for Dataset[Row]
, while Java API users must replace DataFrame
with Dataset<Row>
. Both the typed transformations (e.g. map
, filter
, and groupByKey
) and untyped transformations (e.g. select
and groupBy
) 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, DataFrame
remains the primary programing abstraction, which is analogous to the single-node data frame notion in these languages.
Dataset and DataFrame API unionAll
has been deprecated and replaced by union
Dataset and DataFrame API explode
has been deprecated, alternatively, use functions.explode()
with select
or flatMap
Dataset and DataFrame API registerTempTable
has been deprecated and replaced by createOrReplaceTempView
Changes to CREATE TABLE ... LOCATION
behavior for Hive tables.
CREATE TABLE ... LOCATION
is equivalent to CREATE EXTERNAL TABLE ... LOCATION
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.DROP TABLE
statements on those tables will not remove the data.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 %} ./sbin/start-thriftserver.sh
--conf spark.sql.hive.thriftServer.singleSession=true
... {% endhighlight %}
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.
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 SPARK-11724 for details.
spark.sql.tungsten.enabled
to false
.spark.sql.parquet.mergeSchema
to true
..
) 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
).spark.sql.inMemoryColumnarStorage.partitionPruning
to false
.BigDecimal
objects, a precision of (38, 18) is now used. When no precision is specified in DDL then the default remains Decimal(10, 0)
.sql
dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains unchanged.REFRESH TABLE
SQL command or HiveContext
's refreshTable
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.Based on user feedback, we created a new, more fluid API for reading data in (SQLContext.read
) and writing data out (DataFrame.write
), and deprecated the old APIs (e.g. SQLContext.parquetFile
, SQLContext.jsonFile
).
See the API docs for SQLContext.read
( Scala, Java, Python ) and DataFrame.write
( Scala, Java, Python ) more information.
Based on user feedback, we changed the default behavior of DataFrame.groupBy().agg()
to retain the grouping columns in the resulting DataFrame
. To keep the behavior in 1.3, set spark.sql.retainGroupColumns
to false
.
// In 1.3.x, in order for the grouping column “department” to show up, // it must be included explicitly as part of the agg function call. df.groupBy(“department”).agg($“department”, max(“age”), sum(“expense”))
// In 1.4+, grouping column “department” is included automatically. df.groupBy(“department”).agg(max(“age”), sum(“expense”))
// Revert to 1.3 behavior (not retaining grouping column) by: sqlContext.setConf(“spark.sql.retainGroupColumns”, “false”)
{% endhighlight %}
// In 1.3.x, in order for the grouping column “department” to show up, // it must be included explicitly as part of the agg function call. df.groupBy(“department”).agg(col(“department”), max(“age”), sum(“expense”));
// In 1.4+, grouping column “department” is included automatically. df.groupBy(“department”).agg(max(“age”), sum(“expense”));
// Revert to 1.3 behavior (not retaining grouping column) by: sqlContext.setConf(“spark.sql.retainGroupColumns”, “false”);
{% endhighlight %}
import pyspark.sql.functions as func
df.groupBy(“department”).agg(df[“department”], func.max(“age”), func.sum(“expense”))
df.groupBy(“department”).agg(func.max(“age”), func.sum(“expense”))
sqlContext.setConf(“spark.sql.retainGroupColumns”, “false”)
{% endhighlight %}
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.
Note that this change is only for Scala API, not for PySpark and SparkR.
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).
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 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.
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. Java and Python users will need to update their code.
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 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 use the classes present in org.apache.spark.sql.types
to describe schema programmatically.
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 conversions for converting RDD
s into DataFrame
s into an object inside of the SQLContext
. Users should now write import sqlContext.implicits._
.
Additionally, the implicit conversions now only augment RDDs that are composed of Product
s (i.e., 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: import org.apache.spark.sql.functions._
.
Spark 1.3 removes the type aliases that were present in the base sql package for DataType
. Users should instead import the classes in org.apache.spark.sql.types
sqlContext.udf
(Java & Scala)Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been moved into the udf object in SQLContext
.
sqlContext.udf.register(“strLen”, (s: String) => s.length())
{% endhighlight %}
sqlContext.udf().register(“strLen”, (String s) -> s.length(), DataTypes.IntegerType);
{% endhighlight %}
Python UDF registration is unchanged.
When using DataTypes in Python you will need to construct them (i.e. StringType()
) instead of referencing a singleton.
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 1.2.1. Also see [Interacting with Different Versions of Hive Metastore] (#interacting-with-different-versions-of-hive-metastore)).
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.
Spark SQL supports the vast majority of Hive features, such as:
SELECT
GROUP BY
ORDER BY
CLUSTER BY
SORT BY
=
, ⇔
, ==
, <>
, <
, >
, >=
, <=
, etc)+
, -
, *
, /
, %
, etc)AND
, &&
, OR
, ||
, etc)sign
, ln
, cos
, etc)instr
, length
, printf
, etc)JOIN
{LEFT|RIGHT|FULL} OUTER JOIN
LEFT SEMI JOIN
CROSS JOIN
SELECT col FROM ( SELECT a + b AS col from t1) t2
CREATE TABLE
CREATE TABLE AS SELECT
ALTER TABLE
TINYINT
SMALLINT
INT
BIGINT
BOOLEAN
FLOAT
DOUBLE
STRING
BINARY
TIMESTAMP
DATE
ARRAY<>
MAP<>
STRUCT<>
Below is a list of Hive features that we don't support yet. Most of these features are rarely used in Hive deployments.
Major Hive Features
Esoteric Hive Features
UNION
typeHive Input/Output Formats
Hive Optimizations
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.
SET spark.sql.shuffle.partitions=[num_tasks];
”.STREAMTABLE
hint in join: Spark SQL does not follow the STREAMTABLE
hint.Spark SQL and DataFrames support the following data types:
ByteType
: Represents 1-byte signed integer numbers. The range of numbers is from -128
to 127
.ShortType
: Represents 2-byte signed integer numbers. The range of numbers is from -32768
to 32767
.IntegerType
: Represents 4-byte signed integer numbers. The range of numbers is from -2147483648
to 2147483647
.LongType
: Represents 8-byte signed integer numbers. The range of numbers is from -9223372036854775808
to 9223372036854775807
.FloatType
: Represents 4-byte single-precision floating point numbers.DoubleType
: Represents 8-byte double-precision floating point numbers.DecimalType
: Represents arbitrary-precision signed decimal numbers. Backed internally by java.math.BigDecimal
. A BigDecimal
consists of an arbitrary precision integer unscaled value and a 32-bit integer scale.StringType
: Represents character string values.BinaryType
: Represents byte sequence values.BooleanType
: Represents boolean values.TimestampType
: Represents values comprising values of fields year, month, day, hour, minute, and second.DateType
: Represents values comprising values of fields year, month, day.ArrayType(elementType, containsNull)
: Represents values comprising a sequence of elements with the type of elementType
. containsNull
is used to indicate if elements in a ArrayType
value can have null
values.MapType(keyType, valueType, valueContainsNull)
: Represents values comprising a set of key-value pairs. The data type of keys are described by keyType
and the data type of values are described by valueType
. For a MapType
value, keys are not allowed to have null
values. valueContainsNull
is used to indicate if values of a MapType
value can have null
values.StructType(fields)
: Represents values with the structure described by a sequence of StructField
s (fields
).StructField(name, dataType, nullable)
: Represents a field in a StructType
. The name of a field is indicated by name
. The data type of a field is indicated by dataType
. nullable
is used to indicate if values of this fields can have null
values.All data types of Spark SQL are located in the package org.apache.spark.sql.types
. You can access them by doing
{% include_example data_types scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
All data types of Spark SQL are located in the package of org.apache.spark.sql.types
. To access or create a data type, please use factory methods provided in org.apache.spark.sql.types.DataTypes
.
All data types of Spark SQL are located in the package of pyspark.sql.types
. You can access them by doing {% highlight python %} from pyspark.sql.types import * {% endhighlight %}
There is specially handling for not-a-number (NaN) when dealing with float
or double
types that does not exactly match standard floating point semantics. Specifically: