Getting Started

Starting Point: SparkSession

The entry point into all functionality in Spark is the SparkSession class. To create a basic SparkSession, just use SparkSession.builder():

  1. import org.apache.spark.sql.SparkSession
  2. val spark = SparkSession
  3. .builder()
  4. .appName("Spark SQL basic example")
  5. .config("spark.some.config.option", "some-value")
  6. .getOrCreate()

Find full example code at “examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala” in the Spark repo.

The entry point into all functionality in Spark is the SparkSession class. To create a basic SparkSession, just use SparkSession.builder():

  1. import org.apache.spark.sql.SparkSession;
  2. SparkSession spark = SparkSession
  3. .builder()
  4. .appName("Java Spark SQL basic example")
  5. .config("spark.some.config.option", "some-value")
  6. .getOrCreate();

Find full example code at “examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java” in the Spark repo.

The entry point into all functionality in Spark is the SparkSession class. To create a basic SparkSession, just use SparkSession.builder:

  1. from pyspark.sql import SparkSession
  2. spark = SparkSession \
  3. .builder \
  4. .appName("Python Spark SQL basic example") \
  5. .config("spark.some.config.option", "some-value") \
  6. .getOrCreate()

Find full example code at “examples/src/main/python/sql/basic.py” in the Spark repo.

The entry point into all functionality in Spark is the SparkSession class. To initialize a basic SparkSession, just call sparkR.session():

  1. sparkR.session(appName = "R Spark SQL basic example", sparkConfig = list(spark.some.config.option = "some-value"))

Find full example code at “examples/src/main/r/RSparkSQLExample.R” in the Spark repo.

Note that when invoked for the first time, sparkR.session() initializes a global SparkSession singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the SparkSession once, then SparkR functions like read.df will be able to access this global instance implicitly, and users don’t need to pass the SparkSession instance around.

SparkSession in Spark 2.0 provides builtin support for Hive features including the ability to write queries using HiveQL, access to Hive UDFs, and the ability to read data from Hive tables. To use these features, you do not need to have an existing Hive setup.

Creating DataFrames

With a SparkSession, applications can create DataFrames from an existing RDD, from a Hive table, or from Spark data sources.

As an example, the following creates a DataFrame based on the content of a JSON file:

  1. val df = spark.read.json("examples/src/main/resources/people.json")
  2. // Displays the content of the DataFrame to stdout
  3. df.show()
  4. // +----+-------+
  5. // | age| name|
  6. // +----+-------+
  7. // |null|Michael|
  8. // | 30| Andy|
  9. // | 19| Justin|
  10. // +----+-------+

Find full example code at “examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala” in the Spark repo.

With a SparkSession, applications can create DataFrames from an existing RDD, from a Hive table, or from Spark data sources.

As an example, the following creates a DataFrame based on the content of a JSON file:

  1. import org.apache.spark.sql.Dataset;
  2. import org.apache.spark.sql.Row;
  3. Dataset<Row> df = spark.read().json("examples/src/main/resources/people.json");
  4. // Displays the content of the DataFrame to stdout
  5. df.show();
  6. // +----+-------+
  7. // | age| name|
  8. // +----+-------+
  9. // |null|Michael|
  10. // | 30| Andy|
  11. // | 19| Justin|
  12. // +----+-------+

Find full example code at “examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java” in the Spark repo.

With a SparkSession, applications can create DataFrames from an existing RDD, from a Hive table, or from Spark data sources.

As an example, the following creates a DataFrame based on the content of a JSON file:

  1. # spark is an existing SparkSession
  2. df = spark.read.json("examples/src/main/resources/people.json")
  3. # Displays the content of the DataFrame to stdout
  4. df.show()
  5. # +----+-------+
  6. # | age| name|
  7. # +----+-------+
  8. # |null|Michael|
  9. # | 30| Andy|
  10. # | 19| Justin|
  11. # +----+-------+

Find full example code at “examples/src/main/python/sql/basic.py” in the Spark repo.

