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/09/08 09:05:25 UTC

[incubator-kyuubi] 02/02: Revert "[KYUUBI #3020] kyuubi ldap add new config property kyuubi.authentication.ldap.bindpw and kyuubi.authentication.ldap.attrs"

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

commit 29caadb2ba29e883370be61facf21efeca756c4b
Author: Cheng Pan <ch...@apache.org>
AuthorDate: Thu Sep 8 08:47:01 2022 +0000

    Revert "[KYUUBI #3020] kyuubi ldap add new config property kyuubi.authentication.ldap.bindpw and kyuubi.authentication.ldap.attrs"
    
    This reverts commit 5d88c7b207b97ea2f11b43811f47b2619d04f14e.
---
 docs/deployment/settings.md                        |  5 +-
 .../org/apache/kyuubi/config/KyuubiConf.scala      | 38 +--------
 .../LdapAuthenticationProviderImpl.scala           | 93 +++++-----------------
 .../AuthenticationProviderFactorySuite.scala       | 29 +------
 .../LdapAuthenticationProviderImplSuite.scala      | 80 +------------------
 .../service/authentication/WithLdapServer.scala    | 53 +-----------
 .../org/apache/kyuubi/RestClientTestHelper.scala   |  4 -
 .../KyuubiOperationKerberosAndPlainAuthSuite.scala |  4 -
 ...rationThriftHttpKerberosAndPlainAuthSuite.scala |  4 -
 .../server/rest/client/BatchRestApiSuite.scala     |  2 +-
 10 files changed, 33 insertions(+), 279 deletions(-)

diff --git a/docs/deployment/settings.md b/docs/deployment/settings.md
index f8ab8d246..2020eddcf 100644
--- a/docs/deployment/settings.md
+++ b/docs/deployment/settings.md
@@ -143,12 +143,9 @@ kyuubi.authentication.jdbc.password|&lt;undefined&gt;|Database password for JDBC
 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.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
-kyuubi.authentication.ldap.bindpw|&lt;undefined&gt;|The password for the bind user, to be used to search for the full name of the user being authenticated.|string|1.6.0
 kyuubi.authentication.ldap.domain|&lt;undefined&gt;|LDAP domain.|string|1.0.0
