Skip to content
Snippets Groups Projects
sql-programming-guide.md 86.5 KiB
Newer Older
  • Learn to ignore specific revisions
  • 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 and the Dataset API. When computing a result
    
    the same execution engine is used, independent of which API/language you are using to express the
    
    computation. This unification means that developers can easily switch back and forth between
    different APIs based on which provides the most natural way to express a given transformation.
    
    All of the examples on this page use sample data included in the Spark distribution and can be run in
    the `spark-shell`, `pyspark` shell, or `sparkR` shell.
    
    One use of Spark SQL is to execute SQL queries.
    
    Spark SQL can also be used to read data from an existing Hive installation. For more on how to
    configure this feature, please refer to the [Hive Tables](#hive-tables) section. When running
    
    SQL from within another programming language the results will be returned as a [Dataset/DataFrame](#datasets-and-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 Dataset is a distributed collection of data.
    Dataset is a new interface added in Spark 1.6 that provides the benefits of RDDs (strong
    
    typing, ability to use powerful lambda functions) with the benefits of Spark SQL's optimized
    execution engine. A Dataset can be [constructed](#creating-datasets) from JVM objects and then
    manipulated using functional transformations (`map`, `flatMap`, `filter`, etc.).
    
    The Dataset API is available in [Scala][scala-datasets] and
    [Java][java-datasets]. Python does not have the support for the Dataset API. But due to Python's dynamic nature,
    many of the benefits of the Dataset API are already available (i.e. you can access the field of a row by name naturally
    `row.columnName`). The case for R is similar.
    
    A DataFrame is a *Dataset* organized into named columns. It is conceptually
    equivalent to a table in a relational database or a data frame in R/Python, but with richer
    optimizations under the hood. DataFrames can be constructed from a wide array of [sources](#data-sources) such
    as: structured data files, tables in Hive, external databases, or existing RDDs.
    The DataFrame API is available in Scala,
    Java, [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame), and [R](api/R/index.html).
    In Scala and Java, a DataFrame is represented by a Dataset of `Row`s.
    In [the Scala API][scala-datasets], `DataFrame` is simply a type alias of `Dataset[Row]`.
    While, in [Java API][java-datasets], users need to use `Dataset<Row>` to represent a `DataFrame`.
    
    [scala-datasets]: api/scala/index.html#org.apache.spark.sql.Dataset
    [java-datasets]: api/java/index.html?org/apache/spark/sql/Dataset.html
    
    Throughout this document, we will often refer to Scala/Java Datasets of `Row`s as DataFrames.
    
    <div class="codetabs">
    <div data-lang="scala"  markdown="1">
    
    
    The entry point into all functionality in Spark is the [`SparkSession`](api/scala/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`:
    
    {% include_example init_session scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    
    <div data-lang="java" markdown="1">
    
    The entry point into all functionality in Spark is the [`SparkSession`](api/java/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`:
    
    {% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    
    <div data-lang="python"  markdown="1">
    
    
    The entry point into all functionality in Spark is the [`SparkSession`](api/python/pyspark.sql.html#pyspark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder`:
    
    {% include_example init_session python/sql.py %}
    
    The entry point into all functionality in Spark is the [`SparkSession`](api/R/sparkR.session.html) class. To initialize a basic `SparkSession`, just call `sparkR.session()`:
    
    {% include_example init_session r/RSparkSQLExample.R %}
    
    Note that when invoked for the first time, `sparkR.session()` initializes a global `SparkSession` singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the `SparkSession` once, then SparkR functions like `read.df` will be able to access this global instance implicitly, and users don't need to pass the `SparkSession` instance around.
    
    `SparkSession` in Spark 2.0 provides builtin support for Hive features including the ability to
    
    write queries using HiveQL, access to Hive UDFs, and the ability to read data from Hive tables.
    To use these features, you do not need to have an existing Hive setup.
    
    ## Creating DataFrames
    
    <div class="codetabs">
    <div data-lang="scala"  markdown="1">
    
    With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds),
    from a Hive table, or from [Spark data sources](#data-sources).
    
    As an example, the following creates a DataFrame based on the content of a JSON file:
    
    {% include_example create_df scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    
    </div>
    
    <div data-lang="java" markdown="1">
    
    With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds),
    from a Hive table, or from [Spark data sources](#data-sources).
    
    As an example, the following creates a DataFrame based on the content of a JSON file:
    
    {% include_example create_df java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    
    </div>
    
    <div data-lang="python"  markdown="1">
    
    With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds),
    from a Hive table, or from [Spark data sources](#data-sources).
    
    As an example, the following creates a DataFrame based on the content of a JSON file:
    
    {% highlight python %}
    # spark is an existing SparkSession
    df = spark.read.json("examples/src/main/resources/people.json")
    
    
    # Displays the content of the DataFrame to stdout
    df.show()
    {% endhighlight %}
    
    </div>
    
    With a `SparkSession`, applications can create DataFrames from a local R data.frame,
    
    from a Hive table, or from [Spark data sources](#data-sources).
    
    As an example, the following creates a DataFrame based on the content of a JSON file:
    
    {% include_example create_DataFrames r/RSparkSQLExample.R %}
    
    ## Untyped Dataset Operations (aka DataFrame Operations)
    
    DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset), [Java](api/java/index.html?org/apache/spark/sql/Dataset.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/DataFrame.html).
    
    As mentioned above, in Spark 2.0, DataFrames are just Dataset of `Row`s in Scala and Java API. These operations are also referred as "untyped transformations" in contrast to "typed transformations" come with strongly typed Scala/Java Datasets.
    
    Here we include some basic examples of structured data processing using Datasets:
    
    
    <div class="codetabs">
    <div data-lang="scala"  markdown="1">
    
    {% include_example untyped_ops scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    
    For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.Dataset).
    
    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](api/scala/index.html#org.apache.spark.sql.functions$).
    
    </div>
    
    <div data-lang="java" markdown="1">
    
    {% include_example untyped_ops java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    
    For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/java/org/apache/spark/sql/Dataset.html).
    
    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](api/java/org/apache/spark/sql/functions.html).
    
    </div>
    
    <div data-lang="python"  markdown="1">
    
    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.
    
    
    # spark is an existing SparkSession
    
    df = spark.read.json("examples/src/main/resources/people.json")
    
    
    # Show the content of the DataFrame
    df.show()
    
    
    # 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()
    
    
    # Select everybody, but increment the age by 1
    
    ## 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).
    
    {% include_example dataframe_operations r/RSparkSQLExample.R %}
    
    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/SparkDataFrame.html).
    
    
    ## Running SQL Queries Programmatically
    
    <div class="codetabs">
    <div data-lang="scala"  markdown="1">
    
    The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`.
    
    
    {% include_example run_sql scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    
    </div>
    
    <div data-lang="java" markdown="1">
    
    The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `Dataset<Row>`.
    
    
    {% include_example run_sql java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    
    </div>
    
    <div data-lang="python"  markdown="1">
    
    The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`.
    
    
    # spark is an existing SparkSession
    df = spark.sql("SELECT * FROM table")
    
    The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`.
    
    {% include_example sql_query r/RSparkSQLExample.R %}
    
    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">
    
    {% include_example create_ds scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    
    </div>
    
    <div data-lang="java" markdown="1">
    
    {% include_example create_ds java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    
    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
    
    <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 `Seq`s or `Array`s. This RDD can be implicitly converted to a DataFrame and then be
    
    registered as a table. Tables can be used in subsequent SQL statements.
    
    {% include_example schema_inferring scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    
    </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 `Map` field(s). Nested JavaBeans and `List` or `Array`
    fields are supported though. You can create a JavaBean by creating a class that implements
    Serializable and has getters and setters for all of its fields.
    
    {% include_example schema_inferring java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    
    <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 sampling the whole datase, similar to the inference that is performed on JSON files.
    
    
    {% highlight python %}
    
    # spark is an existing SparkSession.
    from pyspark.sql import Row
    sc = spark.sparkContext
    
    # 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 = spark.createDataFrame(people)
    
    schemaPeople.createOrReplaceTempView("people")
    
    # SQL can be run over DataFrames that have been registered as a table.
    
    teenagers = spark.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():
    
    {% endhighlight %}
    
    </div>
    
    
    ### 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
    
    {% include_example programmatic_schema scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    
    </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 `Dataset<Row>` 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
    
    {% include_example programmatic_schema java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    
    </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 `SparkSession`.
    
    
    For example:
    {% highlight python %}
    
    # Import SparkSession and data types
    
    from pyspark.sql.types import *
    
    # spark is an existing SparkSession.
    sc = spark.sparkContext
    
    
    # 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 = spark.createDataFrame(people, schema)
    
    # Creates a temporary view using the DataFrame
    schemaPeople.createOrReplaceTempView("people")
    
    # SQL can be run over DataFrames that have been registered as a table.
    
    results = spark.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():
    
    {% endhighlight %}
    
    </div>
    
    </div>
    
    Spark SQL supports operating on a variety of data sources through the DataFrame interface.
    A DataFrame can be operated on using relational transformations and can also be used to create a temporary view.
    Registering a DataFrame as a temporary view allows you to run SQL queries over its data. This section
    
    describes the general methods for loading and saving data using the Spark Data Sources and then
    goes into specific options that are available for the built-in data sources.
    
    ## 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">
    
    {% include_example generic_load_save_functions scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    
    </div>
    
    <div data-lang="java"  markdown="1">
    
    {% include_example generic_load_save_functions java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    
    </div>
    
    <div data-lang="python"  markdown="1">
    
    {% highlight python %}
    
    
    df = spark.read.load("examples/src/main/resources/users.parquet")
    
    df.select("name", "favorite_color").write.save("namesAndFavColors.parquet")
    
    {% include_example source_parquet r/RSparkSQLExample.R %}
    
    </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 loaded from any data source type can be converted into other types
    
    using this syntax.
    
    <div class="codetabs">
    <div data-lang="scala"  markdown="1">
    
    {% include_example manual_load_options scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    
    </div>
    
    <div data-lang="java"  markdown="1">
    
    {% include_example manual_load_options java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    
    </div>
    
    <div data-lang="python"  markdown="1">
    
    {% highlight python %}
    
    
    df = spark.read.load("examples/src/main/resources/people.json", format="json")
    
    df.select("name", "age").write.save("namesAndAges.parquet", format="parquet")
    
    {% include_example source_json r/RSparkSQLExample.R %}
    
    ### 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">
    
    {% include_example direct_sql scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    
    </div>
    
    <div data-lang="java"  markdown="1">
    
    {% include_example direct_sql java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    
    </div>
    
    <div data-lang="python"  markdown="1">
    
    {% highlight python %}
    
    df = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`")
    
    {% endhighlight %}
    
    </div>
    
    <div data-lang="r"  markdown="1">
    
    
    {% include_example direct_query r/RSparkSQLExample.R %}
    
    ### 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 an `Overwrite`, the data will be deleted before writing out the
    
    <tr><th>Scala/Java</th><th>Any Language</th><th>Meaning</th></tr>
    
    <tr>
      <td><code>SaveMode.ErrorIfExists</code> (default)</td>
      <td><code>"error"</code> (default)</td>
      <td>
        When saving a DataFrame to a data source, if data already exists,
        an exception is expected to be thrown.
      </td>
    </tr>
    <tr>
      <td><code>SaveMode.Append</code></td>
      <td><code>"append"</code></td>
      <td>
        When saving a DataFrame to a data source, if data/table already exists,
        contents of the DataFrame are expected to be appended to existing data.
      </td>
    </tr>
    <tr>
      <td><code>SaveMode.Overwrite</code></td>
      <td><code>"overwrite"</code></td>
      <td>
        Overwrite mode means that when saving a DataFrame to a data source,
        if data/table already exists, existing data is expected to be overwritten by the contents of
        the DataFrame.
      </td>
    </tr>
    <tr>
      <td><code>SaveMode.Ignore</code></td>
      <td><code>"ignore"</code></td>
      <td>
        Ignore mode means that when saving a DataFrame to a data source, if data already exists,
        the save operation is expected to not save the contents of the DataFrame and to not
    
        change the existing data. This is similar to a <code>CREATE TABLE IF NOT EXISTS</code> in SQL.
    
      </td>
    </tr>
    </table>
    
    ### Saving to Persistent Tables
    
    
    `DataFrames` can also be saved as persistent tables into Hive metastore using the `saveAsTable`
    command. Notice existing Hive deployment is not necessary to use this feature. Spark will create a
    default local Hive metastore (using Derby) for you. Unlike the `createOrReplaceTempView` command,
    `saveAsTable` will materialize the contents of the DataFrame and create a pointer to the data in the
    Hive metastore. Persistent tables will still exist even after your Spark program has restarted, as
    long as you maintain your connection to the same metastore. A DataFrame for a persistent table can
    be created by calling the `table` method on a `SparkSession` with the name of the table.
    
    
    By default `saveAsTable` will create a "managed table", meaning that the location of the data will
    
    be controlled by the metastore. Managed tables will also have their data deleted automatically
    
    [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. When writing Parquet files, all columns are automatically converted to be nullable for
    
    
    ### Loading Data Programmatically
    
    Using the data from the above example:
    
    <div class="codetabs">
    
    <div data-lang="scala"  markdown="1">
    
    {% include_example basic_parquet_example scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    
    </div>
    
    <div data-lang="java"  markdown="1">
    
    {% include_example basic_parquet_example java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    
    </div>
    
    <div data-lang="python"  markdown="1">
    
    {% highlight python %}
    
    # spark from the previous example is used in this example.
    
    schemaPeople # The DataFrame from the previous example.
    
    # DataFrames can be saved as Parquet files, maintaining the schema information.
    
    Peter Hoffmann's avatar
    Peter Hoffmann committed
    schemaPeople.write.parquet("people.parquet")
    
    # Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved.
    
    # The result of loading a parquet file is also a DataFrame.
    
    parquetFile = spark.read.parquet("people.parquet")
    
    # Parquet files can also be used to create a temporary view and then used in SQL statements.
    parquetFile.createOrReplaceTempView("parquetFile");
    
    teenagers = spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19")
    
    teenNames = teenagers.map(lambda p: "Name: " + p.name)
    for teenName in teenNames.collect():
    
    {% include_example load_programmatically r/RSparkSQLExample.R %}
    
    <div data-lang="sql"  markdown="1">
    
    {% highlight sql %}
    
    
    USING org.apache.spark.sql.parquet
    OPTIONS (
      path "examples/src/main/resources/people.parquet"
    )
    
    SELECT * FROM parquetTable
    
    {% endhighlight %}
    
    </div>
    
    
    Table partitioning is a common optimization approach used in systems like Hive. In a partitioned
    
    table, data are usually stored in different directories, with partitioning column values encoded in
    
    the path of each partition directory. The Parquet data source is now able to discover and infer
    partitioning information automatically. For example, we can store all our previously used
    
    population data into a partitioned table using the following directory structure, with two extra
    columns, `gender` and `country` as partitioning columns:
    
    {% highlight text %}
    
    path
    └── to
        └── table
            ├── gender=male
            │   ├── ...
            │   │
            │   ├── country=US
            │   │   └── data.parquet
            │   ├── country=CN
            │   │   └── data.parquet
            │   └── ...
            └── gender=female
                ├── ...
    
                ├── country=US
                │   └── data.parquet
                ├── country=CN
                │   └── data.parquet
                └── ...
    
    {% endhighlight %}
    
    
    By passing `path/to/table` to either `SparkSession.read.parquet` or `SparkSession.read.load`, Spark SQL
    
    will automatically extract the partitioning information from the paths.
    Now the schema of the returned DataFrame becomes:
    
    
    {% highlight text %}
    
    root
    |-- name: string (nullable = true)
    |-- age: long (nullable = true)
    |-- gender: string (nullable = true)
    |-- country: string (nullable = true)
    
    {% endhighlight %}
    
    
    Notice that the data types of the partitioning columns are automatically inferred. Currently,
    
    numeric data types and string type are supported. Sometimes users may not want to automatically
    infer the data types of the partitioning columns. For these use cases, the automatic type inference
    can be configured by `spark.sql.sources.partitionColumnTypeInference.enabled`, which is default to
    `true`. When type inference is disabled, string type will be used for the partitioning columns.
    
    
    Starting from Spark 1.6.0, partition discovery only finds partitions under the given paths
    
    by default. For the above example, if users pass `path/to/table/gender=male` to either
    `SparkSession.read.parquet` or `SparkSession.read.load`, `gender` will not be considered as a
    
    partitioning column. If users need to specify the base path that partition discovery
    should start with, they can set `basePath` in the data source options. For example,
    when `path/to/table/gender=male` is the path of the data and
    users set `basePath` to `path/to/table/`, `gender` will be a partitioning column.
    
    Like ProtocolBuffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with
    a simple schema, and gradually add more columns to the schema as needed. In this way, users may end
    up with multiple Parquet files with different but mutually compatible schemas. The Parquet data
    
    source is now able to automatically detect this case and merge schemas of all these files.
    
    
    Since schema merging is a relatively expensive operation, and is not a necessity in most cases, we
    
    turned it off by default starting from 1.5.0. You may enable it by
    
    
    1. setting data source option `mergeSchema` to `true` when reading Parquet files (as shown in the
       examples below), or
    2. setting the global SQL option `spark.sql.parquet.mergeSchema` to `true`.
    
    
    <div class="codetabs">
    
    <div data-lang="scala"  markdown="1">
    
    {% include_example schema_merging scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    
    <div data-lang="java"  markdown="1">
    
    {% include_example schema_merging java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    
    </div>
    
    <div data-lang="python"  markdown="1">
    
    {% highlight python %}
    
    # spark from the previous example is used in this example.
    
    
    # Create a simple DataFrame, stored into a partition directory
    
    df1 = spark.createDataFrame(sc.parallelize(range(1, 6))\
    
                                       .map(lambda i: Row(single=i, double=i * 2)))
    
    df1.write.parquet("data/test_table/key=1")
    
    
    # Create another DataFrame in a new partition directory,
    # adding a new column and dropping an existing column
    
    df2 = spark.createDataFrame(sc.parallelize(range(6, 11))
    
                                       .map(lambda i: Row(single=i, triple=i * 3)))
    
    df2.write.parquet("data/test_table/key=2")
    
    df3 = spark.read.option("mergeSchema", "true").parquet("data/test_table")
    
    df3.printSchema()
    
    # The final schema consists of all 3 columns in the Parquet files together
    
    # with the partitioning column appeared in the partition directory paths.
    
    # root
    # |-- single: int (nullable = true)
    # |-- double: int (nullable = true)
    # |-- triple: int (nullable = true)
    # |-- key : int (nullable = true)
    {% endhighlight %}
    
    </div>
    
    
    {% include_example schema_merging r/RSparkSQLExample.R %}
    
    ### Hive metastore Parquet table conversion
    
    When reading from and writing to Hive metastore Parquet tables, Spark SQL will try to use its own
    Parquet support instead of Hive SerDe for better performance. This behavior is controlled by the
    `spark.sql.hive.convertMetastoreParquet` configuration, and is turned on by default.
    
    #### Hive/Parquet Schema Reconciliation
    
    There are two key differences between Hive and Parquet from the perspective of table schema
    processing.
    
    1. Hive is case insensitive, while Parquet is not
    1. Hive considers all columns nullable, while nullability in Parquet is significant
    
    Due to this reason, we must reconcile Hive metastore schema with Parquet schema when converting a
    
    Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation rules are:
    
    
    1. Fields that have the same name in both schema must have the same data type regardless of
    
       nullability. The reconciled field should have the data type of the Parquet side, so that
    
       nullability is respected.
    
    1. The reconciled schema contains exactly those fields defined in Hive metastore schema.
    
       - Any fields that only appear in the Parquet schema are dropped in the reconciled schema.
    
       - Any fields that only appear in the Hive metastore schema are added as nullable field in the
    
         reconciled schema.
    
    #### Metadata Refreshing
    
    
    Spark SQL caches Parquet metadata for better performance. When Hive metastore Parquet table
    conversion is enabled, metadata of those converted tables are also cached. If these tables are
    
    updated by Hive or other external tools, you need to refresh them manually to ensure consistent
    metadata.
    
    <div class="codetabs">
    
    <div data-lang="scala"  markdown="1">
    
    {% highlight scala %}
    
    // spark is an existing SparkSession
    spark.catalog.refreshTable("my_table")
    
    {% endhighlight %}
    
    </div>
    
    <div data-lang="java"  markdown="1">
    
    {% highlight java %}
    
    // spark is an existing SparkSession
    spark.catalog().refreshTable("my_table");
    
    {% endhighlight %}
    
    </div>
    
    <div data-lang="python"  markdown="1">
    
    {% highlight python %}
    
    # spark is an existing SparkSession
    spark.catalog.refreshTable("my_table")
    
    {% endhighlight %}
    
    </div>
    
    <div data-lang="sql"  markdown="1">
    
    {% highlight sql %}
    REFRESH TABLE my_table;
    {% endhighlight %}
    
    </div>
    
    </div>
    
    
    ### Configuration
    
    
    Configuration of Parquet can be done using the `setConf` method on `SparkSession` 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, Hive, 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.int96AsTimestamp</code></td>
      <td>true</td>
      <td>
    
        Some Parquet-producing systems, in particular Impala and Hive, store Timestamp into INT96. This
    
        flag tells Spark SQL to interpret INT96 data as a timestamp to provide compatibility with these systems.
      </td>
    </tr>
    
    <tr>
      <td><code>spark.sql.parquet.cacheMetadata</code></td>
    
      <td>true</td>
    
        Turns on caching of Parquet schema metadata. Can speed up querying of static data.
    
      </td>
    </tr>
    <tr>
      <td><code>spark.sql.parquet.compression.codec</code></td>
    
      <td>gzip</td>
    
        Sets the compression codec use when writing Parquet files. Acceptable values include:
    
        uncompressed, snappy, gzip, lzo.
      </td>
    </tr>
    
    <tr>
      <td><code>spark.sql.parquet.filterPushdown</code></td>
    
      <td>true</td>
      <td>Enables Parquet filter push-down optimization when set to true.</td>
    
    <tr>
      <td><code>spark.sql.hive.convertMetastoreParquet</code></td>
      <td>true</td>
      <td>
        When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of the built in
        support.
      </td>
    </tr>
    
    <tr>
      <td><code>spark.sql.parquet.mergeSchema</code></td>
    
      <td>
        <p>
          When true, the Parquet data source merges schemas collected from all data files, otherwise the
          schema is picked from the summary file or a random data file if no summary file is available.
        </p>
      </td>
    </tr>
    
    <tr>
      <td><code>spark.sql.optimizer.metadataOnly</code></td>
      <td>true</td>
      <td>
        <p>
          When true, enable the metadata-only query optimization that use the table's metadata to
          produce the partition columns instead of table scans. It applies when all the columns scanned
          are partition columns and the query has an aggregate operator that satisfies distinct
          semantics.
        </p>
      </td>
    </tr>
    
    ## JSON Datasets
    <div class="codetabs">
    
    <div data-lang="scala"  markdown="1">
    
    Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset[Row]`.
    This conversion can be done using `SparkSession.read.json()` on either an RDD of String,
    
    Note that the file that is offered as _a json file_ is not a typical JSON file. Each
    
    line must contain a separate, self-contained valid JSON object. As a consequence,
    a regular multi-line JSON file will most often fail.
    
    
    {% include_example json_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    
    <div data-lang="java"  markdown="1">
    
    Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset<Row>`.
    This conversion can be done using `SparkSession.read().json()` on either an RDD of String,
    
    Note that the file that is offered as _a json file_ is not a typical JSON file. Each
    
    line must contain a separate, self-contained valid JSON object. As a consequence,
    a regular multi-line JSON file will most often fail.
    
    
    {% include_example json_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    
    </div>
    
    <div data-lang="python"  markdown="1">
    
    Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame.
    
    This conversion can be done using `SparkSession.read.json` on a JSON file.
    
    Note that the file that is offered as _a json file_ is not a typical JSON file. Each
    
    line must contain a separate, self-contained valid JSON object. As a consequence,
    a regular multi-line JSON file will most often fail.
    
    
    # spark is an existing SparkSession.