With a SparkSession, applications can create DataFrames from a local R data.frame, from a Hive table, or from Spark data sources.

As an example, the following creates a DataFrame based on the content of a JSON file:

  1. df <- read.json("examples/src/main/resources/people.json")
  2. # Displays the content of the DataFrame
  3. head(df)
  4. ## age name
  5. ## 1 NA Michael
  6. ## 2 30 Andy
  7. ## 3 19 Justin
  8. # Another method to print the first few rows and optionally truncate the printing of long values
  9. showDF(df)
  10. ## +----+-------+
  11. ## | age| name|
  12. ## +----+-------+
  13. ## |null|Michael|
  14. ## | 30| Andy|
  15. ## | 19| Justin|
  16. ## +----+-------+

Find full example code at “examples/src/main/r/RSparkSQLExample.R” in the Spark repo.

Untyped Dataset Operations (aka DataFrame Operations)

DataFrames provide a domain-specific language for structured data manipulation in Scala, Java, Python and R.

As mentioned above, in Spark 2.0, DataFrames are just Dataset of Rows 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:

  1. // This import is needed to use the $-notation
  2. import spark.implicits._
  3. // Print the schema in a tree format
  4. df.printSchema()
  5. // root
  6. // |-- age: long (nullable = true)
  7. // |-- name: string (nullable = true)
  8. // Select only the "name" column
  9. df.select("name").show()
  10. // +-------+
  11. // | name|
  12. // +-------+
  13. // |Michael|
  14. // | Andy|
  15. // | Justin|
  16. // +-------+
  17. // Select everybody, but increment the age by 1
  18. df.select($"name", $"age" + 1).show()
  19. // +-------+---------+
  20. // | name|(age + 1)|
  21. // +-------+---------+
  22. // |Michael| null|
  23. // | Andy| 31|
  24. // | Justin| 20|
  25. // +-------+---------+
  26. // Select people older than 21
  27. df.filter($"age" > 21).show()
  28. // +---+----+
  29. // |age|name|
  30. // +---+----+
  31. // | 30|Andy|
  32. // +---+----+
  33. // Count people by age
  34. df.groupBy("age").count().show()
  35. // +----+-----+
  36. // | age|count|
  37. // +----+-----+
  38. // | 19| 1|
  39. // |null| 1|
  40. // | 30| 1|
  41. // +----+-----+

Find full example code at “examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala” in the Spark repo.

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.

  1. // col("...") is preferable to df.col("...")
  2. import static org.apache.spark.sql.functions.col;
  3. // Print the schema in a tree format
  4. df.printSchema();
  5. // root
  6. // |-- age: long (nullable = true)
  7. // |-- name: string (nullable = true)
  8. // Select only the "name" column
  9. df.select("name").show();
  10. // +-------+
  11. // | name|
  12. // +-------+
  13. // |Michael|
  14. // | Andy|
  15. // | Justin|
  16. // +-------+
  17. // Select everybody, but increment the age by 1
  18. df.select(col("name"), col("age").plus(1)).show();
  19. // +-------+---------+
  20. // | name|(age + 1)|
  21. // +-------+---------+
  22. // |Michael| null|
  23. // | Andy| 31|
  24. // | Justin| 20|
  25. // +-------+---------+
  26. // Select people older than 21
  27. df.filter(col("age").gt(21)).show();
  28. // +---+----+
  29. // |age|name|
  30. // +---+----+
  31. // | 30|Andy|
  32. // +---+----+
  33. // Count people by age
  34. df.groupBy("age").count().show();
  35. // +----+-----+
  36. // | age|count|
  37. // +----+-----+
  38. // | 19| 1|
  39. // |null| 1|
  40. // | 30| 1|
  41. // +----+-----+

