[
https://issues.apache.org/jira/browse/HADOOP-18851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17755940#comment-17755940
]
Vikas Kumar commented on HADOOP-18851:
--------------------------------------
Hello,
I have made the changes and uploading the patch file here for initial review
and feedback. I parallel, I am trying to do more testing.
Requesting experts to please review the change. I have created branch from
"trunk".
Changes:
1. made {*}AbstractDelegationTokenSecretManager.createPassword()
non-synchronized{*}. There is one places where it requires synchronization,
incrementDelegationTokenSeqNum() method. This method is still synchronized. But
in case of HA,
ZKDelegationTokenSecretManagerImpl.incrementDelegationTokenSeqNum() will be
invoked and this doesn't needs to be synchronized on "this" object's monitor.
As part of fix for HADOOP-16828, this method returns the pre-loaded seq number
and if it reaches the limit, it again reserves the range. This part requires to
be protected from being executed by concurrent threads, and here I used
ReentrantLock to guard this instead of this.
2. *AbstractDelegationTokenSecretManager.checkToken()* : removed the assertion
for this object's monitor lock. As now all the DS has been made thread-safe.
Read and update operations to the maps should be the issue now.
3. {*}Made AbstractDelegationTokenSecretManager.retrievePassword()
non-synchronized{*}. Here getTokenInfo() was expected to be synchronized but
since currentTokens map has been changed to thread-safe map , so this method is
thared-safe. And for overriding class ZKDelegationTokenSecretManagerImpl, it
reads from Zk that is also thread-safe.
4. {*}AbstractDelegationTokenSecretManager.{*}verifyToken(), this method also
reads from the token map that is thread-safe. Or in case of HA, reading from ZK
using CuratorFramework is thread safe.
5. ZKDelegationTokenSecretManager.incrementDelegationTokenSeqNum(): protected
this method using RentrantLock to avoid being accessed concurrently from
multiple threads.
Here intention is to make verifyToken and createPassword ( and few other
methods as well) to run concurrently without blocking each other much . In
current implementation, even multiple verifyToken() method (that only reads )
blocks each other due to "synchronized" nature of the method.
Please let me know if I have missed anything, I would be happy to learn more.
Please find below the patch:
[^0001-HADOOP-18851-Perfm-improvement-for-ZKDT-management.patch]
> 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
> 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: [email protected]
For additional commands, e-mail: [email protected]