aboutsummaryrefslogtreecommitdiff
path: root/docs/sql-programming-guide.md
diff options
context:
space:
mode:
authorMichael Armbrust <michael@databricks.com>2014-09-07 21:34:46 -0400
committerMichael Armbrust <michael@databricks.com>2014-09-07 21:34:46 -0400
commit39db1bfdab434c867044ad4c70fe93a96fb287ad (patch)
tree5766bd6f15fbbeb803fb27a4f56ba573c69fe525 /docs/sql-programming-guide.md
parent6754570d83044c4fbaf0d2ac2378a0e081a93629 (diff)
downloadspark-39db1bfdab434c867044ad4c70fe93a96fb287ad.tar.gz
spark-39db1bfdab434c867044ad4c70fe93a96fb287ad.tar.bz2
spark-39db1bfdab434c867044ad4c70fe93a96fb287ad.zip
[SQL] Update SQL Programming Guide
Author: Michael Armbrust <michael@databricks.com> Author: Yin Huai <huai@cse.ohio-state.edu> Closes #2258 from marmbrus/sqlDocUpdate and squashes the following commits: f3d450b [Michael Armbrust] fix brackets bea3bfa [Michael Armbrust] Davies suggestions 3a29fe2 [Michael Armbrust] tighten visibility a71aa36 [Michael Armbrust] Draft of doc updates 52932c0 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into sqlDocUpdate 1e8c849 [Yin Huai] Update the example used for applySchema. 9457c39 [Yin Huai] Update doc. 31ba240 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeDoc 29bc668 [Yin Huai] Draft doc for data type and schema APIs.
Diffstat (limited to 'docs/sql-programming-guide.md')
-rw-r--r--docs/sql-programming-guide.md952
1 files changed, 857 insertions, 95 deletions
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 8f7fb5431c..1814fef465 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -68,6 +68,16 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc)
import sqlContext.createSchemaRDD
{% endhighlight %}
+In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict
+super set of the functionality provided by the basic SQLContext. Additional features include
+the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the
+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.2 release of Spark. Future releases will focus on bringing SQLContext up to
+feature parity with a HiveContext.
+
</div>
<div data-lang="java" markdown="1">
@@ -81,6 +91,16 @@ JavaSparkContext sc = ...; // An existing JavaSparkContext.
JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc);
{% endhighlight %}
+In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict
+super set of the functionality provided by the basic SQLContext. Additional features include
+the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the
+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.2 release of Spark. Future releases will focus on bringing SQLContext up to
+feature parity with a HiveContext.
+
</div>
<div data-lang="python" markdown="1">
@@ -94,36 +114,52 @@ from pyspark.sql import SQLContext
sqlContext = SQLContext(sc)
{% endhighlight %}
-</div>
+In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict
+super set of the functionality provided by the basic SQLContext. Additional features include
+the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the
+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.2 release of Spark. Future releases will focus on bringing SQLContext up to
+feature parity with a HiveContext.
</div>
-# Data Sources
-
-<div class="codetabs">
-<div data-lang="scala" markdown="1">
-Spark SQL supports operating on a variety of data sources through the `SchemaRDD` interface.
-Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources.
</div>
-<div data-lang="java" markdown="1">
-Spark SQL supports operating on a variety of data sources through the `JavaSchemaRDD` interface.
-Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources.
-</div>
+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,
+ this is recommended for most use cases.
+
+# Data Sources
-<div data-lang="python" markdown="1">
Spark SQL supports operating on a variety of data sources through the `SchemaRDD` interface.
-Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources.
-</div>
-</div>
+A SchemaRDD can be operated on as normal RDDs and can also be registered as a temporary table.
+Registering a SchemaRDD as a table allows you to run SQL queries over its data. This section
+describes the various methods for loading data into a SchemaRDD.
## RDDs
+Spark SQL supports two different methods for converting existing RDDs into SchemaRDDs. 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 went the schema is known ahead
+of time, while you are writing your Spark application.
+
+The second method for creating SchemaRDDs is through a programmatic interface that allows you to
+construct a schema and then apply it to and existing RDD. While this method is more verbose, it allows
+you to construct SchemaRDDs when the columns and their types are not known until runtime.
+
+### Inferring the Schema Using Reflection
<div class="codetabs">
<div data-lang="scala" markdown="1">
-One type of table that is supported by Spark SQL is an RDD of Scala case classes. The case class
+The Scala interaface for Spark SQL supports automatically converting an RDD containing case classes
+to a SchemaRDD. 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 SchemaRDD and then be
@@ -156,8 +192,9 @@ teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
<div data-lang="java" markdown="1">
-One type of table that is supported by Spark SQL is an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly). The BeanInfo
-defines the schema of the table. Currently, Spark SQL does not support JavaBeans that contain
+Spark SQL supports automatically converting an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly)
+into a Schema RDD. 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
class that implements Serializable and has getters and setters for all of its fields.
@@ -192,7 +229,7 @@ for the JavaBean.
{% highlight java %}
// sc is an existing JavaSparkContext.
-JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc)
+JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc);
// Load a text file and convert each line to a JavaBean.
JavaRDD<Person> people = sc.textFile("examples/src/main/resources/people.txt").map(
@@ -229,24 +266,24 @@ List<String> teenagerNames = teenagers.map(new Function<Row, String>() {
<div data-lang="python" markdown="1">
-One type of table that is supported by Spark SQL is an RDD of dictionaries. The keys of the
-dictionary define the columns names of the table, and the types are inferred by looking at the first
-row. Any RDD of dictionaries can converted to a SchemaRDD and then registered as a table. Tables
-can be used in subsequent SQL statements.
+Spark SQL can convert an RDD of Row objects to a SchemaRDD, 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 columns names of the table,
+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 version we
+plan to more completely infer the schema by looking at more data, similar to the inference that is
+performed on JSON files.
{% highlight python %}
# sc is an existing SparkContext.
-from pyspark.sql import SQLContext
+from pyspark.sql import SQLContext, Row
sqlContext = SQLContext(sc)
# Load a text file and convert each line to a dictionary.
lines = sc.textFile("examples/src/main/resources/people.txt")
parts = lines.map(lambda l: l.split(","))
-people = parts.map(lambda p: {"name": p[0], "age": int(p[1])})
+people = parts.map(lambda p: Row(name=p[0], age=int(p[1])))
# Infer the schema, and register the SchemaRDD as a table.
-# In future versions of PySpark we would like to add support for registering RDDs with other
-# datatypes as tables
schemaPeople = sqlContext.inferSchema(people)
schemaPeople.registerTempTable("people")
@@ -263,15 +300,191 @@ for teenName in teenNames.collect():
</div>
-**Note that Spark SQL currently uses a very basic SQL parser.**
-Users that want a more complete dialect of SQL should look at the HiveQL support provided by
-`HiveContext`.
+### Programmatically Specifying the Schema
+
+<div class="codetabs">
+
+<div data-lang="scala" markdown="1">
+
+In cases that 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 `SchemaRDD` can be created programmatically with three steps.
+
+1. Create an RDD of `Row`s from the original RDD;
+2. Create the schema represented by a `StructType` matching the structure of
+`Row`s in the RDD created in the step 1.
+3. Apply the schema to the RDD of `Row`s via `applySchema` method provided
+by `SQLContext`.
+
+For example:
+{% highlight scala %}
+// sc is an existing SparkContext.
+val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+
+// Create an RDD
+val people = sc.textFile("examples/src/main/resources/people.txt")
+
+// The schema is encoded in a string
+val schemaString = "name age"
+
+// Import Spark SQL data types and Row.
+import org.apache.spark.sql._
+
+// Generate the schema based on the string of schema
+val schema =
+ StructType(
+ schemaString.split(" ").map(fieldName => StructField(fieldName, StringType, true)))
+
+// Convert records of the RDD (people) to Rows.
+val rowRDD = people.map(_.split(",")).map(p => Row(p(0), p(1).trim))
+
+// Apply the schema to the RDD.
+val peopleSchemaRDD = sqlContext.applySchema(rowRDD, schema)
+
+// Register the SchemaRDD as a table.
+peopleSchemaRDD.registerTempTable("people")
+
+// SQL statements can be run by using the sql methods provided by sqlContext.
+val results = sqlContext.sql("SELECT name FROM people")
+
+// The results of SQL queries are SchemaRDDs and support all the normal RDD operations.
+// The columns of a row in the result can be accessed by ordinal.
+results.map(t => "Name: " + t(0)).collect().foreach(println)
+{% endhighlight %}
+
+
+</div>
+
+<div data-lang="java" markdown="1">
+
+In cases that 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 `SchemaRDD` can be created programmatically with three steps.
+
+1. Create an RDD of `Row`s from the original RDD;
+2. Create the schema represented by a `StructType` matching the structure of
+`Row`s in the RDD created in the step 1.
+3. Apply the schema to the RDD of `Row`s via `applySchema` method provided
+by `JavaSQLContext`.
+
+For example:
+{% highlight java %}
+// Import factory methods provided by DataType.
+import org.apache.spark.sql.api.java.DataType
+// Import StructType and StructField
+import org.apache.spark.sql.api.java.StructType
+import org.apache.spark.sql.api.java.StructField
+// Import Row.
+import org.apache.spark.sql.api.java.Row
+
+// sc is an existing JavaSparkContext.
+JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc);
+
+// Load a text file and convert each line to a JavaBean.
+JavaRDD<String> people = sc.textFile("examples/src/main/resources/people.txt");
+
+// The schema is encoded in a string
+String schemaString = "name age";
+
+// Generate the schema based on the string of schema
+List<StructField> fields = new ArrayList<StructField>();
+for (String fieldName: schemaString.split(" ")) {
+ fields.add(DataType.createStructField(fieldName, DataType.StringType, true));
+}
+StructType schema = DataType.createStructType(fields);
+
+// Convert records of the RDD (people) to Rows.
+JavaRDD<Row> rowRDD = people.map(
+ new Function<String, Row>() {
+ public Row call(String record) throws Exception {
+ String[] fields = record.split(",");
+ return Row.create(fields[0], fields[1].trim());
+ }
+ });
+
+// Apply the schema to the RDD.
+JavaSchemaRDD peopleSchemaRDD = sqlContext.applySchema(rowRDD, schema);
+
+// Register the SchemaRDD as a table.
+peopleSchemaRDD.registerTempTable("people");
+
+// SQL can be run over RDDs that have been registered as tables.
+JavaSchemaRDD results = sqlContext.sql("SELECT name FROM people");
+
+// The results of SQL queries are SchemaRDDs and support all the normal RDD operations.
+// The columns of a row in the result can be accessed by ordinal.
+List<String> names = results.map(new Function<Row, String>() {
+ public String call(Row row) {
+ return "Name: " + row.getString(0);
+ }
+}).collect();
+
+{% endhighlight %}
+
+</div>
+
+<div data-lang="python" markdown="1">
+
+For some cases (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), it is desired to create `SchemaRDD` with a programmatically way.
+It can be done with three steps.
+
+1. Create an RDD of tuples or lists from the original RDD;
+2. Create the schema represented by a `StructType` matching the structure of
+tuples or lists in the RDD created in the step 1.
+3. Apply the schema to the RDD via `applySchema` method provided by `SQLContext`.
+
+For example:
+{% highlight python %}
+# Import SQLContext and data types
+from pyspark.sql import *
+
+# sc is an existing SparkContext.
+sqlContext = SQLContext(sc)
+
+# Load a text file and convert each line to a tuple.
+lines = sc.textFile("examples/src/main/resources/people.txt")
+parts = lines.map(lambda l: l.split(","))
+people = parts.map(lambda p: (p[0], p[1].strip()))
+
+# The schema is encoded in a string.
+schemaString = "name age"
+
+fields = [StructField(field_name, StringType(), True) for field_name in schemaString.split()]
+schema = StructType(fields)
+
+# Apply the schema to the RDD.
+schemaPeople = sqlContext.applySchema(people, schema)
+
+# Register the SchemaRDD as a table.
+schemaPeople.registerTempTable("people")
+
+# SQL can be run over SchemaRDDs that have been registered as a table.
+results = sqlContext.sql("SELECT name FROM people")
+
+# The results of SQL queries are RDDs and support all the normal RDD operations.
+names = results.map(lambda p: "Name: " + p.name)
+for name in names.collect():
+ print name
+{% endhighlight %}
+
+
+</div>
+
+</div>
## Parquet Files
[Parquet](http://parquet.io) 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. Using the data from the above example:
+of the original data.
+
+### Loading Data Programmatically
+
+Using the data from the above example:
<div class="codetabs">
@@ -349,7 +562,40 @@ for teenName in teenNames.collect():
</div>
-</div>
+</div>
+
+### Configuration
+
+Configuration of parquet can be done using the `setConf` method on SQLContext or by running
+`SET key=value` commands using SQL.
+
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr>
+ <td><code>spark.sql.parquet.binaryAsString</code></td>
+ <td>false</td>
+ <td>
+ Some other parquet producing systems, in particular Impala and older versions of Spark SQL, do
+ not differentiate between binary data and strings when writing out the parquet schema. This
+ flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems.
+ </td>
+</tr>
+<tr>
+ <td><code>spark.sql.parquet.cacheMetadata</code></td>
+ <td>false</td>
+ <td>
+ Turns on caching of parquet schema metadata. Can speed up querying
+ </td>
+</tr>
+<tr>
+ <td><code>spark.sql.parquet.compression.codec</code></td>
+ <td>snappy</td>
+ <td>
+ Sets the compression codec use when writing parquet files. Acceptable values include:
+ uncompressed, snappy, gzip, lzo.
+ </td>
+</tr>
+</table>
## JSON Datasets
<div class="codetabs">
@@ -493,13 +739,13 @@ directory.
{% highlight scala %}
// sc is an existing SparkContext.
-val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc)
+val sqlContext = new org.apache.spark.sql.hive.HiveContext(sc)
-hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
-hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src")
+sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
+sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src")
// Queries are expressed in HiveQL
-hiveContext.sql("FROM src SELECT key, value").collect().foreach(println)
+sqlContext.sql("FROM src SELECT key, value").collect().foreach(println)
{% endhighlight %}
</div>
@@ -513,13 +759,13 @@ expressed in HiveQL.
{% highlight java %}
// sc is an existing JavaSparkContext.
-JavaHiveContext hiveContext = new org.apache.spark.sql.hive.api.java.HiveContext(sc);
+JavaHiveContext sqlContext = new org.apache.spark.sql.hive.api.java.HiveContext(sc);
-hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)");
-hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src");
+sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)");
+sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src");
// Queries are expressed in HiveQL.
-Row[] results = hiveContext.sql("FROM src SELECT key, value").collect();
+Row[] results = sqlContext.sql("FROM src SELECT key, value").collect();
{% endhighlight %}
@@ -535,44 +781,97 @@ expressed in HiveQL.
{% highlight python %}
# sc is an existing SparkContext.
from pyspark.sql import HiveContext
-hiveContext = HiveContext(sc)
+sqlContext = HiveContext(sc)
-hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
-hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src")
+sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
+sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src")
# Queries can be expressed in HiveQL.
-results = hiveContext.sql("FROM src SELECT key, value").collect()
+results = sqlContext.sql("FROM src SELECT key, value").collect()
{% endhighlight %}
</div>
</div>
-# Writing Language-Integrated Relational Queries
+# Performance Tuning
-**Language-Integrated queries are currently only supported in Scala.**
-
-Spark SQL also supports a domain specific language for writing queries. Once again,
-using the data from the above examples:
+For some workloads it is possible to improve performance by either caching data in memory, or by
+turning on some experimental options.
-{% highlight scala %}
-// sc is an existing SparkContext.
-val sqlContext = new org.apache.spark.sql.SQLContext(sc)
-// Importing the SQL context gives access to all the public SQL functions and implicit conversions.
-import sqlContext._
-val people: RDD[Person] = ... // An RDD of case class objects, from the first example.
+## Caching Data In Memory
-// The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19'
-val teenagers = people.where('age >= 10).where('age <= 19).select('name)
-teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
-{% endhighlight %}
+Spark SQL can cache tables using an in-memory columnar format by calling `cacheTable("tableName")`.
+Then Spark SQL will scan only required columns and will automatically tune compression to minimize
+memory usage and GC pressure. You can call `uncacheTable("tableName")` to remove the table from memory.
-The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers
-prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are
-evaluated by the SQL execution engine. A full list of the functions supported can be found in the
-[ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD).
+Note that if you just call `cache` rather than `cacheTable`, tables will _not_ be cached in
+in-memory columnar format. So we strongly recommend using `cacheTable` whenever you want to
+cache tables.
-<!-- TODO: Include the table of operations here. -->
+Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running
+`SET key=value` commands using SQL.
+
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr>
+ <td><code>spark.sql.inMemoryColumnarStorage.compressed</code></td>
+ <td>false</td>
+ <td>
+ When set to true Spark SQL will automatically select a compression codec for each column based
+ on statistics of the data.
+ </td>
+</tr>
+<tr>
+ <td><code>spark.sql.inMemoryColumnarStorage.batchSize</code></td>
+ <td>1000</td>
+ <td>
+ Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization
+ and compression, but risk OOMs when caching data.
+ </td>
+</tr>
+
+</table>
+
+## Other Configuration
+
+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">
+ <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+ <tr>
+ <td><code>spark.sql.autoBroadcastJoinThreshold</code></td>
+ <td>false</td>
+ <td>
+ Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when
+ performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently
+ statistics are only supported for Hive Metastore tables where the command
+ `ANALYZE TABLE &lt;tableName&gt; COMPUTE STATISTICS noscan` has been run.
+ </td>
+ </tr>
+ <tr>
+ <td><code>spark.sql.codegen</code></td>
+ <td>false</td>
+ <td>
+ When true, code will be dynamically generated at runtime for expression evaluation in a specific
+ query. For some queries with complicated expression this option can lead to significant speed-ups.
+ However, for simple queries this can actually slow down query execution.
+ </td>
+ </tr>
+ <tr>
+ <td><code>spark.sql.shuffle.partitions</code></td>
+ <td>200</td>
+ <td>
+ Configures the number of partitions to use when shuffling data for joins or aggregations.
+ </td>
+ </tr>
+</table>
+
+# Other SQL Interfaces
+
+Spark SQL also supports interfaces for running SQL queries directly without the need to write any
+code.
## Running the Thrift JDBC server
@@ -602,14 +901,28 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`.
You may also use the beeline script comes with Hive.
+## Running the Spark SQL CLI
+
+The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute
+queries input from command line. Note: 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` file in `conf/`.
+You may run `./bin/spark-sql --help` for a complete list of all available
+options.
+
+# Compatibility with Other Systems
+
+## Migration Guide for Shark Users
To set a [Fair Scheduler](job-scheduling.html#fair-scheduler-pools) pool for a JDBC client session,
users can set the `spark.sql.thriftserver.scheduler.pool` variable:
SET spark.sql.thriftserver.scheduler.pool=accounting;
-### Migration Guide for Shark Users
-
-#### Reducer number
+### Reducer number
In Shark, default reducer number is 1 and is controlled by the property `mapred.reduce.tasks`. Spark
SQL deprecates this property by a new property `spark.sql.shuffle.partitions`, whose default value
@@ -625,7 +938,7 @@ You may also put this property in `hive-site.xml` to override the default value.
For now, the `mapred.reduce.tasks` property is still recognized, and is converted to
`spark.sql.shuffle.partitions` automatically.
-#### Caching
+### Caching
The `shark.cache` table property no longer exists, and tables whose name end with `_cached` are no
longer automatically cached. Instead, we provide `CACHE TABLE` and `UNCACHE TABLE` statements to
@@ -634,9 +947,9 @@ let user control table caching explicitly:
CACHE TABLE logs_last_month;
UNCACHE TABLE logs_last_month;
-**NOTE:** `CACHE TABLE tbl` is lazy, it only marks table `tbl` as "need to by cached if necessary",
-but doesn't actually cache it until a query that touches `tbl` is executed. To force the table to be
-cached, you may simply count the table immediately after executing `CACHE TABLE`:
+**NOTE:** `CACHE TABLE tbl` is lazy, similar to `.cache` on an RDD. This command only marks `tbl` to ensure that
+partitions are cached when calculated but doesn't actually cache it until a query that touches `tbl` is executed.
+To force the table to be cached, you may simply count the table immediately after executing `CACHE TABLE`:
CACHE TABLE logs_last_month;
SELECT COUNT(1) FROM logs_last_month;
@@ -647,15 +960,18 @@ Several caching related features are not supported yet:
* RDD reloading
* In-memory cache write through policy
-### Compatibility with Apache Hive
+## Compatibility with Apache Hive
+
+Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. Currently Spark
+SQL is based on Hive 0.12.0.
#### Deploying in Existing Hive Warehouses
-Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive
+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.
-#### Supported Hive Features
+### Supported Hive Features
Spark SQL supports the vast majority of Hive features, such as:
@@ -705,13 +1021,14 @@ Spark SQL supports the vast majority of Hive features, such as:
* `MAP<>`
* `STRUCT<>`
-#### Unsupported Hive Functionality
+### Unsupported Hive Functionality
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**
+* Spark SQL does not currently support inserting to tables using dynamic partitioning.
* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL
doesn't support buckets yet.
@@ -721,11 +1038,11 @@ in Hive deployments.
have the same input format.
* Non-equi outer join: For the uncommon use case of using outer joins with non-equi join conditions
(e.g. condition "`key < 10`"), Spark SQL will output wrong result for the `NULL` tuple.
-* `UNIONTYPE`
+* `UNION` type and `DATE` type
* Unique join
* Single query multi insert
* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at
- the moment.
+ the moment and only supports populating the sizeInBytes field of the hive metastore.
**Hive Input/Output Formats**
@@ -735,7 +1052,7 @@ in Hive deployments.
**Hive Optimizations**
A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are
-not necessary due to Spark SQL's in-memory computational model. Others are slotted for future
+less important due to Spark SQL's in-memory computational model. Others are slotted for future
releases of Spark SQL.
* Block level bitmap indexes and virtual columns (used to build indexes)
@@ -743,8 +1060,7 @@ releases of Spark SQL.
Hive automatically converts the join into a map join. We are adding this auto conversion in the
next release.
* Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you
- need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". We are going to add auto-setting of parallelism in the
- next release.
+ need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`".
* Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still
launches tasks to compute the result.
* Skew data flag: Spark SQL does not follow the skew data flags in Hive.
@@ -753,25 +1069,471 @@ releases of Spark SQL.
Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS
metadata. Spark SQL does not support that.
-## Running the Spark SQL CLI
+# Writing Language-Integrated Relational Queries
-The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute
-queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server.
+**Language-Integrated queries are experimental and currently only supported in Scala.**
-To start the Spark SQL CLI, run the following in the Spark directory:
+Spark SQL also supports a domain specific language for writing queries. Once again,
+using the data from the above examples:
- ./bin/spark-sql
+{% highlight scala %}
+// sc is an existing SparkContext.
+val sqlContext = new org.apache.spark.sql.SQLContext(sc)
+// Importing the SQL context gives access to all the public SQL functions and implicit conversions.
+import sqlContext._
+val people: RDD[Person] = ... // An RDD of case class objects, from the first example.
-Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`.
-You may run `./bin/spark-sql --help` for a complete list of all available
-options.
+// The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19'
+val teenagers = people.where('age >= 10).where('age <= 19).select('name)
+teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
+{% endhighlight %}
-# Cached tables
+The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers
+prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are
+evaluated by the SQL execution engine. A full list of the functions supported can be found in the
+[ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD).
-Spark SQL can cache tables using an in-memory columnar format by calling `cacheTable("tableName")`.
-Then Spark SQL will scan only required columns and will automatically tune compression to minimize
-memory usage and GC pressure. You can call `uncacheTable("tableName")` to remove the table from memory.
+<!-- TODO: Include the table of operations here. -->
+
+# Spark SQL DataType Reference
+
+* Numeric 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`:
+* String type
+ - `StringType`: Represents character string values.
+* Binary type
+ - `BinaryType`: Represents byte sequence values.
+* Boolean type
+ - `BooleanType`: Represents boolean values.
+* Datetime type
+ - `TimestampType`: Represents values comprising values of fields year, month, day,
+ hour, minute, and second.
+* Complex types
+ - `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.
+
+<div class="codetabs">
+<div data-lang="scala" markdown="1">
+
+All data types of Spark SQL are located in the package `org.apache.spark.sql`.
+You can access them by doing
+{% highlight scala %}
+import org.apache.spark.sql._
+{% endhighlight %}
+
+<table class="table">
+<tr>
+ <th style="width:20%">Data type</th>
+ <th style="width:40%">Value type in Scala</th>
+ <th>API to access or create a data type</th></tr>
+<tr>
+ <td> <b>ByteType</b> </td>
+ <td> Byte </td>
+ <td>
+ ByteType
+ </td>
+</tr>
+<tr>
+ <td> <b>ShortType</b> </td>
+ <td> Short </td>
+ <td>
+ ShortType
+ </td>
+</tr>
+<tr>
+ <td> <b>IntegerType</b> </td>
+ <td> Int </td>
+ <td>
+ IntegerType
+ </td>
+</tr>
+<tr>
+ <td> <b>LongType</b> </td>
+ <td> Long </td>
+ <td>
+ LongType
+ </td>
+</tr>
+<tr>
+ <td> <b>FloatType</b> </td>
+ <td> Float </td>
+ <td>
+ FloatType
+ </td>
+</tr>
+<tr>
+ <td> <b>DoubleType</b> </td>
+ <td> Double </td>
+ <td>
+ DoubleType
+ </td>
+</tr>
+<tr>
+ <td> <b>DecimalType</b> </td>
+ <td> scala.math.sql.BigDecimal </td>
+ <td>
+ DecimalType
+ </td>
+</tr>
+<tr>
+ <td> <b>StringType</b> </td>
+ <td> String </td>
+ <td>
+ StringType
+ </td>
+</tr>
+<tr>
+ <td> <b>BinaryType</b> </td>
+ <td> Array[Byte] </td>
+ <td>
+ BinaryType
+ </td>
+</tr>
+<tr>
+ <td> <b>BooleanType</b> </td>
+ <td> Boolean </td>
+ <td>
+ BooleanType
+ </td>
+</tr>
+<tr>
+ <td> <b>TimestampType</b> </td>
+ <td> java.sql.Timestamp </td>
+ <td>
+ TimestampType
+ </td>
+</tr>
+<tr>
+ <td> <b>ArrayType</b> </td>
+ <td> scala.collection.Seq </td>
+ <td>
+ ArrayType(<i>elementType</i>, [<i>containsNull</i>])<br />
+ <b>Note:</b> The default value of <i>containsNull</i> is <i>false</i>.
+ </td>
+</tr>
+<tr>
+ <td> <b>MapType</b> </td>
+ <td> scala.collection.Map </td>
+ <td>
+ MapType(<i>keyType</i>, <i>valueType</i>, [<i>valueContainsNull</i>])<br />
+ <b>Note:</b> The default value of <i>valueContainsNull</i> is <i>true</i>.
+ </td>
+</tr>
+<tr>
+ <td> <b>StructType</b> </td>
+ <td> org.apache.spark.sql.Row </td>
+ <td>
+ StructType(<i>fields</i>)<br />
+ <b>Note:</b> <i>fields</i> is a Seq of StructFields. Also, two fields with the same
+ name are not allowed.
+ </td>
+</tr>
+<tr>
+ <td> <b>StructField</b> </td>
+ <td> The value type in Scala of the data type of this field
+ (For example, Int for a StructField with the data type IntegerType) </td>
+ <td>
+ StructField(<i>name</i>, <i>dataType</i>, <i>nullable</i>)
+ </td>
+</tr>
+</table>
+
+</div>
+
+<div data-lang="java" markdown="1">
+
+All data types of Spark SQL are located in the package of
+`org.apache.spark.sql.api.java`. To access or create a data type,
+please use factory methods provided in
+`org.apache.spark.sql.api.java.DataType`.
+
+<table class="table">
+<tr>
+ <th style="width:20%">Data type</th>
+ <th style="width:40%">Value type in Java</th>
+ <th>API to access or create a data type</th></tr>
+<tr>
+ <td> <b>ByteType</b> </td>
+ <td> byte or Byte </td>
+ <td>
+ DataType.ByteType
+ </td>
+</tr>
+<tr>
+ <td> <b>ShortType</b> </td>
+ <td> short or Short </td>
+ <td>
+ DataType.ShortType
+ </td>
+</tr>
+<tr>
+ <td> <b>IntegerType</b> </td>
+ <td> int or Integer </td>
+ <td>
+ DataType.IntegerType
+ </td>
+</tr>
+<tr>
+ <td> <b>LongType</b> </td>
+ <td> long or Long </td>
+ <td>
+ DataType.LongType
+ </td>
+</tr>
+<tr>
+ <td> <b>FloatType</b> </td>
+ <td> float or Float </td>
+ <td>
+ DataType.FloatType
+ </td>
+</tr>
+<tr>
+ <td> <b>DoubleType</b> </td>
+ <td> double or Double </td>
+ <td>
+ DataType.DoubleType
+ </td>
+</tr>
+<tr>
+ <td> <b>DecimalType</b> </td>
+ <td> java.math.BigDecimal </td>
+ <td>
+ DataType.DecimalType
+ </td>
+</tr>
+<tr>
+ <td> <b>StringType</b> </td>
+ <td> String </td>
+ <td>
+ DataType.StringType
+ </td>
+</tr>
+<tr>
+ <td> <b>BinaryType</b> </td>
+ <td> byte[] </td>
+ <td>
+ DataType.BinaryType
+ </td>
+</tr>
+<tr>
+ <td> <b>BooleanType</b> </td>
+ <td> boolean or Boolean </td>
+ <td>
+ DataType.BooleanType
+ </td>
+</tr>
+<tr>
+ <td> <b>TimestampType</b> </td>
+ <td> java.sql.Timestamp </td>
+ <td>
+ DataType.TimestampType
+ </td>
+</tr>
+<tr>
+ <td> <b>ArrayType</b> </td>
+ <td> java.util.List </td>
+ <td>
+ DataType.createArrayType(<i>elementType</i>)<br />
+ <b>Note:</b> The value of <i>containsNull</i> will be <i>false</i><br />
+ DataType.createArrayType(<i>elementType</i>, <i>containsNull</i>).
+ </td>
+</tr>
+<tr>
+ <td> <b>MapType</b> </td>
+ <td> java.util.Map </td>
+ <td>
+ DataType.createMapType(<i>keyType</i>, <i>valueType</i>)<br />
+ <b>Note:</b> The value of <i>valueContainsNull</i> will be <i>true</i>.<br />
+ DataType.createMapType(<i>keyType</i>, <i>valueType</i>, <i>valueContainsNull</i>)<br />
+ </td>
+</tr>
+<tr>
+ <td> <b>StructType</b> </td>
+ <td> org.apache.spark.sql.api.java </td>
+ <td>
+ DataType.createStructType(<i>fields</i>)<br />
+ <b>Note:</b> <i>fields</i> is a List or an array of StructFields.
+ Also, two fields with the same name are not allowed.
+ </td>
+</tr>
+<tr>
+ <td> <b>StructField</b> </td>
+ <td> The value type in Java of the data type of this field
+ (For example, int for a StructField with the data type IntegerType) </td>
+ <td>
+ DataType.createStructField(<i>name</i>, <i>dataType</i>, <i>nullable</i>)
+ </td>
+</tr>
+</table>
+
+</div>
+
+<div data-lang="python" markdown="1">
+
+All data types of Spark SQL are located in the package of `pyspark.sql`.
+You can access them by doing
+{% highlight python %}
+from pyspark.sql import *
+{% endhighlight %}
+
+<table class="table">
+<tr>
+ <th style="width:20%">Data type</th>
+ <th style="width:40%">Value type in Python</th>
+ <th>API to access or create a data type</th></tr>
+<tr>
+ <td> <b>ByteType</b> </td>
+ <td>
+ int or long <br />
+ <b>Note:</b> Numbers will be converted to 1-byte signed integer numbers at runtime.
+ Please make sure that numbers are within the range of -128 to 127.
+ </td>
+ <td>
+ ByteType()
+ </td>
+</tr>
+<tr>
+ <td> <b>ShortType</b> </td>
+ <td>
+ int or long <br />
+ <b>Note:</b> Numbers will be converted to 2-byte signed integer numbers at runtime.
+ Please make sure that numbers are within the range of -32768 to 32767.
+ </td>
+ <td>
+ ShortType()
+ </td>
+</tr>
+<tr>
+ <td> <b>IntegerType</b> </td>
+ <td> int or long </td>
+ <td>
+ IntegerType()
+ </td>
+</tr>
+<tr>
+ <td> <b>LongType</b> </td>
+ <td>
+ long <br />
+ <b>Note:</b> Numbers will be converted to 8-byte signed integer numbers at runtime.
+ Please make sure that numbers are within the range of
+ -9223372036854775808 to 9223372036854775807.
+ Otherwise, please convert data to decimal.Decimal and use DecimalType.
+ </td>
+ <td>
+ LongType()
+ </td>
+</tr>
+<tr>
+ <td> <b>FloatType</b> </td>
+ <td>
+ float <br />
+ <b>Note:</b> Numbers will be converted to 4-byte single-precision floating
+ point numbers at runtime.
+ </td>
+ <td>
+ FloatType()
+ </td>
+</tr>
+<tr>
+ <td> <b>DoubleType</b> </td>
+ <td> float </td>
+ <td>
+ DoubleType()
+ </td>
+</tr>
+<tr>
+ <td> <b>DecimalType</b> </td>
+ <td> decimal.Decimal </td>
+ <td>
+ DecimalType()
+ </td>
+</tr>
+<tr>
+ <td> <b>StringType</b> </td>
+ <td> string </td>
+ <td>
+ StringType()
+ </td>
+</tr>
+<tr>
+ <td> <b>BinaryType</b> </td>
+ <td> bytearray </td>
+ <td>
+ BinaryType()
+ </td>
+</tr>
+<tr>
+ <td> <b>BooleanType</b> </td>
+ <td> bool </td>
+ <td>
+ BooleanType()
+ </td>
+</tr>
+<tr>
+ <td> <b>TimestampType</b> </td>
+ <td> datetime.datetime </td>
+ <td>
+ TimestampType()
+ </td>
+</tr>
+<tr>
+ <td> <b>ArrayType</b> </td>
+ <td> list, tuple, or array </td>
+ <td>
+ ArrayType(<i>elementType</i>, [<i>containsNull</i>])<br />
+ <b>Note:</b> The default value of <i>containsNull</i> is <i>False</i>.
+ </td>
+</tr>
+<tr>
+ <td> <b>MapType</b> </td>
+ <td> dict </td>
+ <td>
+ MapType(<i>keyType</i>, <i>valueType</i>, [<i>valueContainsNull</i>])<br />
+ <b>Note:</b> The default value of <i>valueContainsNull</i> is <i>True</i>.
+ </td>
+</tr>
+<tr>
+ <td> <b>StructType</b> </td>
+ <td> list or tuple </td>
+ <td>
+ StructType(<i>fields</i>)<br />
+ <b>Note:</b> <i>fields</i> is a Seq of StructFields. Also, two fields with the same
+ name are not allowed.
+ </td>
+</tr>
+<tr>
+ <td> <b>StructField</b> </td>
+ <td> The value type in Python of the data type of this field
+ (For example, Int for a StructField with the data type IntegerType) </td>
+ <td>
+ StructField(<i>name</i>, <i>dataType</i>, <i>nullable</i>)
+ </td>
+</tr>
+</table>
+
+</div>
+
+</div>
-Note that if you just call `cache` rather than `cacheTable`, tables will _not_ be cached in
-in-memory columnar format. So we strongly recommend using `cacheTable` whenever you want to
-cache tables.