Find full example code at “examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java” in the Spark repo.

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.

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.

  1. # spark, df are from the previous example
  2. # Print the schema in a tree format
  3. df.printSchema()
  4. # root
  5. # |-- age: long (nullable = true)
  6. # |-- name: string (nullable = true)
  7. # Select only the "name" column
  8. df.select("name").show()
  9. # +-------+
  10. # | name|
  11. # +-------+
  12. # |Michael|
  13. # | Andy|
  14. # | Justin|
  15. # +-------+
  16. # Select everybody, but increment the age by 1
  17. df.select(df['name'], df['age'] + 1).show()
  18. # +-------+---------+
  19. # | name|(age + 1)|
  20. # +-------+---------+
  21. # |Michael| null|
  22. # | Andy| 31|
  23. # | Justin| 20|
  24. # +-------+---------+
  25. # Select people older than 21
  26. df.filter(df['age'] > 21).show()
  27. # +---+----+
  28. # |age|name|
  29. # +---+----+
  30. # | 30|Andy|
  31. # +---+----+
  32. # Count people by age
  33. df.groupBy("age").count().show()
  34. # +----+-----+
  35. # | age|count|
  36. # +----+-----+
  37. # | 19| 1|
  38. # |null| 1|
  39. # | 30| 1|
  40. # +----+-----+

Find full example code at “examples/src/main/python/sql/basic.py” in the Spark repo.

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.

  1. # Create the DataFrame
  2. df <- read.json("examples/src/main/resources/people.json")
  3. # Show the content of the DataFrame
  4. head(df)
  5. ## age name
  6. ## 1 NA Michael
  7. ## 2 30 Andy
  8. ## 3 19 Justin
  9. # Print the schema in a tree format
  10. printSchema(df)
  11. ## root
  12. ## |-- age: long (nullable = true)
  13. ## |-- name: string (nullable = true)
  14. # Select only the "name" column
  15. head(select(df, "name"))
  16. ## name
  17. ## 1 Michael
  18. ## 2 Andy
  19. ## 3 Justin
  20. # Select everybody, but increment the age by 1
  21. head(select(df, df$name, df$age + 1))
  22. ## name (age + 1.0)
  23. ## 1 Michael NA
  24. ## 2 Andy 31
  25. ## 3 Justin 20
  26. # Select people older than 21
  27. head(where(df, df$age > 21))
  28. ## age name
  29. ## 1 30 Andy
  30. # Count people by age
  31. head(count(groupBy(df, "age")))
  32. ## age count
  33. ## 1 19 1
  34. ## 2 NA 1
  35. ## 3 30 1

Find full example code at “examples/src/main/r/RSparkSQLExample.R” in the Spark repo.

For a complete list of the types of operations that can be performed on a DataFrame refer to the API Documentation.

In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the DataFrame Function Reference.

Running SQL Queries Programmatically

The sql function on a SparkSession enables applications to run SQL queries programmatically and returns the result as a DataFrame.

  1. // Register the DataFrame as a SQL temporary view
  2. df.createOrReplaceTempView("people")
  3. val sqlDF = spark.sql("SELECT * FROM people")
  4. sqlDF.show()
  5. // +----+-------+
  6. // | age| name|
  7. // +----+-------+
  8. // |null|Michael|
  9. // | 30| Andy|
  10. // | 19| Justin|
  11. // +----+-------+

Find full example code at “examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala” in the Spark repo.

The sql function on a SparkSession enables applications to run SQL queries programmatically and returns the result as a Dataset<Row>.

  1. import org.apache.spark.sql.Dataset;
  2. import org.apache.spark.sql.Row;
  3. // Register the DataFrame as a SQL temporary view
  4. df.createOrReplaceTempView("people");
  5. Dataset<Row> sqlDF = spark.sql("SELECT * FROM people");
  6. sqlDF.show();
  7. // +----+-------+
  8. // | age| name|
  9. // +----+-------+
  10. // |null|Michael|
  11. // | 30| Andy|
  12. // | 19| Justin|
  13. // +----+-------+

Find full example code at “examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java” in the Spark repo.

