From 4db63fd2b430b0902ce0e50f526b1c2e2a5c6497 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun <dongjoon@apache.org> Date: Tue, 5 Jul 2016 16:47:32 -0700 Subject: [PATCH] [SPARK-16383][SQL] Remove `SessionState.executeSql` ## What changes were proposed in this pull request? This PR removes `SessionState.executeSql` in favor of `SparkSession.sql`. We can remove this safely since the visibility `SessionState` is `private[sql]` and `executeSql` is only used in one **ignored** test, `test("Multiple Hive Instances")`. ## How was this patch tested? Pass the Jenkins tests. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14055 from dongjoon-hyun/SPARK-16383. --- .../scala/org/apache/spark/sql/internal/SessionState.scala | 4 +--- .../spark/sql/hive/execution/ConcurrentHiveSuite.scala | 6 +++--- 2 files changed, 4 insertions(+), 6 deletions(-) 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 01cc13f9df..a228566b6b 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 @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} -import org.apache.spark.sql.catalyst.catalog.{ArchiveResource, _} +import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -162,8 +162,6 @@ private[sql] class SessionState(sparkSession: SparkSession) { // Helper methods, partially leftover from pre-2.0 days // ------------------------------------------------------ - def executeSql(sql: String): QueryExecution = executePlan(sqlParser.parsePlan(sql)) - def executePlan(plan: LogicalPlan): QueryExecution = new QueryExecution(sparkSession, plan) def refreshTable(tableName: String): Unit = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala index 1583a448ef..07d8c5bacb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala @@ -30,9 +30,9 @@ class ConcurrentHiveSuite extends SparkFunSuite with BeforeAndAfterAll { conf.set("spark.ui.enabled", "false") val ts = new TestHiveContext(new SparkContext("local", s"TestSQLContext$i", conf)) - ts.sessionState.executeSql("SHOW TABLES").toRdd.collect() - ts.sessionState.executeSql("SELECT * FROM src").toRdd.collect() - ts.sessionState.executeSql("SHOW TABLES").toRdd.collect() + ts.sparkSession.sql("SHOW TABLES").collect() + ts.sparkSession.sql("SELECT * FROM src").collect() + ts.sparkSession.sql("SHOW TABLES").collect() } } } -- GitLab