You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by fe...@apache.org on 2022/04/24 13:42:58 UTC

[incubator-kyuubi] branch master updated: [KYUUBI #1936][FOLLOWUP] Stop updating credentials when credentials are expired

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

feiwang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-kyuubi.git


The following commit(s) were added to refs/heads/master by this push:
     new 5ae7c9c0c [KYUUBI #1936][FOLLOWUP] Stop updating credentials when credentials are expired
5ae7c9c0c is described below

commit 5ae7c9c0c6a77bed940f0a093a5fb927f6b782df
Author: Tianlin Liao <ti...@ebay.com>
AuthorDate: Sun Apr 24 21:42:52 2022 +0800

    [KYUUBI #1936][FOLLOWUP] Stop updating credentials when credentials are expired
    
    ### _Why are the changes needed?_
    
    This is the follow-up for the comments in https://github.com/apache/incubator-kyuubi/pull/2210
    
    ------
    But considering the following scenario, I think we should also check whether userRef in `userCredentialsRefMap` has been replaced by a new one .
    
    `userRef` was waiting for next renewal.
    `userRef` expired and was removed from `userCredentialsRefMap`
    A new `userRef` was created because of user visiting.
    Renewal of old userRef started again.
    If we do not check, both old and new userRef will renew periodically.
    
    modified the code and added test case for the scenario.
    
    ### _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
    
    Closes #2459 from lightning-L/kyuubi-1936.
    
    Closes #1936
    
    0cef07f0 [Tianlin Liao] [KYUUBI #1936][FOLLOWUP] stop updating credentials when credentials are expired
    
    Authored-by: Tianlin Liao <ti...@ebay.com>
    Signed-off-by: Fei Wang <fw...@ebay.com>
---
 .../apache/kyuubi/credentials/CredentialsRef.scala |  4 +++
 .../credentials/HadoopCredentialsManager.scala     | 33 +++++++++++++---------
 .../HadoopCredentialsManagerSuite.scala            | 12 ++++++++
 3 files changed, 35 insertions(+), 14 deletions(-)

diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/CredentialsRef.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/CredentialsRef.scala
index c15f7f999..077b5023a 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/CredentialsRef.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/CredentialsRef.scala
@@ -49,6 +49,10 @@ class CredentialsRef(appUser: String) {
 
   def getLastAccessTime: Long = lastAccessTime
 
+  def getNoOperationTime: Long = {
+    System.currentTimeMillis() - lastAccessTime
+  }
+
   def getEpoch: Long = epoch
 
   def getAppUser: String = appUser
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/HadoopCredentialsManager.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/HadoopCredentialsManager.scala
index d439f9524..c005de5d8 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/HadoopCredentialsManager.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/HadoopCredentialsManager.scala
@@ -91,6 +91,9 @@ class HadoopCredentialsManager private (name: String) extends AbstractService(na
   private var credentialsWaitTimeout: Long = _
   private var hadoopConf: Configuration = _
 
+  private var credentialsCheckInterval: Long = _
+  private var credentialsTimeout: Long = _
+
   private[credentials] var renewalExecutor: Option[ScheduledExecutorService] = None
   private[credentials] var credentialsTimeoutChecker: Option[ScheduledExecutorService] = None
 
@@ -122,6 +125,10 @@ class HadoopCredentialsManager private (name: String) extends AbstractService(na
     renewalInterval = conf.get(CREDENTIALS_RENEWAL_INTERVAL)
     renewalRetryWait = conf.get(CREDENTIALS_RENEWAL_RETRY_WAIT)
     credentialsWaitTimeout = conf.get(CREDENTIALS_UPDATE_WAIT_TIMEOUT)
+
+    credentialsCheckInterval = conf.get(CREDENTIALS_CHECK_INTERVAL)
+    credentialsTimeout = conf.get(CREDENTIALS_IDLE_TIMEOUT)
+
     super.initialize(conf)
   }
 
@@ -140,12 +147,10 @@ class HadoopCredentialsManager private (name: String) extends AbstractService(na
   override def stop(): Unit = {
     providers.values.foreach(_.close())
     renewalExecutor.foreach { executor =>
-      executor.shutdownNow()
-      try {
-        executor.awaitTermination(10, TimeUnit.SECONDS)
-      } catch {
-        case _: InterruptedException =>
-      }
+      ThreadUtils.shutdown(executor, Duration(10, TimeUnit.SECONDS))
+    }
+    credentialsTimeoutChecker.foreach { executor =>
+      ThreadUtils.shutdown(executor, Duration(10, TimeUnit.SECONDS))
     }
     super.stop()
   }
@@ -246,7 +251,7 @@ class HadoopCredentialsManager private (name: String) extends AbstractService(na
         try {
           promise.trySuccess(updateCredentials(userRef))
 
-          if (userCredentialsRefMap.containsKey(userRef.getAppUser)) {
+          if (userRef.getNoOperationTime < credentialsTimeout) {
             scheduleRenewal(userRef, renewalInterval)
           }
         } catch {
@@ -257,7 +262,7 @@ class HadoopCredentialsManager private (name: String) extends AbstractService(na
               s"Failed to update tokens for ${userRef.getAppUser}, try again in" +
                 s" $renewalRetryWait ms",
               e)
-            if (userCredentialsRefMap.containsKey(userRef.getAppUser)) {
+            if (userRef.getNoOperationTime < credentialsTimeout) {
               scheduleRenewal(userRef, renewalRetryWait)
             }
 
@@ -277,14 +282,10 @@ class HadoopCredentialsManager private (name: String) extends AbstractService(na
   }
 
   private def startTimeoutChecker(): Unit = {
-    val interval = conf.get(CREDENTIALS_CHECK_INTERVAL)
-    val timeout = conf.get(CREDENTIALS_IDLE_TIMEOUT)
-
     val checkTask = new Runnable {
       override def run(): Unit = {
-        val current = System.currentTimeMillis
         for ((user, userCred) <- userCredentialsRefMap.asScala) {
-          if (userCred.getLastAccessTime + timeout <= current) {
+          if (userCred.getNoOperationTime >= credentialsTimeout) {
             userCredentialsRefMap.remove(user)
           }
         }
@@ -292,7 +293,11 @@ class HadoopCredentialsManager private (name: String) extends AbstractService(na
     }
 
     credentialsTimeoutChecker.foreach { executor =>
-      executor.scheduleWithFixedDelay(checkTask, interval, interval, TimeUnit.MILLISECONDS)
+      executor.scheduleWithFixedDelay(
+        checkTask,
+        credentialsCheckInterval,
+        credentialsCheckInterval,
+        TimeUnit.MILLISECONDS)
     }
   }
 
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/credentials/HadoopCredentialsManagerSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/credentials/HadoopCredentialsManagerSuite.scala
index 5c4c2331c..a714f8c55 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/credentials/HadoopCredentialsManagerSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/credentials/HadoopCredentialsManagerSuite.scala
@@ -158,6 +158,18 @@ class HadoopCredentialsManagerSuite extends KyuubiFunSuite {
       eventually(timeout(9000.milliseconds), interval(100.milliseconds)) {
         assert(manager.userCredentialsRefMap.size == 0)
       }
+
+      // New userRef is created
+      val newUserRef = manager.getOrCreateUserCredentialsRef(appUser)
+      assert(manager.userCredentialsRefMap.size == 1)
+
+      // Old renewal schedule is stopped
+      val epoch = userRef.getEpoch
+      Thread.sleep(2000L)
+      assert(userRef.getEpoch == epoch)
+
+      // New renewal schedule is running
+      assert(newUserRef.getEpoch >= 1)
     }
   }