The sql function on a SparkSession enables applications to run SQL queries programmatically and returns the result as a DataFrame.

  1. # Register the DataFrame as a SQL temporary view
  2. df.createOrReplaceTempView("people")
  3. sqlDF = spark.sql("SELECT * FROM people")
  4. sqlDF.show()
  5. # +----+-------+
  6. # | age| name|
  7. # +----+-------+
  8. # |null|Michael|
  9. # | 30| Andy|
  10. # | 19| Justin|
  11. # +----+-------+

Find full example code at “examples/src/main/python/sql/basic.py” in the Spark repo.

The sql function enables applications to run SQL queries programmatically and returns the result as a SparkDataFrame.

  1. df <- sql("SELECT * FROM table")

Find full example code at “examples/src/main/r/RSparkSQLExample.R” in the Spark repo.

Global Temporary View

Temporary views in Spark SQL are session-scoped and will disappear if the session that creates it terminates. If you want to have a temporary view that is shared among all sessions and keep alive until the Spark application terminates, you can create a global temporary view. Global temporary view is tied to a system preserved database global_temp, and we must use the qualified name to refer it, e.g. SELECT * FROM global_temp.view1.

  1. // Register the DataFrame as a global temporary view
  2. df.createGlobalTempView("people")
  3. // Global temporary view is tied to a system preserved database `global_temp`
  4. spark.sql("SELECT * FROM global_temp.people").show()
  5. // +----+-------+
  6. // | age| name|
  7. // +----+-------+
  8. // |null|Michael|
  9. // | 30| Andy|
  10. // | 19| Justin|
  11. // +----+-------+
  12. // Global temporary view is cross-session
  13. spark.newSession().sql("SELECT * FROM global_temp.people").show()
  14. // +----+-------+
  15. // | age| name|
  16. // +----+-------+
  17. // |null|Michael|
  18. // | 30| Andy|
  19. // | 19| Justin|
  20. // +----+-------+

Find full example code at “examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala” in the Spark repo.

  1. // Register the DataFrame as a global temporary view
  2. df.createGlobalTempView("people");
  3. // Global temporary view is tied to a system preserved database `global_temp`
  4. spark.sql("SELECT * FROM global_temp.people").show();
  5. // +----+-------+
  6. // | age| name|
  7. // +----+-------+
  8. // |null|Michael|
  9. // | 30| Andy|
  10. // | 19| Justin|
  11. // +----+-------+
  12. // Global temporary view is cross-session
  13. spark.newSession().sql("SELECT * FROM global_temp.people").show();
  14. // +----+-------+
  15. // | age| name|
  16. // +----+-------+
  17. // |null|Michael|
  18. // | 30| Andy|
  19. // | 19| Justin|
  20. // +----+-------+

Find full example code at “examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java” in the Spark repo.

  1. # Register the DataFrame as a global temporary view
  2. df.createGlobalTempView("people")
  3. # Global temporary view is tied to a system preserved database `global_temp`
  4. spark.sql("SELECT * FROM global_temp.people").show()
  5. # +----+-------+
  6. # | age| name|
  7. # +----+-------+
  8. # |null|Michael|
  9. # | 30| Andy|
  10. # | 19| Justin|
  11. # +----+-------+
  12. # Global temporary view is cross-session
  13. spark.newSession().sql("SELECT * FROM global_temp.people").show()
  14. # +----+-------+
  15. # | age| name|
  16. # +----+-------+
  17. # |null|Michael|
  18. # | 30| Andy|
  19. # | 19| Justin|
  20. # +----+-------+

Find full example code at “examples/src/main/python/sql/basic.py” in the Spark repo.

  1. CREATE GLOBAL TEMPORARY VIEW temp_view AS SELECT a + 1, b * 2 FROM tbl
  2. SELECT * FROM global_temp.temp_view