-kyuubi.authentication.ldap.guidKey|uid|(deprecated)LDAP attribute name whose values are unique in this LDAP server.For example:uid or cn.|string|1.2.0
+kyuubi.authentication.ldap.guidKey|uid|LDAP attribute name whose values are unique in this LDAP server.For example:uid or cn.|string|1.2.0
 kyuubi.authentication.ldap.url|&lt;undefined&gt;|SPACE character separated LDAP connection URL(s).|string|1.0.0
 kyuubi.authentication.sasl.qop|auth|Sasl QOP enable higher levels of protection for Kyuubi communication with clients.<ul> <li>auth - authentication only (default)</li> <li>auth-int - authentication plus integrity protection</li> <li>auth-conf - authentication plus integrity and confidentiality protection. This is applicable only if Kyuubi is configured to use Kerberos authentication.</li> </ul>|string|1.0.0
 
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
index f1843a102..4aa6c0912 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
@@ -640,41 +640,14 @@ object KyuubiConf {
       .stringConf
       .createOptional
 
-  @deprecated(s"using kyuubi.authentication.ldap.binddn instead", "1.6.0")
   val AUTHENTICATION_LDAP_GUIDKEY: ConfigEntry[String] =
     buildConf("kyuubi.authentication.ldap.guidKey")
-      .doc("(deprecated)LDAP attribute name whose values are unique in this LDAP server." +
+      .doc("LDAP attribute name whose values are unique in this LDAP server." +
         "For example:uid or cn.")
       .version("1.2.0")
       .stringConf
       .createWithDefault("uid")
 
-  val AUTHENTICATION_LDAP_BINDDN: OptionalConfigEntry[String] =
-    buildConf("kyuubi.authentication.ldap.binddn")
-      .doc("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")
-      .version("1.6.0")
-      .stringConf
-      .createOptional
-
-  val AUTHENTICATION_LDAP_PASSWORD: OptionalConfigEntry[String] =
-    buildConf("kyuubi.authentication.ldap.bindpw")
-      .doc("The password for the bind user," +
-        " to be used to search for the full name of the user being authenticated.")
-      .version("1.6.0")
-      .stringConf
-      .createOptional
-
-  val AUTHENTICATION_LDAP_ATTRIBUTES: ConfigEntry[Seq[String]] =
-    buildConf("kyuubi.authentication.ldap.attrs")
-      .doc("Specifies part of the search as an attribute returned by LDAP." +
-        " For example: mail,name.")
-      .version("1.6.0")
-      .stringConf
-      .toSequence()
-      .createWithDefault(Seq("mail"))
-
   val AUTHENTICATION_JDBC_DRIVER: OptionalConfigEntry[String] =
     buildConf("kyuubi.authentication.jdbc.driver.class")
       .doc("Driver class name for JDBC Authentication Provider.")
@@ -1973,8 +1946,7 @@ object KyuubiConf {
     SERVER_LIMIT_CONNECTIONS_PER_IPADDRESS,
     SERVER_LIMIT_CONNECTIONS_PER_USER_IPADDRESS,
     SERVER_LIMIT_CONNECTIONS_PER_USER,
-    SESSION_LOCAL_DIR_ALLOW_LIST,
-    AUTHENTICATION_LDAP_PASSWORD)
+    SESSION_LOCAL_DIR_ALLOW_LIST)
 
   /**
    * Holds information about keys that have been deprecated.
@@ -2043,11 +2015,7 @@ object KyuubiConf {
       DeprecatedConfig(
         "kyuubi.ha.zookeeper.acl.enabled",
         "1.3.2",
-        "Use kyuubi.ha.zookeeper.auth.type and kyuubi.ha.zookeeper.engine.auth.type instead"),
-      DeprecatedConfig(
-        AUTHENTICATION_LDAP_GUIDKEY.key,
-        "1.6.0",
-        s"using ${AUTHENTICATION_LDAP_BINDDN} instead"))
+        "Use kyuubi.ha.zookeeper.auth.type and kyuubi.ha.zookeeper.engine.auth.type instead"))
     Map(configs.map { cfg => cfg.key -> cfg }: _*)
   }
 
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImpl.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImpl.scala
index 9bd80cdd7..b5e08def5 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImpl.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImpl.scala
@@ -17,9 +17,8 @@
 
 package org.apache.kyuubi.service.authentication
 
-import javax.naming.{Context, NamingEnumeration, NamingException}
-import javax.naming.directory.{InitialDirContext, SearchControls, SearchResult}
-import javax.naming.ldap.InitialLdapContext
+import javax.naming.{Context, NamingException}
+import javax.naming.directory.InitialDirContext
 import javax.security.sasl.AuthenticationException
 
 import org.apache.commons.lang3.StringUtils
@@ -59,79 +58,29 @@ class LdapAuthenticationProviderImpl(conf: KyuubiConf) extends PasswdAuthenticat
     conf.get(AUTHENTICATION_LDAP_URL).foreach(env.put(Context.PROVIDER_URL, _))
 
     val domain = conf.get(AUTHENTICATION_LDAP_DOMAIN)
-    val mail = if (!hasDomain(user) && domain.nonEmpty) (user + "@" + domain.get) else user
-    var bindDn = conf.get(AUTHENTICATION_LDAP_BINDDN).getOrElse("")
-    val guidKey = conf.get(AUTHENTICATION_LDAP_GUIDKEY)
-
-    if ("".equals(bindDn)) {
-      bindDn = conf.get(AUTHENTICATION_LDAP_BASEDN) match {
-        case Some(dn) => guidKey + "=" + mail + "," + dn
-        case _ => mail
-      }
-      env.put(Context.SECURITY_PRINCIPAL, bindDn)
-      env.put(Context.SECURITY_CREDENTIALS, password)
-      try {
-        val ctx = new InitialDirContext(env)
-        ctx.close()
-      } catch {
-        case e: NamingException =>
-          throw new AuthenticationException(
-            s"Error validating LDAP user: $user," +
-              s" bindDn: $bindDn.",
-            e)
-      }
-    } else {
-      val baseDn = conf.get(AUTHENTICATION_LDAP_BASEDN).getOrElse("")
-      val bindPw = conf.get(AUTHENTICATION_LDAP_PASSWORD).getOrElse("")
-      val attrs = conf.get(AUTHENTICATION_LDAP_ATTRIBUTES).toArray
-      env.put(Context.SECURITY_PRINCIPAL, bindDn)
-      env.put(Context.SECURITY_CREDENTIALS, bindPw)
-      var nameEnuResults: NamingEnumeration[SearchResult] = null
-      try {
-        val ctx = new InitialLdapContext(env, null)
-        val sc = new SearchControls
-        sc.setReturningAttributes(attrs)
-        sc.setSearchScope(SearchControls.SUBTREE_SCOPE)
-        val searchFilter = String.format("(%s=%s)", "mail", mail)
-        nameEnuResults = ctx.search(baseDn, searchFilter, sc)
-      } catch {
-        case e: NamingException =>
-          throw new AuthenticationException(
-            s"LDAP InitialLdapContext failed, Error validating LDAP user: $user," +
-              s" bindDn: $bindDn.",
-            e)
-      }
-      if (nameEnuResults != null && nameEnuResults.hasMore) {
-        try {
-          val searchResult = nameEnuResults.next
-          val attrs = searchResult.getAttributes.getAll
-          if (!attrs.hasMore) {
-            throw new AuthenticationException(
-              s"LDAP attributes are empty, please check config " +
-                s"AUTHENTICATION_LDAP_ATTRIBUTES.key, Error validating LDAP user: $user," +
-                s" bindDn: $bindDn.")
-          }
-          while (attrs.hasMore) {
-            attrs.next
-            env.put(Context.SECURITY_PRINCIPAL, searchResult.getNameInNamespace)
-            env.put(Context.SECURITY_CREDENTIALS, password)
-            val ctx = new InitialDirContext(env)
-            ctx.close()
-          }
-        } catch {
-          case e: NamingException =>
-            throw new AuthenticationException(
-              s"LDAP InitialDirContext failed, Error validating LDAP user: $user," +
-                s" bindDn: $bindDn.",
-              e)
-        }
+    val u =
+      if (!hasDomain(user) && domain.nonEmpty) {
+        user + "@" + domain.get
       } else {
-        throw new AuthenticationException(
-          s"LDAP InitialLdapContext search results are empty, Error validating LDAP user: $user," +
-            s" bindDn: $bindDn.")
+        user
       }
+
+    val guidKey = conf.get(AUTHENTICATION_LDAP_GUIDKEY)
+    val bindDn = conf.get(AUTHENTICATION_LDAP_BASEDN) match {
+      case Some(dn) => guidKey + "=" + u + "," + dn
+      case _ => u
     }
 
+    env.put(Context.SECURITY_PRINCIPAL, bindDn)
+    env.put(Context.SECURITY_CREDENTIALS, password)
+
+    try {
+      val ctx = new InitialDirContext(env)
+      ctx.close()
+    } catch {
+      case e: NamingException =>
+        throw new AuthenticationException(s"Error validating LDAP user: $bindDn", e)
+    }
   }
 
   private def hasDomain(userName: String): Boolean = ServiceUtils.indexOfDomainMatch(userName) > 0
diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/AuthenticationProviderFactorySuite.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/AuthenticationProviderFactorySuite.scala
index 4e829cf55..0bd29ac56 100644
--- a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/AuthenticationProviderFactorySuite.scala
+++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/AuthenticationProviderFactorySuite.scala
@@ -21,41 +21,18 @@ import javax.security.sasl.AuthenticationException
 
 import org.apache.kyuubi.{KyuubiFunSuite, Utils}
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf._
 
 class AuthenticationProviderFactorySuite extends KyuubiFunSuite {
 
   import AuthenticationProviderFactory._
 
-  private val conf = new KyuubiConf()
-
-  override def beforeAll(): Unit = {
-    super.beforeAll()
-  }
-
-  test("get auth provider before 1.6.0") {
-    conf.set(AUTHENTICATION_LDAP_BASEDN, "test")
-    val p1 = getAuthenticationProvider(AuthMethods.withName("NONE"), conf)
-    p1.authenticate(Utils.currentUser, "")
-    val p2 = getAuthenticationProvider(AuthMethods.withName("LDAP"), conf)
-    val e1 = intercept[AuthenticationException](p2.authenticate("test", "test"))
-    assert(e1.getMessage.contains("Error validating LDAP user"))
-    val e2 = intercept[AuthenticationException](
-      AuthenticationProviderFactory.getAuthenticationProvider(null, conf))
-    assert(e2.getMessage === "Not a valid authentication method")
-  }
-
-  test("get auth provider on since 1.6.0") {
-    conf.set(AUTHENTICATION_LDAP_BASEDN, "test")
-    conf.set(AUTHENTICATION_LDAP_BINDDN, "test")
-    conf.set(AUTHENTICATION_LDAP_PASSWORD, "test")
-    conf.set(AUTHENTICATION_LDAP_DOMAIN, "test")
-    conf.set(AUTHENTICATION_LDAP_ATTRIBUTES, Seq("test"))
+  test("get auth provider") {
+    val conf = KyuubiConf()
     val p1 = getAuthenticationProvider(AuthMethods.withName("NONE"), conf)
     p1.authenticate(Utils.currentUser, "")
     val p2 = getAuthenticationProvider(AuthMethods.withName("LDAP"), conf)
     val e1 = intercept[AuthenticationException](p2.authenticate("test", "test"))
-    assert(e1.getMessage.contains("Error validating LDAP user"))
+    assert(e1.getMessage.contains("Error validating LDAP user:"))
     val e2 = intercept[AuthenticationException](
       AuthenticationProviderFactory.getAuthenticationProvider(null, conf))
     assert(e2.getMessage === "Not a valid authentication method")
diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImplSuite.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImplSuite.scala
index 97adf51eb..639411628 100644
--- a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImplSuite.scala
+++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/LdapAuthenticationProviderImplSuite.scala
@@ -42,14 +42,14 @@ class LdapAuthenticationProviderImplSuite extends WithLdapServer {
     assert(ldapServer.getListenPort > 0)
   }
 
-  test("Kyubbi ldap authenticate tests before 1.6.0") {
+  test("authenticate tests") {
     val providerImpl = new LdapAuthenticationProviderImpl(conf)
     val e1 = intercept[AuthenticationException](providerImpl.authenticate("", ""))
     assert(e1.getMessage.contains("user is null"))
     val e2 = intercept[AuthenticationException](providerImpl.authenticate("kyuubi", ""))
     assert(e2.getMessage.contains("password is null"))
 
-    val user = "uid=kentyao,dc=example,dc=com"
+    val user = "uid=kentyao,ou=users"
     providerImpl.authenticate(user, "kentyao")
     val e3 = intercept[AuthenticationException](
       providerImpl.authenticate(user, "kent"))
@@ -68,7 +68,6 @@ class LdapAuthenticationProviderImplSuite extends WithLdapServer {
     val providerImpl3 = new LdapAuthenticationProviderImpl(conf)
     val e5 = intercept[AuthenticationException](
       providerImpl3.authenticate("kentyao", "kentyao"))
-    conf.set(AUTHENTICATION_LDAP_URL, ldapUrl)
 
     assert(e5.getMessage.contains(user))
     assert(e5.getCause.isInstanceOf[CommunicationException])
@@ -77,79 +76,4 @@ class LdapAuthenticationProviderImplSuite extends WithLdapServer {
     val providerImpl4 = new LdapAuthenticationProviderImpl(conf)
     intercept[AuthenticationException](providerImpl4.authenticate("kentyao", "kentyao"))
   }
-
-  test("Kyubbi ldap authenticate tests since 1.6.0") {
-    conf.set(AUTHENTICATION_LDAP_BASEDN, ldapBaseDn)
-    conf.set(AUTHENTICATION_LDAP_ATTRIBUTES, ldapAttrs)
-    conf.set(AUTHENTICATION_LDAP_BINDDN, ldapBinddn)
-    conf.set(AUTHENTICATION_LDAP_PASSWORD, ldapBindpw)
-    conf.set(AUTHENTICATION_LDAP_DOMAIN, ldapDomain)
-    val providerImpl = new LdapAuthenticationProviderImpl(conf)
-    val e1 = intercept[AuthenticationException](providerImpl.authenticate("", ""))
-    assert(e1.getMessage.contains("user is null"))
-    val e2 = intercept[AuthenticationException](providerImpl.authenticate("kyuubi", ""))
-    assert(e2.getMessage.contains("password is null"))
-
-    providerImpl.authenticate(ldapUser, ldapUserPasswd)
-
-    conf.set(AUTHENTICATION_LDAP_BASEDN, "dc=com")
-    val providerImpl1 = new LdapAuthenticationProviderImpl(conf)
-    val e3 = intercept[AuthenticationException](
-      providerImpl1.authenticate(ldapUser, ldapUserPasswd))
-    assert(e3.getMessage contains (ldapUser))
-    assert(e3.getMessage contains "Error validating LDAP user")
-    assert(e3.getCause.isInstanceOf[javax.naming.NameNotFoundException])
-    conf.set(AUTHENTICATION_LDAP_BASEDN, ldapBaseDn)
-
-    conf.set(
-      AUTHENTICATION_LDAP_BINDDN,
-      s"uid=admin,cn=Directory Manager,ou=users,dc=example,dc=com")
-    conf.set(AUTHENTICATION_LDAP_PASSWORD, "adminPasswordTest")
-    val providerImpl2 = new LdapAuthenticationProviderImpl(conf)
-    val e4 = intercept[AuthenticationException](
-      providerImpl2.authenticate(ldapUser, ldapUserPasswd))
-    assert(e4.getMessage contains (ldapUser))
-    assert(e4.getMessage contains "Error validating LDAP user")
-    assert(e4.getCause.isInstanceOf[javax.naming.AuthenticationException])
-    conf.set(AUTHENTICATION_LDAP_PASSWORD, ldapBindpw)
-
-    val providerImpl3 = new LdapAuthenticationProviderImpl(conf)
-    val e5 = intercept[AuthenticationException](
-      providerImpl3.authenticate(ldapUser, "kent"))
-    assert(e5.getMessage contains (ldapUser))
-    assert(e5.getMessage contains "Error validating LDAP user")
-    assert(e5.getCause.isInstanceOf[javax.naming.AuthenticationException])
-
-    val providerImpl4 = new LdapAuthenticationProviderImpl(conf)
-    val e6 = intercept[AuthenticationException](
-      providerImpl4.authenticate("kent", ldapUserPasswd))
-    assert(e6.getMessage contains ("kent"))
-    assert(e6.getMessage contains "Error validating LDAP user")
-
-    conf.set(AUTHENTICATION_LDAP_DOMAIN, "kyuubi.com")
-    val providerImpl5 = new LdapAuthenticationProviderImpl(conf)
-    val e7 =
-      intercept[AuthenticationException](providerImpl5.authenticate(ldapUser, ldapUserPasswd))
-    assert(e7.getMessage contains "Error validating LDAP user")
-    conf.set(AUTHENTICATION_LDAP_DOMAIN, "example")
-
-    conf.set(AUTHENTICATION_LDAP_ATTRIBUTES, Seq("cn"))
-    val providerImpl6 = new LdapAuthenticationProviderImpl(conf)
-    providerImpl6.authenticate(ldapUser, ldapUserPasswd)
-    conf.set(AUTHENTICATION_LDAP_ATTRIBUTES, Seq("mail"))
-
-    conf.set(AUTHENTICATION_LDAP_ATTRIBUTES, Seq("mobile"))
-    val providerImpl7 = new LdapAuthenticationProviderImpl(conf)
-    val e8 =
-      intercept[AuthenticationException](providerImpl7.authenticate(ldapUser, ldapUserPasswd))
-    assert(e8.getMessage contains "Error validating LDAP user")
-    conf.set(AUTHENTICATION_LDAP_ATTRIBUTES, Seq("mail"))
-
-    conf.unset(AUTHENTICATION_LDAP_URL)
-    val providerImpl8 = new LdapAuthenticationProviderImpl(conf)
-    val e9 = intercept[AuthenticationException](
-      providerImpl8.authenticate(ldapUser, ldapUserPasswd))
-    assert(e9.getMessage.contains(ldapUser))
-    assert(e9.getCause.isInstanceOf[CommunicationException])
-  }
 }
diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/WithLdapServer.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/WithLdapServer.scala
index 814364470..0bb38684e 100644
--- a/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/WithLdapServer.scala
+++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/service/authentication/WithLdapServer.scala
@@ -23,23 +23,17 @@ import org.apache.kyuubi.{KyuubiFunSuite, Utils}
 
 trait WithLdapServer extends KyuubiFunSuite {
   protected var ldapServer: InMemoryDirectoryServer = _
-  protected val ldapBaseDn = "dc=example,dc=com"
+  protected val ldapBaseDn = "ou=users"
   protected val ldapUser = Utils.currentUser
   protected val ldapUserPasswd = "ldapPassword"
-  protected val ldapBinddn = "uid=admin,cn=Directory Manager,ou=users,dc=example,dc=com"
-  protected val ldapBindpw = "adminPassword"
-  protected val ldapBaseUserDn = "ou=users,dc=example,dc=com"
-  protected val ldapDomain = "example"
-  protected val ldapAttrs = Seq("mail")
 
   protected def ldapUrl = s"ldap://localhost:${ldapServer.getListenPort}"
 
   override def beforeAll(): Unit = {
     val config = new InMemoryDirectoryServerConfig(ldapBaseDn)
-    config.addAdditionalBindCredentials(ldapBinddn, ldapBindpw)
+    config.addAdditionalBindCredentials(s"uid=$ldapUser,ou=users", ldapUserPasswd)
     ldapServer = new InMemoryDirectoryServer(config)
     ldapServer.startListening()
-    addLdapUser(ldapServer, ldapBaseDn, ldapBaseUserDn, ldapDomain, ldapUser, ldapUserPasswd)
     super.beforeAll()
   }
 
@@ -47,47 +41,4 @@ trait WithLdapServer extends KyuubiFunSuite {
     ldapServer.close()
     super.afterAll()
   }
-
-  def addLdapUser(
-      ldapServer: InMemoryDirectoryServer,
-      ldapBaseDn: String,
-      ldapBaseUserDn: String,
-      ldapDomain: String,
-      ldapUser: String,
-      ldapUserPasswd: String): Unit = {
-    ldapServer.add(
-      s"dn: $ldapBaseDn",
-      "objectClass: domain",
-      "objectClass: top",
-      "dc: example")
-    ldapServer.add(
-      s"dn: $ldapBaseUserDn",
-      "objectClass: top",
-      "objectClass: organizationalUnit",
-      "ou: users")
-    ldapServer.add(
-      s"dn: cn=$ldapUser,$ldapBaseUserDn",
-      s"cn: $ldapUser",
-      s"sn: $ldapUser",
-      s"userPassword: $ldapUserPasswd",
-      "objectClass: person")
-    ldapServer.add(
-      s"dn: uid=$ldapUser,cn=$ldapUser,$ldapBaseUserDn",
-      s"uid: $ldapUser",
-      s"mail: $ldapUser@$ldapDomain",
-      s"cn: $ldapUser",
-      s"sn: $ldapUser",
-      s"userPassword: $ldapUserPasswd",
-      "objectClass: person",
-      "objectClass: inetOrgPerson")
-    ldapServer.add(
-      s"dn: uid=$ldapUser,$ldapBaseDn",
-      s"uid: $ldapUser",
-      s"mail: $ldapUser@$ldapDomain",
-      s"cn: $ldapUser",
-      s"sn: $ldapUser",
-      s"userPassword: $ldapUserPasswd",
-      "objectClass: person",
-      "objectClass: inetOrgPerson")
-  }
 }
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/RestClientTestHelper.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/RestClientTestHelper.scala
index f6c3c22f5..d15993797 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/RestClientTestHelper.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/RestClientTestHelper.scala
@@ -56,10 +56,6 @@ trait RestClientTestHelper extends RestFrontendTestHelper with KerberizedTestHel
       .set(
         KyuubiConf.AUTHENTICATION_CUSTOM_CLASS,
         classOf[UserDefineAuthenticationProviderImpl].getCanonicalName)
-      .set(KyuubiConf.AUTHENTICATION_LDAP_BINDDN, ldapBinddn)
-      .set(KyuubiConf.AUTHENTICATION_LDAP_PASSWORD, ldapBindpw)
-      .set(KyuubiConf.AUTHENTICATION_LDAP_DOMAIN, ldapDomain)
-      .set(KyuubiConf.AUTHENTICATION_LDAP_ATTRIBUTES, ldapAttrs)
   }
 
 }
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiOperationKerberosAndPlainAuthSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiOperationKerberosAndPlainAuthSuite.scala
index 4bcf09c5c..4e9c5dd78 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiOperationKerberosAndPlainAuthSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/KyuubiOperationKerberosAndPlainAuthSuite.scala
@@ -67,10 +67,6 @@ class KyuubiOperationKerberosAndPlainAuthSuite extends WithKyuubiServer with Ker
       .set(
         KyuubiConf.AUTHENTICATION_CUSTOM_CLASS,
         classOf[UserDefineAuthenticationProviderImpl].getCanonicalName)
-      .set(KyuubiConf.AUTHENTICATION_LDAP_BINDDN, ldapBinddn)
-      .set(KyuubiConf.AUTHENTICATION_LDAP_PASSWORD, ldapBindpw)
-      .set(KyuubiConf.AUTHENTICATION_LDAP_DOMAIN, ldapDomain)
-      .set(KyuubiConf.AUTHENTICATION_LDAP_ATTRIBUTES, ldapAttrs)
   }
 
   test("test with KERBEROS TGT cache authentication") {
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/thrift/http/KyuubiOperationThriftHttpKerberosAndPlainAuthSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/thrift/http/KyuubiOperationThriftHttpKerberosAndPlainAuthSuite.scala
index ec9dc4070..e0f32e843 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/thrift/http/KyuubiOperationThriftHttpKerberosAndPlainAuthSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/thrift/http/KyuubiOperationThriftHttpKerberosAndPlainAuthSuite.scala
@@ -56,10 +56,6 @@ class KyuubiOperationThriftHttpKerberosAndPlainAuthSuite
         classOf[UserDefineAuthenticationProviderImpl].getCanonicalName)
       .set(KyuubiConf.SERVER_SPNEGO_KEYTAB, testKeytab)
       .set(KyuubiConf.SERVER_SPNEGO_PRINCIPAL, testSpnegoPrincipal)
-      .set(KyuubiConf.AUTHENTICATION_LDAP_BINDDN, ldapBinddn)
-      .set(KyuubiConf.AUTHENTICATION_LDAP_PASSWORD, ldapBindpw)
-      .set(KyuubiConf.AUTHENTICATION_LDAP_DOMAIN, ldapDomain)
-      .set(KyuubiConf.AUTHENTICATION_LDAP_ATTRIBUTES, ldapAttrs)
   }
 
   override protected def getJdbcUrl: String =
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/BatchRestApiSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/BatchRestApiSuite.scala
index 4148223ea..d144d61be 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/BatchRestApiSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/BatchRestApiSuite.scala
@@ -80,7 +80,7 @@ class BatchRestApiSuite extends RestClientTestHelper with BatchTestHelper {
       // get batch by id
       batchRestApi.getBatchById("1")
     }
-    assert(e.getCause.toString.contains(s"Error validating LDAP user: ${customUser}"))
+    assert(e.getCause.toString.contains(s"Error validating LDAP user: uid=${customUser}"))
 
     basicKyuubiRestClient.close()