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/02/18 17:05:03 UTC

[kyuubi] branch branch-1.7 updated: [KYUUBI #4360] Support to refresh the unlimited users for session limiter

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

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


The following commit(s) were added to refs/heads/branch-1.7 by this push:
     new 709c66454 [KYUUBI #4360] Support to refresh the unlimited users for session limiter
709c66454 is described below

commit 709c66454dfe966a57e39469b4ae862f037c6912
Author: fwang12 <fw...@ebay.com>
AuthorDate: Sat Feb 18 00:54:11 2023 +0800

    [KYUUBI #4360] Support to refresh the unlimited users for session limiter
    
    ### _Why are the changes needed?_
    
    Support to refresh the unlimited users for session limiter, so that we can unblock some customers without restart the kyuubi server.
    
    ### _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.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request
    
    Closes #4360 from turboFei/limier_whitelist.
    
    Closes #4360
    
    c846148bd [fwang12] typo
    839e71365 [fwang12] nit
    904bc9fbc [fwang12] save
    
    Authored-by: fwang12 <fw...@ebay.com>
    Signed-off-by: fwang12 <fw...@ebay.com>
---
 docs/deployment/settings.md                           |  2 +-
 docs/tools/kyuubi-admin.rst                           |  4 +++-
 .../scala/org/apache/kyuubi/config/KyuubiConf.scala   |  2 +-
 .../kyuubi/ctl/cmd/refresh/RefreshConfigCommand.scala |  4 +++-
 .../org/apache/kyuubi/ctl/opt/AdminCommandLine.scala  |  2 +-
 .../kyuubi/ctl/AdminControlCliArgumentsSuite.scala    | 15 ++++++++++++---
 .../java/org/apache/kyuubi/client/AdminRestApi.java   |  5 +++++
 .../scala/org/apache/kyuubi/server/KyuubiServer.scala | 11 +++++++++++
 .../apache/kyuubi/server/api/v1/AdminResource.scala   | 19 +++++++++++++++++++
 .../apache/kyuubi/session/KyuubiSessionManager.scala  |  5 +++++
 .../org/apache/kyuubi/session/SessionLimiter.scala    | 16 +++++++++++++---
 .../kyuubi/server/api/v1/AdminResourceSuite.scala     | 18 ++++++++++++++++++
 12 files changed, 92 insertions(+), 11 deletions(-)

diff --git a/docs/deployment/settings.md b/docs/deployment/settings.md
index 7391f4241..8d0e32436 100644
--- a/docs/deployment/settings.md
+++ b/docs/deployment/settings.md
@@ -457,7 +457,7 @@ You can configure the Kyuubi properties in `$KYUUBI_HOME/conf/kyuubi-defaults.co
 | kyuubi.server.limit.connections.per.ipaddress            | &lt;undefined&gt; | Maximum kyuubi server connections per ipaddress. Any user exceeding this limit will not be allowed to connect.                                                                                                                                 | int    | 1.6.0 |
 | kyuubi.server.limit.connections.per.user                 | &lt;undefined&gt; | Maximum kyuubi server connections per user. Any user exceeding this limit will not be allowed to connect.                                                                                                                                      | int    | 1.6.0 |
 | kyuubi.server.limit.connections.per.user.ipaddress       | &lt;undefined&gt; | Maximum kyuubi server connections per user:ipaddress combination. Any user-ipaddress exceeding this limit will not be allowed to connect.                                                                                                      | int    | 1.6.0 |
-| kyuubi.server.limit.connections.user.unlimited.list                         || The maximin connections of the user in the white list will not be limited.                                                                                                                                                                     | seq    | 1.7.0 |
+| kyuubi.server.limit.connections.user.unlimited.list                         || The maximum connections of the user in the white list will not be limited.                                                                                                                                                                     | seq    | 1.7.0 |
 | kyuubi.server.name                                       | &lt;undefined&gt; | The name of Kyuubi Server.                                                                                                                                                                                                                     | string | 1.5.0 |
 | kyuubi.server.redaction.regex                            | &lt;undefined&gt; | Regex to decide which Kyuubi contain sensitive information. When this regex matches a property key or value, the value is redacted from the various logs.                                                                                              || 1.6.0 |
 
diff --git a/docs/tools/kyuubi-admin.rst b/docs/tools/kyuubi-admin.rst
index c881277fe..606396593 100644
--- a/docs/tools/kyuubi-admin.rst
+++ b/docs/tools/kyuubi-admin.rst
@@ -70,7 +70,9 @@ Usage: ``bin/kyuubi-admin refresh config [options] [<configType>]``
    * - hadoopConf
      - The hadoop conf used for proxy user verification.
    * - userDefaultsConf
-     - Refresh the user defaults configs with key in format in the form of `___{username}___.{config key}` from default property file.
+     - The user defaults configs with key in format in the form of `___{username}___.{config key}` from default property file.
+   * - unlimitedUsers
+     - The users without maximum connections limitation.
 
 .. _list_engine:
 
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 01bd46bd3..14a05e749 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
@@ -2333,7 +2333,7 @@ object KyuubiConf {
 
   val SERVER_LIMIT_CONNECTIONS_USER_UNLIMITED_LIST: ConfigEntry[Seq[String]] =
     buildConf("kyuubi.server.limit.connections.user.unlimited.list")
-      .doc("The maximin connections of the user in the white list will not be limited.")
+      .doc("The maximum connections of the user in the white list will not be limited.")
       .version("1.7.0")
       .serverOnly
       .stringConf
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/refresh/RefreshConfigCommand.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/refresh/RefreshConfigCommand.scala
index b658c0e45..69aa0c3d0 100644
--- a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/refresh/RefreshConfigCommand.scala
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/refresh/RefreshConfigCommand.scala
@@ -21,7 +21,7 @@ import org.apache.kyuubi.KyuubiException
 import org.apache.kyuubi.client.AdminRestApi
 import org.apache.kyuubi.ctl.RestClientFactory.withKyuubiRestClient
 import org.apache.kyuubi.ctl.cmd.AdminCtlCommand
-import org.apache.kyuubi.ctl.cmd.refresh.RefreshConfigCommandConfigType.{HADOOP_CONF, USER_DEFAULTS_CONF}
+import org.apache.kyuubi.ctl.cmd.refresh.RefreshConfigCommandConfigType.{HADOOP_CONF, UNLIMITED_USERS, USER_DEFAULTS_CONF}
 import org.apache.kyuubi.ctl.opt.CliConfig
 import org.apache.kyuubi.ctl.util.{Tabulator, Validator}
 
@@ -36,6 +36,7 @@ class RefreshConfigCommand(cliConfig: CliConfig) extends AdminCtlCommand[String]
       normalizedCliConfig.adminConfigOpts.configType match {
         case HADOOP_CONF => adminRestApi.refreshHadoopConf()
         case USER_DEFAULTS_CONF => adminRestApi.refreshUserDefaultsConf()
+        case UNLIMITED_USERS => adminRestApi.refreshUnlimitedUsers()
         case configType => throw new KyuubiException(s"Invalid config type:$configType")
       }
     }
@@ -48,4 +49,5 @@ class RefreshConfigCommand(cliConfig: CliConfig) extends AdminCtlCommand[String]
 object RefreshConfigCommandConfigType {
   final val HADOOP_CONF = "hadoopConf"
   final val USER_DEFAULTS_CONF = "userDefaultsConf"
+  final val UNLIMITED_USERS = "unlimitedUsers"
 }
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/opt/AdminCommandLine.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/opt/AdminCommandLine.scala
index 59ad7f5fc..b1a70935b 100644
--- a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/opt/AdminCommandLine.scala
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/opt/AdminCommandLine.scala
@@ -102,6 +102,6 @@ object AdminCommandLine extends CommonCommandLine {
           .optional()
           .action((v, c) => c.copy(adminConfigOpts = c.adminConfigOpts.copy(configType = v)))
           .text("The valid config type can be one of the following: " +
-            s"$HADOOP_CONF, $USER_DEFAULTS_CONF."))
+            s"$HADOOP_CONF, $USER_DEFAULTS_CONF, $UNLIMITED_USERS."))
   }
 }
diff --git a/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/AdminControlCliArgumentsSuite.scala b/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/AdminControlCliArgumentsSuite.scala
index afb946e92..dab796127 100644
--- a/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/AdminControlCliArgumentsSuite.scala
+++ b/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/AdminControlCliArgumentsSuite.scala
@@ -63,7 +63,7 @@ class AdminControlCliArgumentsSuite extends KyuubiFunSuite with TestPrematureExi
     val opArgs = new AdminControlCliArguments(args)
     assert(opArgs.cliConfig.action === ControlAction.REFRESH)
     assert(opArgs.cliConfig.resource === ControlObject.CONFIG)
-    assert(opArgs.cliConfig.adminConfigOpts.configType === "hadoopConf")
+    assert(opArgs.cliConfig.adminConfigOpts.configType === HADOOP_CONF)
 
     args = Array(
       "refresh",
@@ -72,7 +72,16 @@ class AdminControlCliArgumentsSuite extends KyuubiFunSuite with TestPrematureExi
     val opArgs2 = new AdminControlCliArguments(args)
     assert(opArgs2.cliConfig.action === ControlAction.REFRESH)
     assert(opArgs2.cliConfig.resource === ControlObject.CONFIG)
-    assert(opArgs2.cliConfig.adminConfigOpts.configType === "userDefaultsConf")
+    assert(opArgs2.cliConfig.adminConfigOpts.configType === USER_DEFAULTS_CONF)
+
+    args = Array(
+      "refresh",
+      "config",
+      "unlimitedUsers")
+    val opArgs3 = new AdminControlCliArguments(args)
+    assert(opArgs3.cliConfig.action === ControlAction.REFRESH)
+    assert(opArgs3.cliConfig.resource === ControlObject.CONFIG)
+    assert(opArgs3.cliConfig.adminConfigOpts.configType === UNLIMITED_USERS)
 
     args = Array(
       "refresh",
@@ -147,7 +156,7 @@ class AdminControlCliArgumentsSuite extends KyuubiFunSuite with TestPrematureExi
          |	Refresh the resource.
          |Command: refresh config [<configType>]
          |	Refresh the config with specified type.
-         |  <configType>             The valid config type can be one of the following: $HADOOP_CONF, $USER_DEFAULTS_CONF.
+         |  <configType>             The valid config type can be one of the following: $HADOOP_CONF, $USER_DEFAULTS_CONF, $UNLIMITED_USERS.
          |
          |  -h, --help               Show help message and exit.""".stripMargin
     // scalastyle:on
diff --git a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/AdminRestApi.java b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/AdminRestApi.java
index da9782df5..b8bfe7ee1 100644
--- a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/AdminRestApi.java
+++ b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/AdminRestApi.java
@@ -44,6 +44,11 @@ public class AdminRestApi {
     return this.getClient().post(path, null, client.getAuthHeader());
   }
 
+  public String refreshUnlimitedUsers() {
+    String path = String.format("%s/%s", API_BASE_PATH, "refresh/unlimited_users");
+    return this.getClient().post(path, null, client.getAuthHeader());
+  }
+
   public String deleteEngine(
       String engineType, String shareLevel, String subdomain, String hs2ProxyUser) {
     Map<String, Object> params = new HashMap<>();
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala
index fbdaf85cc..df163bd1e 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala
@@ -33,6 +33,7 @@ import org.apache.kyuubi.ha.client.{AuthTypes, ServiceDiscovery}
 import org.apache.kyuubi.metrics.{MetricsConf, MetricsSystem}
 import org.apache.kyuubi.server.metadata.jdbc.JDBCMetadataStoreConf
 import org.apache.kyuubi.service.{AbstractBackendService, AbstractFrontendService, Serverable, ServiceState}
+import org.apache.kyuubi.session.KyuubiSessionManager
 import org.apache.kyuubi.util.{KyuubiHadoopUtils, SignalRegister}
 import org.apache.kyuubi.zookeeper.EmbeddedZookeeper
 
@@ -128,6 +129,16 @@ object KyuubiServer extends Logging {
     info(s"Refreshed user defaults configs with changes of " +
       s"unset: $unsetCount, updated: $updatedCount, added: $addedCount")
   }
+
+  private[kyuubi] def refreshUnlimitedUsers(): Unit = synchronized {
+    val existingUnlimitedUsers =
+      kyuubiServer.conf.get(KyuubiConf.SERVER_LIMIT_CONNECTIONS_USER_UNLIMITED_LIST).toSet
+    val refreshedUnlimitedUsers = KyuubiConf().loadFileDefaults().get(
+      KyuubiConf.SERVER_LIMIT_CONNECTIONS_USER_UNLIMITED_LIST).toSet
+    kyuubiServer.backendService.sessionManager.asInstanceOf[KyuubiSessionManager]
+      .refreshUnlimitedUsers(refreshedUnlimitedUsers)
+    info(s"Refreshed unlimited users from $existingUnlimitedUsers to $refreshedUnlimitedUsers")
+  }
 }
 
 class KyuubiServer(name: String) extends Serverable(name) {
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala
index e3fcc7527..6e05ee27c 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala
@@ -82,6 +82,25 @@ private[v1] class AdminResource extends ApiRequestContext with Logging {
     Response.ok(s"Refresh the user defaults conf successfully.").build()
   }
 
+  @ApiResponse(
+    responseCode = "200",
+    content = Array(new Content(mediaType = MediaType.APPLICATION_JSON)),
+    description = "refresh the unlimited users")
+  @POST
+  @Path("refresh/unlimited_users")
+  def refreshUnlimitedUser(): Response = {
+    val userName = fe.getSessionUser(Map.empty[String, String])
+    val ipAddress = fe.getIpAddress
+    info(s"Receive refresh unlimited users request from $userName/$ipAddress")
+    if (!userName.equals(administrator)) {
+      throw new NotAllowedException(
+        s"$userName is not allowed to refresh the unlimited users")
+    }
+    info(s"Reloading unlimited users")
+    KyuubiServer.refreshUnlimitedUsers()
+    Response.ok(s"Refresh the unlimited users successfully.").build()
+  }
+
   @ApiResponse(
     responseCode = "200",
     content = Array(new Content(mediaType = MediaType.APPLICATION_JSON)),
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionManager.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionManager.scala
index 54d5b8b24..be7873d31 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionManager.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionManager.scala
@@ -299,6 +299,11 @@ class KyuubiSessionManager private (name: String) extends SessionManager(name) {
       userUnlimitedList)
   }
 
+  private[kyuubi] def refreshUnlimitedUsers(userUnlimitedList: Set[String]): Unit = {
+    limiter.foreach(SessionLimiter.resetUnlimitedUsers(_, userUnlimitedList))
+    batchLimiter.foreach(SessionLimiter.resetUnlimitedUsers(_, userUnlimitedList))
+  }
+
   private def applySessionLimiter(
       userLimit: Int,
       ipAddressLimit: Int,
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/session/SessionLimiter.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/session/SessionLimiter.scala
index b7acbac3d..6cf739c39 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/session/SessionLimiter.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/session/SessionLimiter.scala
@@ -105,7 +105,7 @@ class SessionLimiterWithUnlimitedUsersImpl(
     userLimit: Int,
     ipAddressLimit: Int,
     userIpAddressLimit: Int,
-    unlimitedUsers: Set[String])
+    var unlimitedUsers: Set[String])
   extends SessionLimiterImpl(userLimit, ipAddressLimit, userIpAddressLimit) {
   override def increment(userIpAddress: UserIpAddress): Unit = {
     if (!unlimitedUsers.contains(userIpAddress.user)) {
@@ -118,6 +118,10 @@ class SessionLimiterWithUnlimitedUsersImpl(
       super.decrement(userIpAddress)
     }
   }
+
+  private[kyuubi] def setUnlimitedUsers(unlimitedUsers: Set[String]): Unit = {
+    this.unlimitedUsers = unlimitedUsers
+  }
 }
 
 object SessionLimiter {
@@ -126,12 +130,18 @@ object SessionLimiter {
       userLimit: Int,
       ipAddressLimit: Int,
       userIpAddressLimit: Int,
-      userWhiteList: Set[String] = Set.empty): SessionLimiter = {
+      unlimitedUsers: Set[String] = Set.empty): SessionLimiter = {
     new SessionLimiterWithUnlimitedUsersImpl(
       userLimit,
       ipAddressLimit,
       userIpAddressLimit,
-      userWhiteList)
+      unlimitedUsers)
   }
 
+  def resetUnlimitedUsers(limiter: SessionLimiter, unlimitedUsers: Set[String]): Unit = {
+    limiter match {
+      case l: SessionLimiterWithUnlimitedUsersImpl => l.setUnlimitedUsers(unlimitedUsers)
+      case _ =>
+    }
+  }
 }
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 bcbdad2ce..d7cd4840e 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
@@ -84,6 +84,24 @@ class AdminResourceSuite extends KyuubiFunSuite with RestFrontendTestHelper {
     assert(200 == response.getStatus)
   }
 
+  test("refresh unlimited users of the kyuubi server") {
+    var response = webTarget.path("api/v1/admin/refresh/unlimited_users")
+      .request()
+      .post(null)
+    assert(405 == response.getStatus)
+
+    val adminUser = Utils.currentUser
+    val encodeAuthorization = new String(
+      Base64.getEncoder.encode(
+        s"$adminUser:".getBytes()),
+      "UTF-8")
+    response = webTarget.path("api/v1/admin/refresh/unlimited_users")
+      .request()
+      .header(AUTHORIZATION_HEADER, s"BASIC $encodeAuthorization")
+      .post(null)
+    assert(200 == response.getStatus)
+  }
+
   test("delete engine - user share level") {
     val id = UUID.randomUUID().toString
     conf.set(KyuubiConf.ENGINE_SHARE_LEVEL, USER.toString)