Creating Datasets

Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use a specialized Encoder to serialize the objects for processing or transmitting over the network. While both encoders and standard serialization are responsible for turning an object into bytes, encoders are code generated dynamically and use a format that allows Spark to perform many operations like filtering, sorting and hashing without deserializing the bytes back into an object.

  1. case class Person(name: String, age: Long)
  2. // Encoders are created for case classes
  3. val caseClassDS = Seq(Person("Andy", 32)).toDS()
  4. caseClassDS.show()
  5. // +----+---+
  6. // |name|age|
  7. // +----+---+
  8. // |Andy| 32|
  9. // +----+---+
  10. // Encoders for most common types are automatically provided by importing spark.implicits._
  11. val primitiveDS = Seq(1, 2, 3).toDS()
  12. primitiveDS.map(_ + 1).collect() // Returns: Array(2, 3, 4)
  13. // DataFrames can be converted to a Dataset by providing a class. Mapping will be done by name
  14. val path = "examples/src/main/resources/people.json"
  15. val peopleDS = spark.read.json(path).as[Person]
  16. peopleDS.show()
  17. // +----+-------+
  18. // | age| name|
  19. // +----+-------+
  20. // |null|Michael|
  21. // | 30| Andy|
  22. // | 19| Justin|
  23. // +----+-------+

Find full example code at “examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala” in the Spark repo.

  1. import java.util.Arrays;
  2. import java.util.Collections;
  3. import java.io.Serializable;
  4. import org.apache.spark.api.java.function.MapFunction;
  5. import org.apache.spark.sql.Dataset;
  6. import org.apache.spark.sql.Row;
  7. import org.apache.spark.sql.Encoder;
  8. import org.apache.spark.sql.Encoders;
  9. public static class Person implements Serializable {
  10. private String name;
  11. private long age;
  12. public String getName() {
  13. return name;
  14. }
  15. public void setName(String name) {
  16. this.name = name;
  17. }
  18. public long getAge() {
  19. return age;
  20. }
  21. public void setAge(long age) {
  22. this.age = age;
  23. }
  24. }
  25. // Create an instance of a Bean class
  26. Person person = new Person();
  27. person.setName("Andy");
  28. person.setAge(32);
  29. // Encoders are created for Java beans
  30. Encoder<Person> personEncoder = Encoders.bean(Person.class);
  31. Dataset<Person> javaBeanDS = spark.createDataset(
  32. Collections.singletonList(person),
  33. personEncoder
  34. );
  35. javaBeanDS.show();
  36. // +---+----+
  37. // |age|name|
  38. // +---+----+
  39. // | 32|Andy|
  40. // +---+----+
  41. // Encoders for most common types are provided in class Encoders
  42. Encoder<Long> longEncoder = Encoders.LONG();
  43. Dataset<Long> primitiveDS = spark.createDataset(Arrays.asList(1L, 2L, 3L), longEncoder);
  44. Dataset<Long> transformedDS = primitiveDS.map(
  45. (MapFunction<Long, Long>) value -> value + 1L,
  46. longEncoder);
  47. transformedDS.collect(); // Returns [2, 3, 4]
  48. // DataFrames can be converted to a Dataset by providing a class. Mapping based on name
  49. String path = "examples/src/main/resources/people.json";
  50. Dataset<Person> peopleDS = spark.read().json(path).as(personEncoder);
  51. peopleDS.show();
  52. // +----+-------+
  53. // | age| name|
  54. // +----+-------+
  55. // |null|Michael|
  56. // | 30| Andy|
  57. // | 19| Justin|
  58. // +----+-------+

Find full example code at “examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java” in the Spark repo.

Interoperating with RDDs

Spark SQL supports two different methods for converting existing RDDs into Datasets. The first method uses reflection to infer the schema of an RDD that contains specific types of objects. This reflection-based approach leads to more concise code and works well when you already know the schema while writing your Spark application.

The second method for creating Datasets is through a programmatic interface that allows you to construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows you to construct Datasets when the columns and their types are not known until runtime.

Inferring the Schema Using Reflection

