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 2023/03/16 10:15:45 UTC

[kyuubi] branch master updated: [KYUUBI #4526][FOLLOWUP] Defer evaluation for GroupProvider

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/kyuubi.git


The following commit(s) were added to refs/heads/master by this push:
     new 427df6831 [KYUUBI #4526][FOLLOWUP] Defer evaluation for GroupProvider
427df6831 is described below

commit 427df68312b4a38b59541546ee73d876e43438be
Author: maruilei <ma...@58.com>
AuthorDate: Thu Mar 16 18:15:32 2023 +0800

    [KYUUBI #4526][FOLLOWUP] Defer evaluation for GroupProvider
    
    ### _Why are the changes needed?_
    
    As described in the discussion: https://github.com/apache/kyuubi/discussions/4512
    
    Since the pr https://github.com/apache/kyuubi/pull/3897 , we introduced the feature of `Supplying pluggable GroupProvider`.
    
    But now when we create a new EngineRef, no matter which engine share level is used, getGroupNames will always be called.
    
    This may cause users to encounter some confusion stack trace when they do not use the GROUP engine share level.
    
    So we solve the problem through moving the `sessionManager.groupProvider.primaryGroup` to the inside of `EngineRef.appUser`, to ensure that the feature of `Supplying pluggable GroupProvider` is only valid for the GROUP engine share level.
    
    ### _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.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request
    
    Closes #4527 from merrily01/branch-1.7-groupProvider.
    
    Closes #4526
    
    0a858a921 [maruilei] Address comments.
    c63a0f97b [maruilei] [KYUUBI #4526][FOLLOWUP] Ensure that the feature of `Supplying pluggable GroupProvider` is only valid for the GROUP engine share level.
    
    Authored-by: maruilei <ma...@58.com>
    Signed-off-by: Cheng Pan <ch...@apache.org>
---
 .../scala/org/apache/kyuubi/engine/EngineRef.scala |  6 +-
 .../apache/kyuubi/session/KyuubiSessionImpl.scala  |  2 +-
 .../org/apache/kyuubi/engine/EngineRefTests.scala  | 67 +++++++++++++++-------
 .../engine/EngineRefWithZookeeperSuite.scala       | 17 +++++-
 .../kyuubi/server/api/v1/AdminResourceSuite.scala  | 22 +++++--
 .../kyuubi/server/rest/client/AdminCtlSuite.scala  |  5 +-
 .../server/rest/client/AdminRestApiSuite.scala     |  4 +-
 7 files changed, 90 insertions(+), 33 deletions(-)

diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/EngineRef.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/EngineRef.scala
index 1f38ea3c2..e2ddb4221 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/EngineRef.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/engine/EngineRef.scala
@@ -19,6 +19,7 @@ package org.apache.kyuubi.engine
 
 import java.util.concurrent.TimeUnit
 
+import scala.collection.JavaConverters._
 import scala.util.Random
 
 import com.codahale.metrics.MetricRegistry
@@ -40,6 +41,7 @@ import org.apache.kyuubi.ha.client.{DiscoveryClient, DiscoveryClientProvider, Di
 import org.apache.kyuubi.metrics.MetricsConstants.{ENGINE_FAIL, ENGINE_TIMEOUT, ENGINE_TOTAL}
 import org.apache.kyuubi.metrics.MetricsSystem
 import org.apache.kyuubi.operation.log.OperationLog
+import org.apache.kyuubi.plugin.GroupProvider
 
 /**
  * The description and functionality of an engine at server side
@@ -51,7 +53,7 @@ import org.apache.kyuubi.operation.log.OperationLog
 private[kyuubi] class EngineRef(
     conf: KyuubiConf,
     user: String,
-    primaryGroup: String,
+    groupProvider: GroupProvider,
     engineRefId: String,
     engineManager: KyuubiApplicationManager)
   extends Logging {
@@ -85,7 +87,7 @@ private[kyuubi] class EngineRef(
   // Launcher of the engine
   private[kyuubi] val appUser: String = shareLevel match {
     case SERVER => Utils.currentUser
-    case GROUP => primaryGroup
+    case GROUP => groupProvider.primaryGroup(user, conf.getAll.asJava)
     case _ => user
   }
 
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionImpl.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionImpl.scala
index e4203b301..80df5c44d 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionImpl.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionImpl.scala
@@ -77,7 +77,7 @@ class KyuubiSessionImpl(
   lazy val engine: EngineRef = new EngineRef(
     sessionConf,
     user,
-    sessionManager.groupProvider.primaryGroup(user, optimizedConf.asJava),
+    sessionManager.groupProvider,
     handle.identifier.toString,
     sessionManager.applicationManager)
   private[kyuubi] val launchEngineOp = sessionManager.operationManager
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/EngineRefTests.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/EngineRefTests.scala
index 5ca8723f5..8b050684a 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/EngineRefTests.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/EngineRefTests.scala
@@ -20,6 +20,8 @@ package org.apache.kyuubi.engine
 import java.util.UUID
 import java.util.concurrent.Executors
 
+import scala.collection.JavaConverters._
+
 import org.scalatest.time.SpanSugar.convertIntToGrainOfTime
 
 import org.apache.kyuubi.{KYUUBI_VERSION, Utils}
@@ -33,6 +35,7 @@ import org.apache.kyuubi.ha.client.DiscoveryClientProvider
 import org.apache.kyuubi.ha.client.DiscoveryPaths
 import org.apache.kyuubi.metrics.MetricsConstants.ENGINE_TOTAL
 import org.apache.kyuubi.metrics.MetricsSystem
+import org.apache.kyuubi.plugin.PluginLoader
 import org.apache.kyuubi.util.NamedThreadFactory
 
 trait EngineRefTests extends KyuubiFunSuite {
@@ -68,7 +71,9 @@ trait EngineRefTests extends KyuubiFunSuite {
     Seq(None, Some("suffix")).foreach { domain =>
       conf.set(KyuubiConf.ENGINE_SHARE_LEVEL, CONNECTION.toString)
       domain.foreach(conf.set(KyuubiConf.ENGINE_SHARE_LEVEL_SUBDOMAIN.key, _))
-      val engine = new EngineRef(conf, user, "grp", id, null)
+      conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
+
+      val engine = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
       assert(engine.engineSpace ===
         DiscoveryPaths.makePath(
           s"kyuubi_${KYUUBI_VERSION}_${CONNECTION}_${engineType}",
@@ -82,7 +87,9 @@ trait EngineRefTests extends KyuubiFunSuite {
     val id = UUID.randomUUID().toString
     conf.set(KyuubiConf.ENGINE_SHARE_LEVEL, USER.toString)
     conf.set(KyuubiConf.ENGINE_TYPE, FLINK_SQL.toString)
-    val appName = new EngineRef(conf, user, "grp", id, null)
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
+
+    val appName = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     assert(appName.engineSpace ===
       DiscoveryPaths.makePath(
         s"kyuubi_${KYUUBI_VERSION}_${USER}_$FLINK_SQL",
@@ -94,7 +101,7 @@ trait EngineRefTests extends KyuubiFunSuite {
       k =>
         conf.unset(KyuubiConf.ENGINE_SHARE_LEVEL_SUBDOMAIN)
         conf.set(k.key, "abc")
-        val appName2 = new EngineRef(conf, user, "grp", id, null)
+        val appName2 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
         assert(appName2.engineSpace ===
           DiscoveryPaths.makePath(
             s"kyuubi_${KYUUBI_VERSION}_${USER}_${FLINK_SQL}",
@@ -108,8 +115,12 @@ trait EngineRefTests extends KyuubiFunSuite {
     val id = UUID.randomUUID().toString
     conf.set(KyuubiConf.ENGINE_SHARE_LEVEL, GROUP.toString)
     conf.set(KyuubiConf.ENGINE_TYPE, SPARK_SQL.toString)
-    val primaryGroupName = "primary_grp"
-    val engineRef = new EngineRef(conf, user, primaryGroupName, id, null)
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
+
+    val primaryGroupName =
+      PluginLoader.loadGroupProvider(conf).primaryGroup(user, Map.empty[String, String].asJava)
+
+    val engineRef = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     assert(engineRef.engineSpace ===
       DiscoveryPaths.makePath(
         s"kyuubi_${KYUUBI_VERSION}_GROUP_SPARK_SQL",
@@ -122,7 +133,7 @@ trait EngineRefTests extends KyuubiFunSuite {
       k =>
         conf.unset(k)
         conf.set(k.key, "abc")
-        val engineRef2 = new EngineRef(conf, user, primaryGroupName, id, null)
+        val engineRef2 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
         assert(engineRef2.engineSpace ===
           DiscoveryPaths.makePath(
             s"kyuubi_${KYUUBI_VERSION}_${GROUP}_${SPARK_SQL}",
@@ -137,7 +148,9 @@ trait EngineRefTests extends KyuubiFunSuite {
     val id = UUID.randomUUID().toString
     conf.set(KyuubiConf.ENGINE_SHARE_LEVEL, SERVER.toString)
     conf.set(KyuubiConf.ENGINE_TYPE, FLINK_SQL.toString)
-    val appName = new EngineRef(conf, user, "grp", id, null)
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
+
+    val appName = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     assert(appName.engineSpace ===
       DiscoveryPaths.makePath(
         s"kyuubi_${KYUUBI_VERSION}_${SERVER}_${FLINK_SQL}",
@@ -146,7 +159,7 @@ trait EngineRefTests extends KyuubiFunSuite {
     assert(appName.defaultEngineName === s"kyuubi_${SERVER}_${FLINK_SQL}_${user}_default_$id")
 
     conf.set(KyuubiConf.ENGINE_SHARE_LEVEL_SUBDOMAIN.key, "abc")
-    val appName2 = new EngineRef(conf, user, "grp", id, null)
+    val appName2 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     assert(appName2.engineSpace ===
       DiscoveryPaths.makePath(
         s"kyuubi_${KYUUBI_VERSION}_${SERVER}_${FLINK_SQL}",
@@ -161,31 +174,33 @@ trait EngineRefTests extends KyuubiFunSuite {
     // set subdomain and disable engine pool
     conf.set(ENGINE_SHARE_LEVEL_SUBDOMAIN.key, "abc")
     conf.set(ENGINE_POOL_SIZE, -1)
-    val engine1 = new EngineRef(conf, user, "grp", id, null)
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
+
+    val engine1 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     assert(engine1.subdomain === "abc")
 
     // unset subdomain and disable engine pool
     conf.unset(ENGINE_SHARE_LEVEL_SUBDOMAIN)
     conf.set(ENGINE_POOL_SIZE, -1)
-    val engine2 = new EngineRef(conf, user, "grp", id, null)
+    val engine2 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     assert(engine2.subdomain === "default")
 
     // set subdomain and 1 <= engine pool size < threshold
     conf.set(ENGINE_SHARE_LEVEL_SUBDOMAIN.key, "abc")
     conf.set(ENGINE_POOL_SIZE, 1)
-    val engine3 = new EngineRef(conf, user, "grp", id, null)
+    val engine3 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     assert(engine3.subdomain === "abc")
 
     // unset subdomain and 1 <= engine pool size < threshold
     conf.unset(ENGINE_SHARE_LEVEL_SUBDOMAIN)
     conf.set(ENGINE_POOL_SIZE, 3)
-    val engine4 = new EngineRef(conf, user, "grp", id, null)
+    val engine4 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     assert(engine4.subdomain.startsWith("engine-pool-"))
 
     // unset subdomain and engine pool size > threshold
     conf.unset(ENGINE_SHARE_LEVEL_SUBDOMAIN)
     conf.set(ENGINE_POOL_SIZE, 100)
-    val engine5 = new EngineRef(conf, user, "grp", id, null)
+    val engine5 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     val engineNumber = Integer.parseInt(engine5.subdomain.substring(12))
     val threshold = ENGINE_POOL_SIZE_THRESHOLD.defaultVal.get
     assert(engineNumber <= threshold)
@@ -195,7 +210,7 @@ trait EngineRefTests extends KyuubiFunSuite {
     val enginePoolName = "test-pool"
     conf.set(ENGINE_POOL_NAME, enginePoolName)
     conf.set(ENGINE_POOL_SIZE, 3)
-    val engine6 = new EngineRef(conf, user, "grp", id, null)
+    val engine6 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     assert(engine6.subdomain.startsWith(s"$enginePoolName-"))
 
     conf.unset(ENGINE_SHARE_LEVEL_SUBDOMAIN)
@@ -206,7 +221,7 @@ trait EngineRefTests extends KyuubiFunSuite {
     conf.set(HighAvailabilityConf.HA_ADDRESSES, getConnectString())
     conf.set(ENGINE_POOL_SELECT_POLICY, "POLLING")
     (0 until (10)).foreach { i =>
-      val engine7 = new EngineRef(conf, user, "grp", id, null)
+      val engine7 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
       val engineNumber = Integer.parseInt(engine7.subdomain.substring(pool_name.length + 1))
       assert(engineNumber == (i % conf.get(ENGINE_POOL_SIZE)))
     }
@@ -219,7 +234,9 @@ trait EngineRefTests extends KyuubiFunSuite {
     conf.set(KyuubiConf.FRONTEND_THRIFT_BINARY_BIND_PORT, 0)
     conf.set(HighAvailabilityConf.HA_NAMESPACE, "engine_test")
     conf.set(HighAvailabilityConf.HA_ADDRESSES, getConnectString())
-    val engine = new EngineRef(conf, user, id, "grp", null)
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
+
+    val engine = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
 
     var port1 = 0
     var port2 = 0
@@ -261,6 +278,7 @@ trait EngineRefTests extends KyuubiFunSuite {
     conf.set(KyuubiConf.ENGINE_INIT_TIMEOUT, 3000L)
     conf.set(HighAvailabilityConf.HA_NAMESPACE, "engine_test2")
     conf.set(HighAvailabilityConf.HA_ADDRESSES, getConnectString())
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
 
     val beforeEngines = MetricsSystem.counterValue(ENGINE_TOTAL).getOrElse(0L)
     val start = System.currentTimeMillis()
@@ -272,7 +290,12 @@ trait EngineRefTests extends KyuubiFunSuite {
         executor.execute(() => {
           DiscoveryClientProvider.withDiscoveryClient(cloned) { client =>
             try {
-              new EngineRef(cloned, user, "grp", id, null).getOrCreate(client)
+              new EngineRef(
+                cloned,
+                user,
+                PluginLoader.loadGroupProvider(conf),
+                id,
+                null).getOrCreate(client)
             } finally {
               times(i) = System.currentTimeMillis()
             }
@@ -300,20 +323,22 @@ trait EngineRefTests extends KyuubiFunSuite {
     conf.set(ENGINE_SHARE_LEVEL_SUBDOMAIN.key, "abc")
     conf.set(ENGINE_POOL_IGNORE_SUBDOMAIN, false)
     conf.set(ENGINE_POOL_SIZE, -1)
-    val engine1 = new EngineRef(conf, user, "grp", id, null)
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
+
+    val engine1 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     assert(engine1.subdomain === "abc")
 
     conf.set(ENGINE_POOL_SIZE, 1)
-    val engine2 = new EngineRef(conf, user, "grp", id, null)
+    val engine2 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     assert(engine2.subdomain === "abc")
 
     conf.unset(ENGINE_SHARE_LEVEL_SUBDOMAIN)
-    val engine3 = new EngineRef(conf, user, "grp", id, null)
+    val engine3 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     assert(engine3.subdomain.startsWith("engine-pool-"))
 
     conf.set(ENGINE_SHARE_LEVEL_SUBDOMAIN.key, "abc")
     conf.set(ENGINE_POOL_IGNORE_SUBDOMAIN, true)
-    val engine4 = new EngineRef(conf, user, "grp", id, null)
+    val engine4 = new EngineRef(conf, user, PluginLoader.loadGroupProvider(conf), id, null)
     assert(engine4.subdomain.startsWith("engine-pool-"))
   }
 }
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/EngineRefWithZookeeperSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/EngineRefWithZookeeperSuite.scala
index 8695e13c4..40fc81870 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/EngineRefWithZookeeperSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/engine/EngineRefWithZookeeperSuite.scala
@@ -29,6 +29,7 @@ import org.apache.kyuubi.engine.EngineType.SPARK_SQL
 import org.apache.kyuubi.engine.ShareLevel.USER
 import org.apache.kyuubi.ha.HighAvailabilityConf
 import org.apache.kyuubi.ha.client.DiscoveryClientProvider
+import org.apache.kyuubi.plugin.PluginLoader
 import org.apache.kyuubi.zookeeper.EmbeddedZookeeper
 import org.apache.kyuubi.zookeeper.ZookeeperConf
 
@@ -62,6 +63,8 @@ class EngineRefWithZookeeperSuite extends EngineRefTests {
     conf.set(KyuubiConf.ENGINE_INIT_TIMEOUT, 3000L)
     conf.set(HighAvailabilityConf.HA_NAMESPACE, "engine_test1")
     conf.set(HighAvailabilityConf.HA_ADDRESSES, getConnectString())
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
+
     val conf1 = conf.clone
     conf1.set(KyuubiConf.ENGINE_TYPE, SPARK_SQL.toString)
     val conf2 = conf.clone
@@ -74,7 +77,12 @@ class EngineRefWithZookeeperSuite extends EngineRefTests {
       executor.execute(() => {
         DiscoveryClientProvider.withDiscoveryClient(conf1) { client =>
           try {
-            new EngineRef(conf1, user, "grp", UUID.randomUUID().toString, null)
+            new EngineRef(
+              conf1,
+              user,
+              PluginLoader.loadGroupProvider(conf),
+              UUID.randomUUID().toString,
+              null)
               .getOrCreate(client)
           } finally {
             times(0) = System.currentTimeMillis()
@@ -84,7 +92,12 @@ class EngineRefWithZookeeperSuite extends EngineRefTests {
       executor.execute(() => {
         DiscoveryClientProvider.withDiscoveryClient(conf2) { client =>
           try {
-            new EngineRef(conf2, user, "grp", UUID.randomUUID().toString, null)
+            new EngineRef(
+              conf2,
+              user,
+              PluginLoader.loadGroupProvider(conf),
+              UUID.randomUUID().toString,
+              null)
               .getOrCreate(client)
           } finally {
             times(1) = System.currentTimeMillis()
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/AdminResourceSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/AdminResourceSuite.scala
index 0e2eb1094..e7281dc5a 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/AdminResourceSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/api/v1/AdminResourceSuite.scala
@@ -36,6 +36,7 @@ import org.apache.kyuubi.engine.ShareLevel.{CONNECTION, USER}
 import org.apache.kyuubi.ha.HighAvailabilityConf
 import org.apache.kyuubi.ha.client.DiscoveryClientProvider.withDiscoveryClient
 import org.apache.kyuubi.ha.client.DiscoveryPaths
+import org.apache.kyuubi.plugin.PluginLoader
 import org.apache.kyuubi.server.http.authentication.AuthenticationHandler.AUTHORIZATION_HEADER
 
 class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper {
@@ -275,7 +276,10 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper {
     conf.set(KyuubiConf.FRONTEND_THRIFT_BINARY_BIND_PORT, 0)
     conf.set(HighAvailabilityConf.HA_NAMESPACE, "kyuubi_test")
     conf.set(KyuubiConf.ENGINE_IDLE_TIMEOUT, 180000L)
-    val engine = new EngineRef(conf.clone, Utils.currentUser, "grp", id, null)
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
+
+    val engine =
+      new EngineRef(conf.clone, Utils.currentUser, PluginLoader.loadGroupProvider(conf), id, null)
 
     val engineSpace = DiscoveryPaths.makePath(
       s"kyuubi_test_${KYUUBI_VERSION}_USER_SPARK_SQL",
@@ -320,9 +324,11 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper {
     conf.set(KyuubiConf.FRONTEND_THRIFT_BINARY_BIND_PORT, 0)
     conf.set(HighAvailabilityConf.HA_NAMESPACE, "kyuubi_test")
     conf.set(KyuubiConf.ENGINE_IDLE_TIMEOUT, 180000L)
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
 
     val id = UUID.randomUUID().toString
-    val engine = new EngineRef(conf.clone, Utils.currentUser, "grp", id, null)
+    val engine =
+      new EngineRef(conf.clone, Utils.currentUser, PluginLoader.loadGroupProvider(conf), id, null)
     val engineSpace = DiscoveryPaths.makePath(
       s"kyuubi_test_${KYUUBI_VERSION}_CONNECTION_SPARK_SQL",
       Utils.currentUser,
@@ -358,7 +364,10 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper {
     conf.set(KyuubiConf.FRONTEND_THRIFT_BINARY_BIND_PORT, 0)
     conf.set(HighAvailabilityConf.HA_NAMESPACE, "kyuubi_test")
     conf.set(KyuubiConf.ENGINE_IDLE_TIMEOUT, 180000L)
-    val engine = new EngineRef(conf.clone, Utils.currentUser, id, "grp", null)
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
+
+    val engine =
+      new EngineRef(conf.clone, Utils.currentUser, PluginLoader.loadGroupProvider(conf), id, null)
 
     val engineSpace = DiscoveryPaths.makePath(
       s"kyuubi_test_${KYUUBI_VERSION}_USER_SPARK_SQL",
@@ -403,6 +412,7 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper {
     conf.set(KyuubiConf.FRONTEND_THRIFT_BINARY_BIND_PORT, 0)
     conf.set(HighAvailabilityConf.HA_NAMESPACE, "kyuubi_test")
     conf.set(KyuubiConf.ENGINE_IDLE_TIMEOUT, 180000L)
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
 
     val engineSpace = DiscoveryPaths.makePath(
       s"kyuubi_test_${KYUUBI_VERSION}_CONNECTION_SPARK_SQL",
@@ -410,14 +420,16 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper {
       "")
 
     val id1 = UUID.randomUUID().toString
-    val engine1 = new EngineRef(conf.clone, Utils.currentUser, "grp", id1, null)
+    val engine1 =
+      new EngineRef(conf.clone, Utils.currentUser, PluginLoader.loadGroupProvider(conf), id1, null)
     val engineSpace1 = DiscoveryPaths.makePath(
       s"kyuubi_test_${KYUUBI_VERSION}_CONNECTION_SPARK_SQL",
       Utils.currentUser,
       id1)
 
     val id2 = UUID.randomUUID().toString
-    val engine2 = new EngineRef(conf.clone, Utils.currentUser, "grp", id2, null)
+    val engine2 =
+      new EngineRef(conf.clone, Utils.currentUser, PluginLoader.loadGroupProvider(conf), id2, null)
     val engineSpace2 = DiscoveryPaths.makePath(
       s"kyuubi_test_${KYUUBI_VERSION}_CONNECTION_SPARK_SQL",
       Utils.currentUser,
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminCtlSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminCtlSuite.scala
index f7cbb2001..389b67e47 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminCtlSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminCtlSuite.scala
@@ -26,6 +26,7 @@ import org.apache.kyuubi.engine.EngineRef
 import org.apache.kyuubi.ha.HighAvailabilityConf
 import org.apache.kyuubi.ha.client.DiscoveryClientProvider.withDiscoveryClient
 import org.apache.kyuubi.ha.client.DiscoveryPaths
+import org.apache.kyuubi.plugin.PluginLoader
 
 class AdminCtlSuite extends RestClientTestHelper with TestPrematureExit {
   override def beforeAll(): Unit = {
@@ -53,8 +54,10 @@ class AdminCtlSuite extends RestClientTestHelper with TestPrematureExit {
     conf.set(HighAvailabilityConf.HA_NAMESPACE, "kyuubi_test")
     conf.set(KyuubiConf.ENGINE_IDLE_TIMEOUT, 180000L)
     conf.set(KyuubiConf.AUTHENTICATION_METHOD, Seq("LDAP", "CUSTOM"))
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
+
     val user = ldapUser
-    val engine = new EngineRef(conf.clone, user, "grp", id, null)
+    val engine = new EngineRef(conf.clone, user, PluginLoader.loadGroupProvider(conf), id, null)
 
     val engineSpace = DiscoveryPaths.makePath(
       s"kyuubi_test_${KYUUBI_VERSION}_USER_SPARK_SQL",
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminRestApiSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminRestApiSuite.scala
index 5165573ed..b79e62a12 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminRestApiSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminRestApiSuite.scala
@@ -30,6 +30,7 @@ import org.apache.kyuubi.engine.EngineRef
 import org.apache.kyuubi.ha.HighAvailabilityConf
 import org.apache.kyuubi.ha.client.DiscoveryClientProvider.withDiscoveryClient
 import org.apache.kyuubi.ha.client.DiscoveryPaths
+import org.apache.kyuubi.plugin.PluginLoader
 
 class AdminRestApiSuite extends RestClientTestHelper {
   test("refresh kyuubi server hadoop conf") {
@@ -48,8 +49,9 @@ class AdminRestApiSuite extends RestClientTestHelper {
     conf.set(HighAvailabilityConf.HA_NAMESPACE, "kyuubi_test")
     conf.set(KyuubiConf.ENGINE_IDLE_TIMEOUT, 180000L)
     conf.set(KyuubiConf.AUTHENTICATION_METHOD, Seq("LDAP", "CUSTOM"))
+    conf.set(KyuubiConf.GROUP_PROVIDER, "hadoop")
     val user = ldapUser
-    val engine = new EngineRef(conf.clone, user, "grp", id, null)
+    val engine = new EngineRef(conf.clone, user, PluginLoader.loadGroupProvider(conf), id, null)
 
     val engineSpace = DiscoveryPaths.makePath(
       s"kyuubi_test_${KYUUBI_VERSION}_USER_SPARK_SQL",