Browse Source

YARN-11627. [GPG] Improve GPGPolicyFacade#getPolicyManager. (#6332) Contributed by Shilun Fan.

Reviewed-by: Inigo Goiri <inigoiri@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 1 year ago
parent
commit
7935eded5e
13 changed files with 331 additions and 17 deletions
  1. 14 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/AbstractPolicyManager.java
  2. 24 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/FederationPolicyManager.java
  3. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/HashBroadcastPolicyManager.java
  4. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/HomePolicyManager.java
  5. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/PriorityBroadcastPolicyManager.java
  6. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/RejectAllPolicyManager.java
  7. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/UniformBroadcastPolicyManager.java
  8. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/WeightedHomePolicyManager.java
  9. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/WeightedLocalityPolicyManager.java
  10. 14 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
  11. 24 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-globalpolicygenerator/src/main/java/org/apache/hadoop/yarn/server/globalpolicygenerator/GPGPolicyFacade.java
  12. 155 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-globalpolicygenerator/src/test/java/org/apache/hadoop/yarn/server/globalpolicygenerator/TestGPGPolicyFacade.java
  13. 15 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestSequentialBroadcastPolicyManager.java

+ 14 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/AbstractPolicyManager.java

@@ -21,6 +21,7 @@ import org.apache.hadoop.yarn.server.federation.policies.ConfigurableFederationP
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContextValidator;
 import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
 import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
@@ -187,4 +188,17 @@ public abstract class AbstractPolicyManager implements
                   federationPolicyContext.getHomeSubcluster());
   }
 
+  /**
+   * We get the WeightedPolicyInfo of the subCluster.
+   *
+   * @return WeightedPolicyInfo.
+   */
+  public abstract WeightedPolicyInfo getWeightedPolicyInfo();
+
+  /**
+   * We set the WeightedPolicyInfo of the subCluster.
+   *
+   * @param weightedPolicyInfo weightedPolicyInfo of the subCluster.
+   */
+  public abstract void setWeightedPolicyInfo(WeightedPolicyInfo weightedPolicyInfo);
 }

+ 24 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/FederationPolicyManager.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.federation.policies.manager;
 
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
 import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
 import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
@@ -115,4 +116,27 @@ public interface FederationPolicyManager {
    */
   void setQueue(String queue);
 
+  /**
+   * This method returns the queue WeightedPolicyInfo
+   * this policy is configured for.
+   *
+   * @return the name of the queue.
+   */
+  WeightedPolicyInfo getWeightedPolicyInfo();
+
+  /**
+   * This methods provides a setter for the queue WeightedPolicyInfo
+   * this policy is specified for.
+   *
+   * @param weightedPolicyInfo weightedPolicyInfo of the subCluster.
+   */
+  void setWeightedPolicyInfo(WeightedPolicyInfo weightedPolicyInfo);
+
+  /**
+   * PolicyManager Whether to support WeightedPolicyInfo.
+   * Some of PolicyManagers do not support WeightedPolicyInfo.
+   * @return true, supports WeightedPolicyInfo;
+   * false, WeightedPolicyInfo is not supported
+   */
+  boolean isSupportWeightedPolicyInfo();
 }

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/HashBroadcastPolicyManager.java

@@ -17,7 +17,9 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.manager;
 
