Repository: spark
Updated Branches:
  refs/heads/branch-1.1 eaa93555a -> c5ed1deba


[SPARK-2784][SQL] Deprecate hql() method in favor of a config option, 
'spark.sql.dialect'

Many users have reported being confused by the distinction between the `sql` 
and `hql` methods.  Specifically, many users think that `sql(...)` cannot be 
used to read hive tables.  In this PR I introduce a new configuration option 
`spark.sql.dialect` that picks which dialect with be used for parsing.  For 
SQLContext this must be set to `sql`.  In `HiveContext` it defaults to `hiveql` 
but can also be set to `sql`.

The `hql` and `hiveql` methods continue to act the same but are now marked as 
deprecated.

**This is a possibly breaking change for some users unless they set the dialect 
manually, though this is unlikely.**

For example: `hiveContex.sql("SELECT 1")` will now throw a parsing exception by 
default.

Author: Michael Armbrust <mich...@databricks.com>

Closes #1746 from marmbrus/sqlLanguageConf and squashes the following commits:

ad375cc [Michael Armbrust] Merge remote-tracking branch 'apache/master' into 
sqlLanguageConf
20c43f8 [Michael Armbrust] override function instead of just setting the value
7e4ae93 [Michael Armbrust] Deprecate hql() method in favor of a config option, 
'spark.sql.dialect'

(cherry picked from commit 236dfac6769016e433b2f6517cda2d308dea74bc)
Signed-off-by: Michael Armbrust <mich...@databricks.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c5ed1deb
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c5ed1deb
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c5ed1deb

Branch: refs/heads/branch-1.1
Commit: c5ed1deba6b3f3e597554a8d0f93f402ae62fab9
Parents: eaa9355
Author: Michael Armbrust <mich...@databricks.com>
Authored: Sun Aug 3 12:28:29 2014 -0700
Committer: Michael Armbrust <mich...@databricks.com>
Committed: Sun Aug 3 12:29:05 2014 -0700

----------------------------------------------------------------------
 .../sbt_app_hive/src/main/scala/HiveApp.scala   |  8 +-
 docs/sql-programming-guide.md                   | 18 ++--
 .../spark/examples/sql/hive/HiveFromSpark.scala | 12 +--
 python/pyspark/sql.py                           | 20 +++--
 .../scala/org/apache/spark/sql/SQLConf.scala    | 17 +++-
 .../scala/org/apache/spark/sql/SQLContext.scala | 11 ++-
 .../spark/sql/api/java/JavaSQLContext.scala     | 14 ++-
 .../sql/hive/thriftserver/SparkSQLDriver.scala  |  2 +-
 .../server/SparkSQLOperationManager.scala       |  2 +-
 .../org/apache/spark/sql/hive/HiveContext.scala | 26 ++++--
 .../sql/hive/api/java/JavaHiveContext.scala     | 15 +++-
 .../spark/sql/hive/CachedTableSuite.scala       | 14 +--
 .../apache/spark/sql/hive/StatisticsSuite.scala | 10 +--
 .../sql/hive/api/java/JavaHiveQLSuite.scala     | 19 ++--
 .../sql/hive/execution/HiveComparisonTest.scala |  4 +-
 .../sql/hive/execution/HiveQuerySuite.scala     | 93 ++++++++++----------
 .../hive/execution/HiveResolutionSuite.scala    |  6 +-
 .../hive/execution/HiveTypeCoercionSuite.scala  |  2 +-
 .../spark/sql/hive/execution/HiveUdfSuite.scala | 10 +--
 .../spark/sql/hive/execution/PruningSuite.scala |  2 +-
 .../spark/sql/parquet/HiveParquetSuite.scala    | 27 +++---
 21 files changed, 199 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala
----------------------------------------------------------------------
diff --git a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala 
b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala
index a21410f..5111bc0 100644
--- a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala
+++ b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala
@@ -37,10 +37,10 @@ object SparkSqlExample {
     val hiveContext = new HiveContext(sc)
 
     import hiveContext._
-    hql("DROP TABLE IF EXISTS src")
-    hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
-    hql("LOAD DATA LOCAL INPATH 'data.txt' INTO TABLE src")
-    val results = hql("FROM src SELECT key, value WHERE key >= 0 AND KEY < 
5").collect()
+    sql("DROP TABLE IF EXISTS src")
+    sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
+    sql("LOAD DATA LOCAL INPATH 'data.txt' INTO TABLE src")
+    val results = sql("FROM src SELECT key, value WHERE key >= 0 AND KEY < 
5").collect()
     results.foreach(println)
     
     def test(f: => Boolean, failureMsg: String) = {

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/docs/sql-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 0465468..cd65439 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -495,11 +495,11 @@ directory.
 // sc is an existing SparkContext.
 val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc)
 
-hiveContext.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
-hiveContext.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' 
INTO TABLE src")
+hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
+hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' 
INTO TABLE src")
 
 // Queries are expressed in HiveQL
-hiveContext.hql("FROM src SELECT key, value").collect().foreach(println)
+hiveContext.sql("FROM src SELECT key, value").collect().foreach(println)
 {% endhighlight %}
 
 </div>
@@ -515,11 +515,11 @@ expressed in HiveQL.
 // sc is an existing JavaSparkContext.
 JavaHiveContext hiveContext = new 
org.apache.spark.sql.hive.api.java.HiveContext(sc);
 
-hiveContext.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)");
-hiveContext.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' 
INTO TABLE src");
+hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)");
+hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' 
INTO TABLE src");
 
 // Queries are expressed in HiveQL.
-Row[] results = hiveContext.hql("FROM src SELECT key, value").collect();
+Row[] results = hiveContext.sql("FROM src SELECT key, value").collect();
 
 {% endhighlight %}
 
@@ -537,11 +537,11 @@ expressed in HiveQL.
 from pyspark.sql import HiveContext
 hiveContext = HiveContext(sc)
 
-hiveContext.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
-hiveContext.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' 
INTO TABLE src")
+hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
+hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' 
INTO TABLE src")
 
 # Queries can be expressed in HiveQL.
-results = hiveContext.hql("FROM src SELECT key, value").collect()
+results = hiveContext.sql("FROM src SELECT key, value").collect()
 
 {% endhighlight %}
 

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
----------------------------------------------------------------------
diff --git 
a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
 
b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
index 12530c8..3423fac 100644
--- 
a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
+++ 
b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
@@ -34,20 +34,20 @@ object HiveFromSpark {
     val hiveContext = new HiveContext(sc)
     import hiveContext._
 
-    hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
-    hql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src")
+    sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
+    sql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src")
 
     // Queries are expressed in HiveQL
     println("Result of 'SELECT *': ")
-    hql("SELECT * FROM src").collect.foreach(println)
+    sql("SELECT * FROM src").collect.foreach(println)
 
     // Aggregation queries are also supported.
-    val count = hql("SELECT COUNT(*) FROM src").collect().head.getLong(0)
+    val count = sql("SELECT COUNT(*) FROM src").collect().head.getLong(0)
     println(s"COUNT(*): $count")
 
     // The results of SQL queries are themselves RDDs and support all normal 
RDD functions.  The
     // items in the RDD are of type Row, which allows you to access each 
column by ordinal.
-    val rddFromSql = hql("SELECT key, value FROM src WHERE key < 10 ORDER BY 
key")
+    val rddFromSql = sql("SELECT key, value FROM src WHERE key < 10 ORDER BY 
key")
 
     println("Result of RDD.map:")
     val rddAsStrings = rddFromSql.map {
@@ -60,6 +60,6 @@ object HiveFromSpark {
 
     // Queries can then join RDD data with data stored in Hive.
     println("Result of SELECT *:")
-    hql("SELECT * FROM records r JOIN src s ON r.key = 
s.key").collect().foreach(println)
+    sql("SELECT * FROM records r JOIN src s ON r.key = 
s.key").collect().foreach(println)
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/python/pyspark/sql.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py
index 42b738e..1a829c6 100644
--- a/python/pyspark/sql.py
+++ b/python/pyspark/sql.py
@@ -1291,16 +1291,20 @@ class HiveContext(SQLContext):
 
     def hiveql(self, hqlQuery):
         """
-        Runs a query expressed in HiveQL, returning the result as
-        a L{SchemaRDD}.
+        DEPRECATED: Use sql()
         """
+        warnings.warn("hiveql() is deprecated as the sql function now parses 
using HiveQL by" +
+                      "default. The SQL dialect for parsing can be set using 
'spark.sql.dialect'",
+                      DeprecationWarning)
         return SchemaRDD(self._ssql_ctx.hiveql(hqlQuery), self)
 
     def hql(self, hqlQuery):
         """
-        Runs a query expressed in HiveQL, returning the result as
-        a L{SchemaRDD}.
+        DEPRECATED: Use sql()
         """
+        warnings.warn("hql() is deprecated as the sql function now parses 
using HiveQL by" +
+                      "default. The SQL dialect for parsing can be set using 
'spark.sql.dialect'",
+                      DeprecationWarning)
         return self.hiveql(hqlQuery)
 
 
@@ -1313,16 +1317,16 @@ class LocalHiveContext(HiveContext):
     >>> import os
     >>> hiveCtx = LocalHiveContext(sc)
     >>> try:
-    ...     supress = hiveCtx.hql("DROP TABLE src")
+    ...     supress = hiveCtx.sql("DROP TABLE src")
     ... except Exception:
     ...     pass
     >>> kv1 = os.path.join(os.environ["SPARK_HOME"],
     ...        'examples/src/main/resources/kv1.txt')
-    >>> supress = hiveCtx.hql(
+    >>> supress = hiveCtx.sql(
     ...     "CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
-    >>> supress = hiveCtx.hql("LOAD DATA LOCAL INPATH '%s' INTO TABLE src"
+    >>> supress = hiveCtx.sql("LOAD DATA LOCAL INPATH '%s' INTO TABLE src"
     ...        % kv1)
-    >>> results = hiveCtx.hql("FROM src SELECT value"
+    >>> results = hiveCtx.sql("FROM src SELECT value"
     ...      ).map(lambda r: int(r.value.split('_')[1]))
     >>> num = results.count()
     >>> reduce_sum = results.reduce(lambda x, y: x + y)

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
index 2d40707..40bfd55 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
@@ -29,6 +29,7 @@ object SQLConf {
   val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions"
   val JOIN_BROADCAST_TABLES = "spark.sql.join.broadcastTables"
   val CODEGEN_ENABLED = "spark.sql.codegen"
+  val DIALECT = "spark.sql.dialect"
 
   object Deprecated {
     val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
@@ -39,7 +40,7 @@ object SQLConf {
  * A trait that enables the setting and getting of mutable config 
parameters/hints.
  *
  * In the presence of a SQLContext, these can be set and queried by passing 
SET commands
- * into Spark SQL's query functions (sql(), hql(), etc.). Otherwise, users of 
this trait can
+ * into Spark SQL's query functions (i.e. sql()). Otherwise, users of this 
trait can
  * modify the hints by programmatically calling the setters and getters of 
this trait.
  *
  * SQLConf is thread-safe (internally synchronized, so safe to be used in 
multiple threads).
@@ -53,6 +54,20 @@ trait SQLConf {
   /** ************************ Spark SQL Params/Hints ******************* */
   // TODO: refactor so that these hints accessors don't pollute the name space 
of SQLContext?
 
+  /**
+   * The SQL dialect that is used when parsing queries.  This defaults to 
'sql' which uses
+   * a simple SQL parser provided by Spark SQL.  This is currently the only 
option for users of
+   * SQLContext.
+   *
+   * When using a HiveContext, this value defaults to 'hiveql', which uses the 
Hive 0.12.0 HiveQL
+   * parser.  Users can change this to 'sql' if they want to run queries that 
aren't supported by
+   * HiveQL (e.g., SELECT 1).
+   *
+   * Note that the choice of dialect does not affect things like what tables 
are available or
+   * how query execution is performed.
+   */
+  private[spark] def dialect: String = get(DIALECT, "sql")
+
   /** When true tables cached using the in-memory columnar caching will be 
compressed. */
   private[spark] def useCompression: Boolean = get(COMPRESS_CACHED, 
"false").toBoolean
 

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 567f4dc..ecd5fba 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -248,11 +248,18 @@ class SQLContext(@transient val sparkContext: 
SparkContext)
   }
 
   /**
-   * Executes a SQL query using Spark, returning the result as a SchemaRDD.
+   * Executes a SQL query using Spark, returning the result as a SchemaRDD.  
The dialect that is
+   * used for SQL parsing can be configured with 'spark.sql.dialect'.
    *
    * @group userf
    */
-  def sql(sqlText: String): SchemaRDD = new SchemaRDD(this, parseSql(sqlText))
+  def sql(sqlText: String): SchemaRDD = {
+    if (dialect == "sql") {
+      new SchemaRDD(this, parseSql(sqlText))
+    } else {
+      sys.error(s"Unsupported SQL dialect: $dialect")
+    }
+  }
 
   /** Returns the specified table as a SchemaRDD */
   def table(tableName: String): SchemaRDD =

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala
index dbaa16e..150ff8a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala
@@ -39,10 +39,18 @@ class JavaSQLContext(val sqlContext: SQLContext) extends 
UDFRegistration {
   def this(sparkContext: JavaSparkContext) = this(new 
SQLContext(sparkContext.sc))
 
   /**
-   * Executes a query expressed in SQL, returning the result as a JavaSchemaRDD
+   * Executes a SQL query using Spark, returning the result as a SchemaRDD.  
The dialect that is
+   * used for SQL parsing can be configured with 'spark.sql.dialect'.
+   *
+   * @group userf
    */
-  def sql(sqlQuery: String): JavaSchemaRDD =
-    new JavaSchemaRDD(sqlContext, sqlContext.parseSql(sqlQuery))
+  def sql(sqlText: String): JavaSchemaRDD = {
+    if (sqlContext.dialect == "sql") {
+      new JavaSchemaRDD(sqlContext, sqlContext.parseSql(sqlText))
+    } else {
+      sys.error(s"Unsupported SQL dialect: $sqlContext.dialect")
+    }
+  }
 
   /**
    * :: Experimental ::

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
index d362d59..7463df1 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
@@ -55,7 +55,7 @@ private[hive] class SparkSQLDriver(val context: HiveContext = 
SparkSQLEnv.hiveCo
   override def run(command: String): CommandProcessorResponse = {
     // TODO unify the error code
     try {
-      val execution = context.executePlan(context.hql(command).logicalPlan)
+      val execution = context.executePlan(context.sql(command).logicalPlan)
       hiveResponse = execution.stringResult()
       tableSchema = getResultSetSchema(execution)
       new CommandProcessorResponse(0)

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
index d4dadfd..dee0921 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
@@ -128,7 +128,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) 
extends OperationManage
         logInfo(s"Running query '$statement'")
         setState(OperationState.RUNNING)
         try {
-          result = hiveContext.hql(statement)
+          result = hiveContext.sql(statement)
           logDebug(result.queryExecution.toString())
           val groupId = round(random * 1000000).toString
           hiveContext.sparkContext.setJobGroup(groupId, statement)

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index 3c70b3f..7db0159 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -71,15 +71,29 @@ class LocalHiveContext(sc: SparkContext) extends 
HiveContext(sc) {
 class HiveContext(sc: SparkContext) extends SQLContext(sc) {
   self =>
 
+  // Change the default SQL dialect to HiveQL
+  override private[spark] def dialect: String = get(SQLConf.DIALECT, "hiveql")
+
   override protected[sql] def executePlan(plan: LogicalPlan): 
this.QueryExecution =
     new this.QueryExecution { val logical = plan }
 
-  /**
-   * Executes a query expressed in HiveQL using Spark, returning the result as 
a SchemaRDD.
-   */
+  override def sql(sqlText: String): SchemaRDD = {
+    // TODO: Create a framework for registering parsers instead of just 
hardcoding if statements.
+    if (dialect == "sql") {
+      super.sql(sqlText)
+    } else if (dialect == "hiveql") {
+      new SchemaRDD(this, HiveQl.parseSql(sqlText))
+    }  else {
+      sys.error(s"Unsupported SQL dialect: $dialect.  Try 'sql' or 'hiveql'")
+    }
+  }
+
+  @deprecated("hiveql() is deprecated as the sql function now parses using 
HiveQL by default. " +
+             s"The SQL dialect for parsing can be set using 
${SQLConf.DIALECT}", "1.1")
   def hiveql(hqlQuery: String): SchemaRDD = new SchemaRDD(this, 
HiveQl.parseSql(hqlQuery))
 
-  /** An alias for `hiveql`. */
+  @deprecated("hql() is deprecated as the sql function now parses using HiveQL 
by default. " +
+             s"The SQL dialect for parsing can be set using 
${SQLConf.DIALECT}", "1.1")
   def hql(hqlQuery: String): SchemaRDD = hiveql(hqlQuery)
 
   /**
@@ -95,7 +109,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
 
   // Circular buffer to hold what hive prints to STDOUT and ERR.  Only printed 
when failures occur.
   @transient
-  protected val outputBuffer =  new java.io.OutputStream {
+  protected lazy val outputBuffer =  new java.io.OutputStream {
     var pos: Int = 0
     var buffer = new Array[Int](10240)
     def write(i: Int): Unit = {
@@ -125,7 +139,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
   /**
    * SQLConf and HiveConf contracts: when the hive session is first 
initialized, params in
    * HiveConf will get picked up by the SQLConf.  Additionally, any properties 
set by
-   * set() or a SET command inside hql() or sql() will be set in the SQLConf 
*as well as*
+   * set() or a SET command inside sql() will be set in the SQLConf *as well 
as*
    * in the HiveConf.
    */
   @transient protected[hive] lazy val hiveconf = new 
HiveConf(classOf[SessionState])

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
index c9ee162..a201d23 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.api.java
 
 import org.apache.spark.api.java.JavaSparkContext
 import org.apache.spark.sql.api.java.{JavaSQLContext, JavaSchemaRDD}
+import org.apache.spark.sql.SQLConf
 import org.apache.spark.sql.hive.{HiveContext, HiveQl}
 
 /**
@@ -28,9 +29,21 @@ class JavaHiveContext(sparkContext: JavaSparkContext) 
extends JavaSQLContext(spa
 
   override val sqlContext = new HiveContext(sparkContext)
 
+  override def sql(sqlText: String): JavaSchemaRDD = {
+    // TODO: Create a framework for registering parsers instead of just 
hardcoding if statements.
+    if (sqlContext.dialect == "sql") {
+      super.sql(sqlText)
+    } else if (sqlContext.dialect == "hiveql") {
+      new JavaSchemaRDD(sqlContext, HiveQl.parseSql(sqlText))
+    }  else {
+      sys.error(s"Unsupported SQL dialect: ${sqlContext.dialect}.  Try 'sql' 
or 'hiveql'")
+    }
+  }
+
   /**
-    * Executes a query expressed in HiveQL, returning the result as a 
JavaSchemaRDD.
+    * DEPRECATED: Use sql(...) Instead
     */
+  @Deprecated
   def hql(hqlQuery: String): JavaSchemaRDD =
     new JavaSchemaRDD(sqlContext, HiveQl.parseSql(hqlQuery))
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
index 08da640..188579e 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
@@ -35,17 +35,17 @@ class CachedTableSuite extends HiveComparisonTest {
     "SELECT * FROM src LIMIT 1", reset = false)
 
   test("Drop cached table") {
-    hql("CREATE TABLE test(a INT)")
+    sql("CREATE TABLE test(a INT)")
     cacheTable("test")
-    hql("SELECT * FROM test").collect()
-    hql("DROP TABLE test")
+    sql("SELECT * FROM test").collect()
+    sql("DROP TABLE test")
     intercept[org.apache.hadoop.hive.ql.metadata.InvalidTableException] {
-      hql("SELECT * FROM test").collect()
+      sql("SELECT * FROM test").collect()
     }
   }
 
   test("DROP nonexistant table") {
-    hql("DROP TABLE IF EXISTS nonexistantTable")
+    sql("DROP TABLE IF EXISTS nonexistantTable")
   }
 
   test("check that table is cached and uncache") {
@@ -74,14 +74,14 @@ class CachedTableSuite extends HiveComparisonTest {
   }
 
   test("'CACHE TABLE' and 'UNCACHE TABLE' HiveQL statement") {
-    TestHive.hql("CACHE TABLE src")
+    TestHive.sql("CACHE TABLE src")
     TestHive.table("src").queryExecution.executedPlan match {
       case _: InMemoryColumnarTableScan => // Found evidence of caching
       case _ => fail(s"Table 'src' should be cached")
     }
     assert(TestHive.isCached("src"), "Table 'src' should be cached")
 
-    TestHive.hql("UNCACHE TABLE src")
+    TestHive.sql("UNCACHE TABLE src")
     TestHive.table("src").queryExecution.executedPlan match {
       case _: InMemoryColumnarTableScan => fail(s"Table 'src' should not be 
cached")
       case _ => // Found evidence of uncaching

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
index a61fd9d..d8c77d6 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
@@ -27,7 +27,7 @@ import org.apache.spark.sql.hive.test.TestHive._
 class StatisticsSuite extends QueryTest {
 
   test("estimates the size of a test MetastoreRelation") {
-    val rdd = hql("""SELECT * FROM src""")
+    val rdd = sql("""SELECT * FROM src""")
     val sizes = rdd.queryExecution.analyzed.collect { case mr: 
MetastoreRelation =>
       mr.statistics.sizeInBytes
     }
@@ -45,7 +45,7 @@ class StatisticsSuite extends QueryTest {
         ct: ClassTag[_]) = {
       before()
 
-      var rdd = hql(query)
+      var rdd = sql(query)
 
       // Assert src has a size smaller than the threshold.
       val sizes = rdd.queryExecution.analyzed.collect {
@@ -65,8 +65,8 @@ class StatisticsSuite extends QueryTest {
       TestHive.settings.synchronized {
         val tmp = autoBroadcastJoinThreshold
 
-        hql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1""")
-        rdd = hql(query)
+        sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1""")
+        rdd = sql(query)
         bhj = rdd.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin 
=> j }
         assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is 
switched off")
 
@@ -74,7 +74,7 @@ class StatisticsSuite extends QueryTest {
         assert(shj.size === 1,
           "ShuffledHashJoin should be planned when BroadcastHashJoin is turned 
off")
 
-        hql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp""")
+        sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp""")
       }
 
       after()

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala
index 578f275..9644b70 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala
@@ -40,7 +40,7 @@ class JavaHiveQLSuite extends FunSuite {
 
   ignore("SELECT * FROM src") {
     assert(
-      javaHiveCtx.hql("SELECT * FROM src").collect().map(_.getInt(0)) ===
+      javaHiveCtx.sql("SELECT * FROM src").collect().map(_.getInt(0)) ===
         TestHive.sql("SELECT * FROM src").collect().map(_.getInt(0)).toSeq)
   }
 
@@ -56,33 +56,34 @@ class JavaHiveQLSuite extends FunSuite {
     val tableName = "test_native_commands"
 
     assertResult(0) {
-      javaHiveCtx.hql(s"DROP TABLE IF EXISTS $tableName").count()
+      javaHiveCtx.sql(s"DROP TABLE IF EXISTS $tableName").count()
     }
 
     assertResult(0) {
-      javaHiveCtx.hql(s"CREATE TABLE $tableName(key INT, value 
STRING)").count()
+      javaHiveCtx.sql(s"CREATE TABLE $tableName(key INT, value 
STRING)").count()
     }
 
-    javaHiveCtx.hql("SHOW TABLES").registerTempTable("show_tables")
+    javaHiveCtx.sql("SHOW TABLES").registerTempTable("show_tables")
 
     assert(
       javaHiveCtx
-        .hql("SELECT result FROM show_tables")
+        .sql("SELECT result FROM show_tables")
         .collect()
         .map(_.getString(0))
         .contains(tableName))
 
     assertResult(Array(Array("key", "int", "None"), Array("value", "string", 
"None"))) {
-      javaHiveCtx.hql(s"DESCRIBE 
$tableName").registerTempTable("describe_table")
+      javaHiveCtx.sql(s"DESCRIBE 
$tableName").registerTempTable("describe_table")
+
 
       javaHiveCtx
-        .hql("SELECT result FROM describe_table")
+        .sql("SELECT result FROM describe_table")
         .collect()
         .map(_.getString(0).split("\t").map(_.trim))
         .toArray
     }
 
-    assert(isExplanation(javaHiveCtx.hql(
+    assert(isExplanation(javaHiveCtx.sql(
       s"EXPLAIN SELECT key, COUNT(*) FROM $tableName GROUP BY key")))
 
     TestHive.reset()
@@ -90,7 +91,7 @@ class JavaHiveQLSuite extends FunSuite {
 
   ignore("Exactly once semantics for DDL and command statements") {
     val tableName = "test_exactly_once"
-    val q0 = javaHiveCtx.hql(s"CREATE TABLE $tableName(key INT, value STRING)")
+    val q0 = javaHiveCtx.sql(s"CREATE TABLE $tableName(key INT, value STRING)")
 
     // If the table was not created, the following assertion would fail
     assert(Try(TestHive.table(tableName)).isSuccess)

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
index 83cfbc6..0ebaf6f 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
@@ -241,13 +241,13 @@ abstract class HiveComparisonTest
         val quotes = "\"\"\""
         queryList.zipWithIndex.map {
           case (query, i) =>
-            s"""val q$i = hql($quotes$query$quotes); q$i.collect()"""
+            s"""val q$i = sql($quotes$query$quotes); q$i.collect()"""
         }.mkString("\n== Console version of this test ==\n", "\n", "\n")
       }
 
       try {
         // MINOR HACK: You must run a query before calling reset the first 
time.
-        TestHive.hql("SHOW TABLES")
+        TestHive.sql("SHOW TABLES")
         if (reset) { TestHive.reset() }
 
         val hiveCacheFiles = queryList.zipWithIndex.map {

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
index 4ed4155..aa810a2 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
@@ -57,8 +57,8 @@ class HiveQuerySuite extends HiveComparisonTest {
     """.stripMargin)
 
   test("CREATE TABLE AS runs once") {
-    hql("CREATE TABLE foo AS SELECT 1 FROM src LIMIT 1").collect()
-    assert(hql("SELECT COUNT(*) FROM foo").collect().head.getLong(0) === 1,
+    sql("CREATE TABLE foo AS SELECT 1 FROM src LIMIT 1").collect()
+    assert(sql("SELECT COUNT(*) FROM foo").collect().head.getLong(0) === 1,
       "Incorrect number of rows in created table")
   }
 
@@ -72,12 +72,14 @@ class HiveQuerySuite extends HiveComparisonTest {
     "SELECT 2 / 1, 1 / 2, 1 / 3, 1 / COUNT(*) FROM src LIMIT 1")
 
   test("Query expressed in SQL") {
+    set("spark.sql.dialect", "sql")
     assert(sql("SELECT 1").collect() === Array(Seq(1)))
+    set("spark.sql.dialect", "hiveql")
+
   }
 
   test("Query expressed in HiveQL") {
-    hql("FROM src SELECT key").collect()
-    hiveql("FROM src SELECT key").collect()
+    sql("FROM src SELECT key").collect()
   }
 
   createQueryTest("Constant Folding Optimization for AVG_SUM_COUNT",
@@ -193,12 +195,12 @@ class HiveQuerySuite extends HiveComparisonTest {
     "SELECT * FROM src LATERAL VIEW explode(map(key+3,key+4)) D as k, v")
 
   test("sampling") {
-    hql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s")
+    sql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s")
   }
 
   test("SchemaRDD toString") {
-    hql("SHOW TABLES").toString
-    hql("SELECT * FROM src").toString
+    sql("SHOW TABLES").toString
+    sql("SELECT * FROM src").toString
   }
 
   createQueryTest("case statements with key #1",
@@ -226,8 +228,8 @@ class HiveQuerySuite extends HiveComparisonTest {
     "SELECT (CASE WHEN key > 2 THEN 3 WHEN 2 > key THEN 2 ELSE 0 END) FROM src 
WHERE key < 15")
 
   test("implement identity function using case statement") {
-    val actual = hql("SELECT (CASE key WHEN key THEN key END) FROM 
src").collect().toSet
-    val expected = hql("SELECT key FROM src").collect().toSet
+    val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM 
src").collect().toSet
+    val expected = sql("SELECT key FROM src").collect().toSet
     assert(actual === expected)
   }
 
@@ -235,7 +237,7 @@ class HiveQuerySuite extends HiveComparisonTest {
   // See https://github.com/apache/spark/pull/1055#issuecomment-45820167 for a 
discussion.
   ignore("non-boolean conditions in a CaseWhen are illegal") {
     intercept[Exception] {
-      hql("SELECT (CASE WHEN key > 2 THEN 3 WHEN 1 THEN 2 ELSE 0 END) FROM 
src").collect()
+      sql("SELECT (CASE WHEN key > 2 THEN 3 WHEN 1 THEN 2 ELSE 0 END) FROM 
src").collect()
     }
   }
 
@@ -250,7 +252,7 @@ class HiveQuerySuite extends HiveComparisonTest {
     testData.registerTempTable("REGisteredTABle")
 
     assertResult(Array(Array(2, "str2"))) {
-      hql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " 
+
+      sql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " 
+
         "WHERE TableAliaS.a > 1").collect()
     }
   }
@@ -261,9 +263,9 @@ class HiveQuerySuite extends HiveComparisonTest {
   }
 
   test("SPARK-1704: Explain commands as a SchemaRDD") {
-    hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
+    sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
 
-    val rdd = hql("explain select key, count(value) from src group by key")
+    val rdd = sql("explain select key, count(value) from src group by key")
     assert(isExplanation(rdd))
 
     TestHive.reset()
@@ -274,7 +276,7 @@ class HiveQuerySuite extends HiveComparisonTest {
       .zipWithIndex.map {case Pair(Pair(value, attr), key) => HavingRow(key, 
value, attr)}
     TestHive.sparkContext.parallelize(fixture).registerTempTable("having_test")
     val results =
-      hql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value 
HAVING attr > 3")
+      sql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value 
HAVING attr > 3")
       .collect()
       .map(x => Pair(x.getString(0), x.getInt(1)))
 
@@ -283,39 +285,39 @@ class HiveQuerySuite extends HiveComparisonTest {
   }
 
   test("SPARK-2180: HAVING with non-boolean clause raises no exceptions") {
-    hql("select key, count(*) c from src group by key having c").collect()
+    sql("select key, count(*) c from src group by key having c").collect()
   }
 
   test("SPARK-2225: turn HAVING without GROUP BY into a simple filter") {
-    assert(hql("select key from src having key > 490").collect().size < 100)
+    assert(sql("select key from src having key > 490").collect().size < 100)
   }
 
   test("Query Hive native command execution result") {
     val tableName = "test_native_commands"
 
     assertResult(0) {
-      hql(s"DROP TABLE IF EXISTS $tableName").count()
+      sql(s"DROP TABLE IF EXISTS $tableName").count()
     }
 
     assertResult(0) {
-      hql(s"CREATE TABLE $tableName(key INT, value STRING)").count()
+      sql(s"CREATE TABLE $tableName(key INT, value STRING)").count()
     }
 
     assert(
-      hql("SHOW TABLES")
+      sql("SHOW TABLES")
         .select('result)
         .collect()
         .map(_.getString(0))
         .contains(tableName))
 
-    assert(isExplanation(hql(s"EXPLAIN SELECT key, COUNT(*) FROM $tableName 
GROUP BY key")))
+    assert(isExplanation(sql(s"EXPLAIN SELECT key, COUNT(*) FROM $tableName 
GROUP BY key")))
 
     TestHive.reset()
   }
 
   test("Exactly once semantics for DDL and command statements") {
     val tableName = "test_exactly_once"
-    val q0 = hql(s"CREATE TABLE $tableName(key INT, value STRING)")
+    val q0 = sql(s"CREATE TABLE $tableName(key INT, value STRING)")
 
     // If the table was not created, the following assertion would fail
     assert(Try(table(tableName)).isSuccess)
@@ -325,9 +327,9 @@ class HiveQuerySuite extends HiveComparisonTest {
   }
 
   test("DESCRIBE commands") {
-    hql(s"CREATE TABLE test_describe_commands1 (key INT, value STRING) 
PARTITIONED BY (dt STRING)")
+    sql(s"CREATE TABLE test_describe_commands1 (key INT, value STRING) 
PARTITIONED BY (dt STRING)")
 
-    hql(
+    sql(
       """FROM src INSERT OVERWRITE TABLE test_describe_commands1 PARTITION 
(dt='2008-06-08')
         |SELECT key, value
       """.stripMargin)
@@ -342,7 +344,7 @@ class HiveQuerySuite extends HiveComparisonTest {
         Array("# col_name", "data_type", "comment"),
         Array("dt", "string", null))
     ) {
-      hql("DESCRIBE test_describe_commands1")
+      sql("DESCRIBE test_describe_commands1")
         .select('col_name, 'data_type, 'comment)
         .collect()
     }
@@ -357,14 +359,14 @@ class HiveQuerySuite extends HiveComparisonTest {
         Array("# col_name", "data_type", "comment"),
         Array("dt", "string", null))
     ) {
-      hql("DESCRIBE default.test_describe_commands1")
+      sql("DESCRIBE default.test_describe_commands1")
         .select('col_name, 'data_type, 'comment)
         .collect()
     }
 
     // Describe a column is a native command
     assertResult(Array(Array("value", "string", "from deserializer"))) {
-      hql("DESCRIBE test_describe_commands1 value")
+      sql("DESCRIBE test_describe_commands1 value")
         .select('result)
         .collect()
         .map(_.getString(0).split("\t").map(_.trim))
@@ -372,7 +374,7 @@ class HiveQuerySuite extends HiveComparisonTest {
 
     // Describe a column is a native command
     assertResult(Array(Array("value", "string", "from deserializer"))) {
-      hql("DESCRIBE default.test_describe_commands1 value")
+      sql("DESCRIBE default.test_describe_commands1 value")
         .select('result)
         .collect()
         .map(_.getString(0).split("\t").map(_.trim))
@@ -390,7 +392,7 @@ class HiveQuerySuite extends HiveComparisonTest {
         Array("", "", ""),
         Array("dt", "string", "None"))
     ) {
-      hql("DESCRIBE test_describe_commands1 PARTITION (dt='2008-06-08')")
+      sql("DESCRIBE test_describe_commands1 PARTITION (dt='2008-06-08')")
         .select('result)
         .collect()
         .map(_.getString(0).split("\t").map(_.trim))
@@ -409,16 +411,16 @@ class HiveQuerySuite extends HiveComparisonTest {
         Array("a", "IntegerType", null),
         Array("b", "StringType", null))
     ) {
-      hql("DESCRIBE test_describe_commands2")
+      sql("DESCRIBE test_describe_commands2")
         .select('col_name, 'data_type, 'comment)
         .collect()
     }
   }
 
   test("SPARK-2263: Insert Map<K, V> values") {
-    hql("CREATE TABLE m(value MAP<INT, STRING>)")
-    hql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10")
-    hql("SELECT * FROM m").collect().zip(hql("SELECT * FROM src LIMIT 
10").collect()).map {
+    sql("CREATE TABLE m(value MAP<INT, STRING>)")
+    sql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10")
+    sql("SELECT * FROM m").collect().zip(sql("SELECT * FROM src LIMIT 
10").collect()).map {
       case (Row(map: Map[_, _]), Row(key: Int, value: String)) =>
         assert(map.size === 1)
         assert(map.head === (key, value))
@@ -430,18 +432,18 @@ class HiveQuerySuite extends HiveComparisonTest {
     val testKey = "spark.sql.key.usedfortestonly"
     val testVal = "val0,val_1,val2.3,my_table"
 
-    hql(s"set $testKey=$testVal")
+    sql(s"set $testKey=$testVal")
     assert(get(testKey, testVal + "_") == testVal)
 
-    hql("set some.property=20")
+    sql("set some.property=20")
     assert(get("some.property", "0") == "20")
-    hql("set some.property = 40")
+    sql("set some.property = 40")
     assert(get("some.property", "0") == "40")
 
-    hql(s"set $testKey=$testVal")
+    sql(s"set $testKey=$testVal")
     assert(get(testKey, "0") == testVal)
 
-    hql(s"set $testKey=")
+    sql(s"set $testKey=")
     assert(get(testKey, "0") == "")
   }
 
@@ -454,33 +456,34 @@ class HiveQuerySuite extends HiveComparisonTest {
     clear()
 
     // "set" itself returns all config variables currently specified in 
SQLConf.
-    assert(hql("SET").collect().size == 0)
+    // TODO: Should we be listing the default here always? probably...
+    assert(sql("SET").collect().size == 0)
 
     assertResult(Array(s"$testKey=$testVal")) {
-      hql(s"SET $testKey=$testVal").collect().map(_.getString(0))
+      sql(s"SET $testKey=$testVal").collect().map(_.getString(0))
     }
 
     assert(hiveconf.get(testKey, "") == testVal)
     assertResult(Array(s"$testKey=$testVal")) {
-      hql(s"SET $testKey=$testVal").collect().map(_.getString(0))
+      sql(s"SET $testKey=$testVal").collect().map(_.getString(0))
     }
 
-    hql(s"SET ${testKey + testKey}=${testVal + testVal}")
+    sql(s"SET ${testKey + testKey}=${testVal + testVal}")
     assert(hiveconf.get(testKey + testKey, "") == testVal + testVal)
     assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal 
+ testVal}")) {
-      hql(s"SET").collect().map(_.getString(0))
+      sql(s"SET").collect().map(_.getString(0))
     }
 
     // "set key"
     assertResult(Array(s"$testKey=$testVal")) {
-      hql(s"SET $testKey").collect().map(_.getString(0))
+      sql(s"SET $testKey").collect().map(_.getString(0))
     }
 
     assertResult(Array(s"$nonexistentKey=<undefined>")) {
-      hql(s"SET $nonexistentKey").collect().map(_.getString(0))
+      sql(s"SET $nonexistentKey").collect().map(_.getString(0))
     }
 
-    // Assert that sql() should have the same effects as hql() by repeating 
the above using sql().
+    // Assert that sql() should have the same effects as sql() by repeating 
the above using sql().
     clear()
     assert(sql("SET").collect().size == 0)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
index 2455c18..6b3ffd1 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
@@ -56,13 +56,13 @@ class HiveResolutionSuite extends HiveComparisonTest {
     TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), 
Seq(Nested(1,2))) :: Nil)
       .registerTempTable("caseSensitivityTest")
 
-    hql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest")
+    sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest")
   }
 
   test("nested repeated resolution") {
     TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), 
Seq(Nested(1,2))) :: Nil)
-     .registerTempTable("nestedRepeatedTest")
-    assert(hql("SELECT nestedArray[0].a FROM 
nestedRepeatedTest").collect().head(0) === 1)
+      .registerTempTable("nestedRepeatedTest")
+    assert(sql("SELECT nestedArray[0].a FROM 
nestedRepeatedTest").collect().head(0) === 1)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
index 7436de2..c3c18cf 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
@@ -35,7 +35,7 @@ class HiveTypeCoercionSuite extends HiveComparisonTest {
 
   test("[SPARK-2210] boolean cast on boolean value should be removed") {
     val q = "select cast(cast(key=0 as boolean) as boolean) from src"
-    val project = TestHive.hql(q).queryExecution.executedPlan.collect { case 
e: Project => e }.head
+    val project = TestHive.sql(q).queryExecution.executedPlan.collect { case 
e: Project => e }.head
 
     // No cast expression introduced
     project.transformAllExpressions { case c: Cast =>

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
index f944d01..b6b8592 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
@@ -37,7 +37,7 @@ import 
org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject
  */
 class HiveUdfSuite extends HiveComparisonTest {
 
-  TestHive.hql(
+  TestHive.sql(
     """
       |CREATE EXTERNAL TABLE hiveUdfTestTable (
       |   pair STRUCT<id: INT, value: INT>
@@ -48,16 +48,16 @@ class HiveUdfSuite extends HiveComparisonTest {
     """.stripMargin.format(classOf[PairSerDe].getName)
   )
 
-  TestHive.hql(
+  TestHive.sql(
     "ALTER TABLE hiveUdfTestTable ADD IF NOT EXISTS 
PARTITION(partition='testUdf') LOCATION '%s'"
       
.format(this.getClass.getClassLoader.getResource("data/files/testUdf").getFile)
   )
 
-  TestHive.hql("CREATE TEMPORARY FUNCTION testUdf AS 
'%s'".format(classOf[PairUdf].getName))
+  TestHive.sql("CREATE TEMPORARY FUNCTION testUdf AS 
'%s'".format(classOf[PairUdf].getName))
 
-  TestHive.hql("SELECT testUdf(pair) FROM hiveUdfTestTable")
+  TestHive.sql("SELECT testUdf(pair) FROM hiveUdfTestTable")
 
-  TestHive.hql("DROP TEMPORARY FUNCTION IF EXISTS testUdf")
+  TestHive.sql("DROP TEMPORARY FUNCTION IF EXISTS testUdf")
 }
 
 class TestPair(x: Int, y: Int) extends Writable with Serializable {

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
index 34d8a06..1a6dbc0 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
@@ -27,7 +27,7 @@ import scala.collection.JavaConversions._
  */
 class PruningSuite extends HiveComparisonTest {
   // MINOR HACK: You must run a query before calling reset the first time.
-  TestHive.hql("SHOW TABLES")
+  TestHive.sql("SHOW TABLES")
 
   // Column/partition pruning is not implemented for 
`InMemoryColumnarTableScan` yet, need to reset
   // the environment to ensure all referenced tables in this suites are not 
cached in-memory.

http://git-wip-us.apache.org/repos/asf/spark/blob/c5ed1deb/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
index 6545e8d..6f57fe8 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
@@ -68,39 +68,40 @@ class HiveParquetSuite extends FunSuite with 
BeforeAndAfterAll with BeforeAndAft
       .saveAsParquetFile(tempFile.getCanonicalPath)
 
     parquetFile(tempFile.getCanonicalPath).registerTempTable("cases")
-    hql("SELECT upper FROM cases").collect().map(_.getString(0)) === (1 to 
10).map(_.toString)
-    hql("SELECT LOWER FROM cases").collect().map(_.getString(0)) === (1 to 
10).map(_.toString)
+    sql("SELECT upper FROM cases").collect().map(_.getString(0)) === (1 to 
10).map(_.toString)
+    sql("SELECT LOWER FROM cases").collect().map(_.getString(0)) === (1 to 
10).map(_.toString)
   }
 
   test("SELECT on Parquet table") {
-    val rdd = hql("SELECT * FROM testsource").collect()
+    val rdd = sql("SELECT * FROM testsource").collect()
     assert(rdd != null)
     assert(rdd.forall(_.size == 6))
   }
 
   test("Simple column projection + filter on Parquet table") {
-    val rdd = hql("SELECT myboolean, mylong FROM testsource WHERE 
myboolean=true").collect()
+    val rdd = sql("SELECT myboolean, mylong FROM testsource WHERE 
myboolean=true").collect()
     assert(rdd.size === 5, "Filter returned incorrect number of rows")
     assert(rdd.forall(_.getBoolean(0)), "Filter returned incorrect Boolean 
field value")
   }
 
   test("Converting Hive to Parquet Table via saveAsParquetFile") {
-    hql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath)
+    sql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath)
     parquetFile(dirname.getAbsolutePath).registerTempTable("ptable")
-    val rddOne = hql("SELECT * FROM src").collect().sortBy(_.getInt(0))
-    val rddTwo = hql("SELECT * from ptable").collect().sortBy(_.getInt(0))
+    val rddOne = sql("SELECT * FROM src").collect().sortBy(_.getInt(0))
+    val rddTwo = sql("SELECT * from ptable").collect().sortBy(_.getInt(0))
+
     compareRDDs(rddOne, rddTwo, "src (Hive)", Seq("key:Int", "value:String"))
   }
 
   test("INSERT OVERWRITE TABLE Parquet table") {
-    hql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath)
+    sql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath)
     parquetFile(dirname.getAbsolutePath).registerTempTable("ptable")
     // let's do three overwrites for good measure
-    hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
-    hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
-    hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
-    val rddCopy = hql("SELECT * FROM ptable").collect()
-    val rddOrig = hql("SELECT * FROM testsource").collect()
+    sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
+    sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
+    sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
+    val rddCopy = sql("SELECT * FROM ptable").collect()
+    val rddOrig = sql("SELECT * FROM testsource").collect()
     assert(rddCopy.size === rddOrig.size, "INSERT OVERWRITE changed size of 
table??")
     compareRDDs(rddOrig, rddCopy, "testsource", 
ParquetTestData.testSchemaFieldNames)
   }


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

Reply via email to