The Scala interface for Spark SQL supports automatically converting an RDD containing case classes to a DataFrame. The case class defines the schema of the table. The names of the arguments to the case class are read using reflection and become the names of the columns. Case classes can also be nested or contain complex types such as Seqs 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.

  1. // For implicit conversions from RDDs to DataFrames
  2. import spark.implicits._
  3. // Create an RDD of Person objects from a text file, convert it to a Dataframe
  4. val peopleDF = spark.sparkContext
  5. .textFile("examples/src/main/resources/people.txt")
  6. .map(_.split(","))
  7. .map(attributes => Person(attributes(0), attributes(1).trim.toInt))
  8. .toDF()
  9. // Register the DataFrame as a temporary view
  10. peopleDF.createOrReplaceTempView("people")
  11. // SQL statements can be run by using the sql methods provided by Spark
  12. val teenagersDF = spark.sql("SELECT name, age FROM people WHERE age BETWEEN 13 AND 19")
  13. // The columns of a row in the result can be accessed by field index
  14. teenagersDF.map(teenager => "Name: " + teenager(0)).show()
  15. // +------------+
  16. // | value|
  17. // +------------+
  18. // |Name: Justin|
  19. // +------------+
  20. // or by field name
  21. teenagersDF.map(teenager => "Name: " + teenager.getAs[String]("name")).show()
  22. // +------------+
  23. // | value|
  24. // +------------+
  25. // |Name: Justin|
  26. // +------------+
  27. // No pre-defined encoders for Dataset[Map[K,V]], define explicitly
  28. implicit val mapEncoder = org.apache.spark.sql.Encoders.kryo[Map[String, Any]]
  29. // Primitive types and case classes can be also defined as
  30. // implicit val stringIntMapEncoder: Encoder[Map[String, Any]] = ExpressionEncoder()
  31. // row.getValuesMap[T] retrieves multiple columns at once into a Map[String, T]
  32. teenagersDF.map(teenager => teenager.getValuesMap[Any](List("name", "age"))).collect()
  33. // Array(Map("name" -> "Justin", "age" -> 19))

Find full example code at “examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala” in the Spark repo.

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.

  1. import org.apache.spark.api.java.JavaRDD;
  2. import org.apache.spark.api.java.function.Function;
  3. import org.apache.spark.api.java.function.MapFunction;
  4. import org.apache.spark.sql.Dataset;
  5. import org.apache.spark.sql.Row;
  6. import org.apache.spark.sql.Encoder;
  7. import org.apache.spark.sql.Encoders;
  8. // Create an RDD of Person objects from a text file
  9. JavaRDD<Person> peopleRDD = spark.read()
  10. .textFile("examples/src/main/resources/people.txt")
  11. .javaRDD()
  12. .map(line -> {
  13. String[] parts = line.split(",");
  14. Person person = new Person();
  15. person.setName(parts[0]);
  16. person.setAge(Integer.parseInt(parts[1].trim()));
  17. return person;
  18. });
  19. // Apply a schema to an RDD of JavaBeans to get a DataFrame
  20. Dataset<Row> peopleDF = spark.createDataFrame(peopleRDD, Person.class);
  21. // Register the DataFrame as a temporary view
  22. peopleDF.createOrReplaceTempView("people");
  23. // SQL statements can be run by using the sql methods provided by spark
  24. Dataset<Row> teenagersDF = spark.sql("SELECT name FROM people WHERE age BETWEEN 13 AND 19");
  25. // The columns of a row in the result can be accessed by field index
  26. Encoder<String> stringEncoder = Encoders.STRING();
  27. Dataset<String> teenagerNamesByIndexDF = teenagersDF.map(
  28. (MapFunction<Row, String>) row -> "Name: " + row.getString(0),
  29. stringEncoder);
  30. teenagerNamesByIndexDF.show();
  31. // +------------+
  32. // | value|
  33. // +------------+
  34. // |Name: Justin|
  35. // +------------+
  36. // or by field name
  37. Dataset<String> teenagerNamesByFieldDF = teenagersDF.map(
  38. (MapFunction<Row, String>) row -> "Name: " + row.<String>getAs("name"),
  39. stringEncoder);
  40. teenagerNamesByFieldDF.show();
  41. // +------------+
  42. // | value|
  43. // +------------+
  44. // |Name: Justin|
  45. // +------------+

