Skip to content

Commit ee237e4

Browse files
committed
Throw exception when use non existing database.
1 parent 43beb4b commit ee237e4

File tree

3 files changed

+18
-5
lines changed

3 files changed

+18
-5
lines changed

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

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -413,8 +413,13 @@ case class SetDatabaseCommand(databaseName: String) extends RunnableCommand {
413413

414414
override def run(sqlContext: SQLContext): Seq[Row] = {
415415
sqlContext.catalog.setCurrentDatabase(databaseName)
416-
Seq.empty[Row]
416+
Seq(Row("OK"))
417417
}
418418

419-
override def output: Seq[Attribute] = Seq.empty
419+
override val output: Seq[Attribute] = {
420+
val schema = StructType(
421+
StructField("output", StringType, nullable = false) :: Nil)
422+
423+
schema.toAttributes
424+
}
420425
}

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

Lines changed: 6 additions & 1 deletion
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}
@@ -230,7 +231,11 @@ private[hive] class ClientWrapper(
230231
}
231232

232233
override def setCurrentDatabase(databaseName: String): Unit = withHiveState {
233-
state.setCurrentDatabase(databaseName)
234+
if (getDatabaseOption(databaseName).isDefined) {
235+
state.setCurrentDatabase(databaseName)
236+
} else {
237+
throw new NoSuchDatabaseException
238+
}
234239
}
235240

236241
override def createDatabase(database: HiveDatabase): Unit = withHiveState {

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

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ import org.scalatest.BeforeAndAfter
2929
import org.apache.spark.{SparkException, SparkFiles}
3030
import org.apache.spark.sql.{AnalysisException, DataFrame, Row}
3131
import org.apache.spark.sql.catalyst.expressions.Cast
32+
import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
3233
import org.apache.spark.sql.catalyst.plans.logical.Project
3334
import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoin
3435
import org.apache.spark.sql.hive._
@@ -1264,13 +1265,15 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
12641265

12651266
test("use database") {
12661267
val currentDatabase = sql("select current_database()").first().getString(0)
1267-
sql("USE test")
1268-
assert("test" == sql("select current_database()").first().getString(0))
12691268

12701269
sql("CREATE DATABASE hive_test_db")
12711270
sql("USE hive_test_db")
12721271
assert("hive_test_db" == sql("select current_database()").first().getString(0))
12731272

1273+
intercept[NoSuchDatabaseException] {
1274+
sql("USE not_existing_db")
1275+
}
1276+
12741277
sql(s"USE $currentDatabase")
12751278
assert(currentDatabase == sql("select current_database()").first().getString(0))
12761279
}

0 commit comments

Comments
 (0)