You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2016/10/17 05:15:43 UTC

spark git commit: [SPARK-17819][SQL][BRANCH-2.0] Support default database in connection URIs for Spark Thrift Server

Repository: spark
Updated Branches:
  refs/heads/branch-2.0 2a1b10b64 -> 3cc2fe5b9


[SPARK-17819][SQL][BRANCH-2.0] Support default database in connection URIs for Spark Thrift Server

## What changes were proposed in this pull request?

Currently, Spark Thrift Server ignores the default database in URI. This PR supports that like the following.

```sql
$ bin/beeline -u jdbc:hive2://localhost:10000 -e "create database testdb"
$ bin/beeline -u jdbc:hive2://localhost:10000/testdb -e "create table t(a int)"
$ bin/beeline -u jdbc:hive2://localhost:10000/testdb -e "show tables"
...
+------------+--------------+--+
| tableName  | isTemporary  |
+------------+--------------+--+
| t          | false        |
+------------+--------------+--+
1 row selected (0.347 seconds)
$ bin/beeline -u jdbc:hive2://localhost:10000 -e "show tables"
...
+------------+--------------+--+
| tableName  | isTemporary  |
+------------+--------------+--+
+------------+--------------+--+
No rows selected (0.098 seconds)
```

## How was this patch tested?

Pass the Jenkins with a newly added testsuite.

Author: Dongjoon Hyun <do...@apache.org>

Closes #15507 from dongjoon-hyun/SPARK-17819-BACK.


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

Branch: refs/heads/branch-2.0
Commit: 3cc2fe5b94d3bcdfb4f28bfa6d8e51fe67d6e1b4
Parents: 2a1b10b
Author: Dongjoon Hyun <do...@apache.org>
Authored: Sun Oct 16 22:15:47 2016 -0700
Committer: Reynold Xin <rx...@databricks.com>
Committed: Sun Oct 16 22:15:47 2016 -0700

----------------------------------------------------------------------
 .../thriftserver/SparkSQLSessionManager.scala   |  3 +
 .../thriftserver/JdbcConnectionUriSuite.scala   | 70 ++++++++++++++++++++
 2 files changed, 73 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/3cc2fe5b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala
----------------------------------------------------------------------
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala
index 1e4c479..fc5b221 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala
@@ -79,6 +79,9 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext:
       sqlContext.newSession()
     }
     ctx.setConf("spark.sql.hive.version", HiveUtils.hiveExecutionVersion)
+    if (sessionConf != null && sessionConf.containsKey("use:database")) {
+      ctx.sql(s"use ${sessionConf.get("use:database")}")
+    }
     sparkSqlOperationManager.sessionToContexts += sessionHandle -> ctx
     sessionHandle
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/3cc2fe5b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala
new file mode 100644
index 0000000..fb8a7e2
--- /dev/null
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala
@@ -0,0 +1,70 @@
+/*
+ * 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.hive.thriftserver
+
+import java.sql.DriverManager
+
+import org.apache.hive.jdbc.HiveDriver
+
+import org.apache.spark.util.Utils
+
+class JdbcConnectionUriSuite extends HiveThriftServer2Test {
+  Utils.classForName(classOf[HiveDriver].getCanonicalName)
+
+  override def mode: ServerMode.Value = ServerMode.binary
+
+  val JDBC_TEST_DATABASE = "jdbc_test_database"
+  val USER = System.getProperty("user.name")
+  val PASSWORD = ""
+
+  override protected def beforeAll(): Unit = {
+    super.beforeAll()
+
+    val jdbcUri = s"jdbc:hive2://localhost:$serverPort/"
+    val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD)
+    val statement = connection.createStatement()
+    statement.execute(s"CREATE DATABASE $JDBC_TEST_DATABASE")
+    connection.close()
+  }
+
+  override protected def afterAll(): Unit = {
+    try {
+      val jdbcUri = s"jdbc:hive2://localhost:$serverPort/"
+      val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD)
+      val statement = connection.createStatement()
+      statement.execute(s"DROP DATABASE $JDBC_TEST_DATABASE")
+      connection.close()
+    } finally {
+      super.afterAll()
+    }
+  }
+
+  test("SPARK-17819 Support default database in connection URIs") {
+    val jdbcUri = s"jdbc:hive2://localhost:$serverPort/$JDBC_TEST_DATABASE"
+    val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD)
+    val statement = connection.createStatement()
+    try {
+      val resultSet = statement.executeQuery("select current_database()")
+      resultSet.next()
+      assert(resultSet.getString(1) === JDBC_TEST_DATABASE)
+    } finally {
+      statement.close()
+      connection.close()
+    }
+  }
+}


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