+import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.policies.router.HashBasedRouterPolicy;
 
 /**
@@ -35,4 +37,20 @@ public class HashBroadcastPolicyManager extends AbstractPolicyManager {
     amrmProxyFederationPolicy = BroadcastAMRMProxyPolicy.class;
   }
 
+  @Override
+  public WeightedPolicyInfo getWeightedPolicyInfo() {
+    throw new NotImplementedException(
+        "HashBroadcastPolicyManager does not implement getWeightedPolicyInfo.");
+  }
+
+  @Override
+  public void setWeightedPolicyInfo(WeightedPolicyInfo weightedPolicyInfo) {
+    throw new NotImplementedException(
+        "HashBroadcastPolicyManager does not implement setWeightedPolicyInfo.");
+  }
+
+  @Override
+  public boolean isSupportWeightedPolicyInfo() {
+    return false;
+  }
 }

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/HomePolicyManager.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.federation.policies.manager;
 import java.nio.ByteBuffer;
 import java.util.Collections;
 
+import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.HomeAMRMProxyPolicy;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
@@ -58,4 +59,21 @@ public class HomePolicyManager extends AbstractPolicyManager {
     return SubClusterPolicyConfiguration.newInstance(
         getQueue(), this.getClass().getCanonicalName(), buf);
   }
+
+  @Override
+  public WeightedPolicyInfo getWeightedPolicyInfo() {
+    throw new NotImplementedException(
+        "HomePolicyManager does not implement getWeightedPolicyInfo.");
+  }
+
+  @Override
+  public void setWeightedPolicyInfo(WeightedPolicyInfo weightedPolicyInfo) {
+    throw new NotImplementedException(
+        "HomePolicyManager does not implement setWeightedPolicyInfo.");
+  }
+
+  @Override
+  public boolean isSupportWeightedPolicyInfo() {
+    return false;
+  }
 }

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/PriorityBroadcastPolicyManager.java

@@ -63,4 +63,8 @@ public class PriorityBroadcastPolicyManager extends AbstractPolicyManager {
     this.weightedPolicyInfo = weightedPolicyInfo;
   }
 
+  @Override
+  public boolean isSupportWeightedPolicyInfo() {
+    return true;
+  }
 }

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/RejectAllPolicyManager.java

@@ -17,7 +17,9 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.manager;
 
+import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.RejectAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.policies.router.RejectRouterPolicy;
 
 /**
@@ -37,4 +39,20 @@ public class RejectAllPolicyManager extends AbstractPolicyManager {
     amrmProxyFederationPolicy = RejectAMRMProxyPolicy.class;
   }
 
+  @Override
+  public WeightedPolicyInfo getWeightedPolicyInfo() {
+    throw new NotImplementedException(
+        "RejectAllPolicyManager does not implement getWeightedPolicyInfo.");
+  }
+
+  @Override
+  public void setWeightedPolicyInfo(WeightedPolicyInfo weightedPolicyInfo) {
+    throw new NotImplementedException(
+        "RejectAllPolicyManager does not implement setWeightedPolicyInfo.");
+  }
+
+  @Override
+  public boolean isSupportWeightedPolicyInfo() {
+    return false;
+  }
 }

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/UniformBroadcastPolicyManager.java

@@ -17,7 +17,9 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.manager;
 
+import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.policies.router.UniformRandomRouterPolicy;
 
 /**
@@ -41,4 +43,20 @@ public class UniformBroadcastPolicyManager extends AbstractPolicyManager {
     amrmProxyFederationPolicy = BroadcastAMRMProxyPolicy.class;
   }
 
+  @Override
+  public WeightedPolicyInfo getWeightedPolicyInfo() {
+    throw new NotImplementedException(
+        "UniformBroadcastPolicyManager does not implement getWeightedPolicyInfo.");
+  }
+
+  @Override
+  public void setWeightedPolicyInfo(WeightedPolicyInfo weightedPolicyInfo) {
+    throw new NotImplementedException(
+        "UniformBroadcastPolicyManager does not implement setWeightedPolicyInfo.");
+  }
+
+  @Override
+  public boolean isSupportWeightedPolicyInfo() {
+    return false;
+  }
 }

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/WeightedHomePolicyManager.java

@@ -62,4 +62,9 @@ public class WeightedHomePolicyManager extends AbstractPolicyManager {
       WeightedPolicyInfo weightedPolicyInfo) {
     this.weightedPolicyInfo = weightedPolicyInfo;
   }
+
+  @Override
+  public boolean isSupportWeightedPolicyInfo() {
+    return true;
+  }
 }

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/WeightedLocalityPolicyManager.java

@@ -65,4 +65,8 @@ public class WeightedLocalityPolicyManager
     this.weightedPolicyInfo = weightedPolicyInfo;
   }
 
+  @Override
+  public boolean isSupportWeightedPolicyInfo() {
+    return true;
+  }
 }

+ 14 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java

@@ -22,6 +22,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
 import org.apache.hadoop.yarn.server.federation.policies.manager.FederationPolicyManager;
 import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
@@ -135,6 +136,19 @@ public class TestFederationPolicyInitializationContextValidator {
 
     }
 
+    @Override
+    public WeightedPolicyInfo getWeightedPolicyInfo() {
+      return null;
+    }
+
+    @Override
+    public void setWeightedPolicyInfo(WeightedPolicyInfo weightedPolicyInfo) {
+    }
+
+    @Override
+    public boolean isSupportWeightedPolicyInfo() {
+      return false;
+    }
   }
 
 }

+ 24 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-globalpolicygenerator/src/main/java/org/apache/hadoop/yarn/server/globalpolicygenerator/GPGPolicyFacade.java

@@ -22,7 +22,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyUtils;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
-import org.apache.hadoop.yarn.server.federation.policies.manager.WeightedLocalityPolicyManager;
 import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
 import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
@@ -32,6 +31,7 @@ import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -100,41 +100,48 @@ public class GPGPolicyFacade {
   public FederationPolicyManager getPolicyManager(String queueName)
       throws YarnException {
     FederationPolicyManager policyManager = policyManagerMap.get(queueName);
+
     // If we don't have the policy manager cached, pull configuration
     // from the FederationStateStore to create and cache it
     if (policyManager == null) {
       try {
+
         // If we don't have the configuration cached, pull it
         // from the stateStore
         SubClusterPolicyConfiguration conf = policyConfMap.get(queueName);
+
         if (conf == null) {
           conf = stateStore.getPolicyConfiguration(queueName);
         }
+
         // If configuration is still null, it does not exist in the
         // FederationStateStore
         if (conf == null) {
-          LOG.info("Read null policy for queue {}", queueName);
+          LOG.info("Read null policy for queue {}.", queueName);
           return null;
         }
-        policyManager =
-            FederationPolicyUtils.instantiatePolicyManager(conf.getType());
+
+        // Generate PolicyManager based on PolicyManagerType.
+        String policyManagerType = conf.getType();
+        policyManager = FederationPolicyUtils.instantiatePolicyManager(policyManagerType);
         policyManager.setQueue(queueName);
 
-        // TODO there is currently no way to cleanly deserialize a policy
-        // manager sub type from just the configuration
-        if (policyManager instanceof WeightedLocalityPolicyManager) {
-          WeightedPolicyInfo wpinfo =
+        // If PolicyManager supports Weighted PolicyInfo, it means that
+        // we need to use this parameter to determine which sub-cluster the router goes to
+        // or which sub-cluster the container goes to.
+        if (policyManager.isSupportWeightedPolicyInfo()) {
+          ByteBuffer weightedPolicyInfoParams = conf.getParams();
+          if (weightedPolicyInfoParams == null) {
+            LOG.warn("Warning: Queue = {}, FederationPolicyManager {} WeightedPolicyInfo is empty.",
+                queueName, policyManagerType);
+            return null;
+          }
+          WeightedPolicyInfo weightedPolicyInfo =
               WeightedPolicyInfo.fromByteBuffer(conf.getParams());
-          WeightedLocalityPolicyManager wlpmanager =
-              (WeightedLocalityPolicyManager) policyManager;
-          LOG.info("Updating policy for queue {} to configured weights router: "
-                  + "{}, amrmproxy: {}", queueName,
-              wpinfo.getRouterPolicyWeights(),
-              wpinfo.getAMRMPolicyWeights());
-          wlpmanager.setWeightedPolicyInfo(wpinfo);
+          policyManager.setWeightedPolicyInfo(weightedPolicyInfo);
         } else {
-          LOG.warn("Warning: FederationPolicyManager of unsupported type {}, "
-              + "initialization may be incomplete ", policyManager.getClass());
+          LOG.warn("Warning: FederationPolicyManager of unsupported WeightedPolicyInfo type {}, " +
+              "initialization may be incomplete.", policyManager.getClass());
         }
 
         policyManagerMap.put(queueName, policyManager);

+ 155 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-globalpolicygenerator/src/test/java/org/apache/hadoop/yarn/server/globalpolicygenerator/TestGPGPolicyFacade.java

@@ -23,11 +23,19 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import org.apache.commons.lang3.NotImplementedException;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.policies.manager.FederationPolicyManager;
 import org.apache.hadoop.yarn.server.federation.policies.manager.WeightedLocalityPolicyManager;
+import org.apache.hadoop.yarn.server.federation.policies.manager.WeightedHomePolicyManager;
+import org.apache.hadoop.yarn.server.federation.policies.manager.UniformBroadcastPolicyManager;
+import org.apache.hadoop.yarn.server.federation.policies.manager.HashBroadcastPolicyManager;
+import org.apache.hadoop.yarn.server.federation.policies.manager.HomePolicyManager;
+import org.apache.hadoop.yarn.server.federation.policies.manager.RejectAllPolicyManager;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
@@ -35,6 +43,7 @@ import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolic
 import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
 import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
 import org.junit.After;
 import org.junit.Assert;
@@ -42,6 +51,10 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Matchers;
 
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Set;
 
@@ -198,4 +211,146 @@ public class TestGPGPolicyFacade {
         Matchers.any(SetSubClusterPolicyConfigurationRequest.class));
   }
 
+  @Test
+  public void testGetWeightedLocalityPolicyManager() throws YarnException {
+    stateStore = new MemoryFederationStateStore();
+    stateStore.init(new Configuration());
+
+    // root.a uses WeightedLocalityPolicyManager.
+    // Step1. Prepare amRMPolicyWeights and routerPolicyWeights
+    Map<SubClusterIdInfo, Float> amrmPolicyWeights = new HashMap<>();
+    amrmPolicyWeights.put(new SubClusterIdInfo("SC-1"), 0.7f);
+    amrmPolicyWeights.put(new SubClusterIdInfo("SC-2"), 0.3f);
+
+    Map<SubClusterIdInfo, Float> routerPolicyWeights = new HashMap<>();
+    routerPolicyWeights.put(new SubClusterIdInfo("SC-1"), 0.6f);
+    routerPolicyWeights.put(new SubClusterIdInfo("SC-2"), 0.4f);
+
+    WeightedPolicyInfo weightedPolicyInfo = new WeightedPolicyInfo();
+    weightedPolicyInfo.setHeadroomAlpha(1);
+    weightedPolicyInfo.setAMRMPolicyWeights(amrmPolicyWeights);
+    weightedPolicyInfo.setRouterPolicyWeights(routerPolicyWeights);
+
+    // Step2. Set PolicyConfiguration.
+    String policyManagerType = WeightedLocalityPolicyManager.class.getName();
+    SubClusterPolicyConfiguration config = SubClusterPolicyConfiguration.newInstance("root.a",
+        policyManagerType, weightedPolicyInfo.toByteBuffer());
+    SetSubClusterPolicyConfigurationRequest request =
+        SetSubClusterPolicyConfigurationRequest.newInstance(config);
+    stateStore.setPolicyConfiguration(request);
+
+    // Step3. Get FederationPolicyManager using policyFacade.
+    facade.reinitialize(stateStore, conf);
+    policyFacade = new GPGPolicyFacade(facade, conf);
+    FederationPolicyManager policyManager = policyFacade.getPolicyManager("root.a");
+    Assert.assertNotNull(policyManager);
+    Assert.assertTrue(policyManager.isSupportWeightedPolicyInfo());
+    WeightedPolicyInfo weightedPolicyInfo1 = policyManager.getWeightedPolicyInfo();
+    Assert.assertNotNull(weightedPolicyInfo1);
+    Assert.assertTrue(policyManager instanceof WeightedLocalityPolicyManager);
+
+    // Step4. Confirm amrmPolicyWeight is accurate.
+    Map<SubClusterIdInfo, Float> amrmPolicyWeights1 = weightedPolicyInfo1.getAMRMPolicyWeights();
+    Assert.assertNotNull(amrmPolicyWeights1);
+    Float sc1Float = amrmPolicyWeights1.get(new SubClusterIdInfo("SC-1"));
+    Float sc2Float = amrmPolicyWeights1.get(new SubClusterIdInfo("SC-2"));
+    Assert.assertEquals(0.7, sc1Float, 0.001);
+    Assert.assertEquals(0.3, sc2Float, 0.001);
+
+    // Step5. Confirm amrmPolicyWeight is accurate.
+    Map<SubClusterIdInfo, Float> routerPolicyWeights1 =
+        weightedPolicyInfo1.getRouterPolicyWeights();
+    Assert.assertNotNull(routerPolicyWeights1);
+    Float sc1Float1 = routerPolicyWeights1.get(new SubClusterIdInfo("SC-1"));
+    Float sc2Float2 = routerPolicyWeights1.get(new SubClusterIdInfo("SC-2"));
+    Assert.assertEquals(0.6, sc1Float1, 0.001);
+    Assert.assertEquals(0.4, sc2Float2, 0.001);
+  }
+
+  @Test
+  public void testGetWeightedHomePolicyManager() throws YarnException {
+    stateStore = new MemoryFederationStateStore();
+    stateStore.init(new Configuration());
+
+    // root.b uses WeightedHomePolicyManager.
+    // Step1. Prepare routerPolicyWeights.
+    Map<SubClusterIdInfo, Float> routerPolicyWeights = new HashMap<>();
+    routerPolicyWeights.put(new SubClusterIdInfo("SC-1"), 0.8f);
+    routerPolicyWeights.put(new SubClusterIdInfo("SC-2"), 0.2f);
+
+    WeightedPolicyInfo weightedPolicyInfo = new WeightedPolicyInfo();
+    weightedPolicyInfo.setHeadroomAlpha(1);
+    weightedPolicyInfo.setRouterPolicyWeights(routerPolicyWeights);
+
+    // Step2. Set PolicyConfiguration.
+    String policyManagerType = WeightedHomePolicyManager.class.getName();
+    SubClusterPolicyConfiguration config = SubClusterPolicyConfiguration.newInstance("root.b",
+        policyManagerType, weightedPolicyInfo.toByteBuffer());
+    SetSubClusterPolicyConfigurationRequest request =
+        SetSubClusterPolicyConfigurationRequest.newInstance(config);
+    stateStore.setPolicyConfiguration(request);
+
+    // Step3. Get FederationPolicyManager using policyFacade.
+    facade.reinitialize(stateStore, conf);
+    policyFacade = new GPGPolicyFacade(facade, conf);
+    FederationPolicyManager policyManager = policyFacade.getPolicyManager("root.b");
+    Assert.assertNotNull(policyManager);
+    Assert.assertTrue(policyManager.isSupportWeightedPolicyInfo());
+    WeightedPolicyInfo weightedPolicyInfo1 = policyManager.getWeightedPolicyInfo();
+    Assert.assertNotNull(weightedPolicyInfo1);
+
+    // Step4. Confirm amrmPolicyWeight is accurate.
+    Map<SubClusterIdInfo, Float> amrmPolicyWeights1 = weightedPolicyInfo1.getAMRMPolicyWeights();
+    Assert.assertNotNull(amrmPolicyWeights1);
+    Assert.assertEquals(0, amrmPolicyWeights1.size());
+
+    // Step5. Confirm amrmPolicyWeight is accurate.
+    Map<SubClusterIdInfo, Float> routerPolicyWeights1 =
+        weightedPolicyInfo1.getRouterPolicyWeights();
+    Assert.assertNotNull(routerPolicyWeights1);
+    Float sc1Float1 = routerPolicyWeights1.get(new SubClusterIdInfo("SC-1"));
+    Float sc2Float2 = routerPolicyWeights1.get(new SubClusterIdInfo("SC-2"));
+    Assert.assertEquals(0.8, sc1Float1, 0.001);
+    Assert.assertEquals(0.2, sc2Float2, 0.001);
+  }
+
+  @Test
+  public void testGetUniformBroadcastPolicyManager() throws Exception {
+    stateStore = new MemoryFederationStateStore();
+    stateStore.init(new Configuration());
+
+    List<String> notSupportWeightedPolicyInfos = new ArrayList<>();
+    notSupportWeightedPolicyInfos.add(HashBroadcastPolicyManager.class.getName());
+    notSupportWeightedPolicyInfos.add(UniformBroadcastPolicyManager.class.getName());
+    notSupportWeightedPolicyInfos.add(HomePolicyManager.class.getName());
+    notSupportWeightedPolicyInfos.add(RejectAllPolicyManager.class.getName());
+    String prefix = "org.apache.hadoop.yarn.server.federation.policies.manager.";
+
+    for (String policyManagerType : notSupportWeightedPolicyInfos) {
+      // root.c uses UniformBroadcastPolicyManager.
+      // Step1. Prepare routerPolicyWeights.
+      WeightedPolicyInfo weightedPolicyInfo = new WeightedPolicyInfo();
+      weightedPolicyInfo.setHeadroomAlpha(1);
+
+      // Step2. Set PolicyConfiguration.
+      SubClusterPolicyConfiguration config = SubClusterPolicyConfiguration.newInstance("root.c",
+          policyManagerType, weightedPolicyInfo.toByteBuffer());
+      SetSubClusterPolicyConfigurationRequest request =
+          SetSubClusterPolicyConfigurationRequest.newInstance(config);
+      stateStore.setPolicyConfiguration(request);
+
+      // Step3. Get FederationPolicyManager using policyFacade.
+      facade.reinitialize(stateStore, conf);
+      policyFacade = new GPGPolicyFacade(facade, conf);
+      FederationPolicyManager policyManager = policyFacade.getPolicyManager("root.c");
+      Assert.assertNotNull(policyManager);
+      Assert.assertFalse(policyManager.isSupportWeightedPolicyInfo());
+      String policyManagerTypeSimple = policyManagerType.replace(prefix, "");
+      // Verify that PolicyManager is initialized successfully,
+      // but getWeightedPolicyInfo is not supported.
+      LambdaTestUtils.intercept(NotImplementedException.class,
+          policyManagerTypeSimple + " does not implement getWeightedPolicyInfo.",
+          () -> policyManager.getWeightedPolicyInfo());
+    }
+  }
 }

+ 15 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestSequentialBroadcastPolicyManager.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.router.clientrm;
 
 import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.policies.manager.AbstractPolicyManager;
 
 /**
@@ -36,4 +37,18 @@ public class TestSequentialBroadcastPolicyManager extends AbstractPolicyManager
     routerFederationPolicy = TestSequentialRouterPolicy.class;
     amrmProxyFederationPolicy = BroadcastAMRMProxyPolicy.class;
   }
+
+  @Override
+  public WeightedPolicyInfo getWeightedPolicyInfo() {
+    return null;
+  }
+
+  @Override
+  public void setWeightedPolicyInfo(WeightedPolicyInfo weightedPolicyInfo) {
+  }
+
+  @Override
+  public boolean isSupportWeightedPolicyInfo() {
+    return false;
+  }
 }