You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@kyuubi.apache.org by GitBox <gi...@apache.org> on 2022/04/14 08:00:14 UTC

[GitHub] [incubator-kyuubi] wForget opened a new pull request, #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

wForget opened a new pull request, #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364

   <!--
   Thanks for sending a pull request!
   
   Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://kyuubi.readthedocs.io/en/latest/community/contributions.html
     2. If the PR is related to an issue in https://github.com/apache/incubator-kyuubi/issues, add '[KYUUBI #XXXX]' in your PR title, e.g., '[KYUUBI #XXXX] Your PR title ...'.
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][KYUUBI #XXXX] Your PR title ...'.
   -->
   
   ### _Why are the changes needed?_
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you add a feature, you can talk about the use case of it.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   close #2301
   
   ### _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
   
   - [ ] [Run test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests) locally before make a pull request
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] wForget commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
wForget commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851044644


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/session/SessionManager.scala:
##########
@@ -69,6 +69,12 @@ abstract class SessionManager(name: String) extends CompositeService(name) {
   private val timeoutChecker =
     ThreadUtils.newDaemonSingleThreadScheduledExecutor(s"$name-timeout-checker")
 
+  @volatile private var _limiter: Option[SessionLimiter] = None
+
+  protected def setSessionLimiter(limiter: SessionLimiter): Unit = {

Review Comment:
   > move all the logic from common to server would be more clear
   
   OK, thanks. I'll do that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] ulysses-you commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851100736


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/session/SessionLimiter.scala:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.session
+
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicInteger
+
+import org.apache.commons.lang3.StringUtils
+
+import org.apache.kyuubi.KyuubiSQLException
+
+trait SessionLimiter {
+  def increment(userIpAddress: UserIpAddress): Unit
+  def decrement(userIpAddress: UserIpAddress): Unit
+}
+
+case class UserIpAddress(user: String, ipAddress: String)
+
+class SessionLimiterImpl(userLimit: Int, ipAddressLimit: Int, userIpAddressLimit: Int)
+  extends SessionLimiter {
+
+  private val _counters: java.util.Map[String, AtomicInteger] =
+    new ConcurrentHashMap[String, AtomicInteger]()
+
+  private[session] def counters(): java.util.Map[String, AtomicInteger] = _counters
+
+  override def increment(userIpAddress: UserIpAddress): Unit = {
+    val user = userIpAddress.user
+    val ipAddress = userIpAddress.ipAddress
+    // increment userIpAddress count
+    if (ipAddressLimit > 0 && StringUtils.isNotBlank(user) && StringUtils.isNotBlank(ipAddress)) {
+      incrLimitCount(
+        s"$user:$ipAddress",
+        userIpAddressLimit,
+        s"Connection limit per user:ipaddress reached" +

Review Comment:
   remove `s`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#issuecomment-1101972773

   thanks, merged to master


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851030271


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/session/SessionManager.scala:
##########
@@ -69,6 +69,12 @@ abstract class SessionManager(name: String) extends CompositeService(name) {
   private val timeoutChecker =
     ThreadUtils.newDaemonSingleThreadScheduledExecutor(s"$name-timeout-checker")
 
+  @volatile private var _limiter: Option[SessionLimiter] = None
+
+  protected def setSessionLimiter(limiter: SessionLimiter): Unit = {

Review Comment:
    shall we add this only in the server?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] wForget commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
wForget commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851104715


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/session/SessionLimiter.scala:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.session
+
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicInteger
+
+import org.apache.commons.lang3.StringUtils
+
+import org.apache.kyuubi.KyuubiSQLException
+
+trait SessionLimiter {
+  def increment(userIpAddress: UserIpAddress): Unit
+  def decrement(userIpAddress: UserIpAddress): Unit
+}
+
+case class UserIpAddress(user: String, ipAddress: String)
+
+class SessionLimiterImpl(userLimit: Int, ipAddressLimit: Int, userIpAddressLimit: Int)
+  extends SessionLimiter {
+
+  private val _counters: java.util.Map[String, AtomicInteger] =
+    new ConcurrentHashMap[String, AtomicInteger]()
+
+  private[session] def counters(): java.util.Map[String, AtomicInteger] = _counters
+
+  override def increment(userIpAddress: UserIpAddress): Unit = {
+    val user = userIpAddress.user
+    val ipAddress = userIpAddress.ipAddress
+    // increment userIpAddress count
+    if (ipAddressLimit > 0 && StringUtils.isNotBlank(user) && StringUtils.isNotBlank(ipAddress)) {
+      incrLimitCount(
+        s"$user:$ipAddress",
+        userIpAddressLimit,
+        s"Connection limit per user:ipaddress reached" +
+          s" (user:ipaddress: $user:$ipAddress limit: $userIpAddressLimit)")
+    }
+    // increment user count
+    if (userLimit > 0 && StringUtils.isNotBlank(user)) {
+      incrLimitCount(
+        user,
+        userLimit,
+        s"Connection limit per user reached (user: $user limit: $userLimit)")
+    }
+    // increment ipAddress count
+    if (ipAddressLimit > 0 && StringUtils.isNotBlank(ipAddress)) {
+      incrLimitCount(
+        ipAddress,
+        ipAddressLimit,
+        s"Connection limit per ipaddress reached (ipaddress: $ipAddress limit: $ipAddressLimit)")
+    }
+  }
+
+  override def decrement(userIpAddress: UserIpAddress): Unit = {
+    val user = userIpAddress.user
+    val ipAddress = userIpAddress.ipAddress
+    // decrement user count
+    if (userLimit > 0 && StringUtils.isNotBlank(user)) {
+      decrLimitCount(user)
+    }
+    // decrement ipAddress count
+    if (ipAddressLimit > 0 && StringUtils.isNotBlank(ipAddress)) {
+      decrLimitCount(ipAddress)
+    }
+    // decrement userIpAddress count
+    if (ipAddressLimit > 0 && StringUtils.isNotBlank(user) && StringUtils.isNotBlank(ipAddress)) {
+      decrLimitCount(s"$user:$ipAddress")
+    }
+  }
+
+  private def incrLimitCount(key: String, limit: Int, errorMsg: String): Unit = {
+    val count = _counters.computeIfAbsent(key, _ => new AtomicInteger())
+    if (count.incrementAndGet() > limit) {
+      count.decrementAndGet()

Review Comment:
   > shall we syncronized the whole method ? we should make sure the `increment and decrement` atomic
   
   I don't think it's necessary, we just need to make sure that increment and comparison operations are atomic.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] wForget commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
wForget commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851089523


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/session/SessionManager.scala:
##########
@@ -69,6 +69,12 @@ abstract class SessionManager(name: String) extends CompositeService(name) {
   private val timeoutChecker =
     ThreadUtils.newDaemonSingleThreadScheduledExecutor(s"$name-timeout-checker")
 
+  @volatile private var _limiter: Option[SessionLimiter] = None

Review Comment:
   > we do not expose this value, so `limiter` is enough
   
   thanks, i have fixed it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] wForget commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
wForget commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851035614


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/session/SessionManager.scala:
##########
@@ -69,6 +69,12 @@ abstract class SessionManager(name: String) extends CompositeService(name) {
   private val timeoutChecker =
     ThreadUtils.newDaemonSingleThreadScheduledExecutor(s"$name-timeout-checker")
 
+  @volatile private var _limiter: Option[SessionLimiter] = None
+
+  protected def setSessionLimiter(limiter: SessionLimiter): Unit = {

Review Comment:
   I just set the limiter in KyuubiSessionManager, so do I need to make some changes here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] wForget commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
wForget commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851056104


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/session/SessionManager.scala:
##########
@@ -69,6 +69,12 @@ abstract class SessionManager(name: String) extends CompositeService(name) {
   private val timeoutChecker =
     ThreadUtils.newDaemonSingleThreadScheduledExecutor(s"$name-timeout-checker")
 
+  @volatile private var _limiter: Option[SessionLimiter] = None
+
+  protected def setSessionLimiter(limiter: SessionLimiter): Unit = {

Review Comment:
   @yaooqinn I need to decrement limit after close session, in KyuubiSessionManager's closeSession method we can't seem to implement it well. Because the session will be removed from handleToSession after closeSession.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] ulysses-you commented on pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#issuecomment-1101043019

   @wForget please resolve the conflicts, thank you


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] wForget commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
wForget commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851087996


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/session/SessionManager.scala:
##########
@@ -69,6 +69,12 @@ abstract class SessionManager(name: String) extends CompositeService(name) {
   private val timeoutChecker =
     ThreadUtils.newDaemonSingleThreadScheduledExecutor(s"$name-timeout-checker")
 
+  @volatile private var _limiter: Option[SessionLimiter] = None
+
+  protected def setSessionLimiter(limiter: SessionLimiter): Unit = {

Review Comment:
   > @yaooqinn I need to decrement limit after close session, in KyuubiSessionManager's closeSession method we can't seem to implement it well. Because the session will be removed from handleToSession after closeSession.
   
   Sorry, please ignore this comment, I have moved it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851840012


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionManager.scala:
##########
@@ -43,10 +43,13 @@ class KyuubiSessionManager private (name: String) extends SessionManager(name) {
   // this lazy is must be specified since the conf is null when the class initialization
   lazy val sessionConfAdvisor: SessionConfAdvisor = PluginLoader.loadSessionConfAdvisor(conf)
 
+  @volatile private var limiter: Option[SessionLimiter] = None

Review Comment:
   volatile necessary?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn closed pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
yaooqinn closed pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress
URL: https://github.com/apache/incubator-kyuubi/pull/2364


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#issuecomment-1100002188

   A general question,does it possible mix with metrics


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] wForget commented on pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
wForget commented on PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#issuecomment-1100035497

   > A general question,does it possible mix with metrics
   
   Sounds good, do we need to ensure atomicity of increment and comparison operations on metric counters?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] wForget commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
wForget commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851853375


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionManager.scala:
##########
@@ -43,10 +43,13 @@ class KyuubiSessionManager private (name: String) extends SessionManager(name) {
   // this lazy is must be specified since the conf is null when the class initialization
   lazy val sessionConfAdvisor: SessionConfAdvisor = PluginLoader.loadSessionConfAdvisor(conf)
 
+  @volatile private var limiter: Option[SessionLimiter] = None

Review Comment:
   It is unnecessary, I will remove it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] wForget commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
wForget commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851031395


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/session/SessionManager.scala:
##########
@@ -69,6 +69,12 @@ abstract class SessionManager(name: String) extends CompositeService(name) {
   private val timeoutChecker =
     ThreadUtils.newDaemonSingleThreadScheduledExecutor(s"$name-timeout-checker")
 
+  @volatile private var _limiter: Option[SessionLimiter] = None
+
+  protected def setSessionLimiter(limiter: SessionLimiter): Unit = {

Review Comment:
   > shall we add this only in the server?
   
   I'm also not sure if we need to add it in Engine too? Since the engine can be shared maybe we can also add limiter to the engine. What do you think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851033771


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/session/SessionManager.scala:
##########
@@ -69,6 +69,12 @@ abstract class SessionManager(name: String) extends CompositeService(name) {
   private val timeoutChecker =
     ThreadUtils.newDaemonSingleThreadScheduledExecutor(s"$name-timeout-checker")
 
+  @volatile private var _limiter: Option[SessionLimiter] = None
+
+  protected def setSessionLimiter(limiter: SessionLimiter): Unit = {

Review Comment:
   Yes, maybe. It won't be late when we have plans on such a thing for engines. So let's focus on the PR's scope itself



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] yaooqinn commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851036592


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/session/SessionManager.scala:
##########
@@ -69,6 +69,12 @@ abstract class SessionManager(name: String) extends CompositeService(name) {
   private val timeoutChecker =
     ThreadUtils.newDaemonSingleThreadScheduledExecutor(s"$name-timeout-checker")
 
+  @volatile private var _limiter: Option[SessionLimiter] = None
+
+  protected def setSessionLimiter(limiter: SessionLimiter): Unit = {

Review Comment:
   move all the logic from common to server would be more clear



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] ulysses-you commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851100328


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/session/SessionLimiter.scala:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.session
+
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicInteger
+
+import org.apache.commons.lang3.StringUtils
+
+import org.apache.kyuubi.KyuubiSQLException
+
+trait SessionLimiter {
+  def increment(userIpAddress: UserIpAddress): Unit
+  def decrement(userIpAddress: UserIpAddress): Unit
+}
+
+case class UserIpAddress(user: String, ipAddress: String)
+
+class SessionLimiterImpl(userLimit: Int, ipAddressLimit: Int, userIpAddressLimit: Int)
+  extends SessionLimiter {
+
+  private val _counters: java.util.Map[String, AtomicInteger] =
+    new ConcurrentHashMap[String, AtomicInteger]()
+
+  private[session] def counters(): java.util.Map[String, AtomicInteger] = _counters
+
+  override def increment(userIpAddress: UserIpAddress): Unit = {
+    val user = userIpAddress.user
+    val ipAddress = userIpAddress.ipAddress
+    // increment userIpAddress count
+    if (ipAddressLimit > 0 && StringUtils.isNotBlank(user) && StringUtils.isNotBlank(ipAddress)) {

Review Comment:
   ipAddressLimit -> userIpAddressLimit



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] ulysses-you commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851102201


##########
kyuubi-server/src/main/scala/org/apache/kyuubi/session/SessionLimiter.scala:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.session
+
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicInteger
+
+import org.apache.commons.lang3.StringUtils
+
+import org.apache.kyuubi.KyuubiSQLException
+
+trait SessionLimiter {
+  def increment(userIpAddress: UserIpAddress): Unit
+  def decrement(userIpAddress: UserIpAddress): Unit
+}
+
+case class UserIpAddress(user: String, ipAddress: String)
+
+class SessionLimiterImpl(userLimit: Int, ipAddressLimit: Int, userIpAddressLimit: Int)
+  extends SessionLimiter {
+
+  private val _counters: java.util.Map[String, AtomicInteger] =
+    new ConcurrentHashMap[String, AtomicInteger]()
+
+  private[session] def counters(): java.util.Map[String, AtomicInteger] = _counters
+
+  override def increment(userIpAddress: UserIpAddress): Unit = {
+    val user = userIpAddress.user
+    val ipAddress = userIpAddress.ipAddress
+    // increment userIpAddress count
+    if (ipAddressLimit > 0 && StringUtils.isNotBlank(user) && StringUtils.isNotBlank(ipAddress)) {
+      incrLimitCount(
+        s"$user:$ipAddress",
+        userIpAddressLimit,
+        s"Connection limit per user:ipaddress reached" +
+          s" (user:ipaddress: $user:$ipAddress limit: $userIpAddressLimit)")
+    }
+    // increment user count
+    if (userLimit > 0 && StringUtils.isNotBlank(user)) {
+      incrLimitCount(
+        user,
+        userLimit,
+        s"Connection limit per user reached (user: $user limit: $userLimit)")
+    }
+    // increment ipAddress count
+    if (ipAddressLimit > 0 && StringUtils.isNotBlank(ipAddress)) {
+      incrLimitCount(
+        ipAddress,
+        ipAddressLimit,
+        s"Connection limit per ipaddress reached (ipaddress: $ipAddress limit: $ipAddressLimit)")
+    }
+  }
+
+  override def decrement(userIpAddress: UserIpAddress): Unit = {
+    val user = userIpAddress.user
+    val ipAddress = userIpAddress.ipAddress
+    // decrement user count
+    if (userLimit > 0 && StringUtils.isNotBlank(user)) {
+      decrLimitCount(user)
+    }
+    // decrement ipAddress count
+    if (ipAddressLimit > 0 && StringUtils.isNotBlank(ipAddress)) {
+      decrLimitCount(ipAddress)
+    }
+    // decrement userIpAddress count
+    if (ipAddressLimit > 0 && StringUtils.isNotBlank(user) && StringUtils.isNotBlank(ipAddress)) {
+      decrLimitCount(s"$user:$ipAddress")
+    }
+  }
+
+  private def incrLimitCount(key: String, limit: Int, errorMsg: String): Unit = {
+    val count = _counters.computeIfAbsent(key, _ => new AtomicInteger())
+    if (count.incrementAndGet() > limit) {
+      count.decrementAndGet()

Review Comment:
   shall we syncronized the whole method ? we should make sure the `increment and decrement` atomic



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] codecov-commenter commented on pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#issuecomment-1098865065

   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#2364](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (557954f) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/659d981c66ac98ccf1ca5a1e525053fc9417c9f6?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (659d981) will **increase** coverage by `0.06%`.
   > The diff coverage is `82.81%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2364      +/-   ##
   ============================================
   + Coverage     62.68%   62.74%   +0.06%     
     Complexity       69       69              
   ============================================
     Files           357      358       +1     
     Lines         16890    16963      +73     
     Branches       2276     2287      +11     
   ============================================
   + Hits          10587    10643      +56     
   - Misses         5321     5327       +6     
   - Partials        982      993      +11     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...g/apache/kyuubi/session/KyuubiSessionManager.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXNzaW9uL0t5dXViaVNlc3Npb25NYW5hZ2VyLnNjYWxh) | `92.30% <71.42%> (-4.57%)` | :arrow_down: |
   | [...ala/org/apache/kyuubi/session/SessionManager.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXNzaW9uL1Nlc3Npb25NYW5hZ2VyLnNjYWxh) | `87.40% <75.00%> (-0.38%)` | :arrow_down: |
   | [...ala/org/apache/kyuubi/session/SessionLimiter.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXNzaW9uL1Nlc3Npb25MaW1pdGVyLnNjYWxh) | `80.48% <80.48%> (ø)` | |
   | [...in/scala/org/apache/kyuubi/config/KyuubiConf.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9jb25maWcvS3l1dWJpQ29uZi5zY2FsYQ==) | `96.40% <100.00%> (+0.05%)` | :arrow_up: |
   | [...apache/kyuubi/engine/hive/HiveProcessBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvaGl2ZS9IaXZlUHJvY2Vzc0J1aWxkZXIuc2NhbGE=) | `84.61% <0.00%> (-6.30%)` | :arrow_down: |
   | [...ache/kyuubi/operation/KyuubiOperationManager.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9vcGVyYXRpb24vS3l1dWJpT3BlcmF0aW9uTWFuYWdlci5zY2FsYQ==) | `94.11% <0.00%> (-1.97%)` | :arrow_down: |
   | [...n/scala/org/apache/kyuubi/engine/ProcBuilder.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9lbmdpbmUvUHJvY0J1aWxkZXIuc2NhbGE=) | `81.87% <0.00%> (-0.93%)` | :arrow_down: |
   | [.../kyuubi/credentials/HadoopCredentialsManager.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9jcmVkZW50aWFscy9IYWRvb3BDcmVkZW50aWFsc01hbmFnZXIuc2NhbGE=) | `93.12% <0.00%> (-0.77%)` | :arrow_down: |
   | [.../org/apache/kyuubi/operation/KyuubiOperation.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9vcGVyYXRpb24vS3l1dWJpT3BlcmF0aW9uLnNjYWxh) | `68.00% <0.00%> (+1.33%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [659d981...557954f](https://codecov.io/gh/apache/incubator-kyuubi/pull/2364?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


[GitHub] [incubator-kyuubi] ulysses-you commented on a diff in pull request #2364: [KYUUBI #2301] Limit the maximum number of concurrent connections per user and ipaddress

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on code in PR #2364:
URL: https://github.com/apache/incubator-kyuubi/pull/2364#discussion_r851087549


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/session/SessionManager.scala:
##########
@@ -69,6 +69,12 @@ abstract class SessionManager(name: String) extends CompositeService(name) {
   private val timeoutChecker =
     ThreadUtils.newDaemonSingleThreadScheduledExecutor(s"$name-timeout-checker")
 
+  @volatile private var _limiter: Option[SessionLimiter] = None

Review Comment:
   we do not expose this value, so `limiter` is enough



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org