Repository: spark
Updated Branches:
  refs/heads/master 6f9a18fe3 -> 4607f6e7f


[SPARK-14991][SQL] Remove HiveNativeCommand

## What changes were proposed in this pull request?
This patch removes HiveNativeCommand, so we can continue to remove the 
dependency on Hive. This pull request also removes the ability to generate 
golden result file using Hive.

## How was this patch tested?
Updated tests to reflect this.

Author: Reynold Xin <[email protected]>

Closes #12769 from rxin/SPARK-14991.


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

Branch: refs/heads/master
Commit: 4607f6e7f7b174c62700f1fe542f77af3203b096
Parents: 6f9a18f
Author: Reynold Xin <[email protected]>
Authored: Thu Apr 28 21:58:48 2016 -0700
Committer: Reynold Xin <[email protected]>
Committed: Thu Apr 28 21:58:48 2016 -0700

----------------------------------------------------------------------
 .../apache/spark/sql/catalyst/parser/SqlBase.g4 |  1 -
 .../scala/org/apache/spark/sql/SQLContext.scala |  9 ---
 .../org/apache/spark/sql/SparkSession.scala     |  9 ---
 .../spark/sql/execution/QueryExecution.scala    | 12 ++-
 .../spark/sql/execution/SparkSqlParser.scala    | 12 ---
 .../execution/command/HiveNativeCommand.scala   | 35 ---------
 .../spark/sql/internal/SessionState.scala       |  5 --
 .../hive/execution/HiveCompatibilitySuite.scala | 42 +++++-----
 .../spark/sql/hive/HiveMetastoreCatalog.scala   | 10 +--
 .../spark/sql/hive/HiveSessionState.scala       | 17 -----
 .../apache/spark/sql/hive/test/TestHive.scala   | 27 +++----
 .../spark/sql/hive/HiveDDLCommandSuite.scala    |  4 +-
 .../sql/hive/HiveMetastoreCatalogSuite.scala    |  7 +-
 .../sql/hive/execution/HiveComparisonTest.scala | 80 +-------------------
 .../sql/hive/execution/HiveQuerySuite.scala     | 64 ----------------
 15 files changed, 51 insertions(+), 283 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 
b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
index c356f0c..4d5d125 100644
--- 
a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
+++ 
b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
@@ -132,7 +132,6 @@ statement
 hiveNativeCommands
     : TRUNCATE TABLE tableIdentifier partitionSpec?
         (COLUMNS identifierList)?
