[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1041589814 ## 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 +203,117 @@ public void testMetricsInited() throws Exception { MetricsRecords.assertMetric(record, "UpdateReservationHomeSubClusterNumOps", expectOps); } - @Test(expected = NotImplementedException.class) + @Test public void testStoreNewMasterKey() throws Exception { super.testStoreNewMasterKey(); } - @Test(expected = NotImplementedException.class) + @Test Review Comment: I agree with you, I will remove this part of the code. -- 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
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1041589119 ## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java: ## @@ -592,4 +588,14 @@ public void testRemoveStoredToken() throws IOException, YarnException { public void testGetTokenByRouterStoreToken() throws IOException, YarnException { super.testGetTokenByRouterStoreToken(); } + + @Override + protected void checkRouterMasterKey(DelegationKey delegationKey, + RouterMasterKey routerMasterKey) throws YarnException, IOException { Review Comment: Thank you for your suggestion! I'll add a comment explaining what the method does. -- 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
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1041588318 ## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java: ## @@ -18,21 +18,17 @@ package org.apache.hadoop.yarn.server.federation.store.impl; import org.apache.commons.lang3.NotImplementedException; +import org.apache.hadoop.security.token.delegation.DelegationKey; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.util.Time; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ReservationId; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; import org.apache.hadoop.yarn.server.federation.store.FederationStateStore; import org.apache.hadoop.yarn.server.federation.store.metrics.FederationStateStoreClientMetrics; -import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; -import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo; -import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest; -import org.apache.hadoop.yarn.server.federation.store.records.ReservationHomeSubCluster; -import org.apache.hadoop.yarn.server.federation.store.records.AddReservationHomeSubClusterRequest; -import org.apache.hadoop.yarn.server.federation.store.records.UpdateReservationHomeSubClusterRequest; -import org.apache.hadoop.yarn.server.federation.store.records.DeleteReservationHomeSubClusterRequest; +import org.apache.hadoop.yarn.server.federation.store.records.*; Review Comment: Thank you very much for helping to review the code, I will fix it. -- 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
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1039707916 ## 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 +203,293 @@ public void testMetricsInited() throws Exception { MetricsRecords.assertMetric(record, "UpdateReservationHomeSubClusterNumOps", expectOps); } - @Test(expected = NotImplementedException.class) + @Test 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) + @Test public void testGetMasterKeyByDelegationKey() throws YarnException, IOException { -super.testGetMasterKeyByDelegationKey(); + +// 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); Review Comment: Thank you very much for helping to review the code, I will fix it. ## 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 +203,293 @@ public void testMetricsInited() throws Exception { MetricsRecords.assertMetric(record, "UpdateReservationHomeSubClusterNumOps", expectOps); } - @Test(expected = NotImplementedException.class) + @Test 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) + @Test
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1036628806 ## 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 +1000,608 @@ 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) 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); +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) 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); + +
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1036559681 ## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java: ## @@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; import org.apache.hadoop.yarn.util.Records; +import java.io.DataInput; Review Comment: The method we completed in the impl defines the abstract method declaration in the abstract class. `RouterStoreToken` is an abstract class, its impl class is `RouterStoreTokenPBImpl`, we have implemented these 2 methods. RouterStoreToken#toByteArray and readFields ``` public abstract byte[] toByteArray() throws IOException; public abstract void readFields(DataInput in) throws IOException; ``` RouterStoreTokenPBImpl#toByteArray and readFields ``` public byte[] toByteArray() throws IOException { return builder.build().toByteArray(); } public void readFields(DataInput in) throws IOException { builder.mergeFrom((DataInputStream) in); } ``` -- 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
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1035905558 ## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java: ## @@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; import org.apache.hadoop.yarn.util.Records; +import java.io.DataInput; Review Comment: We used the `readFields` method, which requires `DataInput` parameters. RouterStoreToken#readFields ``` public abstract void readFields(DataInput in) throws IOException; ``` When we store `RouterStoreToken`, we convert token into a byte array, and then store it in `ZK` - ZookeeperFederationStateStore#storeOrUpdateRouterRMDT ``` private void storeOrUpdateRouterRMDT(RouterRMTokenRequest request, boolean isUpdate) throws Exception { RouterStoreToken routerStoreToken = request.getRouterStoreToken(); String nodeCreatePath = getStoreTokenZNodePathByTokenRequest(request); LOG.debug("nodeCreatePath = {}, isUpdate = {}", nodeCreatePath, isUpdate); put(nodeCreatePath, routerStoreToken.toByteArray(), isUpdate); } ``` When we need to read data, we convert the bytes array stored in `ZK` into `DataInputStream`, and then let `RouterStoreToken` parse the `required fields` from it. - ZookeeperFederationStateStore#getStoreTokenFromZK ``` private RouterStoreToken getStoreTokenFromZK(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); RouterStoreToken storeToken = Records.newRecord(RouterStoreToken.class); storeToken.readFields(din); return storeToken; } catch (Exception ex) { if (!quiet) { LOG.error("No node in path [" + nodePath + "]"); } throw new IOException(ex); } } ``` -- 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
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1035905558 ## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/RouterStoreToken.java: ## @@ -22,6 +22,7 @@ import org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier; import org.apache.hadoop.yarn.util.Records; +import java.io.DataInput; Review Comment: We used the `readFields` method, which requires `DataInput` parameters. RouterStoreToken#readFields ``` public abstract void readFields(DataInput in) throws IOException; ``` When we store `RouterStoreToken`, we convert token into a byte array, and then store it in `ZK` ``` private void storeOrUpdateRouterRMDT(RouterRMTokenRequest request, boolean isUpdate) throws Exception { RouterStoreToken routerStoreToken = request.getRouterStoreToken(); String nodeCreatePath = getStoreTokenZNodePathByTokenRequest(request); LOG.debug("nodeCreatePath = {}, isUpdate = {}", nodeCreatePath, isUpdate); put(nodeCreatePath, routerStoreToken.toByteArray(), isUpdate); } ``` When we need to read data, we convert the bytes array stored in `ZK` into `DataInputStream`, and then let `RouterStoreToken` parse the `required fields` from it. ``` private RouterStoreToken getStoreTokenFromZK(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); RouterStoreToken storeToken = Records.newRecord(RouterStoreToken.class); storeToken.readFields(din); return storeToken; } catch (Exception ex) { if (!quiet) { LOG.error("No node in path [" + nodePath + "]"); } throw new IOException(ex); } } ``` -- 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
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1034873671 ## 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 +975,554 @@ 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); +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
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1030951065 ## 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 +975,554 @@ 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); +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
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1030632183 ## 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 +975,554 @@ 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); +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
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1030632183 ## 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 +975,554 @@ 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); +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
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1029962161 ## 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: ## @@ -905,7 +905,7 @@ public void testGetMasterKeyByDelegationKey() throws YarnException, IOException } @Test - public void testRemoveStoredMasterKey() throws YarnException, IOException { + public void testRemoveStoredMasterKey() throws IOException, YarnException { Review Comment: Thank you so much for reviewing the code, I will fix it. -- 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
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1023980706 ## 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: The exception thrown by the method signature, I will recover. In the process of writing JunitTest code, I used `curatorFramework.getData().forPath` method to get data from ZK. This method throws an `Exception`, so I modified the exception throwing code of the method signature. ``` /** * Commit the currently building operation using the given path * * @param path the path * @return operation result if any * @throws Exception errors */ public T forPath(String path) throws Exception; ``` -- 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
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1023964038 ## 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
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1023521160 ## 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 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: Thanks for your suggestion, I will modify the code. ## 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 =
[GitHub] [hadoop] slfan1989 commented on a diff in pull request #5131: YARN-11350. [Federation] Router Support DelegationToken With ZK.
slfan1989 commented on code in PR #5131: URL: https://github.com/apache/hadoop/pull/5131#discussion_r1023520913 ## 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: Thank you very much for reviewing the code, I will fix it. -- 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