http://git-wip-us.apache.org/repos/asf/spark/blob/71535516/docs/sql-data-sources-troubleshooting.md
----------------------------------------------------------------------
diff --git a/docs/sql-data-sources-troubleshooting.md 
b/docs/sql-data-sources-troubleshooting.md
new file mode 100644
index 0000000..5775eb8
--- /dev/null
+++ b/docs/sql-data-sources-troubleshooting.md
@@ -0,0 +1,9 @@
+---
+layout: global
+title: Troubleshooting
+displayTitle: Troubleshooting
+---
+
+ * The JDBC driver class must be visible to the primordial class loader on the 
client session and on all executors. This is because Java's DriverManager class 
does a security check that results in it ignoring all drivers not visible to 
the primordial class loader when one goes to open a connection. One convenient 
way to do this is to modify compute_classpath.sh on all worker nodes to include 
your driver JARs.
+ * Some databases, such as H2, convert all names to upper case. You'll need to 
use upper case to refer to those names in Spark SQL.
+ * Users can specify vendor-specific JDBC connection properties in the data 
source options to do special treatment. For example, 
`spark.read.format("jdbc").option("url", 
oracleJdbcUrl).option("oracle.jdbc.mapDateToTimestamp", "false")`. 
`oracle.jdbc.mapDateToTimestamp` defaults to true, users often need to disable 
this flag to avoid Oracle date being resolved as timestamp.

