Skip to content

Commit 66449b8

Browse files
viiryarxin
authored andcommitted
[SPARK-12968][SQL] Implement command to set current database
JIRA: https://issues.apache.org/jira/browse/SPARK-12968 Implement command to set current database. Author: Liang-Chi Hsieh <[email protected]> Author: Liang-Chi Hsieh <[email protected]> Closes #10916 from viirya/ddl-use-database.
1 parent b9dfdcc commit 66449b8

File tree

9 files changed

+50
-3
lines changed

9 files changed

+50
-3
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -46,6 +46,10 @@ trait Catalog {
4646

4747
def lookupRelation(tableIdent: TableIdentifier, alias: Option[String] = None): LogicalPlan
4848

49+
def setCurrentDatabase(databaseName: String): Unit = {
50+
throw new UnsupportedOperationException
51+
}
52+
4953
/**
5054
* Returns tuples of (tableName, isTemporary) for all tables in the given database.
5155
* isTemporary is a Boolean value indicates if a table is a temporary or not.

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,9 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly
5555
getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs)
5656
ExplainCommand(nodeToPlan(query), extended = extended.isDefined)
5757

58+
case Token("TOK_SWITCHDATABASE", Token(database, Nil) :: Nil) =>
59+
SetDatabaseCommand(cleanIdentifier(database))
60+
5861
case Token("TOK_DESCTABLE", describeArgs) =>
5962
// Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL
6063
val Some(tableType) :: formatted :: extended :: pretty :: Nil =

sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -408,3 +408,13 @@ case class DescribeFunction(
408408
}
409409
}
410410
}
411+
412+
case class SetDatabaseCommand(databaseName: String) extends RunnableCommand {
413+
414+
override def run(sqlContext: SQLContext): Seq[Row] = {
415+
sqlContext.catalog.setCurrentDatabase(databaseName)
416+
Seq.empty[Row]
417+
}
418+
419+
override val output: Seq[Attribute] = Seq.empty
420+
}

sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -183,7 +183,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
183183
"CREATE DATABASE hive_test_db;"
184184
-> "OK",
185185
"USE hive_test_db;"
186-
-> "OK",
186+
-> "",
187187
"CREATE TABLE hive_test(key INT, val STRING);"
188188
-> "OK",
189189
"SHOW TABLES;"

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -711,6 +711,10 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive
711711
}
712712

713713
override def unregisterAllTables(): Unit = {}
714+
715+
override def setCurrentDatabase(databaseName: String): Unit = {
716+
client.setCurrentDatabase(databaseName)
717+
}
714718
}
715719

716720
/**

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -155,8 +155,6 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging
155155
"TOK_SHOWLOCKS",
156156
"TOK_SHOWPARTITIONS",
157157

158-
"TOK_SWITCHDATABASE",
159-
160158
"TOK_UNLOCKTABLE"
161159
)
162160

sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -109,6 +109,9 @@ private[hive] trait ClientInterface {
109109
/** Returns the name of the active database. */
110110
def currentDatabase: String
111111

112+
/** Sets the name of current database. */
113+
def setCurrentDatabase(databaseName: String): Unit
114+
112115
/** Returns the metadata for specified database, throwing an exception if it doesn't exist */
113116
def getDatabase(name: String): HiveDatabase = {
114117
getDatabaseOption(name).getOrElse(throw new NoSuchDatabaseException)

sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.shims.{HadoopShims, ShimLoader}
3535
import org.apache.hadoop.security.UserGroupInformation
3636

3737
import org.apache.spark.{Logging, SparkConf, SparkException}
38+
import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
3839
import org.apache.spark.sql.catalyst.expressions.Expression
3940
import org.apache.spark.sql.execution.QueryExecutionException
4041
import org.apache.spark.util.{CircularBuffer, Utils}
@@ -229,6 +230,14 @@ private[hive] class ClientWrapper(
229230
state.getCurrentDatabase
230231
}
231232

233+
override def setCurrentDatabase(databaseName: String): Unit = withHiveState {
234+
if (getDatabaseOption(databaseName).isDefined) {
235+
state.setCurrentDatabase(databaseName)
236+
} else {
237+
throw new NoSuchDatabaseException
238+
}
239+
}
240+
232241
override def createDatabase(database: HiveDatabase): Unit = withHiveState {
233242
client.createDatabase(
234243
new Database(

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@ import org.scalatest.BeforeAndAfter
2828

2929
import org.apache.spark.{SparkException, SparkFiles}
3030
import org.apache.spark.sql.{AnalysisException, DataFrame, Row}
31+
import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
3132
import org.apache.spark.sql.catalyst.expressions.Cast
3233
import org.apache.spark.sql.catalyst.plans.logical.Project
3334
import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoin
@@ -1262,6 +1263,21 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
12621263

12631264
}
12641265

1266+
test("use database") {
1267+
val currentDatabase = sql("select current_database()").first().getString(0)
1268+
1269+
sql("CREATE DATABASE hive_test_db")
1270+
sql("USE hive_test_db")
1271+
assert("hive_test_db" == sql("select current_database()").first().getString(0))
1272+
1273+
intercept[NoSuchDatabaseException] {
1274+
sql("USE not_existing_db")
1275+
}
1276+
1277+
sql(s"USE $currentDatabase")
1278+
assert(currentDatabase == sql("select current_database()").first().getString(0))
1279+
}
1280+
12651281
test("lookup hive UDF in another thread") {
12661282
val e = intercept[AnalysisException] {
12671283
range(1).selectExpr("not_a_udf()")

0 commit comments

Comments
 (0)