[KYUUBI #3222][FOLLOWUP] Fixing placeholder and config of user in JDBC Authentication Provider

### _Why are the changes needed?_

To fix the config name and placeholder with `username` introduced in https://github.com/apache/incubator-kyuubi/pull/3235 violate this convention as in JDBC driver use `user` keyword used for connection user rather than `username`,

1. change config name from `kyuubi.authentication.jdbc.username` to `kyuubi.authentication.jdbc.user`
2. change placeholder from `${username}` to `${user}`
3. update docs and config description related to above changes, and sync the update in jdbc auth docs statement details to config docs.
4. fix error in throwing AuthenticationException with auth db password. ut added for the fix.
5. other minor update in docs of custom auth

### _How was this patch tested?_
- [x] 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 #3288 from bowenliang123/jdbc-auth-config-update.

Closes #3222

502703fb [Bowen Liang] skip map for placeholder value lookup
3733be41 [liangbowen] nit
ab00525b [liangbowen] nit
2301c4f0 [liangbowen] fix ut of jdbc auth with wrong_password
06f0c1bb [liangbowen] remove redundant docs
ec4565b3 [liangbowen] remove redundant docs
ae1cce2e [liangbowen] fix compilation error of configLog
5d14103b [liangbowen] simplify configLog
6678e657 [liangbowen] reformat
52c1038e [liangbowen] simplify placeholder checking
21c2d5ea [liangbowen] check whether placeholders in supported list before conn establishment or authenticate
7db0adf5 [liangbowen] ut for unknown placeholder
657de6af [liangbowen] nit
736b3f22 [liangbowen] refactoring placeholder value lookup, for preventing setString multiple times with "i+1"
86c89125 [liangbowen] setMaxRows after prepare placeholder, to postpone operation on jdbc conn
115fae50 [liangbowen] increase test code coverage
b45b28cb [liangbowen] resultSet returned by executeQuery is never null
e1c07274 [liangbowen] update ut for redactPassword in JdbcUtils
b4a52e29 [liangbowen] fix typo in docs of custom auth
371c2c6e [liangbowen] move redactPassword method to JdbcUtils and add ut.
a4973c59 [liangbowen] reformat code
486e1503 [liangbowen] fix error in throwing AuthenticationException with auth db password. add ut for the fix.
efced90a [liangbowen] update settings.md
ef97e354 [liangbowen] add SELECT prefix hint for doc of  kyuubi.authentication.jdbc.query
025f94c7 [liangbowen] fix username to user in JdbcAuthenticationProviderImpl by 1. use config name `kyuubi.authentication.jdbc.user`, 2. use ${user} placeholder instead of ${username}

Lead-authored-by: liangbowen <liangbowen@gf.com.cn>
Co-authored-by: Bowen Liang <liangbowen@gf.com.cn>
Signed-off-by: Cheng Pan <chengpan@apache.org>
This commit is contained in:
liangbowen 2022-08-22 13:36:32 +08:00 committed by Cheng Pan
parent f72d1b7d9d
commit 2b67ab6e69
No known key found for this signature in database
GPG Key ID: 8001952629BCC75D
8 changed files with 116 additions and 56 deletions

View File

@ -140,9 +140,9 @@ kyuubi.authentication|NONE|A comma separated list of client authentication types
kyuubi.authentication.custom.class|&lt;undefined&gt;|User-defined authentication implementation of org.apache.kyuubi.service.authentication.PasswdAuthenticationProvider|string|1.3.0
kyuubi.authentication.jdbc.driver.class|&lt;undefined&gt;|Driver class name for JDBC Authentication Provider.|string|1.6.0
kyuubi.authentication.jdbc.password|&lt;undefined&gt;|Database password for JDBC Authentication Provider.|string|1.6.0
kyuubi.authentication.jdbc.query|&lt;undefined&gt;|Query SQL template with placeholders for JDBC Authentication Provider to execute. Authentication passes if at least one row fetched in the result set.Available placeholders are `${username}` and `${password}`.|string|1.6.0
kyuubi.authentication.jdbc.query|&lt;undefined&gt;|Query SQL template with placeholders for JDBC Authentication Provider to execute. Authentication passes if the result set is not empty.The SQL statement must start with the `SELECT` clause. Available placeholders are `${user}` and `${password}`.|string|1.6.0
kyuubi.authentication.jdbc.url|&lt;undefined&gt;|JDBC URL for JDBC Authentication Provider.|string|1.6.0
kyuubi.authentication.jdbc.username|&lt;undefined&gt;|Database username for JDBC Authentication Provider.|string|1.6.0
kyuubi.authentication.jdbc.user|&lt;undefined&gt;|Database user for JDBC Authentication Provider.|string|1.6.0
kyuubi.authentication.ldap.attrs|mail|Specifies part of the search as an attribute returned by LDAP. For example: mail,name.|seq|1.6.0
kyuubi.authentication.ldap.base.dn|&lt;undefined&gt;|LDAP base DN.|string|1.0.0
kyuubi.authentication.ldap.binddn|&lt;undefined&gt;|The user with which to bind to the LDAP server, and search for the full domain name of the user being authenticated. For example: uid=admin,cn=Directory Manager,ou=users,dc=example,dc=com|string|1.6.0

View File

@ -66,7 +66,7 @@ Enable Custom Authentication
To enable the custom authentication method, we need to
- put the jar package to ``$KYUUBI_HOME/jars`` directory to make it visible for
- Put the jar package to ``$KYUUBI_HOME/jars`` directory to make it visible for
the classpath of the kyuubi server.
- Configure the following properties to ``$KYUUBI_HOME/conf/kyuubi-defaults.conf``
on each node where kyuubi server is installed.

View File

@ -18,14 +18,14 @@
# Configure Kyuubi to Use JDBC Authentication
Kyuubi supports authentication via JDBC query. A query is prepared with username/password value and sent to the database configured in JDBC URL. The authentication passes if the result set is not empty.
Kyuubi supports authentication via JDBC query. A query is prepared with user/password value and sent to the database configured in JDBC URL. Authentication passes if the result set is not empty.
The query sql must start with `SELECT`. The SQL statement must start with the SELECT clause. Placeholders are supported and listed below for substitution:
- `${username}`
The SQL statement must start with the `SELECT` clause. Placeholders are supported and listed below for substitution:
- `${user}`
- `${password}`
For example, `SELECT 1 FROM auth_db.auth_table WHERE user=${username} AND
passwd=MD5(CONCAT(salt,${password}))` will be prepared as: `SELECT 1 FROM auth_db.auth_table WHERE user=? AND passwd=MD5(CONCAT(salt,?))` with value replacement of `username` and `password` in string type.
For example, `SELECT 1 FROM auth_db.auth_table WHERE user=${user} AND
passwd=MD5(CONCAT(salt,${password}))` will be prepared as `SELECT 1 FROM auth_db.auth_table WHERE user=? AND passwd=MD5(CONCAT(salt,?))` with value replacement of `user` and `password` in string type.
## Enable JDBC Authentication
@ -43,7 +43,7 @@ Configure the following properties to `$KYUUBI_HOME/conf/kyuubi-defaults.conf` o
kyuubi.authentication=JDBC
kyuubi.authentication.jdbc.driver.class = com.mysql.jdbc.Driver
kyuubi.authentication.jdbc.url = jdbc:mysql://127.0.0.1:3306/auth_db
kyuubi.authentication.jdbc.username = bowenliang123
kyuubi.authentication.jdbc.user = bowenliang123
kyuubi.authentication.jdbc.password = bowenliang123@kyuubi
kyuubi.authentication.jdbc.query = SELECT 1 FROM auth_table WHERE user=${username} AND passwd=MD5(CONCAT(salt,${password}))
kyuubi.authentication.jdbc.query = SELECT 1 FROM auth_table WHERE user=${user} AND passwd=MD5(CONCAT(salt,${password}))
```

View File

@ -688,9 +688,9 @@ object KyuubiConf {
.stringConf
.createOptional
val AUTHENTICATION_JDBC_USERNAME: OptionalConfigEntry[String] =
buildConf("kyuubi.authentication.jdbc.username")
.doc("Database username for JDBC Authentication Provider.")
val AUTHENTICATION_JDBC_USER: OptionalConfigEntry[String] =
buildConf("kyuubi.authentication.jdbc.user")
.doc("Database user for JDBC Authentication Provider.")
.version("1.6.0")
.stringConf
.createOptional
@ -706,8 +706,9 @@ object KyuubiConf {
buildConf("kyuubi.authentication.jdbc.query")
.doc("Query SQL template with placeholders " +
"for JDBC Authentication Provider to execute. " +
"Authentication passes if at least one row fetched in the result set." +
"Available placeholders are `${username}` and `${password}`.")
"Authentication passes if the result set is not empty." +
"The SQL statement must start with the `SELECT` clause. " +
"Available placeholders are `${user}` and `${password}`.")
.version("1.6.0")
.stringConf
.createOptional

View File

@ -33,28 +33,26 @@ class JdbcAuthenticationProviderImpl(conf: KyuubiConf) extends PasswdAuthenticat
with Logging {
private val SQL_PLACEHOLDER_REGEX = """\$\{.+?}""".r
private val USERNAME_SQL_PLACEHOLDER = "${username}"
private val USER_SQL_PLACEHOLDER = "${user}"
private val PASSWORD_SQL_PLACEHOLDER = "${password}"
private val supportedPlaceholders = Set(
USER_SQL_PLACEHOLDER,
PASSWORD_SQL_PLACEHOLDER)
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 authDbJdbcUrl = conf.get(AUTHENTICATION_JDBC_URL)
private val authDbUser = conf.get(AUTHENTICATION_JDBC_USER)
private val authDbPassword = 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()
implicit private[kyuubi] val ds: DataSource = new DriverDataSource(
jdbcUrl.orNull,
authDbJdbcUrl.orNull,
driverClass.orNull,
new Properties,
username.orNull,
password.orNull)
authDbUser.orNull,
authDbPassword.orNull)
/**
* The authenticate method is called by the Kyuubi Server authentication layer
@ -75,18 +73,18 @@ class JdbcAuthenticationProviderImpl(conf: KyuubiConf) extends PasswdAuthenticat
try {
debug(s"prepared auth query: $preparedQuery")
JdbcUtils.executeQuery(preparedQuery) { stmt =>
stmt.setMaxRows(1) // minimum result size required for authentication
JdbcUtils.executeQuery(preparedQuery) { pStmt =>
queryPlaceholders.zipWithIndex.foreach {
case (USERNAME_SQL_PLACEHOLDER, i) => stmt.setString(i + 1, user)
case (PASSWORD_SQL_PLACEHOLDER, i) => stmt.setString(i + 1, password)
case (USER_SQL_PLACEHOLDER, i) => pStmt.setString(i + 1, user)
case (PASSWORD_SQL_PLACEHOLDER, i) => pStmt.setString(i + 1, password)
case (p, _) => throw new IllegalArgumentException(
s"Unrecognized placeholder in Query SQL: $p")
}
pStmt.setMaxRows(1) // skipping more result rows to minimize I/O
} { resultSet =>
if (resultSet == null || !resultSet.next()) {
if (!resultSet.next()) {
throw new AuthenticationException("Password does not match or no such user. " +
s"user: $user, password: $redactedPasswd")
s"user: $user, password: ${JdbcUtils.redactPassword(Some(password))}")
}
}
} catch {
@ -98,19 +96,20 @@ class JdbcAuthenticationProviderImpl(conf: KyuubiConf) extends PasswdAuthenticat
}
private def checkJdbcConfigs(): Unit = {
def configLog(config: String, value: String): String = s"JDBCAuthConfig: $config = '$value'"
val configLog = (config: String, value: Option[String]) =>
s"JDBCAuthConfig: $config = '${value.orNull}'"
debug(configLog("Driver Class", driverClass.orNull))
debug(configLog("JDBC URL", jdbcUrl.orNull))
debug(configLog("Database username", username.orNull))
debug(configLog("Database password", redactedPasswd))
debug(configLog("Query SQL", authQuery.orNull))
debug(configLog("Driver Class", driverClass))
debug(configLog("JDBC URL", authDbJdbcUrl))
debug(configLog("Database user", authDbUser))
debug(configLog("Database password", Some(JdbcUtils.redactPassword(authDbPassword))))
debug(configLog("Query SQL", authQuery))
// Check if JDBC parameters valid
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(authDbJdbcUrl.nonEmpty, "JDBC url is not configured.")
require(authDbUser.nonEmpty, "JDBC user is not configured.")
// allow empty auth db password
require(authQuery.nonEmpty, "Query SQL is not configured")
val query = authQuery.get.trim.toLowerCase
@ -119,12 +118,18 @@ class JdbcAuthenticationProviderImpl(conf: KyuubiConf) extends PasswdAuthenticat
if (!query.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 (!query.contains(USER_SQL_PLACEHOLDER)) {
warn(s"Query SQL does not contains '$USER_SQL_PLACEHOLDER' placeholder")
}
if (!query.contains(PASSWORD_SQL_PLACEHOLDER)) {
warn(s"Query SQL does not contains '$PASSWORD_SQL_PLACEHOLDER' placeholder")
}
queryPlaceholders.foreach { placeholder =>
require(
supportedPlaceholders.contains(placeholder),
s"Unsupported placeholder in Query SQL: $placeholder")
}
}
private def preparedQuery: String =

View File

@ -22,6 +22,8 @@ import javax.sql.DataSource
import scala.util.control.NonFatal
import org.apache.commons.lang3.StringUtils
import org.apache.kyuubi.Logging
object JdbcUtils extends Logging {
@ -95,4 +97,11 @@ object JdbcUtils extends Logging {
}
}
}
def redactPassword(password: Option[String]): String = {
password match {
case Some(s) if StringUtils.isNotBlank(s) => s"${"*" * s.length}(length:${s.length})"
case _ => "(empty)"
}
}
}

View File

@ -30,30 +30,31 @@ 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 val dbUser: String = "bowenliang123"
protected val dbPasswd: String = "bowenliang123@kyuubi"
protected val authDbName: String = "auth_db"
protected val dbUrl: String = s"jdbc:derby:memory:$authDbName"
protected val jdbcUrl: String = s"$dbUrl;create=true"
private val authDbDriverClz = "org.apache.derby.jdbc.AutoloadedDriver"
implicit private val ds: DataSource = new DriverDataSource(
jdbcUrl,
"org.apache.derby.jdbc.AutoloadedDriver",
authDbDriverClz,
new Properties,
dbUser,
dbPasswd)
protected val authUser: String = "liangtiancheng"
protected val authPasswd: String = "liangtiancheng"
protected val authUser: String = "kyuubiuser"
protected val authPasswd: String = "kyuubiuuserpassword"
protected val conf: KyuubiConf = new KyuubiConf()
.set(AUTHENTICATION_JDBC_DRIVER, "org.apache.derby.jdbc.AutoloadedDriver")
.set(AUTHENTICATION_JDBC_DRIVER, authDbDriverClz)
.set(AUTHENTICATION_JDBC_URL, jdbcUrl)
.set(AUTHENTICATION_JDBC_USERNAME, dbUser)
.set(AUTHENTICATION_JDBC_USER, dbUser)
.set(AUTHENTICATION_JDBC_PASSWORD, dbPasswd)
.set(
AUTHENTICATION_JDBC_QUERY,
"SELECT 1 FROM user_auth WHERE username=${username} and passwd=${password}")
"SELECT 1 FROM user_auth WHERE username=${user} and passwd=${password}")
override def beforeAll(): Unit = {
// init db
@ -89,10 +90,14 @@ class JdbcAuthenticationProviderImplSuite extends KyuubiFunSuite {
}
assert(e1.getMessage.contains("user is null"))
val wrong_password = "wrong_password"
val e4 = intercept[AuthenticationException] {
providerImpl.authenticate(authUser, "wrong_password")
providerImpl.authenticate(authUser, wrong_password)
}
assert(e4.isInstanceOf[AuthenticationException])
assert(e4.getMessage.contains(s"Password does not match or no such user. " +
s"user: $authUser, " +
s"password: ${"*" * wrong_password.length}(length:${wrong_password.length})"))
var _conf = conf.clone
_conf.unset(AUTHENTICATION_JDBC_URL)
@ -100,9 +105,9 @@ class JdbcAuthenticationProviderImplSuite extends KyuubiFunSuite {
assert(e5.getMessage.contains("JDBC url is not configured"))
_conf = conf.clone
_conf.unset(AUTHENTICATION_JDBC_USERNAME)
_conf.unset(AUTHENTICATION_JDBC_USER)
val e6 = intercept[IllegalArgumentException] { new JdbcAuthenticationProviderImpl(_conf) }
assert(e6.getMessage.contains("JDBC username is not configured"))
assert(e6.getMessage.contains("JDBC user is not configured"))
_conf = conf.clone
_conf.unset(AUTHENTICATION_JDBC_QUERY)
@ -111,12 +116,39 @@ class JdbcAuthenticationProviderImplSuite extends KyuubiFunSuite {
_conf.set(
AUTHENTICATION_JDBC_QUERY,
"INSERT INTO user_auth (username, password) VALUES ('demouser','demopassword');")
"INSERT INTO user_auth (user, 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) }
assert(e10.getMessage.contains("JDBC url is not configured"))
_conf = conf.clone
_conf.set(AUTHENTICATION_JDBC_QUERY, "SELECT 1 FROM user_auth")
new JdbcAuthenticationProviderImpl(_conf)
_conf.set(AUTHENTICATION_JDBC_QUERY, "SELECT 1 FROM user_auth WHERE passwd=${password}")
new JdbcAuthenticationProviderImpl(_conf)
_conf.set(AUTHENTICATION_JDBC_QUERY, "SELECT 1 FROM user_auth WHERE username=${user}")
new JdbcAuthenticationProviderImpl(_conf)
// unknown placeholder
_conf.set(
AUTHENTICATION_JDBC_QUERY,
"SELECT 1 FROM user_auth WHERE user=${unsupported_placeholder} and username=${user}")
val e11 = intercept[IllegalArgumentException] { new JdbcAuthenticationProviderImpl(_conf) }
assert(e11.getMessage.contains(
"Unsupported placeholder in Query SQL: ${unsupported_placeholder}"))
// unknown field
_conf.set(
AUTHENTICATION_JDBC_QUERY,
"SELECT 1 FROM user_auth WHERE unknown_column=${user} and passwd=${password}")
val e12 = intercept[AuthenticationException] {
new JdbcAuthenticationProviderImpl(_conf).authenticate(authUser, authPasswd)
}
assert(e12.getCause.getMessage.contains("Column 'UNKNOWN_COLUMN' is either not in any table"))
}
}

View File

@ -65,5 +65,18 @@ class JdbcUtilsSuite extends KyuubiFunSuite {
assert(rs.next())
assert(!rs.next())
}
assertResult("****************(length:16)") {
JdbcUtils.redactPassword(Some("sample_pass_word"))
}
assertResult("(empty)") {
JdbcUtils.redactPassword(Some(""))
}
assertResult("(empty)") {
JdbcUtils.redactPassword(None)
}
}
}