Newer
Older
---
layout: global
displayTitle: Spark SQL, DataFrames and Datasets Guide
title: Spark SQL and DataFrames
---
* This will become a table of contents (this text will be scraped).
{:toc}
# Overview
Spark SQL is a Spark module for structured data processing. Unlike the basic Spark RDD API, the interfaces provided
by Spark SQL provide Spark with more information about the structure of both the data and the computation being performed. Internally,
Spark SQL uses this extra information to perform extra optimizations. There are several ways to
interact with Spark SQL including SQL, 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.
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 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).
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
<div class="codetabs">
<div data-lang="scala" markdown="1">
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.
{% highlight scala %}
val sc: SparkContext // An existing SparkContext.
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
// this is used to implicitly convert an RDD to a DataFrame.
import sqlContext.implicits._
{% endhighlight %}
<div data-lang="java" markdown="1">
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.
JavaSparkContext sc = ...; // An existing JavaSparkContext.
SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
{% endhighlight %}
</div>
<div data-lang="python" markdown="1">
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.
{% highlight python %}
from pyspark.sql import SQLContext
sqlContext = SQLContext(sc)
</div>
<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.
{% highlight r %}
sqlContext <- sparkRSQL.init(sc)
{% endhighlight %}
</div>
</div>
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
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
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,
this is recommended for most use cases.
## Creating DataFrames
With a `SQLContext`, applications can create `DataFrame`s from an <a href='#interoperating-with-rdds'>existing `RDD`</a>, from a Hive table, or from <a href='#data-sources'>data sources</a>.
As an example, the following creates a `DataFrame` based on the content of a JSON file:
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% highlight scala %}
val sc: SparkContext // An existing SparkContext.
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
val df = sqlContext.read.json("examples/src/main/resources/people.json")
// Displays the content of the DataFrame to stdout
df.show()
{% endhighlight %}
</div>
<div data-lang="java" markdown="1">
{% highlight java %}
JavaSparkContext sc = ...; // An existing JavaSparkContext.
SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
DataFrame df = sqlContext.read().json("examples/src/main/resources/people.json");
// Displays the content of the DataFrame to stdout
df.show();
{% endhighlight %}
</div>
<div data-lang="python" markdown="1">
{% highlight python %}
from pyspark.sql import SQLContext
sqlContext = SQLContext(sc)
df = sqlContext.read.json("examples/src/main/resources/people.json")
# Displays the content of the DataFrame to stdout
df.show()
{% endhighlight %}
</div>
<div data-lang="r" markdown="1">
{% highlight r %}
sqlContext <- SQLContext(sc)
df <- jsonFile(sqlContext, "examples/src/main/resources/people.json")
# Displays the content of the DataFrame to stdout
showDF(df)
{% endhighlight %}
</div>
</div>
## DataFrame Operations
DataFrames provide a domain-specific language for structured data manipulation 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/DataFrame.html).
Here we include some basic examples of structured data processing using DataFrames:
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% highlight scala %}
val sc: SparkContext // An existing SparkContext.
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
// Create the DataFrame
val df = sqlContext.read.json("examples/src/main/resources/people.json")
// Show the content of the DataFrame
df.show()
// age name
// null Michael
// 30 Andy
// 19 Justin
// Print the schema in a tree format
df.printSchema()
// root
// |-- age: long (nullable = true)
// |-- name: string (nullable = true)
// Select only the "name" column
df.select("name").show()
// name
// Michael
// Andy
// Justin
// Select everybody, but increment the age by 1
df.select(df("name"), df("age") + 1).show()
// name (age + 1)
// Michael null
// Andy 31
// Justin 20
// Select people older than 21
df.filter(df("age") > 21).show()
// age name
// 30 Andy
// Count people by age
df.groupBy("age").count().show()
// age count
// null 1
// 19 1
// 30 1
{% endhighlight %}
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$).
</div>
<div data-lang="java" markdown="1">
{% highlight java %}
JavaSparkContext sc // An existing SparkContext.
SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc)
// Create the DataFrame
DataFrame df = sqlContext.read().json("examples/src/main/resources/people.json");
// Show the content of the DataFrame
df.show();
// age name
// null Michael
// 30 Andy
// 19 Justin
// Print the schema in a tree format
df.printSchema();
// root
// |-- age: long (nullable = true)
// |-- name: string (nullable = true)
// Select only the "name" column
df.select("name").show();
// name
// Michael
// Andy
// Justin
// Select everybody, but increment the age by 1
df.select(df.col("name"), df.col("age").plus(1)).show();
// name (age + 1)
// Michael null
// Andy 31
// Justin 20
// Select people older than 21
df.filter(df.col("age").gt(21)).show();
// age name
// 30 Andy
// Count people by age
df.groupBy("age").count().show();
// age count
// null 1
// 19 1
// 30 1
{% endhighlight %}
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).
</div>
<div data-lang="python" markdown="1">
ksonj
committed
In Python it's possible to access a DataFrame's columns either by attribute
(`df.age`) or by indexing (`df['age']`). While the former is convenient for
interactive data exploration, users are highly encouraged to use the
latter form, which is future proof and won't break with column names that
are also attributes on the DataFrame class.
{% highlight python %}
from pyspark.sql import SQLContext
sqlContext = SQLContext(sc)
# Create the DataFrame
df = sqlContext.read.json("examples/src/main/resources/people.json")
# Show the content of the DataFrame
df.show()
## age name
## null Michael
## 30 Andy
## 19 Justin
# Print the schema in a tree format
df.printSchema()
## root
## |-- age: long (nullable = true)
## |-- name: string (nullable = true)
# Select only the "name" column
df.select("name").show()
## name
## Michael
## Andy
## Justin
# Select everybody, but increment the age by 1
ksonj
committed
df.select(df['name'], df['age'] + 1).show()
## name (age + 1)
## Michael null
## Andy 31
## Justin 20
# Select people older than 21
ksonj
committed
df.filter(df['age'] > 21).show()
## age name
## 30 Andy
# Count people by age
df.groupBy("age").count().show()
## age count
## null 1
## 19 1
## 30 1
{% endhighlight %}
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).
</div>
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
<div data-lang="r" markdown="1">
{% highlight r %}
sqlContext <- sparkRSQL.init(sc)
# Create the DataFrame
df <- jsonFile(sqlContext, "examples/src/main/resources/people.json")
# Show the content of the DataFrame
showDF(df)
## age name
## null Michael
## 30 Andy
## 19 Justin
# Print the schema in a tree format
printSchema(df)
## root
## |-- age: long (nullable = true)
## |-- name: string (nullable = true)
# Select only the "name" column
showDF(select(df, "name"))
## name
## Michael
## Andy
## Justin
# Select everybody, but increment the age by 1
showDF(select(df, df$name, df$age + 1))
## name (age + 1)
## Michael null
## Andy 31
## Justin 20
# Select people older than 21
showDF(where(df, df$age > 21))
## age name
## 30 Andy
# Count people by age
showDF(count(groupBy(df, "age")))
## age count
## null 1
## 19 1
## 30 1
{% endhighlight %}
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).
</div>
## Running SQL Queries Programmatically
The `sql` function on a `SQLContext` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`.
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% highlight scala %}
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
DataFrame df = sqlContext.sql("SELECT * FROM table")
{% endhighlight %}
</div>
<div data-lang="python" markdown="1">
{% highlight python %}
from pyspark.sql import SQLContext
sqlContext = SQLContext(sc)
df = sqlContext.sql("SELECT * FROM table")
{% endhighlight %}
</div>
<div data-lang="r" markdown="1">
{% highlight r %}
sqlContext <- sparkRSQL.init(sc)
df <- sql(sqlContext, "SELECT * FROM table")
{% endhighlight %}
</div>
</div>
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
## 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
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
you to construct DataFrames 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">
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 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.
{% highlight scala %}
// sc is an existing SparkContext.
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
// this is used to implicitly convert an RDD to a DataFrame.
import sqlContext.implicits._
// Define the schema using a case class.
// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit,
// you can use custom classes that implement the Product interface.
case class Person(name: String, age: Int)
// Create an RDD of Person objects and register it as a table.
val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)).toDF()
people.registerTempTable("people")
// SQL statements can be run by using the sql methods provided by sqlContext.
val teenagers = sqlContext.sql("SELECT name, age FROM people WHERE age >= 13 AND age <= 19")
// The results of SQL queries are DataFrames and support all the normal RDD operations.
// The columns of a row in the result can be accessed by field index:
teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
// or by field name:
teenagers.map(t => "Name: " + t.getAs[String]("name")).collect().foreach(println)
// row.getValuesMap[T] retrieves multiple columns at once into a Map[String, T]
teenagers.map(_.getValuesMap[Any](List("name", "age"))).collect().foreach(println)
// Map("name" -> "Justin", "age" -> 19)
{% endhighlight %}
</div>
<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.
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.
{% highlight java %}
public static class Person implements Serializable {
private String name;
private int age;
public String getName() {
public void setName(String name) {
public void setAge(int age) {
this.age = age;
}
}
{% endhighlight %}
A schema can be applied to an existing RDD by calling `createDataFrame` and providing the Class object
for the JavaBean.
{% highlight java %}
// sc is an existing JavaSparkContext.
SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
// Load a text file and convert each line to a JavaBean.
JavaRDD<Person> people = sc.textFile("examples/src/main/resources/people.txt").map(
new Function<String, Person>() {
public Person call(String line) throws Exception {
String[] parts = line.split(",");
Person person = new Person();
person.setName(parts[0]);
person.setAge(Integer.parseInt(parts[1].trim()));
return person;
}
});
// Apply a schema to an RDD of JavaBeans and register it as a table.
DataFrame schemaPeople = sqlContext.createDataFrame(people, Person.class);
schemaPeople.registerTempTable("people");
// SQL can be run over RDDs that have been registered as tables.
DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19")
// The results of SQL queries are DataFrames and support all the normal RDD operations.
// The columns of a row in the result can be accessed by ordinal.
List<String> teenagerNames = teenagers.javaRDD().map(new Function<Row, String>() {
public String call(Row row) {
return "Name: " + row.getString(0);
}
}).collect();
{% endhighlight %}
</div>
<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
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
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.
# sc is an existing SparkContext.
from pyspark.sql import SQLContext, Row
sqlContext = SQLContext(sc)
# Load a text file and convert each line to a Row.
lines = sc.textFile("examples/src/main/resources/people.txt")
parts = lines.map(lambda l: l.split(","))
people = parts.map(lambda p: Row(name=p[0], age=int(p[1])))
# Infer the schema, and register the DataFrame as a table.
schemaPeople = sqlContext.createDataFrame(people)
schemaPeople.registerTempTable("people")
# SQL can be run over DataFrames that have been registered as a table.
teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19")
# The results of SQL queries are RDDs and support all the normal RDD operations.
teenNames = teenagers.map(lambda p: "Name: " + p.name)
for teenName in teenNames.collect():
Sean Owen
committed
print(teenName)
### Programmatically Specifying the Schema
<div class="codetabs">
<div data-lang="scala" markdown="1">
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.
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 Step 1.
3. Apply the schema to the RDD of `Row`s via `createDataFrame` 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 Row.
import org.apache.spark.sql.Row;
// Import Spark SQL data types
import org.apache.spark.sql.types.{StructType,StructField,StringType};
// 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 peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema)
// Register the DataFrames as a table.
peopleDataFrame.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 DataFrames and support all the normal RDD operations.
// The columns of a row in the result can be accessed by field index or by field name.
results.map(t => "Name: " + t(0)).collect().foreach(println)
{% endhighlight %}
</div>
<div data-lang="java" markdown="1">
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 `DataFrame` 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 Step 1.
3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided
For example:
{% highlight java %}
import org.apache.spark.api.java.function.Function;
// Import factory methods provided by DataTypes.
import org.apache.spark.sql.types.DataTypes;
// Import StructType and StructField
import org.apache.spark.sql.types.StructType;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.Row;
// Import RowFactory.
import org.apache.spark.sql.RowFactory;
// sc is an existing JavaSparkContext.
SQLContext sqlContext = new org.apache.spark.sql.SQLContext(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(DataTypes.createStructField(fieldName, DataTypes.StringType, true));
StructType schema = DataTypes.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 RowFactory.create(fields[0], fields[1].trim());
}
});
// Apply the schema to the RDD.
DataFrame peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema);
// Register the DataFrame as a table.
peopleDataFrame.registerTempTable("people");
// SQL can be run over RDDs that have been registered as tables.
DataFrame results = sqlContext.sql("SELECT name FROM people");
// The results of SQL queries are DataFrames and support all the normal RDD operations.
// The columns of a row in the result can be accessed by ordinal.
List<String> names = results.javaRDD().map(new Function<Row, String>() {
public String call(Row row) {
return "Name: " + row.getString(0);
}
}).collect();
{% endhighlight %}
</div>
<div data-lang="python" markdown="1">
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.
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 `createDataFrame` method provided by `SQLContext`.
For example:
{% highlight python %}
# Import SQLContext and data types
from pyspark.sql import SQLContext
from pyspark.sql.types 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.createDataFrame(people, schema)
# Register the DataFrame as a table.
schemaPeople.registerTempTable("people")
# SQL can be run over DataFrames 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():
Sean Owen
committed
print(name)
{% endhighlight %}
</div>
</div>
# Data Sources
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
describes the general methods for loading and saving data using the Spark Data Sources and then
goes into specific options that are available for the built-in data sources.
## Generic Load/Save Functions
In the simplest form, the default data source (`parquet` unless otherwise configured by
`spark.sql.sources.default`) will be used for all operations.
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% highlight scala %}
val df = sqlContext.read.load("examples/src/main/resources/users.parquet")
df.select("name", "favorite_color").write.save("namesAndFavColors.parquet")
{% endhighlight %}
</div>
<div data-lang="java" markdown="1">
{% highlight java %}
DataFrame df = sqlContext.read().load("examples/src/main/resources/users.parquet");
df.select("name", "favorite_color").write().save("namesAndFavColors.parquet");
{% endhighlight %}
</div>
<div data-lang="python" markdown="1">
{% highlight python %}
df = sqlContext.read.load("examples/src/main/resources/users.parquet")
df.select("name", "favorite_color").write.save("namesAndFavColors.parquet")
{% endhighlight %}
</div>
<div data-lang="r" markdown="1">
{% highlight r %}
df <- loadDF(sqlContext, "people.parquet")
saveDF(select(df, "name", "age"), "namesAndAges.parquet")
{% endhighlight %}
</div>
</div>
### Manually Specifying Options
You can also manually specify the data source that will be used along with any extra options
that you would like to pass to the data source. Data sources are specified by their fully qualified
name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can also use their short
names (`json`, `parquet`, `jdbc`). DataFrames of any type can be converted into other types
using this syntax.
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% highlight scala %}
val df = sqlContext.read.format("json").load("examples/src/main/resources/people.json")
df.select("name", "age").write.format("parquet").save("namesAndAges.parquet")
{% endhighlight %}
</div>
<div data-lang="java" markdown="1">
{% highlight java %}
DataFrame df = sqlContext.read().format("json").load("examples/src/main/resources/people.json");
df.select("name", "age").write().format("parquet").save("namesAndAges.parquet");
{% endhighlight %}
</div>
<div data-lang="python" markdown="1">
{% highlight python %}
df = sqlContext.read.load("examples/src/main/resources/people.json", format="json")
df.select("name", "age").write.save("namesAndAges.parquet", format="parquet")
{% endhighlight %}
</div>
<div data-lang="r" markdown="1">
{% highlight r %}
df <- loadDF(sqlContext, "people.json", "json")
saveDF(select(df, "name", "age"), "namesAndAges.parquet", "parquet")
{% endhighlight %}
</div>
</div>
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
### Run SQL on files directly
Instead of using read API to load a file into DataFrame and query it, you can also query that
file directly with SQL.
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% highlight scala %}
val df = sqlContext.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`")
{% endhighlight %}
</div>
<div data-lang="java" markdown="1">
{% highlight java %}
DataFrame df = sqlContext.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`");
{% endhighlight %}
</div>
<div data-lang="python" markdown="1">
{% highlight python %}
df = sqlContext.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`")
{% endhighlight %}
</div>
<div data-lang="r" markdown="1">
{% highlight r %}
df <- sql(sqlContext, "SELECT * FROM parquet.`examples/src/main/resources/users.parquet`")
{% endhighlight %}
</div>
</div>
### 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
new data.