spark-instrumented-optimizer/sql
Dongjoon Hyun a65bcbc27d [SPARK-16186] [SQL] Support partition batch pruning with IN predicate in InMemoryTableScanExec
## What changes were proposed in this pull request?

One of the most frequent usage patterns for Spark SQL is using **cached tables**. This PR improves `InMemoryTableScanExec` to handle `IN` predicate efficiently by pruning partition batches. Of course, the performance improvement varies over the queries and the datasets. But, for the following simple query, the query duration in Spark UI goes from 9 seconds to 50~90ms. It's about over 100 times faster.

**Before**
```scala
$ bin/spark-shell --driver-memory 6G
scala> val df = spark.range(2000000000)
scala> df.createOrReplaceTempView("t")
scala> spark.catalog.cacheTable("t")
scala> sql("select id from t where id = 1").collect()    // About 2 mins
scala> sql("select id from t where id = 1").collect()    // less than 90ms
scala> sql("select id from t where id in (1,2,3)").collect()  // 9 seconds
```

**After**
```scala
scala> sql("select id from t where id in (1,2,3)").collect() // less than 90ms
```

This PR has impacts over 35 queries of TPC-DS if the tables are cached.
Note that this optimization is applied for `IN`.  To apply `IN` predicate having more than 10 items, `spark.sql.optimizer.inSetConversionThreshold` option should be increased.

## How was this patch tested?

Pass the Jenkins tests (including new testcases).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13887 from dongjoon-hyun/SPARK-16186.
2016-06-24 17:13:13 -07:00
..
catalyst [SPARK-16129][CORE][SQL] Eliminate direct use of commons-lang classes in favor of commons-lang3 2016-06-24 10:35:54 +01:00
core [SPARK-16186] [SQL] Support partition batch pruning with IN predicate in InMemoryTableScanExec 2016-06-24 17:13:13 -07:00
hive [SPARK-13709][SQL] Initialize deserializer with both table and partition properties when reading partitioned tables 2016-06-23 23:11:46 -07:00
hive-thriftserver [SPARK-16129][CORE][SQL] Eliminate direct use of commons-lang classes in favor of commons-lang3 2016-06-24 10:35:54 +01:00
README.md [MINOR][SQL][DOCS] Update sql/README.md and remove some unused imports in sql module. 2016-03-22 23:07:49 -07:00

Spark SQL

This module provides support for executing relational queries expressed in either SQL or a LINQ-like Scala DSL.

Spark SQL is broken up into four subprojects:

  • Catalyst (sql/catalyst) - An implementation-agnostic framework for manipulating trees of relational operators and expressions.
  • Execution (sql/core) - A query planner / execution engine for translating Catalyst's logical query plans into Spark RDDs. This component also includes a new public interface, SQLContext, that allows users to execute SQL or LINQ statements against existing RDDs and Parquet files.
  • Hive Support (sql/hive) - Includes an extension of SQLContext called HiveContext that allows users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allows users to run queries that include Hive UDFs, UDAFs, and UDTFs.
  • HiveServer and CLI support (sql/hive-thriftserver) - Includes support for the SQL CLI (bin/spark-sql) and a HiveServer2 (for JDBC/ODBC) compatible server.

Other dependencies for developers

In order to create new hive test cases (i.e. a test suite based on HiveComparisonTest), you will need to setup your development environment based on the following instructions.

If you are working with Hive 0.12.0, you will need to set several environmental variables as follows.

export HIVE_HOME="<path to>/hive/build/dist"
export HIVE_DEV_HOME="<path to>/hive/"
export HADOOP_HOME="<path to>/hadoop"

If you are working with Hive 0.13.1, the following steps are needed:

  1. Download Hive's 0.13.1 and set HIVE_HOME with export HIVE_HOME="<path to hive>". Please do not set HIVE_DEV_HOME (See SPARK-4119).
  2. Set HADOOP_HOME with export HADOOP_HOME="<path to hadoop>"
  3. Download all Hive 0.13.1a jars (Hive jars actually used by Spark) from here and replace corresponding original 0.13.1 jars in $HIVE_HOME/lib.
  4. Download Kryo 2.21 jar (Note: 2.22 jar does not work) and Javolution 5.5.1 jar to $HIVE_HOME/lib.
  5. This step is optional. But, when generating golden answer files, if a Hive query fails and you find that Hive tries to talk to HDFS or you find weird runtime NPEs, set the following in your test suite...
val testTempDir = Utils.createTempDir()
// We have to use kryo to let Hive correctly serialize some plans.
sql("set hive.plan.serialization.format=kryo")
// Explicitly set fs to local fs.
sql(s"set fs.default.name=file://$testTempDir/")
// Ask Hive to run jobs in-process as a single map and reduce task.
sql("set mapred.job.tracker=local")

Using the console

An interactive scala console can be invoked by running build/sbt hive/console. From here you can execute queries with HiveQl and manipulate DataFrame by using DSL.

$ build/sbt hive/console

[info] Starting scala interpreter...
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.dsl._
import org.apache.spark.sql.catalyst.errors._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.execution
import org.apache.spark.sql.functions._
import org.apache.spark.sql.hive._
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.hive.test.TestHive.implicits._
import org.apache.spark.sql.types._
Type in expressions to have them evaluated.
Type :help for more information.

scala> val query = sql("SELECT * FROM (SELECT * FROM src) a")
query: org.apache.spark.sql.DataFrame = [key: int, value: string]

Query results are DataFrames and can be operated as such.

scala> query.collect()
res0: Array[org.apache.spark.sql.Row] = Array([238,val_238], [86,val_86], [311,val_311], [27,val_27]...

You can also build further queries on top of these DataFrames using the query DSL.

scala> query.where(query("key") > 30).select(avg(query("key"))).collect()
res1: Array[org.apache.spark.sql.Row] = Array([274.79025423728814])