You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by ch...@apache.org on 2022/08/20 10:23:28 UTC

[incubator-kyuubi] branch master updated: [KYUUBI #3222][FOLLOWUP] Introdude JdbcUtils to simplify code

This is an automated email from the ASF dual-hosted git repository.

chengpan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-kyuubi.git


The following commit(s) were added to refs/heads/master by this push:
     new d0f75e879 [KYUUBI #3222][FOLLOWUP] Introdude JdbcUtils to simplify code
d0f75e879 is described below

commit d0f75e879fa268a8d0b34f3c43bba5531e8047ce
Author: Cheng Pan <ch...@apache.org>
AuthorDate: Sat Aug 20 18:23:18 2022 +0800

    [KYUUBI #3222][FOLLOWUP] Introdude JdbcUtils to simplify code
    
    ### _Why are the changes needed?_
    
    This is the followup of #3235, the main change is introdude `JdbcUtils` to simplify code, and allow empty password for Jdbc auth.
    
    Jdbc connection pool has been removed because `JdbcAuthenticationProviderImpl` will be created on each connection, we can improve to use singleton in the future
    
    ### _How was this patch tested?_
    - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible
    
    - [ ] Add screenshots for manual tests if appropriate
    
    - [x] [Run test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests) locally before make a pull request
    
    Closes #3278 from pan3793/jdbc-followup.
    
    Closes #3222
    
    2863cae7 [Cheng Pan] Update kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/JdbcAuthenticationProviderImplSuite.scala
    51a9c458 [Cheng Pan] Update kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/JdbcAuthenticationProviderImplSuite.scala
    eee3c55e [Cheng Pan] Update kyuubi-common/src/test/scala/org/apache/kyuubi/util/JdbcUtilsSuite.scala
    d02bb999 [Cheng Pan] nit
    e001b5b5 [Cheng Pan] nit
    8cf5cd67 [Cheng Pan] nit
    032f2df8 [Cheng Pan] nit
    8a42f184 [Cheng Pan] nit
    c7893fdf [Cheng Pan] JdbcUtilsSuite
    f97f2d90 [Cheng Pan] remove pool
    a8812d0f [Cheng Pan] move render result set to test
    83d7d4cf [Cheng Pan] fix ut
    db787a4b [Cheng Pan] nit
    864f9ddc [Cheng Pan] nit
    b60decf5 [Cheng Pan] nit
    8c66e0be [Cheng Pan] nit
    2063c438 [Cheng Pan] [KYUUBI #3222][FOLLOWUP] Introdude JdbcUtils to simplify code
    
    Authored-by: Cheng Pan <ch...@apache.org>
    Signed-off-by: Cheng Pan <ch...@apache.org>
---
 kyuubi-common/pom.xml                              |   6 +
 .../JdbcAuthenticationProviderImpl.scala           | 189 +++++++--------------
 .../scala/org/apache/kyuubi/util/JdbcUtils.scala   |  98 +++++++++++
 .../test/scala/org/apache/kyuubi/TestUtils.scala   |  16 ++
 .../JdbcAuthenticationProviderImplSuite.scala      | 125 ++++++--------
 .../org/apache/kyuubi/util/JdbcUtilsSuite.scala    |  69 ++++++++
 6 files changed, 300 insertions(+), 203 deletions(-)

diff --git a/kyuubi-common/pom.xml b/kyuubi-common/pom.xml
index c8f736f21..62245e778 100644
--- a/kyuubi-common/pom.xml
+++ b/kyuubi-common/pom.xml
@@ -148,6 +148,12 @@
             <artifactId>derby</artifactId>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>com.jakewharton.fliptables</groupId>
+            <artifactId>fliptables</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/JdbcAuthenticationProviderImpl.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/JdbcAuthenticationProviderImpl.scala
index c4b0abdc7..ffd6911a5 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/JdbcAuthenticationProviderImpl.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/JdbcAuthenticationProviderImpl.scala
@@ -17,33 +17,44 @@
 
 package org.apache.kyuubi.service.authentication
 
-import java.sql.{Connection, PreparedStatement, Statement}
 import java.util.Properties
 import javax.security.sasl.AuthenticationException
+import javax.sql.DataSource
 
-import com.zaxxer.hikari.{HikariConfig, HikariDataSource}
+import com.zaxxer.hikari.util.DriverDataSource
 import org.apache.commons.lang3.StringUtils
 
 import org.apache.kyuubi.Logging
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.config.KyuubiConf._
+import org.apache.kyuubi.util.JdbcUtils
 
 class JdbcAuthenticationProviderImpl(conf: KyuubiConf) extends PasswdAuthenticationProvider
   with Logging {
 
-  private val driverClass = conf.get(AUTHENTICATION_JDBC_DRIVER)
-  private val jdbcUrl = conf.get(AUTHENTICATION_JDBC_URL)
-  private val jdbcUsername = conf.get(AUTHENTICATION_JDBC_USERNAME)
-  private val jdbcUserPassword = conf.get(AUTHENTICATION_JDBC_PASSWORD)
-  private val authQuerySql = conf.get(AUTHENTICATION_JDBC_QUERY)
-
   private val SQL_PLACEHOLDER_REGEX = """\$\{.+?}""".r
   private val USERNAME_SQL_PLACEHOLDER = "${username}"
   private val PASSWORD_SQL_PLACEHOLDER = "${password}"
 
+  private val driverClass = conf.get(AUTHENTICATION_JDBC_DRIVER)
+  private val jdbcUrl = conf.get(AUTHENTICATION_JDBC_URL)
+  private val username = conf.get(AUTHENTICATION_JDBC_USERNAME)
+  private val password = conf.get(AUTHENTICATION_JDBC_PASSWORD)
+  private val authQuery = conf.get(AUTHENTICATION_JDBC_QUERY)
+
+  private val redactedPasswd = password match {
+    case Some(s) if !StringUtils.isBlank(s) => s"${"*" * s.length}(length: ${s.length})"
+    case None => "(empty)"
+  }
+
   checkJdbcConfigs()
 
-  private[kyuubi] val hikariDataSource = getHikariDataSource
+  implicit private[kyuubi] val ds: DataSource = new DriverDataSource(
+    jdbcUrl.orNull,
+    driverClass.orNull,
+    new Properties,
+    username.orNull,
+    password.orNull)
 
   /**
    * The authenticate method is called by the Kyuubi Server authentication layer
@@ -62,37 +73,27 @@ class JdbcAuthenticationProviderImpl(conf: KyuubiConf) extends PasswdAuthenticat
         s" or contains blank space")
     }
 
-    if (StringUtils.isBlank(password)) {
-      throw new AuthenticationException(s"Error validating, password is null" +
-        s" or contains blank space")
-    }
-
-    var connection: Connection = null
-    var queryStatement: PreparedStatement = null
-
     try {
-      connection = hikariDataSource.getConnection
-
-      queryStatement = getAndPrepareQueryStatement(connection, user, password)
-
-      val resultSet = queryStatement.executeQuery()
-
-      if (resultSet == null || !resultSet.next()) {
-        // auth failed
-        throw new AuthenticationException(s"Password does not match or no such user. user:" +
-          s" $user , password length: ${password.length}")
+      debug(s"prepared auth query: $preparedQuery")
+      JdbcUtils.executeQuery(preparedQuery) { stmt =>
+        stmt.setMaxRows(1) // minimum result size required for authentication
+        queryPlaceholders.zipWithIndex.foreach {
+          case (USERNAME_SQL_PLACEHOLDER, i) => stmt.setString(i + 1, user)
+          case (PASSWORD_SQL_PLACEHOLDER, i) => stmt.setString(i + 1, password)
+          case (p, _) => throw new IllegalArgumentException(
+              s"Unrecognized placeholder in Query SQL: $p")
+        }
+      } { resultSet =>
+        if (resultSet == null || !resultSet.next()) {
+          throw new AuthenticationException("Password does not match or no such user. " +
+            s"user: $user, password: $redactedPasswd")
+        }
       }
-
-      // auth passed
-
     } catch {
-      case e: AuthenticationException =>
-        throw e
-      case e: Exception =>
-        error("Cannot get user info", e);
-        throw e
-    } finally {
-      closeDbConnection(connection, queryStatement)
+      case rethrow: AuthenticationException =>
+        throw rethrow
+      case rethrow: Exception =>
+        throw new AuthenticationException("Cannot get user info", rethrow)
     }
   }
 
@@ -101,104 +102,34 @@ class JdbcAuthenticationProviderImpl(conf: KyuubiConf) extends PasswdAuthenticat
 
     debug(configLog("Driver Class", driverClass.orNull))
     debug(configLog("JDBC URL", jdbcUrl.orNull))
-    debug(configLog("Database username", jdbcUsername.orNull))
-    debug(configLog("Database password length", jdbcUserPassword.getOrElse("").length.toString))
-    debug(configLog("Query SQL", authQuerySql.orNull))
+    debug(configLog("Database username", username.orNull))
+    debug(configLog("Database password", redactedPasswd))
+    debug(configLog("Query SQL", authQuery.orNull))
 
     // Check if JDBC parameters valid
-    if (driverClass.isEmpty) {
-      throw new IllegalArgumentException("JDBC driver class is not configured.")
-    }
-
-    if (jdbcUrl.isEmpty) {
-      throw new IllegalArgumentException("JDBC url is not configured")
-    }
-
-    if (jdbcUsername.isEmpty || jdbcUserPassword.isEmpty) {
-      throw new IllegalArgumentException("JDBC username or password is not configured")
+    require(driverClass.nonEmpty, "JDBC driver class is not configured.")
+    require(jdbcUrl.nonEmpty, "JDBC url is not configured.")
+    require(username.nonEmpty, "JDBC username is not configured")
+    // allow empty password
+    require(authQuery.nonEmpty, "Query SQL is not configured")
+
+    val query = authQuery.get.trim.toLowerCase
+    // allow simple select query sql only, complex query like CTE is not allowed
+    require(query.startsWith("select"), "Query SQL must start with 'SELECT'")
+    if (!query.contains("where")) {
+      warn("Query SQL does not contains 'WHERE' keyword")
     }
-
-    // Check Query SQL
-    if (authQuerySql.isEmpty) {
-      throw new IllegalArgumentException("Query SQL is not configured")
-    }
-    val querySqlInLowerCase = authQuerySql.get.trim.toLowerCase
-    if (!querySqlInLowerCase.startsWith("select")) { // allow select query sql only
-      throw new IllegalArgumentException("Query SQL must start with \"SELECT\"");
-    }
-    if (!querySqlInLowerCase.contains("where")) {
-      warn("Query SQL does not contains \"WHERE\" keyword");
+    if (!query.contains(USERNAME_SQL_PLACEHOLDER)) {
+      warn(s"Query SQL does not contains '$USERNAME_SQL_PLACEHOLDER' placeholder")
     }
-    if (!querySqlInLowerCase.contains("${username}")) {
-      warn("Query SQL does not contains \"${username}\" placeholder");
+    if (!query.contains(PASSWORD_SQL_PLACEHOLDER)) {
+      warn(s"Query SQL does not contains '$PASSWORD_SQL_PLACEHOLDER' placeholder")
     }
   }
 
-  private def getPlaceholderList(sql: String): List[String] = {
-    SQL_PLACEHOLDER_REGEX.findAllMatchIn(sql)
-      .map(m => m.matched)
-      .toList
-  }
-
-  private def getAndPrepareQueryStatement(
-      connection: Connection,
-      user: String,
-      password: String): PreparedStatement = {
+  private def preparedQuery: String =
+    SQL_PLACEHOLDER_REGEX.replaceAllIn(authQuery.get, "?")
 
-    val preparedSql: String = {
-      SQL_PLACEHOLDER_REGEX.replaceAllIn(authQuerySql.get, "?")
-    }
-    debug(s"prepared auth query sql: $preparedSql")
-
-    val stmt = connection.prepareStatement(preparedSql)
-    stmt.setMaxRows(1) // minimum result size required for authentication
-
-    // Extract placeholder list and fill parameters to placeholders
-    val placeholderList: List[String] = getPlaceholderList(authQuerySql.get)
-    for (i <- placeholderList.indices) {
-      val param = placeholderList(i) match {
-        case USERNAME_SQL_PLACEHOLDER => user
-        case PASSWORD_SQL_PLACEHOLDER => password
-        case otherPlaceholder =>
-          throw new IllegalArgumentException(
-            s"Unrecognized Placeholder In Query SQL: $otherPlaceholder")
-      }
-
-      stmt.setString(i + 1, param)
-    }
-
-    stmt
-  }
-
-  private def closeDbConnection(connection: Connection, statement: Statement): Unit = {
-    if (statement != null && !statement.isClosed) {
-      try {
-        statement.close()
-      } catch {
-        case e: Exception =>
-          error("Cannot close PreparedStatement to auth database ", e)
-      }
-    }
-
-    if (connection != null && !connection.isClosed) {
-      try {
-        connection.close()
-      } catch {
-        case e: Exception =>
-          error("Cannot close connection to auth database ", e)
-      }
-    }
-  }
-
-  private def getHikariDataSource: HikariDataSource = {
-    val datasourceProperties = new Properties()
-    val hikariConfig = new HikariConfig(datasourceProperties)
-    hikariConfig.setDriverClassName(driverClass.orNull)
-    hikariConfig.setJdbcUrl(jdbcUrl.orNull)
-    hikariConfig.setUsername(jdbcUsername.orNull)
-    hikariConfig.setPassword(jdbcUserPassword.orNull)
-    hikariConfig.setPoolName("jdbc-auth-pool")
-
-    new HikariDataSource(hikariConfig)
-  }
+  private def queryPlaceholders: Iterator[String] =
+    SQL_PLACEHOLDER_REGEX.findAllMatchIn(authQuery.get).map(_.matched)
 }
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/util/JdbcUtils.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/util/JdbcUtils.scala
new file mode 100644
index 000000000..29ce98247
--- /dev/null
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/util/JdbcUtils.scala
@@ -0,0 +1,98 @@
+/*
+ * 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.kyuubi.util
+
+import java.sql.{Connection, PreparedStatement, ResultSet}
+import javax.sql.DataSource
+
+import scala.util.control.NonFatal
+
+import org.apache.kyuubi.Logging
+
+object JdbcUtils extends Logging {
+
+  def close(c: AutoCloseable): Unit = {
+    if (c != null) {
+      try {
+        c.close()
+      } catch {
+        case NonFatal(t) => warn(s"Error on closing", t)
+      }
+    }
+  }
+
+  def withCloseable[R, C <: AutoCloseable](c: C)(block: C => R): R = {
+    try {
+      block(c)
+    } finally {
+      close(c)
+    }
+  }
+
+  def withConnection[R](block: Connection => R)(implicit ds: DataSource): R = {
+    withCloseable(ds.getConnection)(block)
+  }
+
+  def execute(
+      sqlTemplate: String)(
+      setParameters: PreparedStatement => Unit = _ => {})(
+      implicit ds: DataSource): Boolean = withConnection { conn =>
+    withCloseable(conn.prepareStatement(sqlTemplate)) { pStmt =>
+      setParameters(pStmt)
+      pStmt.execute()
+    }
+  }
+
+  def executeUpdate(
+      sqlTemplate: String)(
+      setParameters: PreparedStatement => Unit = _ => {})(
+      implicit ds: DataSource): Int = withConnection { conn =>
+    withCloseable(conn.prepareStatement(sqlTemplate)) { pStmt =>
+      setParameters(pStmt)
+      pStmt.executeUpdate()
+    }
+  }
+
+  def executeQuery[R](
+      sqlTemplate: String)(
+      setParameters: PreparedStatement => Unit = _ => {})(
+      processResultSet: ResultSet => R)(
+      implicit ds: DataSource): R = withConnection { conn =>
+    withCloseable(conn.prepareStatement(sqlTemplate)) { pStmt =>
+      setParameters(pStmt)
+      withCloseable(pStmt.executeQuery()) { rs =>
+        processResultSet(rs)
+      }
+    }
+  }
+
+  def executeQueryWithRowMapper[R](
+      sqlTemplate: String)(
+      setParameters: PreparedStatement => Unit = _ => {})(
+      rowMapper: ResultSet => R)(
+      implicit ds: DataSource): Seq[R] = withConnection { conn =>
+    withCloseable(conn.prepareStatement(sqlTemplate)) { pStmt =>
+      setParameters(pStmt)
+      withCloseable(pStmt.executeQuery()) { rs =>
+        val builder = Seq.newBuilder[R]
+        while (rs.next()) builder += rowMapper(rs)
+        builder.result
+      }
+    }
+  }
+}
diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/TestUtils.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/TestUtils.scala
index 4f9a4f126..0c1f9dee1 100644
--- a/kyuubi-common/src/test/scala/org/apache/kyuubi/TestUtils.scala
+++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/TestUtils.scala
@@ -19,9 +19,11 @@ package org.apache.kyuubi
 
 import java.nio.charset.StandardCharsets
 import java.nio.file.{Files, Path, StandardOpenOption}
+import java.sql.ResultSet
 
 import scala.collection.mutable.ArrayBuffer
 
+import com.jakewharton.fliptables.FlipTable
 import org.scalatest.Assertions.convertToEqualizer
 
 object TestUtils {
@@ -59,4 +61,18 @@ object TestUtils {
       newOutput.zip(expected).foreach { case (out, in) => assert(out === in, hint) }
     }
   }
+
+  def displayResultSet(resultSet: ResultSet): Unit = {
+    if (resultSet == null) throw new NullPointerException("resultSet == null")
+    val resultSetMetaData = resultSet.getMetaData
+    val columnCount: Int = resultSetMetaData.getColumnCount
+    val headers = (1 to columnCount).map(resultSetMetaData.getColumnName).toArray
+    val data = ArrayBuffer.newBuilder[Array[String]]
+    while (resultSet.next) {
+      data += (1 to columnCount).map(resultSet.getString).toArray
+    }
+    // scalastyle:off println
+    println(FlipTable.of(headers, data.result().toArray))
+    // scalastyle:on println
+  }
 }
diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/JdbcAuthenticationProviderImplSuite.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/JdbcAuthenticationProviderImplSuite.scala
index 617a64fba..f35d4f6b0 100644
--- a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/JdbcAuthenticationProviderImplSuite.scala
+++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/JdbcAuthenticationProviderImplSuite.scala
@@ -17,51 +17,56 @@
 
 package org.apache.kyuubi.service.authentication
 
-import java.sql.{Connection, DriverManager}
+import java.sql.DriverManager
 import java.util.Properties
 import javax.security.sasl.AuthenticationException
+import javax.sql.DataSource
 
-import org.apache.kyuubi.KyuubiFunSuite
+import com.zaxxer.hikari.util.DriverDataSource
+
+import org.apache.kyuubi.{KyuubiFunSuite, Utils}
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.config.KyuubiConf._
+import org.apache.kyuubi.util.JdbcUtils
 
 class JdbcAuthenticationProviderImplSuite extends KyuubiFunSuite {
   protected val dbUser: String = "liangbowen"
   protected val dbPasswd: String = "liangbowen"
-  protected var jdbcUrl: String = _
+  protected val authDbName: String = "auth_db"
+  protected val dbUrl: String = s"jdbc:derby:memory:$authDbName"
+  protected val jdbcUrl: String = s"$dbUrl;create=true"
+
+  implicit private val ds: DataSource = new DriverDataSource(
+    jdbcUrl,
+    "org.apache.derby.jdbc.AutoloadedDriver",
+    new Properties,
+    dbUser,
+    dbPasswd)
 
   protected val authUser: String = "liangtiancheng"
   protected val authPasswd: String = "liangtiancheng"
 
-  protected var conf = new KyuubiConf()
-  var conn: Connection = _
-  var authDbName: String = "auth_db"
+  protected val conf: KyuubiConf = new KyuubiConf()
+    .set(AUTHENTICATION_JDBC_DRIVER, "org.apache.derby.jdbc.AutoloadedDriver")
+    .set(AUTHENTICATION_JDBC_URL, jdbcUrl)
+    .set(AUTHENTICATION_JDBC_USERNAME, dbUser)
+    .set(AUTHENTICATION_JDBC_PASSWORD, dbPasswd)
+    .set(
+      AUTHENTICATION_JDBC_QUERY,
+      "SELECT 1 FROM user_auth WHERE username=${username} and passwd=${password}")
 
   override def beforeAll(): Unit = {
     // init db
-    val datasourceProperties = new Properties()
-    datasourceProperties.put("user", dbUser)
-    datasourceProperties.put("password", dbPasswd)
-
-    jdbcUrl = s"jdbc:derby:memory:$authDbName;create=true"
-    conn = DriverManager.getConnection(
-      s"$jdbcUrl;user=$dbUser;password=$dbPasswd",
-      datasourceProperties)
-
-    conn.prepareStatement(s"CREATE SCHEMA $dbUser").execute
-
-    conn.prepareStatement(
+    JdbcUtils.execute(s"CREATE SCHEMA $dbUser")()
+    JdbcUtils.execute(
       """CREATE TABLE user_auth (
-        |username VARCHAR(64) NOT NULL PRIMARY KEY,
-        |passwd VARCHAR(64))""".stripMargin).execute();
-
-    val insertStmt = conn.prepareStatement("INSERT INTO user_auth " +
-      "(username, passwd) VALUES (?,?)")
-    insertStmt.setString(1, authUser)
-    insertStmt.setString(2, authPasswd)
-    insertStmt.execute();
-
-    conf = genJdbcAuthConfigs
+        |  username VARCHAR(64) NOT NULL PRIMARY KEY,
+        |  passwd   VARCHAR(64)
+        |)""".stripMargin)()
+    JdbcUtils.execute("INSERT INTO user_auth (username, passwd) VALUES (?, ?)") { stmt =>
+      stmt.setString(1, authUser)
+      stmt.setString(2, authPasswd)
+    }
 
     super.beforeAll()
   }
@@ -70,16 +75,12 @@ class JdbcAuthenticationProviderImplSuite extends KyuubiFunSuite {
     super.afterAll()
 
     // cleanup db
-    try {
-      DriverManager.getConnection(s"jdbc:derby:memory:$authDbName;shutdown=true")
-    } catch {
-      case e: Throwable =>
+    Utils.tryLogNonFatalError {
+      DriverManager.getConnection(s"$dbUrl;shutdown=true")
     }
   }
 
   test("authenticate tests") {
-    val unchangedConf = genJdbcAuthConfigs
-
     val providerImpl = new JdbcAuthenticationProviderImpl(conf)
     providerImpl.authenticate(authUser, authPasswd)
 
@@ -88,58 +89,34 @@ class JdbcAuthenticationProviderImplSuite extends KyuubiFunSuite {
     }
     assert(e1.getMessage.contains("user is null"))
 
-    val e2 = intercept[AuthenticationException] {
-      providerImpl.authenticate(authUser, "")
-    }
-    assert(e2.getMessage.contains("password is null"))
-
     val e4 = intercept[AuthenticationException] {
       providerImpl.authenticate(authUser, "wrong_password")
     }
     assert(e4.isInstanceOf[AuthenticationException])
 
-    conf = unchangedConf.clone()
-    conf.unset(AUTHENTICATION_JDBC_URL)
-    val e5 = intercept[IllegalArgumentException] { new JdbcAuthenticationProviderImpl(conf) }
+    var _conf = conf.clone
+    _conf.unset(AUTHENTICATION_JDBC_URL)
+    val e5 = intercept[IllegalArgumentException] { new JdbcAuthenticationProviderImpl(_conf) }
     assert(e5.getMessage.contains("JDBC url is not configured"))
 
-    conf = unchangedConf.clone()
-    conf.unset(AUTHENTICATION_JDBC_USERNAME)
-    val e6 = intercept[IllegalArgumentException] { new JdbcAuthenticationProviderImpl(conf) }
-    assert(e6.getMessage.contains("JDBC username or password is not configured"))
+    _conf = conf.clone
+    _conf.unset(AUTHENTICATION_JDBC_USERNAME)
+    val e6 = intercept[IllegalArgumentException] { new JdbcAuthenticationProviderImpl(_conf) }
+    assert(e6.getMessage.contains("JDBC username is not configured"))
 
-    conf = unchangedConf.clone()
-    conf.unset(AUTHENTICATION_JDBC_PASSWORD)
-    val e7 = intercept[IllegalArgumentException] { new JdbcAuthenticationProviderImpl(conf) }
-    assert(e7.getMessage.contains("JDBC username or password is not configured"))
-
-    conf = unchangedConf.clone()
-    conf.unset(AUTHENTICATION_JDBC_QUERY)
-    val e8 = intercept[IllegalArgumentException] { new JdbcAuthenticationProviderImpl(conf) }
+    _conf = conf.clone
+    _conf.unset(AUTHENTICATION_JDBC_QUERY)
+    val e8 = intercept[IllegalArgumentException] { new JdbcAuthenticationProviderImpl(_conf) }
     assert(e8.getMessage.contains("Query SQL is not configured"))
 
-    conf.set(
+    _conf.set(
       AUTHENTICATION_JDBC_QUERY,
-      "INSERT INTO user_auth (username, password) " +
-        " VALUES ('demouser','demopassword'); ")
-    val e9 = intercept[IllegalArgumentException] { new JdbcAuthenticationProviderImpl(conf) }
-    assert(e9.getMessage.contains("Query SQL must start with \"SELECT\""))
+      "INSERT INTO user_auth (username, password) VALUES ('demouser','demopassword');")
+    val e9 = intercept[IllegalArgumentException] { new JdbcAuthenticationProviderImpl(_conf) }
+    assert(e9.getMessage.contains("Query SQL must start with 'SELECT'"))
 
-    conf.unset(AUTHENTICATION_JDBC_URL)
-    val e10 = intercept[IllegalArgumentException] { new JdbcAuthenticationProviderImpl(conf) }
+    _conf.unset(AUTHENTICATION_JDBC_URL)
+    val e10 = intercept[IllegalArgumentException] { new JdbcAuthenticationProviderImpl(_conf) }
     assert(e10.getMessage.contains("JDBC url is not configured"))
   }
-
-  private def genJdbcAuthConfigs: KyuubiConf = {
-    conf = new KyuubiConf()
-    conf.set(AUTHENTICATION_JDBC_DRIVER, "org.apache.derby.jdbc.AutoloadedDriver")
-    conf.set(AUTHENTICATION_JDBC_URL, jdbcUrl)
-    conf.set(AUTHENTICATION_JDBC_USERNAME, dbUser)
-    conf.set(AUTHENTICATION_JDBC_PASSWORD, dbPasswd)
-    conf.set(
-      AUTHENTICATION_JDBC_QUERY,
-      "SELECT 1 FROM user_auth " +
-        " WHERE username=${username} and passwd=${password}")
-    conf
-  }
 }
diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/util/JdbcUtilsSuite.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/util/JdbcUtilsSuite.scala
new file mode 100644
index 000000000..1523052af
--- /dev/null
+++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/util/JdbcUtilsSuite.scala
@@ -0,0 +1,69 @@
+/*
+ * 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.kyuubi.util
+
+import java.util.Properties
+import javax.sql.DataSource
+
+import com.zaxxer.hikari.util.DriverDataSource
+
+import org.apache.kyuubi.KyuubiFunSuite
+
+class JdbcUtilsSuite extends KyuubiFunSuite {
+
+  private val dbUrl = s"jdbc:derby:memory:jdbc_utils_test"
+  private val connUrl = s"$dbUrl;create=true"
+  private val shutdownUrl = s"$dbUrl;shutdown=true"
+  private val driverClz = "org.apache.derby.jdbc.AutoloadedDriver"
+
+  implicit private val ds: DataSource =
+    new DriverDataSource(connUrl, driverClz, new Properties, "test", "test")
+
+  case class Person(id: Int, name: String)
+
+  test("JdbcUtils methods") {
+    JdbcUtils.execute(
+      """CREATE TABLE person(
+        |  id   INT NOT NULL PRIMARY KEY,
+        |  name VARCHAR(255)
+        |)
+        |""".stripMargin)()
+    val affected = JdbcUtils.executeUpdate("INSERT INTO person VALUES (?, ?), (?, ?)") { stmt =>
+      stmt.setInt(1, 3)
+      stmt.setString(2, "Apache")
+      stmt.setInt(3, 9)
+      stmt.setString(4, "Kyuubi")
+    }
+    assert(affected == 2)
+
+    val persons = JdbcUtils.executeQueryWithRowMapper {
+      "SELECT * FROM person WHERE id=?"
+    } { stmt =>
+      stmt.setInt(1, 9)
+    } { rs =>
+      Person(rs.getInt(1), rs.getString(2))
+    }
+    assert(persons.length == 1)
+    assert(persons.head == Person(9, "Kyuubi"))
+
+    JdbcUtils.executeQuery("SELECT count(*) FROM person")() { rs =>
+      assert(rs.next())
+      assert(!rs.next())
+    }
+  }
+}