-    | (CREATE | ALTER | DROP | SHOW | DESC | DESCRIBE) .*?
     ;
 
 unsupportedHiveNativeCommands

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/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 dbbdf11..1f08a61 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
@@ -674,15 +674,6 @@ class SQLContext private[sql](
   def sql(sqlText: String): DataFrame = sparkSession.sql(sqlText)
 
   /**
-   * Executes a SQL query without parsing it, but instead passing it directly 
to an underlying
-   * system to process. This is currently only used for Hive DDLs and will be 
removed as soon
-   * as Spark can parse all supported Hive DDLs itself.
-   */
-  private[sql] def runNativeSql(sqlText: String): Seq[Row] = {
-    sparkSession.runNativeSql(sqlText)
-  }
-
-  /**
    * Returns the specified table as a [[DataFrame]].
    *
    * @group ddl_ops

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
index f05546a..2814b70 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
@@ -649,15 +649,6 @@ class SparkSession private(
   }
 
   /**
-   * Executes a SQL query without parsing it, but instead passing it directly 
to an underlying
-   * system to process. This is currently only used for Hive DDLs and will be 
removed as soon
-   * as Spark can parse all supported Hive DDLs itself.
-   */
-  protected[sql] def runNativeSql(sqlText: String): Seq[Row] = {
-    sessionState.runNativeSql(sqlText).map { r => Row(r) }
-  }
-
-  /**
    * Parses the data type in our internal string representation. The data type 
string should
    * have the same format as the one generated by `toString` in scala.
    * It is only used by PySpark.

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
index d3d83b0..3e77228 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
@@ -27,7 +27,7 @@ import 
org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer}
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.catalyst.util.DateTimeUtils
-import org.apache.spark.sql.execution.command.{DescribeTableCommand, 
ExecutedCommandExec, HiveNativeCommand}
+import org.apache.spark.sql.execution.command.{DescribeTableCommand, 
ExecutedCommandExec}
 import org.apache.spark.sql.execution.exchange.{EnsureRequirements, 
ReuseExchange}
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types.{BinaryType, DateType, DecimalType, 
TimestampType, _}
@@ -199,12 +199,10 @@ class QueryExecution(val sparkSession: SparkSession, val 
logical: LogicalPlan) {
     }
   }
 
-  def simpleString: String = logical match {
-    case _: HiveNativeCommand => "<Native command: executed by Hive>"
-    case _ =>
-      s"""== Physical Plan ==
-         |${stringOrError(executedPlan)}
-        """.stripMargin.trim
+  def simpleString: String = {
+    s"""== Physical Plan ==
+       |${stringOrError(executedPlan)}
+      """.stripMargin.trim
   }
 
   override def toString: String = {

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
index e4c837a..c8e1003 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
@@ -44,10 +44,6 @@ class SparkSqlParser(conf: SQLConf) extends 
AbstractSqlParser {
   protected override def parse[T](command: String)(toResult: SqlBaseParser => 
T): T = {
     super.parse(substitutor.substitute(command))(toResult)
   }
-
-  protected override def nativeCommand(sqlText: String): LogicalPlan = {
-    HiveNativeCommand(substitutor.substitute(sqlText))
-  }
 }
 
 /**
@@ -57,14 +53,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
   import org.apache.spark.sql.catalyst.parser.ParserUtils._
 
   /**
-   * Pass a command to Hive using a [[HiveNativeCommand]].
-   */
-  override def visitExecuteNativeCommand(
-      ctx: ExecuteNativeCommandContext): LogicalPlan = withOrigin(ctx) {
-    HiveNativeCommand(command(ctx))
-  }
-
-  /**
    * Create a [[SetCommand]] logical plan.
    *
    * Note that we assume that everything after the SET keyword is assumed to 
be a part of the

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/sql/core/src/main/scala/org/apache/spark/sql/execution/command/HiveNativeCommand.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/HiveNativeCommand.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/HiveNativeCommand.scala
deleted file mode 100644
index bf66ea4..0000000
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/HiveNativeCommand.scala
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.command
-
-import org.apache.spark.sql.{Row, SparkSession}
-import org.apache.spark.sql.catalyst.expressions.AttributeReference
-import org.apache.spark.sql.types.StringType
-
-/**
- * A command that we delegate to Hive. Eventually we should remove this.
- */
-case class HiveNativeCommand(sql: String) extends RunnableCommand {
-
-  override def output: Seq[AttributeReference] =
-    Seq(AttributeReference("result", StringType, nullable = false)())
-
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    sparkSession.sessionState.runNativeSql(sql).map(Row(_))
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
index 63e0dc7..c05fe37 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala
@@ -192,9 +192,4 @@ private[sql] class SessionState(sparkSession: SparkSession) 
{
   def analyze(tableName: String): Unit = {
     AnalyzeTable(tableName).run(sparkSession)
   }
-
-  def runNativeSql(sql: String): Seq[String] = {
-    throw new AnalysisException("Unsupported query: " + sql)
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index 3cf82bc..f082035 100644
--- 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -475,6 +475,30 @@ class HiveCompatibilitySuite extends HiveQueryFileTest 
with BeforeAndAfter {
     "partition_wise_fileformat7",
     "rename_column",
 
+    // The following fails due to describe extended.
+    "alter3",
+    "alter5",
+    "alter_table_serde",
+    "input_part10",
+    "input_part10_win",
+    "inputddl6",
+    "inputddl7",
+    "part_inherit_tbl_props_empty",
+    "serde_reported_schema",
+    "stats0",
+    "stats_empty_partition",
+    "unicode_notation",
+    "union_remove_11",
+    "union_remove_3",
+
+    // The following fails due to alter table partitions with predicate.
+    "drop_partitions_filter",
+    "drop_partitions_filter2",
+    "drop_partitions_filter3",
+
+    // The following failes due to truncate table
+    "truncate_table",
+
     // We do not support DFS command.
     // We have converted the useful parts of these tests to tests
     // in org.apache.spark.sql.hive.execution.SQLQuerySuite.
@@ -493,11 +517,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest 
with BeforeAndAfter {
     "add_partition_with_whitelist",
     "alias_casted_column",
     "alter2",
-    "alter3",
-    "alter5",
     "alter_partition_with_whitelist",
     "alter_rename_partition",
-    "alter_table_serde",
     "ambiguous_col",
     "annotate_stats_join",
     "annotate_stats_limit",
@@ -590,9 +611,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with 
BeforeAndAfter {
     "distinct_stats",
     "drop_function",
     "drop_multi_partitions",
-    "drop_partitions_filter",
-    "drop_partitions_filter2",
-    "drop_partitions_filter3",
     "drop_table",
     "drop_table2",
     "drop_view",
@@ -693,8 +711,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with 
BeforeAndAfter {
     "input_limit",
     "input_part0",
     "input_part1",
-    "input_part10",
-    "input_part10_win",
     "input_part2",
     "input_part3",
     "input_part4",
@@ -707,8 +723,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with 
BeforeAndAfter {
     "inputddl1",
     "inputddl2",
     "inputddl3",
-    "inputddl6",
-    "inputddl7",
     "inputddl8",
     "insert1",
     "insert1_overwrite_partitions",
@@ -845,7 +859,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with 
BeforeAndAfter {
     "parallel",
     "parenthesis_star_by",
     "part_inherit_tbl_props",
-    "part_inherit_tbl_props_empty",
     "part_inherit_tbl_props_with_star",
     "partcols1",
     "partition_date",
@@ -901,7 +914,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with 
BeforeAndAfter {
     "semicolon",
     "semijoin",
     "serde_regex",
-    "serde_reported_schema",
     "set_variable_sub",
     "show_columns",
     "show_describe_func_quotes",
@@ -909,9 +921,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with 
BeforeAndAfter {
     "show_partitions",
     "show_tblproperties",
     "sort",
-    "stats0",
     "stats_aggregator_error_1",
-    "stats_empty_partition",
     "stats_publisher_error_1",
     "subq2",
     "tablename_with_select",
@@ -921,7 +931,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with 
BeforeAndAfter {
     "timestamp_null",
     "transform_ppr1",
     "transform_ppr2",
-    "truncate_table",
     "type_cast_1",
     "type_widening",
     "udaf_collect_set",
@@ -1068,7 +1077,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest 
with BeforeAndAfter {
     "udf_xpath_long",
     "udf_xpath_short",
     "udf_xpath_string",
-    "unicode_notation",
     "union10",
     "union11",
     "union13",
@@ -1102,8 +1110,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest 
with BeforeAndAfter {
     "union_date",
     "union_lateralview",
     "union_ppr",
-    "union_remove_11",
-    "union_remove_3",
     "union_remove_6",
     "union_script",
     "varchar_2",

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 58c10b7..0b24d35 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -18,24 +18,22 @@
 package org.apache.spark.sql.hive
 
 import scala.collection.JavaConverters._
-import scala.collection.mutable
 
 import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
 import org.apache.hadoop.fs.{FileStatus, Path}
 
 import org.apache.spark.internal.Logging
-import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession, 
SQLContext}
+import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession}
 import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
 import org.apache.spark.sql.catalyst.catalog._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.execution.command.{CreateTableAsSelectLogicalPlan, 
CreateViewCommand, HiveNativeCommand}
+import org.apache.spark.sql.execution.command.{CreateTableAsSelectLogicalPlan, 
CreateViewCommand}
 import org.apache.spark.sql.execution.datasources.{Partition => _, _}
 import org.apache.spark.sql.execution.datasources.parquet.{DefaultSource => 
ParquetDefaultSource, ParquetRelation}
 import org.apache.spark.sql.hive.orc.{DefaultSource => OrcDefaultSource}
-import org.apache.spark.sql.internal.HiveSerDe
 import org.apache.spark.sql.types._
 
 
@@ -436,10 +434,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
       case p: LogicalPlan if !p.childrenResolved => p
       case p: LogicalPlan if p.resolved => p
 
-      case CreateViewCommand(table, child, allowExisting, replace, sql)
-        if !sessionState.conf.nativeView =>
-        HiveNativeCommand(sql)
-
       case p @ CreateTableAsSelectLogicalPlan(table, child, allowExisting) =>
         val schema = if (table.schema.nonEmpty) {
           table.schema

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala
index f071df7..e085094 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala
@@ -154,23 +154,6 @@ private[hive] class HiveSessionState(sparkSession: 
SparkSession)
   }
 
   /**
-   * Execute a SQL statement by passing the query text directly to Hive.
-   */
-  override def runNativeSql(sql: String): Seq[String] = {
-    val command = sql.trim.toLowerCase
-    val functionOrMacroDDLPattern = Pattern.compile(
-      ".*(create|drop)\\s+(temporary\\s+)?(function|macro).+", Pattern.DOTALL)
-    if (functionOrMacroDDLPattern.matcher(command).matches()) {
-      executionHive.runSqlHive(sql)
-    } else if (command.startsWith("set")) {
-      metadataHive.runSqlHive(sql)
-      executionHive.runSqlHive(sql)
-    } else {
-      metadataHive.runSqlHive(sql)
-    }
-  }
-
-  /**
    * When true, enables an experimental feature where metastore tables that 
use the parquet SerDe
    * are automatically converted to use the Spark SQL parquet table scan, 
instead of the Hive
    * SerDe.

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
index 1d8f24c..ddb72fb 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
@@ -40,7 +40,7 @@ import 
org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder
 import org.apache.spark.sql.catalyst.expressions.ExpressionInfo
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.QueryExecution
-import org.apache.spark.sql.execution.command.{CacheTableCommand, 
HiveNativeCommand}
+import org.apache.spark.sql.execution.command.CacheTableCommand
 import org.apache.spark.sql.hive._
 import org.apache.spark.sql.hive.client.HiveClient
 import org.apache.spark.sql.internal.SQLConf
@@ -237,20 +237,20 @@ private[hive] class TestHiveSparkSession(
       "CREATE TABLE src1 (key INT, value STRING)".cmd,
       s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO 
TABLE src1".cmd),
     TestTable("srcpart", () => {
-      sessionState.runNativeSql(
+      sql(
         "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds 
STRING, hr STRING)")
       for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) {
-        sessionState.runNativeSql(
+        sql(
           s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}'
              |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr')
            """.stripMargin)
       }
     }),
     TestTable("srcpart1", () => {
-      sessionState.runNativeSql(
+      sql(
         "CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds 
STRING, hr INT)")
       for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) {
-        sessionState.runNativeSql(
+        sql(
           s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}'
              |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr')
            """.stripMargin)
@@ -261,7 +261,7 @@ private[hive] class TestHiveSparkSession(
       import org.apache.hadoop.mapred.{SequenceFileInputFormat, 
SequenceFileOutputFormat}
       import org.apache.thrift.protocol.TBinaryProtocol
 
-      sessionState.runNativeSql(
+      sql(
         s"""
          |CREATE TABLE src_thrift(fake INT)
          |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}'
@@ -274,7 +274,7 @@ private[hive] class TestHiveSparkSession(
          |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_, _]].getName}'
         """.stripMargin)
 
-      sessionState.runNativeSql(
+      sql(
         s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/complex.seq")}' 
INTO TABLE src_thrift")
     }),
     TestTable("serdeins",
@@ -436,20 +436,16 @@ private[hive] class TestHiveSparkSession(
       sessionState.hiveconf.set("fs.default.name", new 
File(".").toURI.toString)
       // It is important that we RESET first as broken hooks that might have 
been set could break
       // other sql exec here.
-      sessionState.executionHive.runSqlHive("RESET")
       sessionState.metadataHive.runSqlHive("RESET")
       // For some reason, RESET does not reset the following variables...
       // https://issues.apache.org/jira/browse/HIVE-9004
-      sessionState.runNativeSql("set hive.table.parameters.default=")
-      sessionState.runNativeSql("set datanucleus.cache.collections=true")
-      sessionState.runNativeSql("set datanucleus.cache.collections.lazy=true")
+      sessionState.metadataHive.runSqlHive("set 
hive.table.parameters.default=")
+      sessionState.metadataHive.runSqlHive("set 
datanucleus.cache.collections=true")
+      sessionState.metadataHive.runSqlHive("set 
datanucleus.cache.collections.lazy=true")
       // Lots of tests fail if we do not change the partition whitelist from 
the default.
-      sessionState.runNativeSql("set 
hive.metastore.partition.name.whitelist.pattern=.*")
+      sessionState.metadataHive.runSqlHive("set 
hive.metastore.partition.name.whitelist.pattern=.*")
 
       // In case a test changed any of these values, restore all the original 
ones here.
-      TestHiveContext.hiveClientConfigurations(
-        sessionState.hiveconf, warehousePath, scratchDirPath, 
metastoreTemporaryConf)
-          .foreach { case (k, v) => sessionState.metadataHive.runSqlHive(s"SET 
$k=$v") }
       sessionState.setDefaultOverrideConfs()
 
       sessionState.catalog.setCurrentDatabase("default")
@@ -477,7 +473,6 @@ private[hive] class TestHiveQueryExecution(
 
   override lazy val analyzed: LogicalPlan = {
     val describedTables = logical match {
-      case HiveNativeCommand(sparkSession.describedTable(tbl)) => tbl :: Nil
       case CacheTableCommand(tbl, _, _) => tbl :: Nil
       case _ => Nil
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala
index ec581b6..c97c28c 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.JsonTuple
 import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.catalyst.plans.PlanTest
 import org.apache.spark.sql.catalyst.plans.logical.{Generate, 
ScriptTransformation}
-import org.apache.spark.sql.execution.command.{CreateTable, 
CreateTableAsSelectLogicalPlan, CreateTableLike, CreateViewCommand, 
HiveNativeCommand, LoadData}
+import org.apache.spark.sql.execution.command.{CreateTable, 
CreateTableAsSelectLogicalPlan, CreateTableLike, CreateViewCommand, LoadData}
 import org.apache.spark.sql.hive.test.TestHive
 
 class HiveDDLCommandSuite extends PlanTest {
@@ -548,7 +548,7 @@ class HiveDDLCommandSuite extends PlanTest {
   test("create view -- partitioned view") {
     val v1 = "CREATE VIEW view1 partitioned on (ds, hr) as select * from 
srcpart"
     intercept[ParseException] {
-      parser.parsePlan(v1).isInstanceOf[HiveNativeCommand]
+      parser.parsePlan(v1)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
index 0d6a2e7..3665bb4 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
@@ -97,7 +97,7 @@ class DataSourceWithHiveMetastoreCatalogSuite
         assert(columns.map(_.dataType) === Seq("decimal(10,3)", "string"))
 
         checkAnswer(table("t"), testDF)
-        assert(sessionState.runNativeSql("SELECT * FROM t") === Seq("1.1\t1", 
"2.1\t2"))
+        assert(sessionState.metadataHive.runSqlHive("SELECT * FROM t") === 
Seq("1.1\t1", "2.1\t2"))
       }
     }
 
@@ -130,7 +130,8 @@ class DataSourceWithHiveMetastoreCatalogSuite
           assert(columns.map(_.dataType) === Seq("decimal(10,3)", "string"))
 
           checkAnswer(table("t"), testDF)
-          assert(sessionState.runNativeSql("SELECT * FROM t") === 
Seq("1.1\t1", "2.1\t2"))
+          assert(sessionState.metadataHive.runSqlHive("SELECT * FROM t") ===
+            Seq("1.1\t1", "2.1\t2"))
         }
       }
     }
@@ -160,7 +161,7 @@ class DataSourceWithHiveMetastoreCatalogSuite
           assert(columns.map(_.dataType) === Seq("int", "string"))
 
           checkAnswer(table("t"), Row(1, "val_1"))
-          assert(sessionState.runNativeSql("SELECT * FROM t") === 
Seq("1\tval_1"))
+          assert(sessionState.metadataHive.runSqlHive("SELECT * FROM t") === 
Seq("1\tval_1"))
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/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 3a9c981..b12f3aa 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
@@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.SQLBuilder
 import org.apache.spark.sql.catalyst.planning.PhysicalOperation
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.util._
-import org.apache.spark.sql.execution.command.{DescribeTableCommand, 
ExplainCommand, HiveNativeCommand, SetCommand, ShowColumnsCommand}
+import org.apache.spark.sql.execution.command.{DescribeTableCommand, 
ExplainCommand, SetCommand, ShowColumnsCommand}
 import org.apache.spark.sql.hive.{InsertIntoHiveTable => 
LogicalInsertIntoHiveTable}
 import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution}
 
@@ -166,14 +166,6 @@ abstract class HiveComparisonTest
       // Hack: Hive simply prints the result of a SET command to screen,
       // and does not return it as a query answer.
       case _: SetCommand => Seq("0")
-      case HiveNativeCommand(c) if c.toLowerCase.contains("desc") =>
-        answer
-          .filterNot(nonDeterministicLine)
-          .map(_.replaceAll("from deserializer", ""))
-          .map(_.replaceAll("None", ""))
-          .map(_.trim)
-          .filterNot(_ == "")
-      case _: HiveNativeCommand => 
answer.filterNot(nonDeterministicLine).filterNot(_ == "")
       case _: ExplainCommand => answer
       case _: DescribeTableCommand | ShowColumnsCommand(_) =>
         // Filter out non-deterministic lines and lines which do not have 
actual results but
@@ -342,53 +334,8 @@ abstract class HiveComparisonTest
             logInfo(s"Using answer cache for test: $testCaseName")
             hiveCachedResults
           } else {
-
-            val hiveQueries = queryList.map(new TestHiveQueryExecution(_))
-            // Make sure we can at least parse everything before attempting 
hive execution.
-            // Note this must only look at the logical plan as we might not be 
able to analyze if
-            // other DDL has not been executed yet.
-            hiveQueries.foreach(_.logical)
-            val computedResults = (queryList.zipWithIndex, hiveQueries, 
hiveCacheFiles).zipped.map {
-              case ((queryString, i), hiveQuery, cachedAnswerFile) =>
-                try {
-                  // Hooks often break the harness and don't really affect our 
test anyway, don't
-                  // even try running them.
-                  if 
(installHooksCommand.findAllMatchIn(queryString).nonEmpty) {
-                    sys.error("hive exec hooks not supported for tests.")
-                  }
-
-                  logWarning(s"Running query ${i + 1}/${queryList.size} with 
hive.")
-                  // Analyze the query with catalyst to ensure test tables are 
loaded.
-                  val answer = hiveQuery.analyzed match {
-                    case _: ExplainCommand =>
-                      // No need to execute EXPLAIN queries as we don't check 
the output.
-                      Nil
-                    case _ => TestHive.sessionState.runNativeSql(queryString)
-                  }
-
-                  // We need to add a new line to non-empty answers so we can 
differentiate Seq()
-                  // from Seq("").
-                  stringToFile(
-                    cachedAnswerFile, answer.mkString("\n") + (if 
(answer.nonEmpty) "\n" else ""))
-                  answer
-                } catch {
-                  case e: Exception =>
-                    val errorMessage =
-                      s"""
-                        |Failed to generate golden answer for query:
-                        |Error: ${e.getMessage}
-                        |${stackTraceToString(e)}
-                        |$queryString
-                      """.stripMargin
-                    stringToFile(
-                      new File(hiveFailedDirectory, testCaseName),
-                      errorMessage + consoleTestCase)
-                    fail(errorMessage)
-                }
-            }.toSeq
-            if (reset) { TestHive.reset() }
-
-            computedResults
+            throw new UnsupportedOperationException(
+              "Cannot find result file for test case: " + testCaseName)
           }
 
         // Run w/ catalyst
@@ -568,27 +515,6 @@ abstract class HiveComparisonTest
         }
       } catch {
         case tf: org.scalatest.exceptions.TestFailedException => throw tf
-        case originalException: Exception =>
-          if (System.getProperty("spark.hive.canarytest") != null) {
-            // When we encounter an error we check to see if the environment 
is still
-            // okay by running a simple query. If this fails then we halt 
testing since
-            // something must have gone seriously wrong.
-            try {
-              new TestHiveQueryExecution("SELECT key FROM 
src").hiveResultString()
-              TestHive.sessionState.runNativeSql("SELECT key FROM src")
-            } catch {
-              case e: Exception =>
-                logError(s"FATAL ERROR: Canary query threw $e This implies 
that the " +
-                  "testing environment has likely been corrupted.")
-                // The testing setup traps exits so wait here for a long time 
so the developer
-                // can see when things started to go wrong.
-                Thread.sleep(1000000)
-            }
-          }
-
-          // If the canary query didn't fail then the environment is still 
okay,
-          // so just throw the original exception.
-          throw originalException
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/4607f6e7/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 d7f6d18..e5a7706 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
@@ -73,13 +73,6 @@ class HiveQuerySuite extends HiveComparisonTest with 
BeforeAndAfter {
     assert(e.getMessage.toLowerCase.contains("operation not allowed"))
   }
 
-  test("SPARK-4908: concurrent hive native commands") {
-    (1 to 100).par.map { _ =>
-      sql("USE default")
-      sql("SHOW DATABASES")
-    }
-  }
-
   // Testing the Broadcast based join for cartesian join (cross join)
   // We assume that the Broadcast Join Threshold will works since the src is a 
small table
   private val spark_10484_1 = """
@@ -777,29 +770,6 @@ class HiveQuerySuite extends HiveComparisonTest with 
BeforeAndAfter {
     assert(sql("select array(key, *) from src limit 5").collect().size == 5)
   }
 
-  test("Query Hive native command execution result") {
-    val databaseName = "test_native_commands"
-
-    assertResult(0) {
-      sql(s"DROP DATABASE IF EXISTS $databaseName").count()
-    }
-
-    assertResult(0) {
-      sql(s"CREATE DATABASE $databaseName").count()
-    }
-
-    assert(
-      sql("SHOW DATABASES")
-        .select('result)
-        .collect()
-        .map(_.getString(0))
-        .contains(databaseName))
-
-    assert(isExplanation(sql(s"EXPLAIN SELECT key, COUNT(*) FROM src GROUP BY 
key")))
-
-    TestHive.reset()
-  }
-
   test("Exactly once semantics for DDL and command statements") {
     val tableName = "test_exactly_once"
     val q0 = sql(s"CREATE TABLE $tableName(key INT, value STRING)")
@@ -849,40 +819,6 @@ class HiveQuerySuite extends HiveComparisonTest with 
BeforeAndAfter {
         .collect()
     }
 
-    // Describe a column is a native command
-    assertResult(Array(Array("value", "string", "from deserializer"))) {
-      sql("DESCRIBE test_describe_commands1 value")
-        .select('result)
-        .collect()
-        .map(_.getString(0).split("\t").map(_.trim))
-    }
-
-    // Describe a column is a native command
-    assertResult(Array(Array("value", "string", "from deserializer"))) {
-      sql("DESCRIBE default.test_describe_commands1 value")
-        .select('result)
-        .collect()
-        .map(_.getString(0).split("\t").map(_.trim))
-    }
-
-    // Describe a partition is a native command
-    assertResult(
-      Array(
-        Array("key", "int"),
-        Array("value", "string"),
-        Array("dt", "string"),
-        Array(""),
-        Array("# Partition Information"),
-        Array("# col_name", "data_type", "comment"),
-        Array(""),
-        Array("dt", "string"))
-    ) {
-      sql("DESCRIBE test_describe_commands1 PARTITION (dt='2008-06-08')")
-        .select('result)
-        .collect()
-        .map(_.getString(0).replaceAll("None", 
"").trim.split("\t").map(_.trim))
-    }
-
     // Describe a registered temporary table.
     val testData =
       TestHive.sparkContext.parallelize(


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to