[ https://issues.apache.org/jira/browse/HDFS-16455?focusedWorklogId=727325&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-727325 ]
ASF GitHub Bot logged work on HDFS-16455: ----------------------------------------- Author: ASF GitHub Bot Created on: 15/Feb/22 18:40 Start Date: 15/Feb/22 18:40 Worklog Time Spent: 10m Work Description: goiri commented on a change in pull request #3983: URL: https://github.com/apache/hadoop/pull/3983#discussion_r806077863 ########## File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java ########## @@ -98,6 +98,8 @@ + "kerberos.keytab"; public static final String ZK_DTSM_ZK_KERBEROS_PRINCIPAL = ZK_CONF_PREFIX + "kerberos.principal"; + public static final String ZK_DTSM_ZK_JUTE_MAXBUFFER = ZK_CONF_PREFIX + + "jute.maxbuffer"; Review comment: The indentation is not correct. Check the checkstyle. ########## File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java ########## @@ -199,6 +202,10 @@ public ZKDelegationTokenSecretManager(Configuration conf) { ZK_DTSM_ZK_SESSION_TIMEOUT_DEFAULT); int numRetries = conf.getInt(ZK_DTSM_ZK_NUM_RETRIES, ZK_DTSM_ZK_NUM_RETRIES_DEFAULT); + String juteMaxBuffer = + conf.get(ZK_DTSM_ZK_JUTE_MAXBUFFER, ZK_DTSM_ZK_JUTE_MAXBUFFER_DEFAULT); Review comment: Indentation fix. ########## File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java ########## @@ -199,6 +202,10 @@ public ZKDelegationTokenSecretManager(Configuration conf) { ZK_DTSM_ZK_SESSION_TIMEOUT_DEFAULT); int numRetries = conf.getInt(ZK_DTSM_ZK_NUM_RETRIES, ZK_DTSM_ZK_NUM_RETRIES_DEFAULT); + String juteMaxBuffer = + conf.get(ZK_DTSM_ZK_JUTE_MAXBUFFER, ZK_DTSM_ZK_JUTE_MAXBUFFER_DEFAULT); + System.setProperty(ZKClientConfig.JUTE_MAXBUFFER, + juteMaxBuffer); Review comment: This could go to the previous line. -- 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: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 727325) Time Spent: 1h 40m (was: 1.5h) > RBF: Router should explicitly specify the value of `jute.maxbuffer` in hadoop > configuration files like core-site.xml > -------------------------------------------------------------------------------------------------------------------- > > Key: HDFS-16455 > URL: https://issues.apache.org/jira/browse/HDFS-16455 > Project: Hadoop HDFS > Issue Type: Bug > Components: rbf > Affects Versions: 3.3.0, 3.4.0 > Reporter: Max Xie > Assignee: Max Xie > Priority: Minor > Labels: pull-request-available > Time Spent: 1h 40m > Remaining Estimate: 0h > > Based on the current design for delegation token in secure Router, the total > number of tokens store and update in zookeeper using > ZKDelegationTokenManager. > But the default value of system property `jute.maxbuffer` is just 4MB, if > Router store too many tokens in zk, it will throw IOException `{{{}Packet > lenxx is out of range{}}}` and all Router will crash. > > In our cluster, Routers crashed because of it. The crash logs are below > {code:java} > 2022-02-09 02:15:51,607 INFO > org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager: > Token renewal for identifier: (token for xxx: HDFS_DELEGATION_TOKEN > owner=xxx/scheduler, renewer=hadoop, realUser=, issueDate=1644344146305, > maxDate=1644948946305, sequenceNumbe > r=27136070, masterKeyId=1107); total currentTokens 279548 2022-02-09 > 02:16:07,632 WARN org.apache.zookeeper.ClientCnxn: Session 0x1000172775a0012 > for server zkurl:2181, unexpected e > rror, closing socket connection and attempting reconnect > java.io.IOException: Packet len4194553 is out of range! > at org.apache.zookeeper.ClientCnxnSocket.readLength(ClientCnxnSocket.java:113) > at org.apache.zookeeper.ClientCnxnSocketNIO.doIO(ClientCnxnSocketNIO.java:79) > at > org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:366) > at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1145) > 2022-02-09 02:16:07,733 WARN org.apache.hadoop.ipc.Server: IPC Server handler > 1254 on default port 9001, call Call#144 Retry#0 > org.apache.hadoop.hdfs.protocol.ClientProtocol.getDelegationToken from > ip:46534 > java.lang.RuntimeException: Could not increment shared counter !! > at > org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager.incrementDelegationTokenSeqNum(ZKDelegationTokenSecretManager.java:582) > {code} > When we restart a Router, it crashed again > {code:java} > 2022-02-09 03:14:17,308 INFO > org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager: > Starting to load key cache. > 2022-02-09 03:14:17,310 INFO > org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager: > Loaded key cache. > 2022-02-09 03:14:32,930 WARN org.apache.zookeeper.ClientCnxn: Session > 0x205584be35b0001 for server zkurl:2181, unexpected > error, closing socket connection and attempting reconnect > java.io.IOException: Packet len4194478 is out of range! > at > org.apache.zookeeper.ClientCnxnSocket.readLength(ClientCnxnSocket.java:113) > at > org.apache.zookeeper.ClientCnxnSocketNIO.doIO(ClientCnxnSocketNIO.java:79) > at > org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:366) > at > org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1145) > 2022-02-09 03:14:33,030 ERROR > org.apache.hadoop.hdfs.server.federation.router.security.token.ZKDelegationTokenSecretManagerImpl: > Error starting threads for z > kDelegationTokens > java.io.IOException: Could not start PathChildrenCache for tokens {code} > Finnally, we config `-Djute.maxbuffer=10000000` in hadoop-env,sh to fix this > issue. > After dig it, we found the number of the znode > `/ZKDTSMRoot/ZKDTSMTokensRoot`'s children node was more than 250000, which's > data size was over 4MB. > > Maybe we should explicitly specify the value of `jute.maxbuffer` in hadoop > configuration files like core-site.xml, hdfs-rbf-site.xml to configure a > larger value > > -- This message was sent by Atlassian Jira (v8.20.1#820001) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org