http://git-wip-us.apache.org/repos/asf/spark/blob/71535516/docs/sql-data-sources.md
----------------------------------------------------------------------
diff --git a/docs/sql-data-sources.md b/docs/sql-data-sources.md
new file mode 100644
index 0000000..aa607ec
--- /dev/null
+++ b/docs/sql-data-sources.md
@@ -0,0 +1,42 @@
+---
+layout: global
+title: Data Sources
+displayTitle: Data Sources
+---
+
+
+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](sql-data-sources-load-save-functions.html)
+  * [Manually Specifying 
Options](sql-data-sources-load-save-functions.html#manually-specifying-options)
+  * [Run SQL on files 
directly](sql-data-sources-load-save-functions.html#run-sql-on-files-directly)
+  * [Save Modes](sql-data-sources-load-save-functions.html#save-modes)
+  * [Saving to Persistent 
Tables](sql-data-sources-load-save-functions.html#run-sql-on-files-directly)
+  * [Bucketing, Sorting and 
Partitioning](sql-data-sources-load-save-functions.html#run-sql-on-files-directly)
+* [Parquet Files](sql-data-sources-parquet.html)
+  * [Loading Data 
Programmatically](sql-data-sources-parquet.html#loading-data-programmatically)
+  * [Partition Discovery](sql-data-sources-parquet.html#partition-discovery)
+  * [Schema Merging](sql-data-sources-parquet.html#schema-merging)
+  * [Hive metastore Parquet table 
conversion](sql-data-sources-parquet.html#hive-metastore-parquet-table-conversion)
+  * [Configuration](sql-data-sources-parquet.html#configuration)
+* [ORC Files](sql-data-sources-orc.html)
+* [JSON Files](sql-data-sources-json.html)
+* [Hive Tables](sql-data-sources-hive-tables.html)
+  * [Specifying storage format for Hive 
tables](sql-data-sources-hive-tables.html#specifying-storage-format-for-hive-tables)
+  * [Interacting with Different Versions of Hive 
Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore)
+* [JDBC To Other Databases](sql-data-sources-jdbc.html)
+* [Avro Files](sql-data-sources-avro.html)
+  * [Deploying](sql-data-sources-avro.html#deploying)
+  * [Load and Save 
Functions](sql-data-sources-avro.html#load-and-save-functions)
+  * [to_avro() and 
from_avro()](sql-data-sources-avro.html#to_avro-and-from_avro)
+  * [Data Source Option](sql-data-sources-avro.html#data-source-option)
+  * [Configuration](sql-data-sources-avro.html#configuration)
+  * [Compatibility with Databricks 
spark-avro](sql-data-sources-avro.html#compatibility-with-databricks-spark-avro)
+  * [Supported types for Avro -> Spark SQL 
conversion](sql-data-sources-avro.html#supported-types-for-avro---spark-sql-conversion)
+  * [Supported types for Spark SQL -> Avro 
conversion](sql-data-sources-avro.html#supported-types-for-spark-sql---avro-conversion)
+* [Troubleshooting](sql-data-sources-troubleshooting.html)

http://git-wip-us.apache.org/repos/asf/spark/blob/71535516/docs/sql-distributed-sql-engine.md
----------------------------------------------------------------------
diff --git a/docs/sql-distributed-sql-engine.md 
b/docs/sql-distributed-sql-engine.md
new file mode 100644
index 0000000..66d6fda
--- /dev/null
+++ b/docs/sql-distributed-sql-engine.md
@@ -0,0 +1,84 @@
+---
+layout: global
+title: Distributed SQL Engine
+displayTitle: Distributed SQL Engine
+---
+
+* Table of contents
+{:toc}
+
+Spark SQL can also act as a distributed query engine using its JDBC/ODBC or 
command-line interface.
+In this mode, end-users or applications can interact with Spark SQL directly 
to run SQL queries,
+without the need to write any code.
+
+## Running the Thrift JDBC/ODBC server
+
+The Thrift JDBC/ODBC server implemented here corresponds to the 
[`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2)
+in Hive 1.2.1. You can test the JDBC server with the beeline script that comes 
with either Spark or Hive 1.2.1.
+
+To start the JDBC/ODBC server, run the following in the Spark directory:
+
+    ./sbin/start-thriftserver.sh
+
+This script accepts all `bin/spark-submit` command line options, plus a 
`--hiveconf` option to
+specify Hive properties. You may run `./sbin/start-thriftserver.sh --help` for 
a complete list of
+all available options. By default, the server listens on localhost:10000. You 
may override this
+behaviour via either environment variables, i.e.:
+
+{% highlight bash %}
+export HIVE_SERVER2_THRIFT_PORT=<listening-port>
+export HIVE_SERVER2_THRIFT_BIND_HOST=<listening-host>
+./sbin/start-thriftserver.sh \
+  --master <master-uri> \
+  ...
+{% endhighlight %}
+
+or system properties:
+
+{% highlight bash %}
+./sbin/start-thriftserver.sh \
+  --hiveconf hive.server2.thrift.port=<listening-port> \
+  --hiveconf hive.server2.thrift.bind.host=<listening-host> \
+  --master <master-uri>
+  ...
+{% endhighlight %}
+
+Now you can use beeline to test the Thrift JDBC/ODBC server:
+
+    ./bin/beeline
+
+Connect to the JDBC/ODBC server in beeline with:
+
+    beeline> !connect jdbc:hive2://localhost:10000
+
+Beeline will ask you for a username and password. In non-secure mode, simply 
enter the username on
+your machine and a blank password. For secure mode, please follow the 
instructions given in the
+[beeline 
documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients).
+
+Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` 
and `hdfs-site.xml` files in `conf/`.
+
+You may also use the beeline script that comes with Hive.
+
+Thrift JDBC server also supports sending thrift RPC messages over HTTP 
transport.
+Use the following setting to enable HTTP mode as system property or in 
`hive-site.xml` file in `conf/`:
+
+    hive.server2.transport.mode - Set this to value: http
+    hive.server2.thrift.http.port - HTTP port number to listen on; default is 
10001
+    hive.server2.http.endpoint - HTTP endpoint; default is cliservice
+
+To test, use beeline to connect to the JDBC/ODBC server in http mode with:
+
+    beeline> !connect 
jdbc:hive2://<host>:<port>/<database>?hive.server2.transport.mode=http;hive.server2.thrift.http.path=<http_endpoint>
+
+
+## Running the Spark SQL CLI
+
+The Spark SQL CLI is a convenient tool to run the Hive metastore service in 
local mode and execute
+queries input from the command line. Note that the Spark SQL CLI cannot talk 
to the Thrift JDBC server.
+
+To start the Spark SQL CLI, run the following in the Spark directory:
+
+    ./bin/spark-sql
+
+Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` 
and `hdfs-site.xml` files in `conf/`.
+You may run `./bin/spark-sql --help` for a complete list of all available 
options.

http://git-wip-us.apache.org/repos/asf/spark/blob/71535516/docs/sql-getting-started.md
----------------------------------------------------------------------
diff --git a/docs/sql-getting-started.md b/docs/sql-getting-started.md
new file mode 100644
index 0000000..8851220
--- /dev/null
+++ b/docs/sql-getting-started.md
@@ -0,0 +1,369 @@
+---
+layout: global
+title: Getting Started
+displayTitle: Getting Started
+---
+
+* Table of contents
+{:toc}
+
+## Starting Point: SparkSession
+
+<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>
+
+<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>
+
+<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/basic.py %}
+</div>
+
+<div data-lang="r"  markdown="1">
+
+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.
+</div>
+</div>
+
+`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](sql-data-sources.html).
+
+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](sql-data-sources.html).
+
+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](sql-data-sources.html).
+
+As an example, the following creates a DataFrame based on the content of a 
JSON file:
+
+{% include_example create_df python/sql/basic.py %}
+</div>
+
+<div data-lang="r"  markdown="1">
+With a `SparkSession`, applications can create DataFrames from a local R 
data.frame,
+from a Hive table, or from [Spark data sources](sql-data-sources.html).
+
+As an example, the following creates a DataFrame based on the content of a 
JSON file:
+
+{% include_example create_df r/RSparkSQLExample.R %}
+
+</div>
+</div>
+
+
+## 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/SparkDataFrame.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.
+
+{% include_example untyped_ops python/sql/basic.py %}
+For a complete list of the types of operations that can be performed on a 
DataFrame refer to the [API 
Documentation](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>
+
+<div data-lang="r"  markdown="1">
+
+{% include_example untyped_ops r/RSparkSQLExample.R %}
+
+For a complete list of the types of operations that can be performed on a 
DataFrame refer to the [API Documentation](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).
+
+</div>
+
+</div>
+
+## 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`.
+
+{% include_example run_sql python/sql/basic.py %}
+</div>
+
+<div data-lang="r"  markdown="1">
+The `sql` function enables applications to run SQL queries programmatically 
and returns the result as a `SparkDataFrame`.
+
+{% include_example run_sql r/RSparkSQLExample.R %}
+
+</div>
+</div>
+
+
+## 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`.
+
+<div class="codetabs">
+<div data-lang="scala"  markdown="1">
+{% include_example global_temp_view 
scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
+</div>
+
+<div data-lang="java" markdown="1">
+{% include_example global_temp_view 
java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
+</div>
+
+<div data-lang="python"  markdown="1">
+{% include_example global_temp_view python/sql/basic.py %}
+</div>
+
+<div data-lang="sql"  markdown="1">
+
+{% highlight sql %}
+
+CREATE GLOBAL TEMPORARY VIEW temp_view AS SELECT a + 1, b * 2 FROM tbl
+
+SELECT * FROM global_temp.temp_view
+
+{% endhighlight %}
+
+</div>
+</div>
+
+
+## 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">
+{% 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 %}
+</div>
+</div>
+
+## 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
+<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>
+
+<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 dataset, similar to the 
inference that is performed on JSON files.
+
+{% include_example schema_inferring python/sql/basic.py %}
+</div>
+
+</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
+by `SparkSession`.
+
+For example:
+
+{% 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
+by `SparkSession`.
+
+For example:
+
+{% 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:
+
+{% include_example programmatic_schema python/sql/basic.py %}
+</div>
+
+</div>
+
+## Aggregations
+
+The [built-in DataFrames 
functions](api/scala/index.html#org.apache.spark.sql.functions$) provide common
+aggregations such as `count()`, `countDistinct()`, `avg()`, `max()`, `min()`, 
etc.
+While those functions are designed for DataFrames, Spark SQL also has 
type-safe versions for some of them in
+[Scala](api/scala/index.html#org.apache.spark.sql.expressions.scalalang.typed$)
 and
+[Java](api/java/org/apache/spark/sql/expressions/javalang/typed.html) to work 
with strongly typed Datasets.
+Moreover, users are not limited to the predefined aggregate functions and can 
create their own.
+
+### Untyped User-Defined Aggregate Functions
+Users have to extend the 
[UserDefinedAggregateFunction](api/scala/index.html#org.apache.spark.sql.expressions.UserDefinedAggregateFunction)
+abstract class to implement a custom untyped aggregate function. For example, 
a user-defined average
+can look like:
+
+<div class="codetabs">
+<div data-lang="scala"  markdown="1">
+{% include_example untyped_custom_aggregation 
scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala%}
+</div>
+<div data-lang="java"  markdown="1">
+{% include_example untyped_custom_aggregation 
java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java%}
+</div>
+</div>
+
+### Type-Safe User-Defined Aggregate Functions
+
+User-defined aggregations for strongly typed Datasets revolve around the 
[Aggregator](api/scala/index.html#org.apache.spark.sql.expressions.Aggregator) 
abstract class.
+For example, a type-safe user-defined average can look like:
+
+<div class="codetabs">
+<div data-lang="scala"  markdown="1">
+{% include_example typed_custom_aggregation 
scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala%}
+</div>
+<div data-lang="java"  markdown="1">
+{% include_example typed_custom_aggregation 
java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java%}
+</div>
+</div>

http://git-wip-us.apache.org/repos/asf/spark/blob/71535516/docs/sql-migration-guide-hive-compatibility.md
----------------------------------------------------------------------
diff --git a/docs/sql-migration-guide-hive-compatibility.md 
b/docs/sql-migration-guide-hive-compatibility.md
new file mode 100644
index 0000000..0234ea2
--- /dev/null
+++ b/docs/sql-migration-guide-hive-compatibility.md
@@ -0,0 +1,137 @@
+---
+layout: global
+title: Compatibility with Apache Hive
+displayTitle: Compatibility with Apache Hive
+---
+
+* Table of contents
+{:toc}
+
+Spark SQL is designed to be compatible with the Hive Metastore, SerDes and 
UDFs.
+Currently, Hive SerDes and UDFs are based on Hive 1.2.1,
+and Spark SQL can be connected to different versions of Hive Metastore
+(from 0.12.0 to 2.3.3. Also see [Interacting with Different Versions of Hive 
Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore)).
+
+#### Deploying in Existing Hive Warehouses
+
+The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible 
with existing Hive
+installations. You do not need to modify your existing Hive Metastore or 
change the data placement
+or partitioning of your tables.
+
+### Supported Hive Features
+
+Spark SQL supports the vast majority of Hive features, such as:
+
+* Hive query statements, including:
+  * `SELECT`
+  * `GROUP BY`
+  * `ORDER BY`
+  * `CLUSTER BY`
+  * `SORT BY`
+* All Hive operators, including:
+  * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc)
+  * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc)
+  * Logical operators (`AND`, `&&`, `OR`, `||`, etc)
+  * Complex type constructors
+  * Mathematical functions (`sign`, `ln`, `cos`, etc)
+  * String functions (`instr`, `length`, `printf`, etc)
+* User defined functions (UDF)
+* User defined aggregation functions (UDAF)
+* User defined serialization formats (SerDes)
+* Window functions
+* Joins
+  * `JOIN`
+  * `{LEFT|RIGHT|FULL} OUTER JOIN`
+  * `LEFT SEMI JOIN`
+  * `CROSS JOIN`
+* Unions
+* Sub-queries
+  * `SELECT col FROM ( SELECT a + b AS col from t1) t2`
+* Sampling
+* Explain
+* Partitioned tables including dynamic partition insertion
+* View
+* All Hive DDL Functions, including:
+  * `CREATE TABLE`
+  * `CREATE TABLE AS SELECT`
+  * `ALTER TABLE`
+* Most Hive Data types, including:
+  * `TINYINT`
+  * `SMALLINT`
+  * `INT`
+  * `BIGINT`
+  * `BOOLEAN`
+  * `FLOAT`
+  * `DOUBLE`
+  * `STRING`
+  * `BINARY`
+  * `TIMESTAMP`
+  * `DATE`
+  * `ARRAY<>`
+  * `MAP<>`
+  * `STRUCT<>`
+
+### Unsupported Hive Functionality
+
+Below is a list of Hive features that we don't support yet. Most of these 
features are rarely used
+in Hive deployments.
+
+**Major Hive Features**
+
+* Tables with buckets: bucket is the hash partitioning within a Hive table 
partition. Spark SQL
+  doesn't support buckets yet.
+
+
+**Esoteric Hive Features**
+
+* `UNION` type
+* Unique join
+* Column statistics collecting: Spark SQL does not piggyback scans to collect 
column statistics at
+  the moment and only supports populating the sizeInBytes field of the hive 
metastore.
+
+**Hive Input/Output Formats**
+
+* File format for CLI: For results showing back to the CLI, Spark SQL only 
supports TextOutputFormat.
+* Hadoop archive
+
+**Hive Optimizations**
+
+A handful of Hive optimizations are not yet included in Spark. Some of these 
(such as indexes) are
+less important due to Spark SQL's in-memory computational model. Others are 
slotted for future
+releases of Spark SQL.
+
+* Block-level bitmap indexes and virtual columns (used to build indexes)
+* Automatically determine the number of reducers for joins and groupbys: 
Currently, in Spark SQL, you
+  need to control the degree of parallelism post-shuffle using "`SET 
spark.sql.shuffle.partitions=[num_tasks];`".
+* Meta-data only query: For queries that can be answered by using only 
metadata, Spark SQL still
+  launches tasks to compute the result.
+* Skew data flag: Spark SQL does not follow the skew data flags in Hive.
+* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint.
+* Merge multiple small files for query results: if the result output contains 
multiple small files,
+  Hive can optionally merge the small files into fewer large files to avoid 
overflowing the HDFS
+  metadata. Spark SQL does not support that.
+
+**Hive UDF/UDTF/UDAF**
+
+Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below 
are the unsupported APIs:
+
+* `getRequiredJars` and `getRequiredFiles` (`UDF` and `GenericUDF`) are 
functions to automatically
+  include additional resources required by this UDF.
+* `initialize(StructObjectInspector)` in `GenericUDTF` is not supported yet. 
Spark SQL currently uses
+  a deprecated interface `initialize(ObjectInspector[])` only.
+* `configure` (`GenericUDF`, `GenericUDTF`, and `GenericUDAFEvaluator`) is a 
function to initialize
+  functions with `MapredContext`, which is inapplicable to Spark.
+* `close` (`GenericUDF` and `GenericUDAFEvaluator`) is a function to release 
associated resources.
+  Spark SQL does not call this function when tasks finish.
+* `reset` (`GenericUDAFEvaluator`) is a function to re-initialize aggregation 
for reusing the same aggregation.
+  Spark SQL currently does not support the reuse of aggregation.
+* `getWindowingEvaluator` (`GenericUDAFEvaluator`) is a function to optimize 
aggregation by evaluating
+  an aggregate over a fixed window.
+
+### Incompatible Hive UDF
+
+Below are the scenarios in which Hive and Spark generate different results:
+
+* `SQRT(n)` If n < 0, Hive returns null, Spark SQL returns NaN.
+* `ACOS(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN.
+* `ASIN(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN.

http://git-wip-us.apache.org/repos/asf/spark/blob/71535516/docs/sql-migration-guide-upgrade.md
----------------------------------------------------------------------
diff --git a/docs/sql-migration-guide-upgrade.md 
b/docs/sql-migration-guide-upgrade.md
new file mode 100644
index 0000000..3476aa8
--- /dev/null
+++ b/docs/sql-migration-guide-upgrade.md
@@ -0,0 +1,516 @@
+---
+layout: global
+title: Spark SQL Upgrading Guide
+displayTitle: Spark SQL Upgrading Guide
+---
+
+* Table of contents
+{:toc}
+
+## Upgrading From Spark SQL 2.3 to 2.4
+
+  - In Spark version 2.3 and earlier, the second parameter to array_contains 
function is implicitly promoted to the element type of first array type 
parameter. This type promotion can be lossy and may cause `array_contains` 
function to return wrong result. This problem has been addressed in 2.4 by 
employing a safer type promotion mechanism. This can cause some change in 
behavior and are illustrated in the table below.
+  <table class="table">
+        <tr>
+          <th>
+            <b>Query</b>
+          </th>
+          <th>
+            <b>Result Spark 2.3 or Prior</b>
+          </th>
+          <th>
+            <b>Result Spark 2.4</b>
+          </th>
+          <th>
+            <b>Remarks</b>
+          </th>
+        </tr>
+        <tr>
+          <th>
+            <b>SELECT <br> array_contains(array(1), 1.34D);</b>
+          </th>
+          <th>
+            <b>true</b>
+          </th>
+          <th>
+            <b>false</b>
+          </th>
+          <th>
+            <b>In Spark 2.4, left and right parameters are  promoted to 
array(double) and double type respectively.</b>
+          </th>
+        </tr>
+        <tr>
+          <th>
+            <b>SELECT <br> array_contains(array(1), '1');</b>
+          </th>
+          <th>
+            <b>true</b>
+          </th>
+          <th>
+            <b>AnalysisException is thrown since integer type can not be 
promoted to string type in a loss-less manner.</b>
+          </th>
+          <th>
+            <b>Users can use explicit cast</b>
+          </th>
+        </tr>
+        <tr>
+          <th>
+            <b>SELECT <br> array_contains(array(1), 'anystring');</b>
+          </th>
+          <th>
+            <b>null</b>
+          </th>
+          <th>
+            <b>AnalysisException is thrown since integer type can not be 
promoted to string type in a loss-less manner.</b>
+          </th>
+          <th>
+            <b>Users can use explicit cast</b>
+          </th>
+        </tr>
+  </table>
+
+  - Since Spark 2.4, when there is a struct field in front of the IN operator 
before a subquery, the inner query must contain a struct field as well. In 
previous versions, instead, the fields of the struct were compared to the 
output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 
2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a 
in (select 1, 'a' from range(1))` is not. In previous version it was the 
opposite.
+  - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, 
then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became 
case-sensitive and would resolve to columns (unless typed in lower case). In 
Spark 2.4 this has been fixed and the functions are no longer case-sensitive.
+  - Since Spark 2.4, Spark will evaluate the set operations referenced in a 
query by following a precedence rule as per the SQL standard. If the order is 
not specified by parentheses, set operations are performed from left to right 
with the exception that all INTERSECT operations are performed before any 
UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence 
to all the set operations are preserved under a newly added configuration 
`spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. 
When this property is set to `true`, spark will evaluate the set operators from 
left to right as they appear in the query given no explicit ordering is 
enforced by usage of parenthesis.
+  - Since Spark 2.4, Spark will display table description column Last Access 
value as UNKNOWN when the value was Jan 01 1970.
+  - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for 
ORC files by default. To do that, `spark.sql.orc.impl` and 
`spark.sql.orc.filterPushdown` change their default values to `native` and 
`true` respectively.
+  - In PySpark, when Arrow optimization is enabled, previously `toPandas` just 
failed when Arrow optimization is unable to be used whereas `createDataFrame` 
from Pandas DataFrame allowed the fallback to non-optimization. Now, both 
`toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by 
default, which can be switched off by 
`spark.sql.execution.arrow.fallback.enabled`.
+  - Since Spark 2.4, writing an empty dataframe to a directory launches at 
least one write task, even if physically the dataframe has no partition. This 
introduces a small behavior change that for self-describing file formats like 
Parquet and Orc, Spark creates a metadata-only file in the target directory 
when writing a 0-partition dataframe, so that schema inference can still work 
if users read that directory later. The new behavior is more reasonable and 
more consistent regarding writing empty dataframe.
+  - Since Spark 2.4, expression IDs in UDF arguments do not appear in column 
names. For example, a column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` 
but ``UDF:f(col0 AS `colA`)``.
+  - Since Spark 2.4, writing a dataframe with an empty or nested empty schema 
using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An 
exception is thrown when attempting to write dataframes with empty schema.
+  - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after 
promotes both sides to TIMESTAMP. To set `false` to 
`spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous 
behavior. This option will be removed in Spark 3.0.
+  - Since Spark 2.4, creating a managed table with nonempty location is not 
allowed. An exception is thrown when attempting to create a managed table with 
nonempty location. To set `true` to 
`spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the 
previous behavior. This option will be removed in Spark 3.0.
+  - Since Spark 2.4, renaming a managed table to existing location is not 
allowed. An exception is thrown when attempting to rename a managed table to 
existing location.
+  - Since Spark 2.4, the type coercion rules can automatically promote the 
argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest 
common type, no matter how the input arguments order. In prior Spark versions, 
the promotion could fail in some specific orders (e.g., TimestampType, 
IntegerType and StringType) and throw an exception.
+  - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in 
addition to the traditional cache invalidation mechanism. The non-cascading 
cache invalidation mechanism allows users to remove a cache without impacting 
its dependent caches. This new cache invalidation mechanism is used in 
scenarios where the data of the cache to be removed is still valid, e.g., 
calling unpersist() on a Dataset, or dropping a temporary view. This allows 
users to free up memory and keep the desired caches valid at the same time.
+  - In version 2.3 and earlier, Spark converts Parquet Hive tables by default 
but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. 
This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 
'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 
2.4, Spark respects Parquet/ORC specific table properties while converting 
Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS 
PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy 
parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would 
be uncompressed parquet files.
+  - Since Spark 2.0, Spark converts Parquet Hive tables by default for better 
performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. 
It means Spark uses its own ORC support by default instead of Hive SerDe. As an 
example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive 
SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC 
data source table and ORC vectorization would be applied. To set `false` to 
`spark.sql.hive.convertMetastoreOrc` restores the previous behavior.
+  - In version 2.3 and earlier, CSV rows are considered as malformed if at 
least one column value in the row is malformed. CSV parser dropped such rows in 
the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 
2.4, CSV row is considered as malformed only when it contains malformed column 
values requested from CSV datasource, other values can be ignored. As an 
example, CSV file contains the "id,name" header and one row "1234". In Spark 
2.4, selection of the id column consists of a row with one column value 1234 
but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore 
the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to 
`false`.
+  - Since Spark 2.4, File listing for compute statistics is done in parallel 
by default. This can be disabled by setting 
`spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `False`.
+  - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary 
files are not counted as data files when calculating table size during 
Statistics computation.
+  - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In 
version 2.3 and earlier, empty strings are equal to `null` values and do not 
reflect to any characters in saved CSV files. For example, the row of `"a", 
null, "", 1` was writted as `a,,,1`. Since Spark 2.4, the same row is saved as 
`a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to 
empty (not quoted) string.  
+  - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, 
which match any one character, and zero or more characters, respectively. 
Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH 
'/tmp/part-?'`. Special Characters like `space` also now work in paths. 
Example: `LOAD DATA INPATH '/tmp/folder name/'`.
+  - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as 
WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 
1 FROM range(10) WHERE true`  and returns 10 rows. This violates SQL standard, 
and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is 
treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING 
true` will return only one row. To restore the previous behavior, set 
`spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`.
+
+## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above
+
+  - As of version 2.3.1 Arrow functionality, including `pandas_udf` and 
`toPandas()`/`createDataFrame()` with `spark.sql.execution.arrow.enabled` set 
to `True`, has been marked as experimental. These are still evolving and not 
currently recommended for use in production.
+
+## Upgrading From Spark SQL 2.2 to 2.3
+
+  - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when 
the referenced columns only include the internal corrupt record column (named 
`_corrupt_record` by default). For example, 
`spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()`
 and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. 
Instead, you can cache or save the parsed results and then send the same query. 
For example, `val df = spark.read.schema(schema).json(file).cache()` and then 
`df.filter($"_corrupt_record".isNotNull).count()`.
+  - The `percentile_approx` function previously accepted numeric type input 
and output double type results. Now it supports date type, timestamp type and 
numeric types as input types. The result type is also changed to be the same as 
the input type, which is more reasonable for percentiles.
+  - Since Spark 2.3, the Join/Filter's deterministic predicates that are after 
the first non-deterministic predicates are also pushed down/through the child 
operators, if possible. In prior Spark versions, these filters are not eligible 
for predicate pushdown.
+  - Partition column inference previously found incorrect common type for 
different inferred types, for example, previously it ended up with double type 
as the common type for double type and date type. Now it finds the correct 
common type for such conflicts. The conflict resolution follows the table below:
+    <table class="table">
+      <tr>
+        <th>
+          <b>InputA \ InputB</b>
+        </th>
+        <th>
+          <b>NullType</b>
+        </th>
+        <th>
+          <b>IntegerType</b>
+        </th>
+        <th>
+          <b>LongType</b>
+        </th>
+        <th>
+          <b>DecimalType(38,0)*</b>
+        </th>
+        <th>
+          <b>DoubleType</b>
+        </th>
+        <th>
+          <b>DateType</b>
+        </th>
+        <th>
+          <b>TimestampType</b>
+        </th>
+        <th>
+          <b>StringType</b>
+        </th>
+      </tr>
+      <tr>
+        <td>
+          <b>NullType</b>
+        </td>
+        <td>NullType</td>
+        <td>IntegerType</td>
+        <td>LongType</td>
+        <td>DecimalType(38,0)</td>
+        <td>DoubleType</td>
+        <td>DateType</td>
+        <td>TimestampType</td>
+        <td>StringType</td>
+      </tr>
+      <tr>
+        <td>
+          <b>IntegerType</b>
+        </td>
+        <td>IntegerType</td>
+        <td>IntegerType</td>
+        <td>LongType</td>
+        <td>DecimalType(38,0)</td>
+        <td>DoubleType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+      </tr>
+      <tr>
+        <td>
+          <b>LongType</b>
+        </td>
+        <td>LongType</td>
+        <td>LongType</td>
+        <td>LongType</td>
+        <td>DecimalType(38,0)</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+      </tr>
+      <tr>
+        <td>
+          <b>DecimalType(38,0)*</b>
+        </td>
+        <td>DecimalType(38,0)</td>
+        <td>DecimalType(38,0)</td>
+        <td>DecimalType(38,0)</td>
+        <td>DecimalType(38,0)</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+      </tr>
+      <tr>
+        <td>
+          <b>DoubleType</b>
+        </td>
+        <td>DoubleType</td>
+        <td>DoubleType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>DoubleType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+      </tr>
+      <tr>
+        <td>
+          <b>DateType</b>
+        </td>
+        <td>DateType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>DateType</td>
+        <td>TimestampType</td>
+        <td>StringType</td>
+      </tr>
+      <tr>
+        <td>
+          <b>TimestampType</b>
+        </td>
+        <td>TimestampType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>TimestampType</td>
+        <td>TimestampType</td>
+        <td>StringType</td>
+      </tr>
+      <tr>
+        <td>
+          <b>StringType</b>
+        </td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+        <td>StringType</td>
+      </tr>
+    </table>
+
+    Note that, for <b>DecimalType(38,0)*</b>, the table above intentionally 
does not cover all other combinations of scales and precisions because 
currently we only infer decimal type like `BigInteger`/`BigInt`. For example, 
1.1 is inferred as double type.
+  - In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas 
related functionalities, such as `toPandas`, `createDataFrame` from Pandas 
DataFrame, etc.
+  - In PySpark, the behavior of timestamp values for Pandas related 
functionalities was changed to respect session timezone. If you want to use the 
old behavior, you need to set a configuration 
`spark.sql.execution.pandas.respectSessionTimeZone` to `False`. See 
[SPARK-22395](https://issues.apache.org/jira/browse/SPARK-22395) for details.
+  - In PySpark, `na.fill()` or `fillna` also accepts boolean and replaces 
nulls with booleans. In prior Spark versions, PySpark just ignores it and 
returns the original Dataset/DataFrame.
+  - Since Spark 2.3, when either broadcast hash join or broadcast nested loop 
join is applicable, we prefer to broadcasting the table that is explicitly 
specified in a broadcast hint. For details, see the section [Broadcast 
Hint](sql-performance-turing.html#broadcast-hint-for-sql-queries) and 
[SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489).
+  - Since Spark 2.3, when all inputs are binary, `functions.concat()` returns 
an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it 
always returns as a string despite of input types. To keep the old behavior, 
set `spark.sql.function.concatBinaryAsString` to `true`.
+  - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output 
as binary. Otherwise, it returns as a string. Until Spark 2.3, it always 
returns as a string despite of input types. To keep the old behavior, set 
`spark.sql.function.eltOutputAsString` to `true`.
+
+ - Since Spark 2.3, by default arithmetic operations between decimals return a 
rounded value if an exact representation is not possible (instead of returning 
NULL). This is compliant with SQL ANSI 2011 specification and Hive's new 
behavior introduced in Hive 2.2 (HIVE-15331). This involves the following 
changes
+    - The rules to determine the result type of an arithmetic operation have 
been updated. In particular, if the precision / scale needed are out of the 
range of available values, the scale is reduced up to 6, in order to prevent 
the truncation of the integer part of the decimals. All the arithmetic 
operations are affected by the change, ie. addition (`+`), subtraction (`-`), 
multiplication (`*`), division (`/`), remainder (`%`) and positive module 
(`pmod`).
+    - Literal values used in SQL operations are converted to DECIMAL with the 
exact precision and scale needed by them.
+    - The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has 
been introduced. It defaults to `true`, which means the new behavior described 
here; if set to `false`, Spark uses previous rules, ie. it doesn't adjust the 
needed scale to represent the values and it returns NULL if an exact 
representation of the value is not possible.
+  - In PySpark, `df.replace` does not allow to omit `value` when `to_replace` 
is not a dictionary. Previously, `value` could be omitted in the other cases 
and had `None` by default, which is counterintuitive and error-prone.
+  - Un-aliased subquery's semantic has not been well defined with confusing 
behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: 
`SELECT v.i from (SELECT i FROM v)`, Spark will throw an analysis exception in 
this case because users should not be able to use the qualifier inside a 
subquery. See [SPARK-20690](https://issues.apache.org/jira/browse/SPARK-20690) 
and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more 
details.
+
+  - When creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, 
if there is an existing `SparkContext`, the builder was trying to update the 
`SparkConf` of the existing `SparkContext` with configurations specified to the 
builder, but the `SparkContext` is shared by all `SparkSession`s, so we should 
not update them. Since 2.3, the builder comes to not update the configurations. 
If you want to update them, you need to update them prior to creating a 
`SparkSession`.
+
+## Upgrading From Spark SQL 2.1 to 2.2
+
+  - Spark 2.1.1 introduced a new configuration key: 
`spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of 
`NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 
changes this setting's default value to `INFER_AND_SAVE` to restore 
compatibility with reading Hive metastore tables whose underlying file schema 
have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on 
first access Spark will perform schema inference on any Hive metastore table 
for which it has not already saved an inferred schema. Note that schema 
inference can be a very time-consuming operation for tables with thousands of 
partitions. If compatibility with mixed-case column names is not a concern, you 
can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to 
avoid the initial overhead of schema inference. Note that with the new default 
`INFER_AND_SAVE` setting, the results of the schema inference are saved as a 
metastore key for future use
 . Therefore, the initial schema inference occurs only at a table's first 
access.
+  
+  - Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when 
the data source tables have the columns that exist in both partition schema and 
data schema. The inferred schema does not have the partitioned columns. When 
reading the table, Spark respects the partition values of these overlapping 
columns instead of the values stored in the data source files. In 2.2.0 and 
2.1.x release, the inferred schema is partitioned but the data of the table is 
invisible to users (i.e., the result set is empty).
+
+## Upgrading From Spark SQL 2.0 to 2.1
+
+ - Datasource tables now store partition metadata in the Hive metastore. This 
means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now 
available for tables created with the Datasource API.
+    - Legacy datasource tables can be migrated to this format via the `MSCK 
REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage 
of Hive DDL support and improved planning performance.
+    - To determine if a table has been migrated, look for the 
`PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the 
table.
+ - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for 
Datasource tables.
+    - In prior Spark versions `INSERT OVERWRITE` overwrote the entire 
Datasource table, even when given a partition specification. Now only 
partitions matching the specification are overwritten.
+    - Note that this still differs from the behavior of Hive tables, which is 
to overwrite only partitions overlapping with newly inserted data.
+
+## Upgrading From Spark SQL 1.6 to 2.0
+
+ - `SparkSession` is now the new entry point of Spark that replaces the old 
`SQLContext` and
+   `HiveContext`. Note that the old SQLContext and HiveContext are kept for 
backward compatibility. A new `catalog` interface is accessible from 
`SparkSession` - existing API on databases and tables access such as 
`listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved 
here.
+
+ - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a 
type alias for
+   `Dataset[Row]`, while Java API users must replace `DataFrame` with 
`Dataset<Row>`. Both the typed
+   transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped 
transformations (e.g.,
+   `select` and `groupBy`) are available on the Dataset class. Since 
compile-time type-safety in
+   Python and R is not a language feature, the concept of Dataset does not 
apply to these languages’
+   APIs. Instead, `DataFrame` remains the primary programming abstraction, 
which is analogous to the
+   single-node data frame notion in these languages.
+
+ - Dataset and DataFrame API `unionAll` has been deprecated and replaced by 
`union`
+ - Dataset and DataFrame API `explode` has been deprecated, alternatively, use 
`functions.explode()` with `select` or `flatMap`
+ - Dataset and DataFrame API `registerTempTable` has been deprecated and 
replaced by `createOrReplaceTempView`
+
+ - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables.
+    - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE 
EXTERNAL TABLE ... LOCATION`
+      in order to prevent accidental dropping the existing data in the 
user-provided locations.
+      That means, a Hive table created in Spark SQL with the user-specified 
location is always a Hive external table.
+      Dropping external tables will not remove the data. Users are not allowed 
to specify the location for Hive managed tables.
+      Note that this is different from the Hive behavior.
+    - As a result, `DROP TABLE` statements on those tables will not remove the 
data.
+
+ - `spark.sql.parquet.cacheMetadata` is no longer used.
+   See [SPARK-13664](https://issues.apache.org/jira/browse/SPARK-13664) for 
details.
+
+## Upgrading From Spark SQL 1.5 to 1.6
+
+ - From Spark 1.6, by default, the Thrift server runs in multi-session mode. 
Which means each JDBC/ODBC
+   connection owns a copy of their own SQL configuration and temporary 
function registry. Cached
+   tables are still shared though. If you prefer to run the Thrift server in 
the old single-session
+   mode, please set option `spark.sql.hive.thriftServer.singleSession` to 
`true`. You may either add
+   this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` 
via `--conf`:
+
+   {% highlight bash %}
+   ./sbin/start-thriftserver.sh \
+     --conf spark.sql.hive.thriftServer.singleSession=true \
+     ...
+   {% endhighlight %}
+ - Since 1.6.1, withColumn method in sparkR supports adding a new column to or 
replacing existing columns
+   of the same name of a DataFrame.
+
+ - From Spark 1.6, LongType casts to TimestampType expect seconds instead of 
microseconds. This
+   change was made to match the behavior of Hive 1.2 for more consistent type 
casting to TimestampType
+   from numeric types. See 
[SPARK-11724](https://issues.apache.org/jira/browse/SPARK-11724) for
+   details.
+
+## Upgrading From Spark SQL 1.4 to 1.5
+
+ - Optimized execution using manually managed memory (Tungsten) is now enabled 
by default, along with
+   code generation for expression evaluation. These features can both be 
disabled by setting
+   `spark.sql.tungsten.enabled` to `false`.
+ - Parquet schema merging is no longer enabled by default. It can be 
re-enabled by setting
+   `spark.sql.parquet.mergeSchema` to `true`.
+ - Resolution of strings to columns in python now supports using dots (`.`) to 
qualify the column or
+   access nested values. For example `df['table.column.nestedField']`. 
However, this means that if
+   your column name contains any dots you must now escape them using backticks 
(e.g., ``table.`column.with.dots`.nested``).
+ - In-memory columnar storage partition pruning is on by default. It can be 
disabled by setting
+   `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`.
+ - Unlimited precision decimal columns are no longer supported, instead Spark 
SQL enforces a maximum
+   precision of 38. When inferring schema from `BigDecimal` objects, a 
precision of (38, 18) is now
+   used. When no precision is specified in DDL then the default remains 
`Decimal(10, 0)`.
+ - Timestamps are now stored at a precision of 1us, rather than 1ns
+ - In the `sql` dialect, floating point numbers are now parsed as decimal. 
HiveQL parsing remains
+   unchanged.
+ - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum 
vs SUM).
+ - JSON data source will not automatically load new files that are created by 
other applications
+   (i.e. files that are not inserted to the dataset through Spark SQL).
+   For a JSON persistent table (i.e. the metadata of the table is stored in 
Hive Metastore),
+   users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` 
method
+   to include those new files to the table. For a DataFrame representing a 
JSON dataset, users need to recreate
+   the DataFrame and the new DataFrame will include new files.
+ - DataFrame.withColumn method in pySpark supports adding a new column or 
replacing existing columns of the same name.
+
+## Upgrading from Spark SQL 1.3 to 1.4
+
+#### DataFrame data reader/writer interface
+
+Based on user feedback, we created a new, more fluid API for reading data in 
(`SQLContext.read`)
+and writing data out (`DataFrame.write`),
+and deprecated the old APIs (e.g., `SQLContext.parquetFile`, 
`SQLContext.jsonFile`).
+
+See the API docs for `SQLContext.read` (
+  <a 
href="api/scala/index.html#org.apache.spark.sql.SQLContext@read:DataFrameReader">Scala</a>,
+  <a href="api/java/org/apache/spark/sql/SQLContext.html#read()">Java</a>,
+  <a href="api/python/pyspark.sql.html#pyspark.sql.SQLContext.read">Python</a>
+) and `DataFrame.write` (
+  <a 
href="api/scala/index.html#org.apache.spark.sql.DataFrame@write:DataFrameWriter">Scala</a>,
+  <a href="api/java/org/apache/spark/sql/Dataset.html#write()">Java</a>,
+  <a href="api/python/pyspark.sql.html#pyspark.sql.DataFrame.write">Python</a>
+) more information.
+
+
+#### DataFrame.groupBy retains grouping columns
+
+Based on user feedback, we changed the default behavior of 
`DataFrame.groupBy().agg()` to retain the
+grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, 
set `spark.sql.retainGroupColumns` to `false`.
+
+<div class="codetabs">
+<div data-lang="scala"  markdown="1">
+{% highlight scala %}
+
+// In 1.3.x, in order for the grouping column "department" to show up,
+// it must be included explicitly as part of the agg function call.
+df.groupBy("department").agg($"department", max("age"), sum("expense"))
+
+// In 1.4+, grouping column "department" is included automatically.
+df.groupBy("department").agg(max("age"), sum("expense"))
+
+// Revert to 1.3 behavior (not retaining grouping column) by:
+sqlContext.setConf("spark.sql.retainGroupColumns", "false")
+
+{% endhighlight %}
+</div>
+
+<div data-lang="java"  markdown="1">
+{% highlight java %}
+
+// In 1.3.x, in order for the grouping column "department" to show up,
+// it must be included explicitly as part of the agg function call.
+df.groupBy("department").agg(col("department"), max("age"), sum("expense"));
+
+// In 1.4+, grouping column "department" is included automatically.
+df.groupBy("department").agg(max("age"), sum("expense"));
+
+// Revert to 1.3 behavior (not retaining grouping column) by:
+sqlContext.setConf("spark.sql.retainGroupColumns", "false");
+
+{% endhighlight %}
+</div>
+
+<div data-lang="python"  markdown="1">
+{% highlight python %}
+
+import pyspark.sql.functions as func
+
+# In 1.3.x, in order for the grouping column "department" to show up,
+# it must be included explicitly as part of the agg function call.
+df.groupBy("department").agg(df["department"], func.max("age"), 
func.sum("expense"))
+
+# In 1.4+, grouping column "department" is included automatically.
+df.groupBy("department").agg(func.max("age"), func.sum("expense"))
+
+# Revert to 1.3.x behavior (not retaining grouping column) by:
+sqlContext.setConf("spark.sql.retainGroupColumns", "false")
+
+{% endhighlight %}
+</div>
+
+</div>
+
+
+#### Behavior change on DataFrame.withColumn
+
+Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column 
will always be added
+as a new column with its specified name in the result DataFrame even if there 
may be any existing
+columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a 
column of a different
+name from names of all existing columns or replacing existing columns of the 
same name.
+
+Note that this change is only for Scala API, not for PySpark and SparkR.
+
+
+## Upgrading from Spark SQL 1.0-1.2 to 1.3
+
+In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this 
did a cleanup of the
+available APIs. From Spark 1.3 onwards, Spark SQL will provide binary 
compatibility with other
+releases in the 1.X series. This compatibility guarantee excludes APIs that 
are explicitly marked
+as unstable (i.e., DeveloperAPI or Experimental).
+
+#### Rename of SchemaRDD to DataFrame
+
+The largest change that users will notice when upgrading to Spark SQL 1.3 is 
that `SchemaRDD` has
+been renamed to `DataFrame`. This is primarily because DataFrames no longer 
inherit from RDD
+directly, but instead provide most of the functionality that RDDs provide 
though their own
+implementation. DataFrames can still be converted to RDDs by calling the 
`.rdd` method.
+
+In Scala, there is a type alias from `SchemaRDD` to `DataFrame` to provide 
source compatibility for
+some use cases. It is still recommended that users update their code to use 
`DataFrame` instead.
+Java and Python users will need to update their code.
+
+#### Unification of the Java and Scala APIs
+
+Prior to Spark 1.3 there were separate Java compatible classes 
(`JavaSQLContext` and `JavaSchemaRDD`)
+that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been 
unified. Users
+of either language should use `SQLContext` and `DataFrame`. In general these 
classes try to
+use types that are usable from both languages (i.e. `Array` instead of 
language-specific collections).
+In some cases where no common type exists (e.g., for passing in closures or 
Maps) function overloading
+is used instead.
+
+Additionally, the Java specific types API has been removed. Users of both 
Scala and Java should
+use the classes present in `org.apache.spark.sql.types` to describe schema 
programmatically.
+
+
+#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only)
+
+Many of the code examples prior to Spark 1.3 started with `import 
sqlContext._`, which brought
+all of the functions from sqlContext into scope. In Spark 1.3 we have isolated 
the implicit
+conversions for converting `RDD`s into `DataFrame`s into an object inside of 
the `SQLContext`.
+Users should now write `import sqlContext.implicits._`.
+
+Additionally, the implicit conversions now only augment RDDs that are composed 
of `Product`s (i.e.,
+case classes or tuples) with a method `toDF`, instead of applying 
automatically.
+
+When using function inside of the DSL (now replaced with the `DataFrame` API) 
users used to import
+`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions 
API should be used:
+`import org.apache.spark.sql.functions._`.
+
+#### Removal of the type aliases in org.apache.spark.sql for DataType 
(Scala-only)
+
+Spark 1.3 removes the type aliases that were present in the base sql package 
for `DataType`. Users
+should instead import the classes in `org.apache.spark.sql.types`
+
+#### UDF Registration Moved to `sqlContext.udf` (Java & Scala)
+
+Functions that are used to register UDFs, either for use in the DataFrame DSL 
or SQL, have been
+moved into the udf object in `SQLContext`.
+
+<div class="codetabs">
+<div data-lang="scala"  markdown="1">
+{% highlight scala %}
+
+sqlContext.udf.register("strLen", (s: String) => s.length())
+
+{% endhighlight %}
+</div>
+
+<div data-lang="java"  markdown="1">
+{% highlight java %}
+
+sqlContext.udf().register("strLen", (String s) -> s.length(), 
DataTypes.IntegerType);
+
+{% endhighlight %}
+</div>
+
+</div>
+
+Python UDF registration is unchanged.
+
+#### Python DataTypes No Longer Singletons
+
+When using DataTypes in Python you will need to construct them (i.e. 
`StringType()`) instead of
+referencing a singleton.

http://git-wip-us.apache.org/repos/asf/spark/blob/71535516/docs/sql-migration-guide.md
----------------------------------------------------------------------
diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md
new file mode 100644
index 0000000..71d83e8
--- /dev/null
+++ b/docs/sql-migration-guide.md
@@ -0,0 +1,23 @@
+---
+layout: global
+title: Migration Guide
+displayTitle: Migration Guide
+---
+
+* [Spark SQL Upgrading Guide](sql-migration-guide-upgrade.html)
+  * [Upgrading From Spark SQL 2.4 to 
3.0](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-24-to-30)
+  * [Upgrading From Spark SQL 2.3 to 
2.4](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-23-to-24)
+  * [Upgrading From Spark SQL 2.3.0 to 2.3.1 and 
above](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-230-to-231-and-above)
+  * [Upgrading From Spark SQL 2.2 to 
2.3](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-22-to-23)
+  * [Upgrading From Spark SQL 2.1 to 
2.2](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-21-to-22)
+  * [Upgrading From Spark SQL 2.0 to 
2.1](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-20-to-21)
+  * [Upgrading From Spark SQL 1.6 to 
2.0](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-16-to-20)
+  * [Upgrading From Spark SQL 1.5 to 
1.6](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-15-to-16)
+  * [Upgrading From Spark SQL 1.4 to 
1.5](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-14-to-15)
+  * [Upgrading from Spark SQL 1.3 to 
1.4](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-13-to-14)
+  * [Upgrading from Spark SQL 1.0-1.2 to 
1.3](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-10-12-to-13)
+* [Compatibility with Apache Hive](sql-migration-guide-hive-compatibility.html)
+  * [Deploying in Existing Hive 
Warehouses](sql-migration-guide-hive-compatibility.html#deploying-in-existing-hive-warehouses)
+  * [Supported Hive 
Features](sql-migration-guide-hive-compatibility.html#supported-hive-features)
+  * [Unsupported Hive 
Functionality](sql-migration-guide-hive-compatibility.html#unsupported-hive-functionality)
+  * [Incompatible Hive 
UDF](sql-migration-guide-hive-compatibility.html#incompatible-hive-udf)

http://git-wip-us.apache.org/repos/asf/spark/blob/71535516/docs/sql-performance-turing.md
----------------------------------------------------------------------
diff --git a/docs/sql-performance-turing.md b/docs/sql-performance-turing.md
new file mode 100644
index 0000000..7c7c4a8
--- /dev/null
+++ b/docs/sql-performance-turing.md
@@ -0,0 +1,151 @@
+---
+layout: global
+title: Performance Tuning
+displayTitle: Performance Tuning
+---
+
+* Table of contents
+{:toc}
+
+For some workloads, it is possible to improve performance by either caching 
data in memory, or by
+turning on some experimental options.
+
+## Caching Data In Memory
+
+Spark SQL can cache tables using an in-memory columnar format by calling 
`spark.catalog.cacheTable("tableName")` or `dataFrame.cache()`.
+Then Spark SQL will scan only required columns and will automatically tune 
compression to minimize
+memory usage and GC pressure. You can call 
`spark.catalog.uncacheTable("tableName")` to remove the table from memory.
+
+Configuration of in-memory caching can be done using the `setConf` method on 
`SparkSession` or by running
+`SET key=value` commands using SQL.
+
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr>
+  <td><code>spark.sql.inMemoryColumnarStorage.compressed</code></td>
+  <td>true</td>
+  <td>
+    When set to true Spark SQL will automatically select a compression codec 
for each column based
+    on statistics of the data.
+  </td>
+</tr>
+<tr>
+  <td><code>spark.sql.inMemoryColumnarStorage.batchSize</code></td>
+  <td>10000</td>
+  <td>
+    Controls the size of batches for columnar caching. Larger batch sizes can 
improve memory utilization
+    and compression, but risk OOMs when caching data.
+  </td>
+</tr>
+
+</table>
+
+## Other Configuration Options
+
+The following options can also be used to tune the performance of query 
execution. It is possible
+that these options will be deprecated in future release as more optimizations 
are performed automatically.
+
+<table class="table">
+  <tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+  <tr>
+    <td><code>spark.sql.files.maxPartitionBytes</code></td>
+    <td>134217728 (128 MB)</td>
+    <td>
+      The maximum number of bytes to pack into a single partition when reading 
files.
+    </td>
+  </tr>
+  <tr>
+    <td><code>spark.sql.files.openCostInBytes</code></td>
+    <td>4194304 (4 MB)</td>
+    <td>
+      The estimated cost to open a file, measured by the number of bytes could 
be scanned in the same
+      time. This is used when putting multiple files into a partition. It is 
better to over-estimated,
+      then the partitions with small files will be faster than partitions with 
bigger files (which is
+      scheduled first).
+    </td>
+  </tr>
+  <tr>
+    <td><code>spark.sql.broadcastTimeout</code></td>
+    <td>300</td>
+    <td>
+    <p>
+      Timeout in seconds for the broadcast wait time in broadcast joins
+    </p>
+    </td>
+  </tr>
+  <tr>
+    <td><code>spark.sql.autoBroadcastJoinThreshold</code></td>
+    <td>10485760 (10 MB)</td>
+    <td>
+      Configures the maximum size in bytes for a table that will be broadcast 
to all worker nodes when
+      performing a join. By setting this value to -1 broadcasting can be 
disabled. Note that currently
+      statistics are only supported for Hive Metastore tables where the command
+      <code>ANALYZE TABLE &lt;tableName&gt; COMPUTE STATISTICS noscan</code> 
has been run.
+    </td>
+  </tr>
+  <tr>
+    <td><code>spark.sql.shuffle.partitions</code></td>
+    <td>200</td>
+    <td>
+      Configures the number of partitions to use when shuffling data for joins 
or aggregations.
+    </td>
+  </tr>
+</table>
+
+## Broadcast Hint for SQL Queries
+
+The `BROADCAST` hint guides Spark to broadcast each specified table when 
joining them with another table or view.
+When Spark deciding the join methods, the broadcast hash join (i.e., BHJ) is 
preferred,
+even if the statistics is above the configuration 
`spark.sql.autoBroadcastJoinThreshold`.
+When both sides of a join are specified, Spark broadcasts the one having the 
lower statistics.
+Note Spark does not guarantee BHJ is always chosen, since not all cases (e.g. 
full outer join)
+support BHJ. When the broadcast nested loop join is selected, we still respect 
the hint.
+
+<div class="codetabs">
+
+<div data-lang="scala"  markdown="1">
+
+{% highlight scala %}
+import org.apache.spark.sql.functions.broadcast
+broadcast(spark.table("src")).join(spark.table("records"), "key").show()
+{% endhighlight %}
+
+</div>
+
+<div data-lang="java"  markdown="1">
+
+{% highlight java %}
+import static org.apache.spark.sql.functions.broadcast;
+broadcast(spark.table("src")).join(spark.table("records"), "key").show();
+{% endhighlight %}
+
+</div>
+
+<div data-lang="python"  markdown="1">
+
+{% highlight python %}
+from pyspark.sql.functions import broadcast
+broadcast(spark.table("src")).join(spark.table("records"), "key").show()
+{% endhighlight %}
+
+</div>
+
+<div data-lang="r"  markdown="1">
+
+{% highlight r %}
+src <- sql("SELECT * FROM src")
+records <- sql("SELECT * FROM records")
+head(join(broadcast(src), records, src$key == records$key))
+{% endhighlight %}
+
+</div>
+
+<div data-lang="sql"  markdown="1">
+
+{% highlight sql %}
+-- We accept BROADCAST, BROADCASTJOIN and MAPJOIN for broadcast hint
+SELECT /*+ BROADCAST(r) */ * FROM records r JOIN src s ON r.key = s.key
+{% endhighlight %}
+
+</div>
+</div>


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to