Procházet zdrojové kódy

YARN-11675. Update MemoryResourceHandler implementation for cgroup v2 support (#6760)

Peter Szucs před 1 rokem
rodič
revize
08419c4233

+ 158 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/AbstractCGroupsMemoryResourceHandler.java

@@ -0,0 +1,158 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+@InterfaceStability.Unstable
+@InterfaceAudience.Private
+public abstract class AbstractCGroupsMemoryResourceHandler implements MemoryResourceHandler {
+
+  static final Logger LOG =
+      LoggerFactory.getLogger(CGroupsMemoryResourceHandlerImpl.class);
+  protected static final CGroupsHandler.CGroupController MEMORY =
+      CGroupsHandler.CGroupController.MEMORY;
+
+  private CGroupsHandler cGroupsHandler;
+
+  protected static final int OPPORTUNISTIC_SOFT_LIMIT = 0;
+  // multiplier to set the soft limit - value should be between 0 and 1
+  private float softLimit = 0.0f;
+  private boolean enforce = true;
+
+  public AbstractCGroupsMemoryResourceHandler(CGroupsHandler cGroupsHandler) {
+    this.cGroupsHandler = cGroupsHandler;
+  }
+
+  protected CGroupsHandler getCGroupsHandler() {
+    return cGroupsHandler;
+  }
+
+  @Override
+  public List<PrivilegedOperation> bootstrap(Configuration conf)
+      throws ResourceHandlerException {
+    this.cGroupsHandler.initializeCGroupController(MEMORY);
+    enforce = conf.getBoolean(
+        YarnConfiguration.NM_MEMORY_RESOURCE_ENFORCED,
+        YarnConfiguration.DEFAULT_NM_MEMORY_RESOURCE_ENFORCED);
+    float softLimitPerc = conf.getFloat(
+        YarnConfiguration.NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE,
+        YarnConfiguration.
+            DEFAULT_NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE);
+    softLimit = softLimitPerc / 100.0f;
+    if (softLimitPerc < 0.0f || softLimitPerc > 100.0f) {
+      throw new ResourceHandlerException(
+          "Illegal value '" + softLimitPerc + "' "
+              + YarnConfiguration.
+              NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE
+              + ". Value must be between 0 and 100.");
+    }
+    return null;
+  }
+
+  @Override
+  public List<PrivilegedOperation> updateContainer(Container container)
+      throws ResourceHandlerException {
+    String cgroupId = container.getContainerId().toString();
+    File cgroup = new File(cGroupsHandler.getPathForCGroup(MEMORY, cgroupId));
+    if (cgroup.exists()) {
+      //memory is in MB
+      long containerSoftLimit =
+          (long) (container.getResource().getMemorySize() * this.softLimit);
+      long containerHardLimit = container.getResource().getMemorySize();
+      if (enforce) {
+        try {
+          updateMemoryHardLimit(cgroupId, containerHardLimit);
+          ContainerTokenIdentifier id = container.getContainerTokenIdentifier();
+          if (id != null && id.getExecutionType() ==
+              ExecutionType.OPPORTUNISTIC) {
+            updateOpportunisticMemoryLimits(cgroupId);
+          } else {
+            updateGuaranteedMemoryLimits(cgroupId, containerSoftLimit);
+          }
+        } catch (ResourceHandlerException re) {
+          cGroupsHandler.deleteCGroup(MEMORY, cgroupId);
+          LOG.warn("Could not update cgroup for container", re);
+          throw re;
+        }
+      }
+    }
+    return null;
+  }
+
+  protected abstract void updateMemoryHardLimit(String cgroupId, long containerHardLimit)
+      throws ResourceHandlerException;
+
+  protected abstract void updateOpportunisticMemoryLimits(String cgroupId)
+      throws ResourceHandlerException;
+
+  protected abstract void updateGuaranteedMemoryLimits(String cgroupId, long containerSoftLimit)
+      throws ResourceHandlerException;
+
+  @Override
+  public List<PrivilegedOperation> reacquireContainer(ContainerId containerId)
+      throws ResourceHandlerException {
+    return null;
+  }
+
+  @Override
+  public List<PrivilegedOperation> preStart(Container container)
+      throws ResourceHandlerException {
+    String cgroupId = container.getContainerId().toString();
+    cGroupsHandler.createCGroup(MEMORY, cgroupId);
+    updateContainer(container);
+    List<PrivilegedOperation> ret = new ArrayList<>();
+    ret.add(new PrivilegedOperation(
+        PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+        PrivilegedOperation.CGROUP_ARG_PREFIX
+            + cGroupsHandler.getPathForCGroupTasks(MEMORY, cgroupId)));
+    return ret;
+  }
+
+  @Override
+  public List<PrivilegedOperation> postComplete(ContainerId containerId)
+      throws ResourceHandlerException {
+    cGroupsHandler.deleteCGroup(MEMORY, containerId.toString());
+    return null;
+  }
+
+  @Override
+  public List<PrivilegedOperation> teardown() throws ResourceHandlerException {
+    return null;
+  }
+
+  @Override
+  public String toString() {
+    return AbstractCGroupsMemoryResourceHandler.class.getName();
+  }
+}

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

@@ -123,6 +123,8 @@ public interface CGroupsHandler {
   String CGROUP_CONTROLLERS_FILE = "cgroup.controllers";
   String CGROUP_SUBTREE_CONTROL_FILE = "cgroup.subtree_control";
   String CGROUP_CPU_MAX = "max";
+  String CGROUP_MEMORY_MAX = "max";
+  String CGROUP_MEMORY_LOW = "low";
 
   // present in v1 and v2
   String CGROUP_PROCS_FILE = "cgroup.procs";

+ 21 - 104
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsMemoryResourceHandlerImpl.java

@@ -19,20 +19,12 @@
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
 
 import org.apache.hadoop.classification.VisibleForTesting;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
 
-import java.io.File;
-import java.util.ArrayList;
 import java.util.List;
 
 /**
@@ -43,32 +35,19 @@ import java.util.List;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class CGroupsMemoryResourceHandlerImpl implements MemoryResourceHandler {
+public class CGroupsMemoryResourceHandlerImpl extends AbstractCGroupsMemoryResourceHandler {
 
-  static final Logger LOG =
-       LoggerFactory.getLogger(CGroupsMemoryResourceHandlerImpl.class);
-  private static final CGroupsHandler.CGroupController MEMORY =
-      CGroupsHandler.CGroupController.MEMORY;
   private static final int OPPORTUNISTIC_SWAPPINESS = 100;
-  private static final int OPPORTUNISTIC_SOFT_LIMIT = 0;
-
-  private CGroupsHandler cGroupsHandler;
-  private boolean enforce = true;
   private int swappiness = 0;
-  // multiplier to set the soft limit - value should be between 0 and 1
-  private float softLimit = 0.0f;
 
   CGroupsMemoryResourceHandlerImpl(CGroupsHandler cGroupsHandler) {
-    this.cGroupsHandler = cGroupsHandler;
+    super(cGroupsHandler);
   }
 
   @Override
   public List<PrivilegedOperation> bootstrap(Configuration conf)
       throws ResourceHandlerException {
-    this.cGroupsHandler.initializeCGroupController(MEMORY);
-    enforce = conf.getBoolean(
-        YarnConfiguration.NM_MEMORY_RESOURCE_ENFORCED,
-        YarnConfiguration.DEFAULT_NM_MEMORY_RESOURCE_ENFORCED);
+    super.bootstrap(conf);
     swappiness = conf
         .getInt(YarnConfiguration.NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS,
             YarnConfiguration.DEFAULT_NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS);
@@ -78,18 +57,6 @@ public class CGroupsMemoryResourceHandlerImpl implements MemoryResourceHandler {
               + YarnConfiguration.NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS
               + ". Value must be between 0 and 100.");
     }
-    float softLimitPerc = conf.getFloat(
-        YarnConfiguration.NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE,
-        YarnConfiguration.
-            DEFAULT_NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE);
-    softLimit = softLimitPerc / 100.0f;
-    if (softLimitPerc < 0.0f || softLimitPerc > 100.0f) {
-      throw new ResourceHandlerException(
-          "Illegal value '" + softLimitPerc + "' "
-              + YarnConfiguration.
-                NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE
-              + ". Value must be between 0 and 100.");
-    }
     return null;
   }
 
@@ -99,81 +66,31 @@ public class CGroupsMemoryResourceHandlerImpl implements MemoryResourceHandler {
   }
 
   @Override
-  public List<PrivilegedOperation> reacquireContainer(ContainerId containerId)
-      throws ResourceHandlerException {
-    return null;
-  }
-
-  @Override
-  public List<PrivilegedOperation> updateContainer(Container container)
+  protected void updateMemoryHardLimit(String cgroupId, long containerHardLimit)
       throws ResourceHandlerException {
-    String cgroupId = container.getContainerId().toString();
-    File cgroup = new File(cGroupsHandler.getPathForCGroup(MEMORY, cgroupId));
-    if (cgroup.exists()) {
-      //memory is in MB
-      long containerSoftLimit =
-          (long) (container.getResource().getMemorySize() * this.softLimit);
-      long containerHardLimit = container.getResource().getMemorySize();
-      if (enforce) {
-        try {
-          cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
-              CGroupsHandler.CGROUP_PARAM_MEMORY_HARD_LIMIT_BYTES,
-              String.valueOf(containerHardLimit) + "M");
-          ContainerTokenIdentifier id = container.getContainerTokenIdentifier();
-          if (id != null && id.getExecutionType() ==
-              ExecutionType.OPPORTUNISTIC) {
-            cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
-                CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES,
-                String.valueOf(OPPORTUNISTIC_SOFT_LIMIT) + "M");
-            cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
-                CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS,
-                String.valueOf(OPPORTUNISTIC_SWAPPINESS));
-          } else {
-            cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
-                CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES,
-                String.valueOf(containerSoftLimit) + "M");
-            cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
-                CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS,
-                String.valueOf(swappiness));
-          }
-        } catch (ResourceHandlerException re) {
-          cGroupsHandler.deleteCGroup(MEMORY, cgroupId);
-          LOG.warn("Could not update cgroup for container", re);
-          throw re;
-        }
-      }
-    }
-    return null;
+    getCGroupsHandler().updateCGroupParam(MEMORY, cgroupId,
+        CGroupsHandler.CGROUP_PARAM_MEMORY_HARD_LIMIT_BYTES,
+        String.valueOf(containerHardLimit) + "M");
   }
 
   @Override
-  public List<PrivilegedOperation> preStart(Container container)
-      throws ResourceHandlerException {
-    String cgroupId = container.getContainerId().toString();
-    cGroupsHandler.createCGroup(MEMORY, cgroupId);
-    updateContainer(container);
-    List<PrivilegedOperation> ret = new ArrayList<>();
-    ret.add(new PrivilegedOperation(
-        PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
-        PrivilegedOperation.CGROUP_ARG_PREFIX
-            + cGroupsHandler.getPathForCGroupTasks(MEMORY, cgroupId)));
-    return ret;
+  protected void updateOpportunisticMemoryLimits(String cgroupId) throws ResourceHandlerException {
+    getCGroupsHandler().updateCGroupParam(MEMORY, cgroupId,
+        CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES,
+        String.valueOf(OPPORTUNISTIC_SOFT_LIMIT) + "M");
+    getCGroupsHandler().updateCGroupParam(MEMORY, cgroupId,
+        CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS,
+        String.valueOf(OPPORTUNISTIC_SWAPPINESS));
   }
 
   @Override
-  public List<PrivilegedOperation> postComplete(ContainerId containerId)
+  protected void updateGuaranteedMemoryLimits(String cgroupId, long containerSoftLimit)
       throws ResourceHandlerException {
-    cGroupsHandler.deleteCGroup(MEMORY, containerId.toString());
-    return null;
-  }
-
-  @Override
-  public List<PrivilegedOperation> teardown() throws ResourceHandlerException {
-    return null;
-  }
-
-  @Override
-  public String toString() {
-    return CGroupsMemoryResourceHandlerImpl.class.getName();
+    getCGroupsHandler().updateCGroupParam(MEMORY, cgroupId,
+        CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES,
+        String.valueOf(containerSoftLimit) + "M");
+    getCGroupsHandler().updateCGroupParam(MEMORY, cgroupId,
+        CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS,
+        String.valueOf(swappiness));
   }
 }

+ 51 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/resources/CGroupsV2MemoryResourceHandlerImpl.java

@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+/**
+ * Handler class to handle the memory controller. YARN already ships a
+ * physical memory monitor in Java but it isn't as
+ * good as CGroups. This handler sets the soft and hard memory limits. The soft
+ * limit is set to 90% of the hard limit.
+ */
+public class CGroupsV2MemoryResourceHandlerImpl extends AbstractCGroupsMemoryResourceHandler {
+
+  CGroupsV2MemoryResourceHandlerImpl(CGroupsHandler cGroupsHandler) {
+    super(cGroupsHandler);
+  }
+
+  @Override
+  protected void updateMemoryHardLimit(String cgroupId, long containerHardLimit)
+      throws ResourceHandlerException {
+    getCGroupsHandler().updateCGroupParam(MEMORY, cgroupId,
+        CGroupsHandler.CGROUP_MEMORY_MAX, String.valueOf(containerHardLimit) + "M");
+  }
+
+  @Override
+  protected void updateOpportunisticMemoryLimits(String cgroupId) throws ResourceHandlerException {
+    updateGuaranteedMemoryLimits(cgroupId, OPPORTUNISTIC_SOFT_LIMIT);
+  }
+
+  @Override
+  protected void updateGuaranteedMemoryLimits(String cgroupId, long containerSoftLimit)
+      throws ResourceHandlerException {
+    getCGroupsHandler().updateCGroupParam(MEMORY, cgroupId,
+        CGroupsHandler.CGROUP_MEMORY_LOW, String.valueOf(containerSoftLimit) + "M");
+  }
+}

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

@@ -0,0 +1,209 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class TestCgroupsV2MemoryResourceHandlerImpl {
+
+  private CGroupsHandler mockCGroupsHandler;
+  private CGroupsV2MemoryResourceHandlerImpl cGroupsMemoryResourceHandler;
+
+  @Before
+  public void setup() {
+    mockCGroupsHandler = mock(CGroupsHandler.class);
+    when(mockCGroupsHandler.getPathForCGroup(any(), any())).thenReturn(".");
+    cGroupsMemoryResourceHandler =
+        new CGroupsV2MemoryResourceHandlerImpl(mockCGroupsHandler);
+  }
+
+  @Test
+  public void testBootstrap() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false);
+    conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false);
+    List<PrivilegedOperation> ret =
+        cGroupsMemoryResourceHandler.bootstrap(conf);
+    verify(mockCGroupsHandler, times(1))
+        .initializeCGroupController(CGroupsHandler.CGroupController.MEMORY);
+    Assert.assertNull(ret);
+    conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, true);
+    try {
+      cGroupsMemoryResourceHandler.bootstrap(conf);
+    } catch (ResourceHandlerException re) {
+      Assert.fail("Pmem check should be allowed to run with cgroups");
+    }
+    conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false);
+    conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, true);
+    try {
+      cGroupsMemoryResourceHandler.bootstrap(conf);
+    } catch (ResourceHandlerException re) {
+      Assert.fail("Vmem check should be allowed to run with cgroups");
+    }
+  }
+
+  @Test
+  public void testPreStart() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false);
+    conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false);
+    cGroupsMemoryResourceHandler.bootstrap(conf);
+    String id = "container_01_01";
+    String path = "test-path/" + id;
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    Container mockContainer = mock(Container.class);
+    when(mockContainer.getContainerId()).thenReturn(mockContainerId);
+    when(mockCGroupsHandler
+        .getPathForCGroupTasks(CGroupsHandler.CGroupController.MEMORY, id))
+        .thenReturn(path);
+    int memory = 1024;
+    when(mockContainer.getResource())
+        .thenReturn(Resource.newInstance(memory, 1));
+    List<PrivilegedOperation> ret =
+        cGroupsMemoryResourceHandler.preStart(mockContainer);
+    verify(mockCGroupsHandler, times(1))
+        .createCGroup(CGroupsHandler.CGroupController.MEMORY, id);
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_MEMORY_MAX,
+            String.valueOf(memory) + "M");
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_MEMORY_LOW,
+            String.valueOf((int) (memory * 0.9)) + "M");
+    Assert.assertNotNull(ret);
+    Assert.assertEquals(1, ret.size());
+    PrivilegedOperation op = ret.get(0);
+    Assert.assertEquals(PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+        op.getOperationType());
+    List<String> args = op.getArguments();
+    Assert.assertEquals(1, args.size());
+    Assert.assertEquals(PrivilegedOperation.CGROUP_ARG_PREFIX + path,
+        args.get(0));
+  }
+
+  @Test
+  public void testPreStartNonEnforced() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false);
+    conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false);
+    conf.setBoolean(YarnConfiguration.NM_MEMORY_RESOURCE_ENFORCED, false);
+    cGroupsMemoryResourceHandler.bootstrap(conf);
+    String id = "container_01_01";
+    String path = "test-path/" + id;
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    Container mockContainer = mock(Container.class);
+    when(mockContainer.getContainerId()).thenReturn(mockContainerId);
+    when(mockCGroupsHandler
+        .getPathForCGroupTasks(CGroupsHandler.CGroupController.MEMORY, id))
+        .thenReturn(path);
+    int memory = 1024;
+    when(mockContainer.getResource())
+        .thenReturn(Resource.newInstance(memory, 1));
+    List<PrivilegedOperation> ret =
+        cGroupsMemoryResourceHandler.preStart(mockContainer);
+    verify(mockCGroupsHandler, times(1))
+        .createCGroup(CGroupsHandler.CGroupController.MEMORY, id);
+    verify(mockCGroupsHandler, times(0))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_MEMORY_MAX,
+            String.valueOf(memory) + "M");
+    verify(mockCGroupsHandler, times(0))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_MEMORY_LOW,
+            String.valueOf((int) (memory * 0.9)) + "M");
+    Assert.assertNotNull(ret);
+    Assert.assertEquals(1, ret.size());
+    PrivilegedOperation op = ret.get(0);
+    Assert.assertEquals(PrivilegedOperation.OperationType.ADD_PID_TO_CGROUP,
+        op.getOperationType());
+    List<String> args = op.getArguments();
+    Assert.assertEquals(1, args.size());
+    Assert.assertEquals(PrivilegedOperation.CGROUP_ARG_PREFIX + path,
+        args.get(0));
+  }
+
+  @Test
+  public void testReacquireContainer() throws Exception {
+    ContainerId containerIdMock = mock(ContainerId.class);
+    Assert.assertNull(
+        cGroupsMemoryResourceHandler.reacquireContainer(containerIdMock));
+  }
+
+  @Test
+  public void testPostComplete() throws Exception {
+    String id = "container_01_01";
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    Assert
+        .assertNull(cGroupsMemoryResourceHandler.postComplete(mockContainerId));
+    verify(mockCGroupsHandler, times(1))
+        .deleteCGroup(CGroupsHandler.CGroupController.MEMORY, id);
+  }
+
+  @Test
+  public void testTeardown() throws Exception {
+    Assert.assertNull(cGroupsMemoryResourceHandler.teardown());
+  }
+
+  @Test
+  public void testOpportunistic() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false);
+    conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false);
+
+    cGroupsMemoryResourceHandler.bootstrap(conf);
+    ContainerTokenIdentifier tokenId = mock(ContainerTokenIdentifier.class);
+    when(tokenId.getExecutionType()).thenReturn(ExecutionType.OPPORTUNISTIC);
+    Container container = mock(Container.class);
+    String id = "container_01_01";
+    ContainerId mockContainerId = mock(ContainerId.class);
+    when(mockContainerId.toString()).thenReturn(id);
+    when(container.getContainerId()).thenReturn(mockContainerId);
+    when(container.getContainerTokenIdentifier()).thenReturn(tokenId);
+    when(container.getResource()).thenReturn(Resource.newInstance(1024, 2));
+    cGroupsMemoryResourceHandler.preStart(container);
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_MEMORY_LOW, "0M");
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_MEMORY_MAX, "1024M");
+  }
+}