[ 
https://issues.apache.org/jira/browse/HADOOP-18851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17763526#comment-17763526
 ] 

ASF GitHub Bot commented on HADOOP-18851:
-----------------------------------------

Hexiaoqiao commented on code in PR #6001:
URL: https://github.com/apache/hadoop/pull/6001#discussion_r1320937858


##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java:
##########
@@ -520,24 +525,32 @@ protected int incrementDelegationTokenSeqNum() {
     // The secret manager will keep a local range of seq num which won't be
     // seen by peers, so only when the range is exhausted it will ask zk for
     // another range again
-    if (currentSeqNum >= currentMaxSeqNum) {
-      try {
-        // after a successful batch request, we can get the range starting 
point
-        currentSeqNum = incrSharedCount(delTokSeqCounter, seqNumBatchSize);
-        currentMaxSeqNum = currentSeqNum + seqNumBatchSize;
-        LOG.info("Fetched new range of seq num, from {} to {} ",
-            currentSeqNum+1, currentMaxSeqNum);
-      } catch (InterruptedException e) {
-        // The ExpirationThread is just finishing.. so dont do anything..
-        LOG.debug(
-            "Thread interrupted while performing token counter increment", e);
-        Thread.currentThread().interrupt();
-      } catch (Exception e) {
-        throw new RuntimeException("Could not increment shared counter !!", e);
+    try{
+      this.currentSeqNumLock.lock();

Review Comment:
   @vikaskr22 Thanks for your works and sorry for the late response. After 
carefully review. I think it is possible of the race condition you mentioned 
above. And before changes, there is one `synchronized` to protect this critical 
section.
   Back to this PR, I prefer to the first version, because the performance 
could not improve obviously but with lock escalations. What do you think about?
   Let's wait if other guys would like to have some other opinions. cc 
@jojochuang 





> AbstractDelegationTokenSecretManager- Performance improvement by optimising 
> the synchronization context
> -------------------------------------------------------------------------------------------------------
>
>                 Key: HADOOP-18851
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18851
>             Project: Hadoop Common
>          Issue Type: Task
>            Reporter: Vikas Kumar
>            Assignee: Vikas Kumar
>            Priority: Major
>              Labels: pull-request-available
>         Attachments: 
> 0001-HADOOP-18851-Perfm-improvement-for-ZKDT-management.patch, Screenshot 
> 2023-08-16 at 5.36.57 PM.png
>
>
> *Context:*
> KMS depends on hadoop-common for DT management. Recently we were analysing 
> one performance issue and following is out findings:
>  # Around 96% (196 out of 200) KMS container threads were in BLOCKED state at 
> following:
>  ## *AbstractDelegationTokenSecretManager.verifyToken()*
>  ## *AbstractDelegationTokenSecretManager.createPassword()* 
>  # And then process crashed.
>  
> {code:java}
> http-nio-9292-exec-200PRIORITY : 5THREAD ID : 0X00007F075C157800NATIVE ID : 
> 0X2C87FNATIVE ID (DECIMAL) : 182399STATE : BLOCKED
> stackTrace:
> java.lang.Thread.State: BLOCKED (on object monitor)
> at 
> org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.verifyToken(AbstractDelegationTokenSecretManager.java:474)
> - waiting to lock <0x00000005f2f545e8> (a 
> org.apache.hadoop.security.token.delegation.web.DelegationTokenManager$ZKSecretManager)
> at 
> org.apache.hadoop.security.token.delegation.web.DelegationTokenManager.verifyToken(DelegationTokenManager.java:213)
> at 
> org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler.authenticate(DelegationTokenAuthenticationHandler.java:396)
> at  {code}
> All the 199 out of 200 were blocked at above point.
> And the lock they are waiting for is acquired by a thread that was trying to 
> createPassword and publishing the same on ZK.
>  
> {code:java}
> stackTrace:
> java.lang.Thread.State: WAITING (on object monitor)
> at java.lang.Object.wait(Native Method)
> at java.lang.Object.wait(Object.java:502)
> at org.apache.zookeeper.ClientCnxn.submitRequest(ClientCnxn.java:1598)
> - locked <0x0000000749263ec0> (a org.apache.zookeeper.ClientCnxn$Packet)
> at org.apache.zookeeper.ClientCnxn.submitRequest(ClientCnxn.java:1570)
> at org.apache.zookeeper.ZooKeeper.setData(ZooKeeper.java:2235)
> at 
> org.apache.curator.framework.imps.SetDataBuilderImpl$7.call(SetDataBuilderImpl.java:398)
> at 
> org.apache.curator.framework.imps.SetDataBuilderImpl$7.call(SetDataBuilderImpl.java:385)
> at org.apache.curator.RetryLoop.callWithRetry(RetryLoop.java:93)
> at 
> org.apache.curator.framework.imps.SetDataBuilderImpl.pathInForeground(SetDataBuilderImpl.java:382)
> at 
> org.apache.curator.framework.imps.SetDataBuilderImpl.forPath(SetDataBuilderImpl.java:358)
> at 
> org.apache.curator.framework.imps.SetDataBuilderImpl.forPath(SetDataBuilderImpl.java:36)
> at 
> org.apache.curator.framework.recipes.shared.SharedValue.trySetValue(SharedValue.java:201)
> at 
> org.apache.curator.framework.recipes.shared.SharedCount.trySetCount(SharedCount.java:116)
> at 
> org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager.incrSharedCount(ZKDelegationTokenSecretManager.java:586)
> at 
> org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager.incrementDelegationTokenSeqNum(ZKDelegationTokenSecretManager.java:601)
> at 
> org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.createPassword(AbstractDelegationTokenSecretManager.java:402)
> - locked <0x00000005f2f545e8> (a 
> org.apache.hadoop.security.token.delegation.web.DelegationTokenManager$ZKSecretManager)
> at 
> org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.createPassword(AbstractDelegationTokenSecretManager.java:48)
> at org.apache.hadoop.security.token.Token.<init>(Token.java:67)
> at 
> org.apache.hadoop.security.token.delegation.web.DelegationTokenManager.createToken(DelegationTokenManager.java:183)
>  {code}
> We can say that this thread is slow and has blocked remaining all. But 
> following is my observation:
>  
>  # verifyToken() and createPaswword() has been synchronized because one is 
> reading the tokenMap and another is updating the map. If it's only to protect 
> the map, then can't we simply use ConcurrentHashMap and remove the 
> "synchronized" keyword. Because due to this, all reader threads ( to 
> verifyToken()) are also blocking each other.
>  # IN HA env, It is recommended to use ZK to store DTs. We know that 
> CuratorFramework is thread safe.  
> ZKDelegationTokenSecretManager.incrementDelegationTokenSeqNum() only requires 
> to be protected from concurrent execution and it should be protected using 
> some other locks instead of "this". 
>  # With these changes, verifyToken() and createPaswword() will not block each 
> other. It will be blocked only at the time of updating the map.
>  # Similarly other methods can also be considered but these two are critical.
> I made these changes on my local and got the significant performance 
> improvement. 
> I request community to provide their input and if we agree, I can provide the 
> patch. Please let me know if any other details are required.
> Thanks. 
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to