소스 검색

YARN-9909. Offline format of YarnConfigurationStore. Contributed by Prabhu Joseph.

Sunil G 5 년 전
부모
커밋
3990ffa083

+ 47 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java

@@ -105,12 +105,15 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.YarnConfigurationStore;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.YarnConfigurationStoreFactory;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.AllocationTagsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.MemoryPlacementConstraintManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.PlacementConstraintManagerService;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.MultiNodeSortingManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ProxyCAManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
@@ -1563,6 +1566,8 @@ public class ResourceManager extends CompositeService
       if (argv.length >= 1) {
         if (argv[0].equals("-format-state-store")) {
           deleteRMStateStore(conf);
+        } else if (argv[0].equals("-format-conf-store")) {
+          deleteRMConfStore(conf);
         } else if (argv[0].equals("-remove-application-from-state-store")
             && argv.length == 2) {
           removeApplication(conf, argv[1]);
@@ -1671,6 +1676,45 @@ public class ResourceManager extends CompositeService
     }
   }
 
+  /**
+   * Deletes the YarnConfigurationStore
+   *
+   * @param conf
+   * @throws Exception
+   */
+  @VisibleForTesting
+  static void deleteRMConfStore(Configuration conf) throws Exception {
+    ResourceManager rm = new ResourceManager();
+    rm.conf = conf;
+    ResourceScheduler scheduler = rm.createScheduler();
+    RMContextImpl rmContext = new RMContextImpl();
+    rmContext.setResourceManager(rm);
+
+    boolean isConfigurationMutable = false;
+    String confProviderStr = conf.get(
+        YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
+        YarnConfiguration.DEFAULT_CONFIGURATION_STORE);
+    switch (confProviderStr) {
+      case YarnConfiguration.MEMORY_CONFIGURATION_STORE:
+      case YarnConfiguration.LEVELDB_CONFIGURATION_STORE:
+      case YarnConfiguration.ZK_CONFIGURATION_STORE:
+      case YarnConfiguration.FS_CONFIGURATION_STORE:
+        isConfigurationMutable = true;
+        break;
+      default:
+    }
+
+    if (scheduler instanceof MutableConfScheduler && isConfigurationMutable) {
+      YarnConfigurationStore confStore = YarnConfigurationStoreFactory
+          .getStore(conf);
+      confStore.initialize(conf, conf, rmContext);
+      confStore.format();
+    } else {
+      System.out.println("Scheduler Configuration format only " +
+          "supported by MutableConfScheduler.");
+    }
+  }
+
   @VisibleForTesting
   static void removeApplication(Configuration conf, String applicationId)
       throws Exception {
@@ -1691,7 +1735,9 @@ public class ResourceManager extends CompositeService
   private static void printUsage(PrintStream out) {
     out.println("Usage: yarn resourcemanager [-format-state-store]");
     out.println("                            "
-        + "[-remove-application-from-state-store <appId>]" + "\n");
+        + "[-remove-application-from-state-store <appId>]");
+    out.println("                            "
+        + "[-format-conf-store]" + "\n");
 
     out.println("[-convert-fs-configuration ");
     out.println(FSConfigToCSConfigConverter.WARNING_TEXT);

+ 1 - 20
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/MutableCSConfigurationProvider.java

@@ -68,26 +68,7 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
 
   @Override
   public void init(Configuration config) throws IOException {
-    String store = config.get(
-        YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
-        YarnConfiguration.MEMORY_CONFIGURATION_STORE);
-    switch (store) {
-    case YarnConfiguration.MEMORY_CONFIGURATION_STORE:
-      this.confStore = new InMemoryConfigurationStore();
-      break;
-    case YarnConfiguration.LEVELDB_CONFIGURATION_STORE:
-      this.confStore = new LeveldbConfigurationStore();
-      break;
-    case YarnConfiguration.ZK_CONFIGURATION_STORE:
-      this.confStore = new ZKConfigurationStore();
-      break;
-    case YarnConfiguration.FS_CONFIGURATION_STORE:
-      this.confStore = new FSSchedulerConfigurationStore();
-      break;
-    default:
-      this.confStore = YarnConfigurationStoreFactory.getStore(config);
-      break;
-    }
+    this.confStore = YarnConfigurationStoreFactory.getStore(config);
     Configuration initialSchedConf = new Configuration(false);
     initialSchedConf.addResource(YarnConfiguration.CS_CONFIGURATION_FILE);
     this.schedConf = new Configuration(false);

+ 18 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/YarnConfigurationStoreFactory.java

@@ -37,10 +37,24 @@ public final class YarnConfigurationStoreFactory {
   }
 
   public static YarnConfigurationStore getStore(Configuration conf) {
-    Class<? extends YarnConfigurationStore> storeClass =
-        conf.getClass(YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
+    String store = conf.get(
+        YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
+        YarnConfiguration.MEMORY_CONFIGURATION_STORE);
+    switch (store) {
+      case YarnConfiguration.MEMORY_CONFIGURATION_STORE:
+        return new InMemoryConfigurationStore();
+      case YarnConfiguration.LEVELDB_CONFIGURATION_STORE:
+        return new LeveldbConfigurationStore();
+      case YarnConfiguration.ZK_CONFIGURATION_STORE:
+        return new ZKConfigurationStore();
+      case YarnConfiguration.FS_CONFIGURATION_STORE:
+        return new FSSchedulerConfigurationStore();
+      default:
+        Class<? extends YarnConfigurationStore> storeClass =
+            conf.getClass(YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
             InMemoryConfigurationStore.class, YarnConfigurationStore.class);
-    LOG.info("Using YarnConfigurationStore implementation - " + storeClass);
-    return ReflectionUtils.newInstance(storeClass, conf);
+        LOG.info("Using YarnConfigurationStore implementation - " + storeClass);
+        return ReflectionUtils.newInstance(storeClass, conf);
+    }
   }
 }

+ 35 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMStoreCommands.java

@@ -19,6 +19,8 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -66,6 +68,39 @@ public class TestRMStoreCommands {
     }
   }
 
+  @Test
+  public void testFormatConfStoreCmdForZK() throws Exception {
+    try (TestingServer curatorTestingServer =
+        TestZKRMStateStore.setupCuratorServer();
+        CuratorFramework curatorFramework = TestZKRMStateStore.
+            setupCuratorFramework(curatorTestingServer)) {
+      Configuration conf = TestZKRMStateStore.createHARMConf("rm1,rm2", "rm1",
+          1234, false, curatorTestingServer);
+      conf.set(YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
+          YarnConfiguration.ZK_CONFIGURATION_STORE);
+
+      ResourceManager rm = new MockRM(conf);
+      rm.start();
+
+      String confStorePath = conf.get(
+          YarnConfiguration.RM_SCHEDCONF_STORE_ZK_PARENT_PATH,
+          YarnConfiguration.DEFAULT_RM_SCHEDCONF_STORE_ZK_PARENT_PATH)
+          + "/CONF_STORE";
+      assertNotNull("Failed to initialize ZKConfigurationStore",
+          curatorFramework.checkExists().forPath(confStorePath));
+
+      rm.close();
+      try {
+        ResourceManager.deleteRMConfStore(conf);
+      } catch (Exception e) {
+        fail("Exception should not be thrown during format rm conf store" +
+            " operation.");
+      }
+      assertNull("Failed to format ZKConfigurationStore",
+          curatorFramework.checkExists().forPath(confStorePath));
+    }
+  }
+
   @Test
   public void testRemoveApplicationFromStateStoreCmdForZK() throws Exception {
     StateChangeRequestInfo req = new StateChangeRequestInfo(

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md

@@ -193,6 +193,7 @@ Usage: `yarn resourcemanager [-format-state-store]`
 |:---- |:---- |
 | -format-state-store | Formats the RMStateStore. This will clear the RMStateStore and is useful if past applications are no longer needed. This should be run only when the ResourceManager is not running. |
 | -remove-application-from-state-store \<appId\> | Remove the application from RMStateStore. This should be run only when the ResourceManager is not running. |
+| -format-conf-store | Formats the YarnConfigurationStore. This will clear the persisted scheduler configuration under YarnConfigurationStore. This should be run only when the ResourceManager is not running. |
 | -convert-fs-configuration [-y&#124;yarnsiteconfig] [-f&#124;fsconfig] [-r&#124;rulesconfig] [-o&#124;output-directory] [-p&#124;print] [-c&#124;cluster-resource] | WARNING: This feature is experimental and not intended for production use! Development is still in progress so the converter should not be considered complete! <br/> Converts the specified Fair Scheduler configuration to Capacity Scheduler configuration. Requires two mandatory input files. First, the yarn-site.xml with the following format: [-y&#124;yarnsiteconfig [\<Path to the yarn-site.xml file\>]. Secondly, the fair-scheduler.xml with the following format: [-f&#124;fsconfig [\<Path to the fair-scheduler.xml file\>]. This config is not mandatory if there is a reference in yarn-site.xml to the fair-scheduler.xml with the property 'yarn.scheduler.fair.allocation.file'. If both are defined, the -f option has precedence. The output directory of the config files should be specified as well, with: \[-o&#124;output-directory\ \<directory\>]. An optional rules config file could be also specified with the following format: [-r&#124;rulesconfig \<Path to the conversion rules file\>]. The rule config file's format is a property file. There's an additional \[-p&#124;print\] parameter, which is optional. If defined, the configuration will be emitted to the console instead. In its normal operation, the output files (yarn-site.xml and capacity-scheduler.xml) of this command is generated to the specified output directory. The cluster resource parameter (\[-c&#124;cluster-resource\] \<resource\>\]) needs to be specified if any queue has a maxResources setting with value as percentage. The format of the resource string is the same as in fair-scheduler.xml.) ] |
 
 Start the ResourceManager