Pārlūkot izejas kodu

YARN-11692. Support mixed cgroup v1/v2 controller structure (#6821)

Peter Szucs 11 mēneši atpakaļ
vecāks
revīzija
129d91f7b2

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -2733,6 +2733,10 @@ public class YarnConfiguration extends Configuration {
   public static final String NM_LINUX_CONTAINER_CGROUPS_MOUNT_PATH =
     NM_PREFIX + "linux-container-executor.cgroups.mount-path";
 
+  /** Where the linux container executor should mount cgroups v2 if not found. */
+  public static final String NM_LINUX_CONTAINER_CGROUPS_V2_MOUNT_PATH =
+      NM_PREFIX + "linux-container-executor.cgroups.v2.mount-path";
+
   /**
    * Whether the apps should run in strict resource usage mode(not allowed to
    * use spare CPU)

+ 14 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -2087,6 +2087,20 @@
     <name>yarn.nodemanager.linux-container-executor.cgroups.mount-path</name>
   </property>
 
+  <property>
+    <description>This property sets the mount path for CGroups v2.
+      This parameter is optional, and needed to be set only in mixed mode,
+      when CGroups v2 is mounted alongside with Cgroups v1.
+      For example, in hybrid mode, CGroups v1 controllers can be mounted under /sys/fs/cgroup/
+      (for example /sys/fs/cgroup/cpu,cpuacct), while v2 can be mounted in /sys/fs/cgroup/unified folder.
+
+      If this value is not set, the value of
+      yarn.nodemanager.linux-container-executor.cgroups.mount-path
+      will be used for CGroups v2 as well.
+    </description>
+    <name>yarn.nodemanager.linux-container-executor.cgroups.v2.mount-path</name>
+  </property>
+
   <property>
     <description>Delay in ms between attempts to remove linux cgroup</description>
     <name>yarn.nodemanager.linux-container-executor.cgroups.delete-delay-ms</name>

+ 12 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsMountConfig.java

@@ -26,11 +26,17 @@ public class CGroupsMountConfig {
   private final boolean enableMount;
   private final String mountPath;
 
+  // CGroups v2 mount path is only relevant in mixed CGroups v1/v2 mode,
+  // where v2 is mounted alongside with v1.
+  private final String v2MountPath;
+
   public CGroupsMountConfig(Configuration conf) {
     this.enableMount = conf.getBoolean(YarnConfiguration.
         NM_LINUX_CONTAINER_CGROUPS_MOUNT, false);
     this.mountPath = conf.get(YarnConfiguration.
         NM_LINUX_CONTAINER_CGROUPS_MOUNT_PATH, null);
+    this.v2MountPath = conf.get(YarnConfiguration.
+        NM_LINUX_CONTAINER_CGROUPS_V2_MOUNT_PATH, mountPath);
   }
 
   public boolean ensureMountPathIsDefined() throws ResourceHandlerException {
@@ -62,11 +68,16 @@ public class CGroupsMountConfig {
     return mountPath;
   }
 
+  public String getV2MountPath() {
+    return v2MountPath;
+  }
+
   @Override
   public String toString() {
     return "CGroupsMountConfig{" +
         "enableMount=" + enableMount +
-        ", mountPath='" + mountPath + '\'' +
+        ", mountPath='" + mountPath +
+        ", v2MountPath='" + v2MountPath + '\'' +
         '}';
   }
 }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsV2HandlerImpl.java

@@ -97,8 +97,8 @@ class CGroupsV2HandlerImpl extends AbstractCGroupsHandler {
   @Override
   protected Map<String, Set<String>> parsePreConfiguredMountPath() throws IOException {
     Map<String, Set<String>> controllerMappings = new HashMap<>();
-    controllerMappings.put(this.cGroupsMountConfig.getMountPath(),
-        readControllersFile(this.cGroupsMountConfig.getMountPath()));
+    controllerMappings.put(this.cGroupsMountConfig.getV2MountPath(),
+        readControllersFile(this.cGroupsMountConfig.getV2MountPath()));
     return controllerMappings;
   }
 

+ 60 - 42
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/ResourceHandlerModule.java

@@ -63,11 +63,12 @@ public class ResourceHandlerModule {
    * as resource metrics functionality. We need to ensure that the same
    * instance is used for both.
    */
+  private static volatile CGroupsHandler cGroupV1Handler;
+  private static volatile CGroupsHandler cGroupV2Handler;
   private static volatile TrafficControlBandwidthHandlerImpl
       trafficControlBandwidthHandler;
   private static volatile NetworkPacketTaggingHandlerImpl
       networkPacketTaggingHandlerImpl;
-  private static volatile CGroupsHandler cGroupsHandler;
   private static volatile CGroupsBlkioResourceHandlerImpl
       cGroupsBlkioResourceHandler;
   private static volatile MemoryResourceHandler
@@ -75,23 +76,42 @@ public class ResourceHandlerModule {
   private static volatile CpuResourceHandler
       cGroupsCpuResourceHandler;
 
-  /**
-   * Returns an initialized, thread-safe CGroupsHandler instance.
-   */
-  private static CGroupsHandler getInitializedCGroupsHandler(Configuration conf)
+  private static void initializeCGroupHandlers(Configuration conf)
+      throws ResourceHandlerException {
+    initializeCGroupV1Handler(conf);
+    if (cgroupsV2Enabled) {
+      initializeCGroupV2Handler(conf);
+    }
+  }
+
+  private static void initializeCGroupV1Handler(Configuration conf)
       throws ResourceHandlerException {
-    if (cGroupsHandler == null) {
+    if (cGroupV1Handler == null) {
       synchronized (CGroupsHandler.class) {
-        if (cGroupsHandler == null) {
-          cGroupsHandler = cgroupsV2Enabled
-              ? new CGroupsV2HandlerImpl(conf, PrivilegedOperationExecutor.getInstance(conf))
-              : new CGroupsHandlerImpl(conf, PrivilegedOperationExecutor.getInstance(conf));
-          LOG.debug("Value of CGroupsHandler is: {}", cGroupsHandler);
+        if (cGroupV1Handler == null) {
+          cGroupV1Handler = new CGroupsHandlerImpl(
+              conf, PrivilegedOperationExecutor.getInstance(conf));
+          LOG.debug("Value of CGroupsV1Handler is: {}", cGroupV1Handler);
         }
       }
     }
+  }
 
-    return cGroupsHandler;
+  private static void initializeCGroupV2Handler(Configuration conf)
+      throws ResourceHandlerException {
+    if (cGroupV2Handler == null) {
+      synchronized (CGroupsHandler.class) {
+        if (cGroupV2Handler == null) {
+          cGroupV2Handler = new CGroupsV2HandlerImpl(
+              conf, PrivilegedOperationExecutor.getInstance(conf));
+          LOG.debug("Value of CGroupsV2Handler is: {}", cGroupV2Handler);
+        }
+      }
+    }
+  }
+
+  private static boolean isMountedInCGroupsV2(CGroupsHandler.CGroupController controller) {
+    return (cGroupV2Handler != null && cGroupV2Handler.getControllerPath(controller) != null);
   }
 
   /**
@@ -101,7 +121,7 @@ public class ResourceHandlerModule {
    */
 
   public static CGroupsHandler getCGroupsHandler() {
-    return cGroupsHandler;
+    return cGroupV1Handler;
   }
 
   /**
@@ -109,10 +129,10 @@ public class ResourceHandlerModule {
    * not initialized, or if the path is empty.
    */
   public static String getCgroupsRelativeRoot() {
-    if (cGroupsHandler == null) {
+    if (getCGroupsHandler() == null) {
       return null;
     }
-    String cGroupPath = cGroupsHandler.getRelativePathForCGroup("");
+    String cGroupPath = getCGroupsHandler().getRelativePathForCGroup("");
     if (cGroupPath == null || cGroupPath.isEmpty()) {
       return null;
     }
@@ -153,9 +173,13 @@ public class ResourceHandlerModule {
         synchronized (CpuResourceHandler.class) {
           if (cGroupsCpuResourceHandler == null) {
             LOG.debug("Creating new cgroups cpu handler");
-            cGroupsCpuResourceHandler = cgroupsV2Enabled
-                ? new CGroupsV2CpuResourceHandlerImpl(getInitializedCGroupsHandler(conf))
-                : new CGroupsCpuResourceHandlerImpl(getInitializedCGroupsHandler(conf));
+
+            initializeCGroupHandlers(conf);
+            if (isMountedInCGroupsV2(CGroupsHandler.CGroupController.CPU)) {
+              cGroupsCpuResourceHandler = new CGroupsV2CpuResourceHandlerImpl(cGroupV2Handler);
+            } else {
+              cGroupsCpuResourceHandler = new CGroupsCpuResourceHandlerImpl(cGroupV1Handler);
+            }
             return cGroupsCpuResourceHandler;
           }
         }
@@ -173,9 +197,11 @@ public class ResourceHandlerModule {
         synchronized (OutboundBandwidthResourceHandler.class) {
           if (trafficControlBandwidthHandler == null) {
             LOG.info("Creating new traffic control bandwidth handler.");
+
+            initializeCGroupHandlers(conf);
             trafficControlBandwidthHandler = new
                 TrafficControlBandwidthHandlerImpl(PrivilegedOperationExecutor
-                .getInstance(conf), getInitializedCGroupsHandler(conf),
+                .getInstance(conf), cGroupV1Handler,
                 new TrafficController(conf, PrivilegedOperationExecutor
                     .getInstance(conf)));
           }
@@ -208,10 +234,11 @@ public class ResourceHandlerModule {
       synchronized (OutboundBandwidthResourceHandler.class) {
         if (networkPacketTaggingHandlerImpl == null) {
           LOG.info("Creating new network-tagging-handler.");
+
+          initializeCGroupHandlers(conf);
           networkPacketTaggingHandlerImpl =
               new NetworkPacketTaggingHandlerImpl(
-                  PrivilegedOperationExecutor.getInstance(conf),
-                  getInitializedCGroupsHandler(conf));
+                  PrivilegedOperationExecutor.getInstance(conf), cGroupV1Handler);
         }
       }
     }
@@ -239,9 +266,10 @@ public class ResourceHandlerModule {
       synchronized (DiskResourceHandler.class) {
         if (cGroupsBlkioResourceHandler == null) {
           LOG.debug("Creating new cgroups blkio handler");
+
+          initializeCGroupHandlers(conf);
           cGroupsBlkioResourceHandler =
-              new CGroupsBlkioResourceHandlerImpl(
-                  getInitializedCGroupsHandler(conf));
+              new CGroupsBlkioResourceHandlerImpl(cGroupV1Handler);
         }
       }
     }
@@ -263,9 +291,13 @@ public class ResourceHandlerModule {
     if (cGroupsMemoryResourceHandler == null) {
       synchronized (MemoryResourceHandler.class) {
         if (cGroupsMemoryResourceHandler == null) {
-          cGroupsMemoryResourceHandler = cgroupsV2Enabled
-              ? new CGroupsV2MemoryResourceHandlerImpl(getInitializedCGroupsHandler(conf))
-              : new CGroupsMemoryResourceHandlerImpl(getInitializedCGroupsHandler(conf));
+
+          initializeCGroupHandlers(conf);
+          if (isMountedInCGroupsV2(CGroupsHandler.CGroupController.MEMORY)) {
+            cGroupsMemoryResourceHandler = new CGroupsV2MemoryResourceHandlerImpl(cGroupV2Handler);
+          } else {
+            cGroupsMemoryResourceHandler = new CGroupsMemoryResourceHandlerImpl(cGroupV1Handler);
+          }
         }
       }
     }
@@ -327,9 +359,10 @@ public class ResourceHandlerModule {
     }
 
     for (ResourcePlugin plugin : pluginMap.values()) {
+      initializeCGroupHandlers(conf);
       addHandlerIfNotNull(handlerList,
           plugin.createResourceHandler(nmContext,
-              getInitializedCGroupsHandler(conf),
+              cGroupV1Handler,
               PrivilegedOperationExecutor.getInstance(conf)));
     }
   }
@@ -360,21 +393,6 @@ public class ResourceHandlerModule {
     resourceHandlerChain = null;
   }
 
-  @VisibleForTesting
-  static void resetCgroupsHandler() {
-    cGroupsHandler = null;
-  }
-
-  @VisibleForTesting
-  static void resetCpuResourceHandler() {
-    cGroupsCpuResourceHandler = null;
-  }
-
-  @VisibleForTesting
-  static void resetMemoryResourceHandler() {
-    cGroupsMemoryResourceHandler = null;
-  }
-
   /**
    * If a cgroup mount directory is specified, it returns cgroup directories
    * with valid names.

+ 145 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestCGroupsV2HandlerImpl.java

@@ -210,6 +210,111 @@ public class TestCGroupsV2HandlerImpl extends TestCGroupsHandlerBase {
     Assert.assertEquals(parentDir.getAbsolutePath(), memoryDir);
   }
 
+  /*
+   * Create a mock mtab file with the following content for hybrid v1/v2:
+   * cgroup2 /path/to/parentV2Dir cgroup2 rw,nosuid,nodev,noexec,relatime,memory_recursiveprot 0 0
+   * cgroup /path/to/parentDir/memory cgroup rw,nosuid,nodev,noexec,relatime,memory 0 0
+   *
+   * Create the following cgroup hierarchy:
+   *
+   *                                           parentDir
+   *                              __________________________________
+   *                             /                                  \
+   *                          unified                             memory
+   *       _________________________________________________
+   *      /                     \                           \
+   *  cgroup.controllers     cgroup.subtree_control   test-hadoop-yarn (hierarchyDir)
+   *                                                        _________________
+   *                                                       /                 \
+   *                                              cgroup.controllers   cgroup.subtree_control
+   */
+  public File createPremountedHybridCgroups(File v1ParentDir)
+      throws IOException {
+    File v2ParentDir = new File(v1ParentDir, "unified");
+
+    String mtabContent =
+        "cgroup " + v1ParentDir.getAbsolutePath() + "/memory"
+            + " cgroup rw,nosuid,nodev,noexec,relatime,memory 0 0\n"
+        + "cgroup2 " + v2ParentDir.getAbsolutePath()
+            + " cgroup2 rw,nosuid,nodev,noexec,relatime,memory_recursiveprot 0 0\n";
+
+    File mockMtab = createFileWithContent(v1ParentDir, UUID.randomUUID().toString(), mtabContent);
+
+    String enabledV2Controllers = "cpuset cpu io hugetlb pids rdma misc\n";
+    File controllersFile = createFileWithContent(v2ParentDir,
+        CGroupsHandler.CGROUP_CONTROLLERS_FILE, enabledV2Controllers);
+
+    File subtreeControlFile = new File(v2ParentDir, CGroupsHandler.CGROUP_SUBTREE_CONTROL_FILE);
+    Assert.assertTrue("empty subtree_control file should be created",
+        subtreeControlFile.createNewFile());
+
+    File hierarchyDir = new File(v2ParentDir, hierarchy);
+    if (!hierarchyDir.mkdirs()) {
+      String message = "Could not create directory " + hierarchyDir.getAbsolutePath();
+      throw new IOException(message);
+    }
+    hierarchyDir.deleteOnExit();
+
+    FileUtils.copyFile(controllersFile, new File(hierarchyDir,
+        CGroupsHandler.CGROUP_CONTROLLERS_FILE));
+    FileUtils.copyFile(subtreeControlFile, new File(hierarchyDir,
+        CGroupsHandler.CGROUP_SUBTREE_CONTROL_FILE));
+
+    return mockMtab;
+  }
+
+  @Test
+  public void testHybridMtabParsing() throws Exception {
+    // Initialize mtab and cgroup dir
+    File v1ParentDir = new File(tmpPath);
+
+    File v2ParentDir = new File(v1ParentDir, "unified");
+    Assert.assertTrue("temp dir should be created", v2ParentDir.mkdirs());
+    v2ParentDir.deleteOnExit();
+
+    // create mock cgroup
+    File mockMtabFile = createPremountedHybridCgroups(v1ParentDir);
+
+    // create memory cgroup for v1
+    File memoryCgroup = new File(v1ParentDir, "memory");
+    assertTrue("Directory should be created", memoryCgroup.mkdirs());
+
+    // init v1 and v2 handlers
+    CGroupsHandlerImpl cGroupsHandler = new CGroupsHandlerImpl(
+        createMountConfiguration(),
+        privilegedOperationExecutorMock, mockMtabFile.getAbsolutePath());
+    CGroupsV2HandlerImpl cGroupsV2Handler = new CGroupsV2HandlerImpl(
+        createMountConfiguration(),
+        privilegedOperationExecutorMock, mockMtabFile.getAbsolutePath());
+
+    // Verify resource handlers that are enabled in v1
+    Map<String, Set<String>> newMtab =
+        cGroupsHandler.parseMtab(mockMtabFile.getAbsolutePath());
+    Map<CGroupsHandler.CGroupController, String> controllerv1Paths =
+        cGroupsHandler.initializeControllerPathsFromMtab(
+            newMtab);
+
+    Assert.assertEquals(1, controllerv1Paths.size());
+    assertTrue(controllerv1Paths
+        .containsKey(CGroupsHandler.CGroupController.MEMORY));
+    String memoryDir =
+        controllerv1Paths.get(CGroupsHandler.CGroupController.MEMORY);
+    Assert.assertEquals(memoryCgroup.getAbsolutePath(), memoryDir);
+
+    // Verify resource handlers that are enabled in v2
+    newMtab =
+        cGroupsV2Handler.parseMtab(mockMtabFile.getAbsolutePath());
+    Map<CGroupsHandler.CGroupController, String> controllerPaths =
+        cGroupsV2Handler.initializeControllerPathsFromMtab(
+            newMtab);
+
+    Assert.assertEquals(3, controllerPaths.size());
+    assertTrue(controllerPaths
+        .containsKey(CGroupsHandler.CGroupController.CPU));
+    String cpuDir = controllerPaths.get(CGroupsHandler.CGroupController.CPU);
+    Assert.assertEquals(v2ParentDir.getAbsolutePath(), cpuDir);
+  }
+
   @Test
   public void testManualCgroupSetting() throws Exception {
     YarnConfiguration conf = new YarnConfiguration();
@@ -217,8 +322,27 @@ public class TestCGroupsV2HandlerImpl extends TestCGroupsHandlerBase {
     conf.set(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_HIERARCHY,
         "/hadoop-yarn");
 
-    File baseCgroup = new File(tmpPath);
-    File subCgroup = new File(tmpPath, "/hadoop-yarn");
+    validateCgroupV2Controllers(conf, tmpPath);
+  }
+
+  @Test
+  public void testManualHybridCgroupSetting() throws Exception {
+    String unifiedPath = tmpPath + "/unified";
+
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_MOUNT_PATH, tmpPath);
+    conf.set(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_V2_MOUNT_PATH, unifiedPath);
+    conf.set(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_HIERARCHY,
+        "/hadoop-yarn");
+
+    validateCgroupV1Controllers(conf, tmpPath);
+    validateCgroupV2Controllers(conf, unifiedPath);
+  }
+
+  private void validateCgroupV2Controllers(YarnConfiguration conf, String mountPath)
+      throws Exception {
+    File baseCgroup = new File(mountPath);
+    File subCgroup = new File(mountPath, "/hadoop-yarn");
     Assert.assertTrue("temp dir should be created", subCgroup.mkdirs());
     subCgroup.deleteOnExit();
 
@@ -235,8 +359,8 @@ public class TestCGroupsV2HandlerImpl extends TestCGroupsHandlerBase {
     cGroupsHandler.initializeCGroupController(CGroupsHandler.CGroupController.CPU);
 
     Assert.assertEquals("CPU cgroup path was not set", subCgroup.getAbsolutePath(),
-            new File(cGroupsHandler.getPathForCGroup(
-                CGroupsHandler.CGroupController.CPU, "")).getAbsolutePath());
+        new File(cGroupsHandler.getPathForCGroup(
+            CGroupsHandler.CGroupController.CPU, "")).getAbsolutePath());
 
     // Verify that the subtree control file was updated
     String subtreeControllersEnabledString = FileUtils.readFileToString(subtreeControlFile,
@@ -276,4 +400,21 @@ public class TestCGroupsV2HandlerImpl extends TestCGroupsHandlerBase {
     Assert.assertTrue("Controllers not enabled in subtree control file",
         cGroupsHandler.getValidCGroups().containsAll(subtreeControllersEnabled));
   }
+
+  private void validateCgroupV1Controllers(YarnConfiguration conf, String mountPath)
+      throws ResourceHandlerException {
+    File blkio = new File(new File(mountPath, "blkio"), "/hadoop-yarn");
+
+    Assert.assertTrue("temp dir should be created", blkio.mkdirs());
+
+    CGroupsHandlerImpl cGroupsv1Handler = new CGroupsHandlerImpl(conf, null);
+    cGroupsv1Handler.initializeCGroupController(
+        CGroupsHandler.CGroupController.BLKIO);
+
+    Assert.assertEquals("BLKIO CGRoup path was not set", blkio.getAbsolutePath(),
+        new File(cGroupsv1Handler.getPathForCGroup(
+            CGroupsHandler.CGroupController.BLKIO, "")).getAbsolutePath());
+
+    FileUtils.deleteQuietly(blkio);
+  }
 }

+ 0 - 68
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/TestResourceHandlerModule.java

@@ -49,9 +49,6 @@ public class TestResourceHandlerModule {
     networkEnabledConf.setBoolean(YarnConfiguration.NM_NETWORK_RESOURCE_ENABLED,
         true);
     ResourceHandlerModule.nullifyResourceHandlerChain();
-    ResourceHandlerModule.resetCgroupsHandler();
-    ResourceHandlerModule.resetCpuResourceHandler();
-    ResourceHandlerModule.resetMemoryResourceHandler();
   }
 
   @Test
@@ -114,69 +111,4 @@ public class TestResourceHandlerModule {
       Assert.fail("Null returned");
     }
   }
-
-  @Test
-  public void testCpuResourceHandlerClassForCgroupV1() throws ResourceHandlerException {
-    Configuration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.NM_CPU_RESOURCE_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_V2_ENABLED, false);
-
-    initResourceHandlerChain(conf);
-
-    Assert.assertTrue(ResourceHandlerModule.getCpuResourceHandler()
-        instanceof CGroupsCpuResourceHandlerImpl);
-    Assert.assertTrue(ResourceHandlerModule.getCGroupsHandler()
-        instanceof CGroupsHandlerImpl);
-  }
-
-  @Test
-  public void testCpuResourceHandlerClassForCgroupV2() throws ResourceHandlerException {
-    Configuration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.NM_CPU_RESOURCE_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_V2_ENABLED, true);
-
-    initResourceHandlerChain(conf);
-
-    Assert.assertTrue(ResourceHandlerModule.getCpuResourceHandler()
-        instanceof CGroupsV2CpuResourceHandlerImpl);
-    Assert.assertTrue(ResourceHandlerModule.getCGroupsHandler()
-        instanceof CGroupsV2HandlerImpl);
-  }
-
-  @Test
-  public void testMemoryResourceHandlerClassForCgroupV1() throws ResourceHandlerException {
-    Configuration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.NM_MEMORY_RESOURCE_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_V2_ENABLED, false);
-
-    initResourceHandlerChain(conf);
-
-    Assert.assertTrue(ResourceHandlerModule.getMemoryResourceHandler()
-        instanceof CGroupsMemoryResourceHandlerImpl);
-    Assert.assertTrue(ResourceHandlerModule.getCGroupsHandler()
-        instanceof CGroupsHandlerImpl);
-  }
-
-  @Test
-  public void testMemoryResourceHandlerClassForCgroupV2() throws ResourceHandlerException {
-    Configuration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.NM_MEMORY_RESOURCE_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.NM_LINUX_CONTAINER_CGROUPS_V2_ENABLED, true);
-
-    initResourceHandlerChain(conf);
-
-    Assert.assertTrue(ResourceHandlerModule.getMemoryResourceHandler()
-        instanceof CGroupsV2MemoryResourceHandlerImpl);
-    Assert.assertTrue(ResourceHandlerModule.getCGroupsHandler()
-        instanceof CGroupsV2HandlerImpl);
-  }
-
-  private void initResourceHandlerChain(Configuration conf) throws ResourceHandlerException {
-    ResourceHandlerChain resourceHandlerChain =
-        ResourceHandlerModule.getConfiguredResourceHandlerChain(conf,
-            mock(Context.class));
-    if (resourceHandlerChain == null) {
-      Assert.fail("Could not initialize resource handler chain");
-    }
-  }
 }