goiri commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1023317011
########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java: ########## @@ -879,7 +879,7 @@ public void testStoreNewMasterKey() throws Exception { } @Test - public void testGetMasterKeyByDelegationKey() throws YarnException, IOException { + public void testGetMasterKeyByDelegationKey() throws Exception { Review Comment: This is correct but do we need to add all this churn? ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java: ########## @@ -121,6 +134,14 @@ * |--- RESERVATION * | |----- RESERVATION1 * | |----- RESERVATION2 + * |--- ROUTER_RM_DT_SECRET_MANAGER_ROOT + * | |----- ROUTER_RM_DELEGATION_TOKENS_ROOT + * | | |----- RM_DELEGATION_TOKEN_1 + * | | |----- RM_DELEGATION_TOKEN_2 + * | | |----- RM_DELEGATION_TOKEN_3 + * | |----- ROUTER_RM_DT_MASTER_KEYS_ROOT + * | | |----- DELEGATION_KEY_1 + * |--- |----- ROUTER_RM_DT_SEQUENTIAL_NUMBER Review Comment: ```suggestion * | |----- ROUTER_RM_DT_SEQUENTIAL_NUMBER ``` ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestZookeeperFederationStateStore.java: ########## @@ -171,38 +202,270 @@ public void testMetricsInited() throws Exception { MetricsRecords.assertMetric(record, "UpdateReservationHomeSubClusterNumOps", expectOps); } - @Test(expected = NotImplementedException.class) public void testStoreNewMasterKey() throws Exception { - super.testStoreNewMasterKey(); + + // Manually create a DelegationKey, + // and call the interface storeNewMasterKey to write the data to zk. + DelegationKey key = new DelegationKey(1234, Time.now() + 60 * 60, "keyBytes".getBytes()); + RouterMasterKey paramRouterMasterKey = RouterMasterKey.newInstance(key.getKeyId(), + ByteBuffer.wrap(key.getEncodedKey()), key.getExpiryDate()); + FederationStateStore stateStore = this.getStateStore(); + + assertTrue(stateStore instanceof ZookeeperFederationStateStore); + + // Compare the data returned by the storeNewMasterKey + // interface with the data queried by zk, and ensure that the data is consistent. + RouterMasterKeyRequest routerMasterKeyRequest = + RouterMasterKeyRequest.newInstance(paramRouterMasterKey); + RouterMasterKeyResponse response = stateStore.storeNewMasterKey(routerMasterKeyRequest); + assertNotNull(response); + RouterMasterKey respRouterMasterKey = response.getRouterMasterKey(); + assertNotNull(respRouterMasterKey); + + // Get Data From zk. + String nodeName = ROUTER_RM_DELEGATION_KEY_PREFIX + key.getKeyId(); + String nodePath = ZNODE_MASTER_KEY_PREFIX + nodeName; + RouterMasterKey zkRouterMasterKey = getRouterMasterKeyFromZK(nodePath); + + assertNotNull(zkRouterMasterKey); + assertEquals(paramRouterMasterKey, respRouterMasterKey); + assertEquals(paramRouterMasterKey, zkRouterMasterKey); + assertEquals(zkRouterMasterKey, respRouterMasterKey); } - @Test(expected = NotImplementedException.class) - public void testGetMasterKeyByDelegationKey() throws YarnException, IOException { - super.testGetMasterKeyByDelegationKey(); + public void testGetMasterKeyByDelegationKey() throws Exception { + + // Manually create a DelegationKey, + // and call the interface storeNewMasterKey to write the data to zk. + DelegationKey key = new DelegationKey(5678, Time.now() + 60 * 60, "keyBytes".getBytes()); + RouterMasterKey paramRouterMasterKey = RouterMasterKey.newInstance(key.getKeyId(), + ByteBuffer.wrap(key.getEncodedKey()), key.getExpiryDate()); + FederationStateStore stateStore = this.getStateStore(); + + assertTrue(stateStore instanceof ZookeeperFederationStateStore); + + // Call the getMasterKeyByDelegationKey interface of stateStore to get the MasterKey data. + RouterMasterKeyRequest routerMasterKeyRequest = + RouterMasterKeyRequest.newInstance(paramRouterMasterKey); + RouterMasterKeyResponse response = stateStore.storeNewMasterKey(routerMasterKeyRequest); + assertNotNull(response); + + // Get Data From zk. + String nodeName = ROUTER_RM_DELEGATION_KEY_PREFIX + key.getKeyId(); + String nodePath = ZNODE_MASTER_KEY_PREFIX + nodeName; + RouterMasterKey zkRouterMasterKey = getRouterMasterKeyFromZK(nodePath); + + // Call the getMasterKeyByDelegationKey interface to get the returned result. + // The zk data should be consistent with the returned data. + RouterMasterKeyResponse response1 = + stateStore.getMasterKeyByDelegationKey(routerMasterKeyRequest); + assertNotNull(response1); + RouterMasterKey respRouterMasterKey = response1.getRouterMasterKey(); + assertEquals(paramRouterMasterKey, respRouterMasterKey); + assertEquals(paramRouterMasterKey, zkRouterMasterKey); + assertEquals(zkRouterMasterKey, respRouterMasterKey); + } + + public void testRemoveStoredMasterKey() throws Exception { + + // Manually create a DelegationKey, + // and call the interface storeNewMasterKey to write the data to zk. + DelegationKey key = new DelegationKey(2345, Time.now() + 60 * 60, "keyBytes".getBytes()); + RouterMasterKey paramRouterMasterKey = RouterMasterKey.newInstance(key.getKeyId(), + ByteBuffer.wrap(key.getEncodedKey()), key.getExpiryDate()); + FederationStateStore stateStore = this.getStateStore(); + + assertTrue(stateStore instanceof ZookeeperFederationStateStore); + + // We need to ensure that the returned result is not empty. + RouterMasterKeyRequest routerMasterKeyRequest = + RouterMasterKeyRequest.newInstance(paramRouterMasterKey); + RouterMasterKeyResponse response = stateStore.storeNewMasterKey(routerMasterKeyRequest); + assertNotNull(response); + + // We will check if delegationToken exists in zk. + String nodeName = ROUTER_RM_DELEGATION_KEY_PREFIX + key.getKeyId(); + String nodePath = ZNODE_MASTER_KEY_PREFIX + nodeName; + assertTrue(curatorFramework.checkExists().forPath(nodePath) != null); + + // Call removeStoredMasterKey to remove the MasterKey data in zk. + RouterMasterKeyResponse response1 = stateStore.removeStoredMasterKey(routerMasterKeyRequest); + assertNotNull(response1); + RouterMasterKey respRouterMasterKey = response1.getRouterMasterKey(); + assertNotNull(respRouterMasterKey); + assertEquals(paramRouterMasterKey, respRouterMasterKey); + + // We have removed the RouterMasterKey data from zk, + // the path should be empty at this point. + assertTrue(curatorFramework.checkExists().forPath(nodePath) == null); + } + + public void testStoreNewToken() throws Exception { + + // We manually generate the DelegationToken, + // and then call the StoreNewToken method to store the Token in zk. + RMDelegationTokenIdentifier identifier = new RMDelegationTokenIdentifier( + new Text("owner2"), new Text("renewer2"), new Text("realuser2")); + FederationStateStore stateStore = this.getStateStore(); + int seqNum = stateStore.incrementDelegationTokenSeqNum(); + identifier.setSequenceNumber(seqNum); + Long renewDate = Time.now(); + + // Store new rm-token + RouterStoreToken paramStoreToken = RouterStoreToken.newInstance(identifier, renewDate); + RouterRMTokenRequest request = RouterRMTokenRequest.newInstance(paramStoreToken); + RouterRMTokenResponse routerRMTokenResponse = stateStore.storeNewToken(request); + assertNotNull(routerRMTokenResponse); + RouterStoreToken respStoreToken = routerRMTokenResponse.getRouterStoreToken(); + assertNotNull(respStoreToken); + + // Get delegationToken Path + String nodeName = ROUTER_RM_DELEGATION_TOKEN_PREFIX + identifier.getSequenceNumber(); + String nodePath = getNodePath(ZNODE_DT_PREFIX, nodeName); + + // Check if the path exists, we expect the result to exist. + assertTrue(curatorFramework.checkExists().forPath(nodePath) != null); + + // Check whether the token (paramStoreToken) + // We generated is consistent with the data stored in zk. + // We expect data to be consistent. + RouterStoreToken zkRouterStoreToken = getStoreTokenFromZK(nodePath); + assertNotNull(zkRouterStoreToken); + assertEquals(paramStoreToken, zkRouterStoreToken); + assertEquals(respStoreToken, zkRouterStoreToken); } - @Test(expected = NotImplementedException.class) - public void testRemoveStoredMasterKey() throws YarnException, IOException { - super.testRemoveStoredMasterKey(); + public void testUpdateStoredToken() throws Exception { Review Comment: Can we keep the Test annotation in all these methods? ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java: ########## @@ -886,45 +974,558 @@ public UpdateReservationHomeSubClusterResponse updateReservationHomeSubCluster( return UpdateReservationHomeSubClusterResponse.newInstance(); } + /** + * ZookeeperFederationStateStore Supports Store NewMasterKey. + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override - public RouterMasterKeyResponse storeNewMasterKey(RouterMasterKeyRequest request) + public synchronized RouterMasterKeyResponse storeNewMasterKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + long start = clock.getTime(); + // For the verification of the request, after passing the verification, + // the request and the internal objects will not be empty and can be used directly. + FederationRouterRMTokenInputValidator.validate(request); + + // Parse the delegationKey from the request and get the ZK storage path. + DelegationKey delegationKey = convertMasterKeyToDelegationKey(request); + String nodeCreatePath = getMasterKeyZNodePathByDelegationKey(delegationKey); + LOG.debug("Storing RMDelegationKey_{}, ZkNodePath = {}.", delegationKey.getKeyId(), + nodeCreatePath); + + // Write master key data to zk. + try(ByteArrayOutputStream os = new ByteArrayOutputStream(); + DataOutputStream fsOut = new DataOutputStream(os)) { + delegationKey.write(fsOut); + put(nodeCreatePath, os.toByteArray(), false); + } + + // Get the stored masterKey from zk. + RouterMasterKey masterKeyFromZK = getRouterMasterKeyFromZK(nodeCreatePath, false); + long end = clock.getTime(); + opDurations.addStoreNewMasterKeyDuration(start, end); + return RouterMasterKeyResponse.newInstance(masterKeyFromZK); } + /** + * ZookeeperFederationStateStore Supports Remove MasterKey. + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override - public RouterMasterKeyResponse removeStoredMasterKey(RouterMasterKeyRequest request) + public synchronized RouterMasterKeyResponse removeStoredMasterKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + long start = clock.getTime(); + // For the verification of the request, after passing the verification, + // the request and the internal objects will not be empty and can be used directly. + FederationRouterRMTokenInputValidator.validate(request); + + try { + // Parse the delegationKey from the request and get the ZK storage path. + RouterMasterKey masterKey = request.getRouterMasterKey(); + DelegationKey delegationKey = convertMasterKeyToDelegationKey(request); + String nodeRemovePath = getMasterKeyZNodePathByDelegationKey(delegationKey); + LOG.debug("Removing RMDelegationKey_{}, ZkNodePath = {}.", delegationKey.getKeyId(), + nodeRemovePath); + + // Check if the path exists, Throws an exception if the path does not exist. + if (!exists(nodeRemovePath)) { + throw new YarnException("ZkNodePath = " + nodeRemovePath + " not exists!"); + } + + // try to remove masterKey. + zkManager.delete(nodeRemovePath); + long end = clock.getTime(); + opDurations.removeStoredMasterKeyDuration(start, end); + return RouterMasterKeyResponse.newInstance(masterKey); + } catch (Exception e) { + throw new YarnException(e); + } } + /** + * ZookeeperFederationStateStore Supports Remove MasterKey. + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterMasterKeyResponse getMasterKeyByDelegationKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + long start = clock.getTime(); + // For the verification of the request, after passing the verification, + // the request and the internal objects will not be empty and can be used directly. + FederationRouterRMTokenInputValidator.validate(request); + + try { + + // Parse the delegationKey from the request and get the ZK storage path. + DelegationKey delegationKey = convertMasterKeyToDelegationKey(request); + String nodePath = getMasterKeyZNodePathByDelegationKey(delegationKey); + + // Check if the path exists, Throws an exception if the path does not exist. + if (!exists(nodePath)) { + throw new YarnException("ZkNodePath = " + nodePath + " not exists!"); + } + + // Get the stored masterKey from zk. + RouterMasterKey routerMasterKey = getRouterMasterKeyFromZK(nodePath, false); + long end = clock.getTime(); + opDurations.getMasterKeyByDelegationKeyDuration(start, end); + return RouterMasterKeyResponse.newInstance(routerMasterKey); + } catch (Exception e) { + throw new YarnException(e); + } + } + + /** + * Get MasterKeyZNodePath based on DelegationKey. + * + * @param delegationKey delegationKey. + * @return masterKey ZNodePath. + */ + private String getMasterKeyZNodePathByDelegationKey(DelegationKey delegationKey) { + return getMasterKeyZNodePathByKeyId(delegationKey.getKeyId()); + } + + /** + * Get MasterKeyZNodePath based on KeyId. + * + * @param keyId master key id. + * @return masterKey ZNodePath. + */ + private String getMasterKeyZNodePathByKeyId(int keyId) { + String nodeName = ROUTER_RM_DELEGATION_KEY_PREFIX + keyId; + return getNodePath(routerRMDTMasterKeysRootPath, nodeName); } + /** + * Get RouterMasterKey from ZK. + * + * @param nodePath The path where masterKey is stored in zk. + * @param quiet If true is silent mode, no error message is printed at this time, + * if false is non-silent mode, error message is printed at this time. + * + * @return + * @throws IOException + */ + private RouterMasterKey getRouterMasterKeyFromZK(String nodePath, boolean quiet) + throws IOException { + try { + byte[] data = get(nodePath); + if ((data == null) || (data.length == 0)) { + return null; + } + + ByteArrayInputStream bin = new ByteArrayInputStream(data); + DataInputStream din = new DataInputStream(bin); + DelegationKey key = new DelegationKey(); + key.readFields(din); + + return RouterMasterKey.newInstance(key.getKeyId(), + ByteBuffer.wrap(key.getEncodedKey()), key.getExpiryDate()); + } catch (Exception ex) { + if (!quiet) { + LOG.error("No node in path [" + nodePath + "]"); Review Comment: Use {} ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java: ########## @@ -174,21 +219,64 @@ public void init(Configuration conf) throws YarnException { policiesZNode = getNodePath(baseZNode, ROOT_ZNODE_NAME_POLICY); reservationsZNode = getNodePath(baseZNode, ROOT_ZNODE_NAME_RESERVATION); + // delegation token znodes + routerRMDTSecretManagerRoot = getNodePath(baseZNode, ROUTER_RM_DT_SECRET_MANAGER_ROOT); + routerRMDTMasterKeysRootPath = getNodePath(routerRMDTSecretManagerRoot, + ROUTER_RM_DT_MASTER_KEYS_ROOT_ZNODE_NAME); + routerRMDelegationTokensRootPath = getNodePath(routerRMDTSecretManagerRoot, + ROUTER_RM_DELEGATION_TOKENS_ROOT_ZNODE_NAME); + routerRMSequenceNumberPath = getNodePath(routerRMDTSecretManagerRoot, + ROUTER_RM_DT_SEQUENTIAL_NUMBER_ZNODE_NAME); + // Create base znode for each entity try { List<ACL> zkAcl = ZKCuratorManager.getZKAcls(conf); zkManager.createRootDirRecursively(membershipZNode, zkAcl); zkManager.createRootDirRecursively(appsZNode, zkAcl); zkManager.createRootDirRecursively(policiesZNode, zkAcl); zkManager.createRootDirRecursively(reservationsZNode, zkAcl); + zkManager.createRootDirRecursively(routerRMDTSecretManagerRoot, zkAcl); + zkManager.createRootDirRecursively(routerRMDTMasterKeysRootPath, zkAcl); + zkManager.createRootDirRecursively(routerRMDelegationTokensRootPath, zkAcl); } catch (Exception e) { String errMsg = "Cannot create base directories: " + e.getMessage(); FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg); } + + // Distributed sequenceNum. + try { + seqNumBatchSize = conf.getInt(ZK_DTSM_TOKEN_SEQNUM_BATCH_SIZE, + ZK_DTSM_TOKEN_SEQNUM_BATCH_SIZE_DEFAULT); + + delTokSeqCounter = new SharedCount(zkManager.getCurator(), routerRMSequenceNumberPath, 0); + + if (delTokSeqCounter != null) { + delTokSeqCounter.start(); + } + + // the first batch range should be allocated during this starting window + // by calling the incrSharedCount + currentSeqNum = incrSharedCount(delTokSeqCounter, seqNumBatchSize); + currentMaxSeqNum = currentSeqNum + seqNumBatchSize; + + LOG.info("Fetched initial range of seq num, from {} to {} ", + currentSeqNum + 1, currentMaxSeqNum); + } catch (Exception e) { + throw new YarnException("Could not start Sequence Counter.", e); + } } @Override public void close() throws Exception { + + try { + if (delTokSeqCounter != null) { + delTokSeqCounter.close(); Review Comment: Add ``` delTokSeqCounter = null; ``` ? ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java: ########## @@ -886,45 +974,558 @@ public UpdateReservationHomeSubClusterResponse updateReservationHomeSubCluster( return UpdateReservationHomeSubClusterResponse.newInstance(); } + /** + * ZookeeperFederationStateStore Supports Store NewMasterKey. + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override - public RouterMasterKeyResponse storeNewMasterKey(RouterMasterKeyRequest request) + public synchronized RouterMasterKeyResponse storeNewMasterKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + long start = clock.getTime(); + // For the verification of the request, after passing the verification, + // the request and the internal objects will not be empty and can be used directly. + FederationRouterRMTokenInputValidator.validate(request); + + // Parse the delegationKey from the request and get the ZK storage path. + DelegationKey delegationKey = convertMasterKeyToDelegationKey(request); + String nodeCreatePath = getMasterKeyZNodePathByDelegationKey(delegationKey); + LOG.debug("Storing RMDelegationKey_{}, ZkNodePath = {}.", delegationKey.getKeyId(), + nodeCreatePath); + + // Write master key data to zk. + try(ByteArrayOutputStream os = new ByteArrayOutputStream(); + DataOutputStream fsOut = new DataOutputStream(os)) { + delegationKey.write(fsOut); + put(nodeCreatePath, os.toByteArray(), false); + } + + // Get the stored masterKey from zk. + RouterMasterKey masterKeyFromZK = getRouterMasterKeyFromZK(nodeCreatePath, false); + long end = clock.getTime(); + opDurations.addStoreNewMasterKeyDuration(start, end); + return RouterMasterKeyResponse.newInstance(masterKeyFromZK); } + /** + * ZookeeperFederationStateStore Supports Remove MasterKey. + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override - public RouterMasterKeyResponse removeStoredMasterKey(RouterMasterKeyRequest request) + public synchronized RouterMasterKeyResponse removeStoredMasterKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + long start = clock.getTime(); + // For the verification of the request, after passing the verification, + // the request and the internal objects will not be empty and can be used directly. + FederationRouterRMTokenInputValidator.validate(request); + + try { + // Parse the delegationKey from the request and get the ZK storage path. + RouterMasterKey masterKey = request.getRouterMasterKey(); + DelegationKey delegationKey = convertMasterKeyToDelegationKey(request); + String nodeRemovePath = getMasterKeyZNodePathByDelegationKey(delegationKey); + LOG.debug("Removing RMDelegationKey_{}, ZkNodePath = {}.", delegationKey.getKeyId(), + nodeRemovePath); + + // Check if the path exists, Throws an exception if the path does not exist. + if (!exists(nodeRemovePath)) { + throw new YarnException("ZkNodePath = " + nodeRemovePath + " not exists!"); + } + + // try to remove masterKey. + zkManager.delete(nodeRemovePath); + long end = clock.getTime(); + opDurations.removeStoredMasterKeyDuration(start, end); + return RouterMasterKeyResponse.newInstance(masterKey); + } catch (Exception e) { + throw new YarnException(e); + } } + /** + * ZookeeperFederationStateStore Supports Remove MasterKey. + * + * @param request The request contains RouterMasterKey, which is an abstraction for DelegationKey + * @return routerMasterKeyResponse, the response contains the RouterMasterKey. + * @throws YarnException if the call to the state store is unsuccessful. + * @throws IOException An IO Error occurred. + */ @Override public RouterMasterKeyResponse getMasterKeyByDelegationKey(RouterMasterKeyRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + + long start = clock.getTime(); + // For the verification of the request, after passing the verification, + // the request and the internal objects will not be empty and can be used directly. + FederationRouterRMTokenInputValidator.validate(request); + + try { + + // Parse the delegationKey from the request and get the ZK storage path. + DelegationKey delegationKey = convertMasterKeyToDelegationKey(request); + String nodePath = getMasterKeyZNodePathByDelegationKey(delegationKey); + + // Check if the path exists, Throws an exception if the path does not exist. + if (!exists(nodePath)) { + throw new YarnException("ZkNodePath = " + nodePath + " not exists!"); + } + + // Get the stored masterKey from zk. + RouterMasterKey routerMasterKey = getRouterMasterKeyFromZK(nodePath, false); + long end = clock.getTime(); + opDurations.getMasterKeyByDelegationKeyDuration(start, end); + return RouterMasterKeyResponse.newInstance(routerMasterKey); + } catch (Exception e) { + throw new YarnException(e); + } + } + + /** + * Get MasterKeyZNodePath based on DelegationKey. + * + * @param delegationKey delegationKey. + * @return masterKey ZNodePath. + */ + private String getMasterKeyZNodePathByDelegationKey(DelegationKey delegationKey) { + return getMasterKeyZNodePathByKeyId(delegationKey.getKeyId()); + } + + /** + * Get MasterKeyZNodePath based on KeyId. + * + * @param keyId master key id. + * @return masterKey ZNodePath. + */ + private String getMasterKeyZNodePathByKeyId(int keyId) { + String nodeName = ROUTER_RM_DELEGATION_KEY_PREFIX + keyId; + return getNodePath(routerRMDTMasterKeysRootPath, nodeName); } + /** + * Get RouterMasterKey from ZK. + * + * @param nodePath The path where masterKey is stored in zk. + * @param quiet If true is silent mode, no error message is printed at this time, + * if false is non-silent mode, error message is printed at this time. + * + * @return + * @throws IOException + */ + private RouterMasterKey getRouterMasterKeyFromZK(String nodePath, boolean quiet) Review Comment: I don't see much value in this quiet option. -- 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 --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org