Find full example code at “examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java” in the Spark repo.

Spark SQL can convert an RDD of Row objects to a DataFrame, inferring the datatypes. Rows are constructed by passing a list of key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table, and the types are inferred by sampling the whole dataset, similar to the inference that is performed on JSON files.

  1. from pyspark.sql import Row
  2. sc = spark.sparkContext
  3. # Load a text file and convert each line to a Row.
  4. lines = sc.textFile("examples/src/main/resources/people.txt")
  5. parts = lines.map(lambda l: l.split(","))
  6. people = parts.map(lambda p: Row(name=p[0], age=int(p[1])))
  7. # Infer the schema, and register the DataFrame as a table.
  8. schemaPeople = spark.createDataFrame(people)
  9. schemaPeople.createOrReplaceTempView("people")
  10. # SQL can be run over DataFrames that have been registered as a table.
  11. teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19")
  12. # The results of SQL queries are Dataframe objects.
  13. # rdd returns the content as an :class:`pyspark.RDD` of :class:`Row`.
  14. teenNames = teenagers.rdd.map(lambda p: "Name: " + p.name).collect()
  15. for name in teenNames:
  16. print(name)
  17. # Name: Justin

Find full example code at “examples/src/main/python/sql/basic.py” in the Spark repo.

Programmatically Specifying the Schema

When case classes cannot be defined ahead of time (for example, the structure of records is encoded in a string, or a text dataset will be parsed and fields will be projected differently for different users), a DataFrame can be created programmatically with three steps.

  1. Create an RDD of Rows from the original RDD;
  2. Create the schema represented by a StructType matching the structure of Rows in the RDD created in Step 1.
  3. Apply the schema to the RDD of Rows via createDataFrame method provided by SparkSession.

For example:

  1. import org.apache.spark.sql.Row
  2. import org.apache.spark.sql.types._
  3. // Create an RDD
  4. val peopleRDD = spark.sparkContext.textFile("examples/src/main/resources/people.txt")
  5. // The schema is encoded in a string
  6. val schemaString = "name age"
  7. // Generate the schema based on the string of schema
  8. val fields = schemaString.split(" ")
  9. .map(fieldName => StructField(fieldName, StringType, nullable = true))
  10. val schema = StructType(fields)
  11. // Convert records of the RDD (people) to Rows
  12. val rowRDD = peopleRDD
  13. .map(_.split(","))
  14. .map(attributes => Row(attributes(0), attributes(1).trim))
  15. // Apply the schema to the RDD
  16. val peopleDF = spark.createDataFrame(rowRDD, schema)
  17. // Creates a temporary view using the DataFrame
  18. peopleDF.createOrReplaceTempView("people")
  19. // SQL can be run over a temporary view created using DataFrames
  20. val results = spark.sql("SELECT name FROM people")
  21. // The results of SQL queries are DataFrames and support all the normal RDD operations
  22. // The columns of a row in the result can be accessed by field index or by field name
  23. results.map(attributes => "Name: " + attributes(0)).show()
  24. // +-------------+
  25. // | value|
  26. // +-------------+
  27. // |Name: Michael|
  28. // | Name: Andy|
  29. // | Name: Justin|
  30. // +-------------+

Find full example code at “examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala” in the Spark repo.

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.

  1. Create an RDD of Rows from the original RDD;
  2. Create the schema represented by a StructType matching the structure of Rows in the RDD created in Step 1.
  3. Apply the schema to the RDD of Rows via createDataFrame method provided by SparkSession.

For example:

  1. import java.util.ArrayList;
  2. import java.util.List;
  3. import org.apache.spark.api.java.JavaRDD;
  4. import org.apache.spark.api.java.function.Function;
  5. import org.apache.spark.sql.Dataset;
  6. import org.apache.spark.sql.Row;
  7. import org.apache.spark.sql.types.DataTypes;
  8. import org.apache.spark.sql.types.StructField;
  9. import org.apache.spark.sql.types.StructType;
  10. // Create an RDD
  11. JavaRDD<String> peopleRDD = spark.sparkContext()
  12. .textFile("examples/src/main/resources/people.txt", 1)
  13. .toJavaRDD();
  14. // The schema is encoded in a string
  15. String schemaString = "name age";
  16. // Generate the schema based on the string of schema
  17. List<StructField> fields = new ArrayList<>();
  18. for (String fieldName : schemaString.split(" ")) {
  19. StructField field = DataTypes.createStructField(fieldName, DataTypes.StringType, true);
  20. fields.add(field);
  21. }
  22. StructType schema = DataTypes.createStructType(fields);
  23. // Convert records of the RDD (people) to Rows
  24. JavaRDD<Row> rowRDD = peopleRDD.map((Function<String, Row>) record -> {
  25. String[] attributes = record.split(",");
  26. return RowFactory.create(attributes[0], attributes[1].trim());
  27. });
  28. // Apply the schema to the RDD
  29. Dataset<Row> peopleDataFrame = spark.createDataFrame(rowRDD, schema);
  30. // Creates a temporary view using the DataFrame
  31. peopleDataFrame.createOrReplaceTempView("people");
  32. // SQL can be run over a temporary view created using DataFrames
  33. Dataset<Row> results = spark.sql("SELECT name FROM people");
  34. // The results of SQL queries are DataFrames and support all the normal RDD operations
  35. // The columns of a row in the result can be accessed by field index or by field name
  36. Dataset<String> namesDS = results.map(
  37. (MapFunction<Row, String>) row -> "Name: " + row.getString(0),
  38. Encoders.STRING());
  39. namesDS.show();
  40. // +-------------+
  41. // | value|
  42. // +-------------+
  43. // |Name: Michael|
  44. // | Name: Andy|
  45. // | Name: Justin|
  46. // +-------------+

Find full example code at “examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java” in the Spark repo.

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 SparkSession.

For example:

  1. # Import data types
  2. from pyspark.sql.types import StringType, StructType, StructField
  3. sc = spark.sparkContext
  4. # Load a text file and convert each line to a Row.
  5. lines = sc.textFile("examples/src/main/resources/people.txt")
  6. parts = lines.map(lambda l: l.split(","))
  7. # Each line is converted to a tuple.
  8. people = parts.map(lambda p: (p[0], p[1].strip()))
  9. # The schema is encoded in a string.
  10. schemaString = "name age"
  11. fields = [StructField(field_name, StringType(), True) for field_name in schemaString.split()]
  12. schema = StructType(fields)
  13. # Apply the schema to the RDD.
  14. schemaPeople = spark.createDataFrame(people, schema)
  15. # Creates a temporary view using the DataFrame
  16. schemaPeople.createOrReplaceTempView("people")
  17. # SQL can be run over DataFrames that have been registered as a table.
  18. results = spark.sql("SELECT name FROM people")
  19. results.show()
  20. # +-------+
  21. # | name|
  22. # +-------+
  23. # |Michael|
  24. # | Andy|
  25. # | Justin|
  26. # +-------+

Find full example code at “examples/src/main/python/sql/basic.py” in the Spark repo.

Scalar Functions

Scalar functions are functions that return a single value per row, as opposed to aggregation functions, which return a value for a group of rows. Spark SQL supports a variety of Built-in Scalar Functions. It also supports User Defined Scalar Functions.

Aggregate Functions

Aggregate functions are functions that return a single value on a group of rows. The Built-in Aggregation Functions provide common aggregations such as count(), count_distinct(), avg(), max(), min(), etc. Users are not limited to the predefined aggregate functions and can create their own. For more details about user defined aggregate functions, please refer to the documentation of User Defined Aggregate Functions.