浏览代码

YARN-1856. Added cgroups based memory monitoring for containers as another alternative to custom memory-monitoring. Contributed by Varun Vasudev.

Vinod Kumar Vavilapalli (I am also known as @tshooter.) 9 年之前
父节点
当前提交
4e7d32c0db

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -61,6 +61,9 @@ Release 2.9.0 - UNRELEASED
 
   NEW FEATURES
 
+    YARN-1856. Added cgroups based memory monitoring for containers as another
+    alternative to custom memory-monitoring. (Varun Vasudev via vinodkv)
+
   IMPROVEMENTS
 
     YARN-4072. ApplicationHistoryServer, WebAppProxyServer, NodeManager and

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

@@ -905,6 +905,30 @@ public class YarnConfiguration extends Configuration {
   public static final boolean DEFAULT_NM_ENABLE_HARDWARE_CAPABILITY_DETECTION =
       false;
 
+  @Private
+  public static final String NM_MEMORY_RESOURCE_PREFIX = NM_PREFIX
+      + "resource.memory.";
+
+  @Private
+  public static final String NM_MEMORY_RESOURCE_ENABLED =
+      NM_MEMORY_RESOURCE_PREFIX + "enabled";
+  @Private
+  public static final boolean DEFAULT_NM_MEMORY_RESOURCE_ENABLED = false;
+
+  @Private
+  public static final String NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS =
+      NM_MEMORY_RESOURCE_PREFIX + "cgroups.swappiness";
+  @Private
+  public static final int DEFAULT_NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS = 0;
+
+  @Private
+  public static final String NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE =
+      NM_MEMORY_RESOURCE_PREFIX + "cgroups.soft-limit-percentage";
+  @Private
+  public static final float
+      DEFAULT_NM_MEMORY_RESOURCE_CGROUPS_SOFT_LIMIT_PERCENTAGE =
+      90.0f;
+
   /**
    * Prefix for disk configurations. Work in progress: This configuration
    * parameter may be changed/removed in the future.

+ 2 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java

@@ -108,6 +108,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
         .add(YarnConfiguration.NM_NETWORK_RESOURCE_OUTBOUND_BANDWIDTH_YARN_MBIT);
     configurationPrefixToSkipCompare
         .add(YarnConfiguration.NM_DISK_RESOURCE_ENABLED);
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.NM_MEMORY_RESOURCE_PREFIX);
 
     // Set by container-executor.cfg
     configurationPrefixToSkipCompare.add(YarnConfiguration.NM_USER_HOME_DIR);

+ 8 - 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/CGroupsHandler.java

@@ -31,10 +31,12 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public interface CGroupsHandler {
+
   public enum CGroupController {
     CPU("cpu"),
     NET_CLS("net_cls"),
-    BLKIO("blkio");
+    BLKIO("blkio"),
+    MEMORY("memory");
 
     private final String name;
 
@@ -51,6 +53,11 @@ public interface CGroupsHandler {
   public static final String CGROUP_PARAM_CLASSID = "classid";
   public static final String CGROUP_PARAM_BLKIO_WEIGHT = "weight";
 
+  String CGROUP_PARAM_MEMORY_HARD_LIMIT_BYTES = "limit_in_bytes";
+  String CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES = "soft_limit_in_bytes";
+  String CGROUP_PARAM_MEMORY_SWAPPINESS = "swappiness";
+
+
   /**
    * Mounts a cgroup controller
    * @param controller - the controller being mounted

+ 156 - 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/CGroupsMemoryResourceHandlerImpl.java

@@ -0,0 +1,156 @@
+/**
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+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.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class CGroupsMemoryResourceHandlerImpl implements MemoryResourceHandler {
+
+  static final Log LOG = LogFactory.getLog(
+      CGroupsMemoryResourceHandlerImpl.class);
+  private static final CGroupsHandler.CGroupController MEMORY =
+      CGroupsHandler.CGroupController.MEMORY;
+
+  private CGroupsHandler cGroupsHandler;
+  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;
+  }
+
+  @Override
+  public List<PrivilegedOperation> bootstrap(Configuration conf)
+      throws ResourceHandlerException {
+    boolean pmemEnabled =
+        conf.getBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED,
+            YarnConfiguration.DEFAULT_NM_PMEM_CHECK_ENABLED);
+    boolean vmemEnabled =
+        conf.getBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED,
+            YarnConfiguration.DEFAULT_NM_VMEM_CHECK_ENABLED);
+    if (pmemEnabled || vmemEnabled) {
+      String msg = "The default YARN physical and/or virtual memory health"
+          + " checkers as well as the CGroups memory controller are enabled. "
+          + "If you wish to use the Cgroups memory controller, please turn off"
+          + " the default physical/virtual memory checkers by setting "
+          + YarnConfiguration.NM_PMEM_CHECK_ENABLED + " and "
+          + YarnConfiguration.NM_VMEM_CHECK_ENABLED + " to false.";
+      throw new ResourceHandlerException(msg);
+    }
+    this.cGroupsHandler.mountCGroupController(MEMORY);
+    swappiness = conf
+        .getInt(YarnConfiguration.NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS,
+            YarnConfiguration.DEFAULT_NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS);
+    if (swappiness < 0 || swappiness > 100) {
+      throw new ResourceHandlerException(
+          "Illegal value '" + swappiness + "' for "
+              + 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;
+  }
+
+  @VisibleForTesting
+  int getSwappiness() {
+    return swappiness;
+  }
+
+  @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();
+    //memory is in MB
+    long containerSoftLimit =
+        (long) (container.getResource().getMemory() * this.softLimit);
+    long containerHardLimit = container.getResource().getMemory();
+    cGroupsHandler.createCGroup(MEMORY, cgroupId);
+    try {
+      cGroupsHandler.updateCGroupParam(MEMORY, cgroupId,
+          CGroupsHandler.CGROUP_PARAM_MEMORY_HARD_LIMIT_BYTES,
+          String.valueOf(containerHardLimit) + "M");
+      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;
+    }
+    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;
+  }
+
+}

+ 27 - 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/MemoryResourceHandler.java

@@ -0,0 +1,27 @@
+/**
+ * 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;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface MemoryResourceHandler extends ResourceHandler {
+}

+ 28 - 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/ResourceHandlerModule.java

@@ -50,6 +50,8 @@ public class ResourceHandlerModule {
   private static volatile CGroupsHandler cGroupsHandler;
   private static volatile CGroupsBlkioResourceHandlerImpl
       cGroupsBlkioResourceHandler;
+  private static volatile CGroupsMemoryResourceHandlerImpl
+      cGroupsMemoryResourceHandler;
 
   /**
    * Returns an initialized, thread-safe CGroupsHandler instance.
@@ -69,7 +71,7 @@ public class ResourceHandlerModule {
   }
 
   private static TrafficControlBandwidthHandlerImpl
-  getTrafficControlBandwidthHandler(Configuration conf)
+    getTrafficControlBandwidthHandler(Configuration conf)
       throws ResourceHandlerException {
     if (conf.getBoolean(YarnConfiguration.NM_NETWORK_RESOURCE_ENABLED,
         YarnConfiguration.DEFAULT_NM_NETWORK_RESOURCE_ENABLED)) {
@@ -92,7 +94,7 @@ public class ResourceHandlerModule {
   }
 
   public static OutboundBandwidthResourceHandler
-  getOutboundBandwidthResourceHandler(Configuration conf)
+    getOutboundBandwidthResourceHandler(Configuration conf)
       throws ResourceHandlerException {
     return getTrafficControlBandwidthHandler(conf);
   }
@@ -119,6 +121,29 @@ public class ResourceHandlerModule {
     return cGroupsBlkioResourceHandler;
   }
 
+  public static MemoryResourceHandler getMemoryResourceHandler(
+      Configuration conf) throws ResourceHandlerException {
+    if (conf.getBoolean(YarnConfiguration.NM_MEMORY_RESOURCE_ENABLED,
+        YarnConfiguration.DEFAULT_NM_MEMORY_RESOURCE_ENABLED)) {
+      return getCgroupsMemoryResourceHandler(conf);
+    }
+    return null;
+  }
+
+  private static CGroupsMemoryResourceHandlerImpl
+    getCgroupsMemoryResourceHandler(
+      Configuration conf) throws ResourceHandlerException {
+    if (cGroupsMemoryResourceHandler == null) {
+      synchronized (MemoryResourceHandler.class) {
+        if (cGroupsMemoryResourceHandler == null) {
+          cGroupsMemoryResourceHandler =
+              new CGroupsMemoryResourceHandlerImpl(getCGroupsHandler(conf));
+        }
+      }
+    }
+    return cGroupsMemoryResourceHandler;
+  }
+
   private static void addHandlerIfNotNull(List<ResourceHandler> handlerList,
       ResourceHandler handler) {
     if (handler != null) {
@@ -132,6 +157,7 @@ public class ResourceHandlerModule {
 
     addHandlerIfNotNull(handlerList, getOutboundBandwidthResourceHandler(conf));
     addHandlerIfNotNull(handlerList, getDiskResourceHandler(conf));
+    addHandlerIfNotNull(handlerList, getMemoryResourceHandler(conf));
     resourceHandlerChain = new ResourceHandlerChain(handlerList);
   }
 

+ 172 - 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/TestCGroupsMemoryResourceHandlerImpl.java

@@ -0,0 +1,172 @@
+/**
+ * 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.Resource;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.Assert;
+
+import java.util.List;
+
+import static org.mockito.Mockito.*;
+
+public class TestCGroupsMemoryResourceHandlerImpl {
+
+  private CGroupsHandler mockCGroupsHandler;
+  private CGroupsMemoryResourceHandlerImpl cGroupsMemoryResourceHandler;
+
+  @Before
+  public void setup() {
+    mockCGroupsHandler = mock(CGroupsHandler.class);
+    cGroupsMemoryResourceHandler =
+        new CGroupsMemoryResourceHandlerImpl(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))
+        .mountCGroupController(CGroupsHandler.CGroupController.MEMORY);
+    Assert.assertNull(ret);
+    Assert.assertEquals("Default swappiness value incorrect", 0,
+        cGroupsMemoryResourceHandler.getSwappiness());
+    conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, true);
+    try {
+      cGroupsMemoryResourceHandler.bootstrap(conf);
+      Assert.fail("Pmem check should not be allowed to run with cgroups");
+    }
+    catch(ResourceHandlerException re) {
+      // do nothing
+    }
+    conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false);
+    conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, true);
+    try {
+      cGroupsMemoryResourceHandler.bootstrap(conf);
+      Assert.fail("Vmem check should not be allowed to run with cgroups");
+    }
+    catch(ResourceHandlerException re) {
+      // do nothing
+    }
+  }
+
+  @Test
+  public void testSwappinessValues() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false);
+    conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false);
+    conf.setInt(YarnConfiguration.NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS, -1);
+    try {
+      cGroupsMemoryResourceHandler.bootstrap(conf);
+      Assert.fail("Negative values for swappiness should not be allowed.");
+    }
+    catch (ResourceHandlerException re) {
+      // do nothing
+    }
+    try {
+      conf.setInt(YarnConfiguration.NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS, 101);
+      cGroupsMemoryResourceHandler.bootstrap(conf);
+      Assert.fail("Values greater than 100 for swappiness"
+          + " should not be allowed.");
+    }
+    catch (ResourceHandlerException re) {
+      // do nothing
+    }
+    conf.setInt(YarnConfiguration.NM_MEMORY_RESOURCE_CGROUPS_SWAPPINESS, 60);
+    cGroupsMemoryResourceHandler.bootstrap(conf);
+    Assert.assertEquals("Swappiness value incorrect", 60,
+        cGroupsMemoryResourceHandler.getSwappiness());
+  }
+
+  @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_PARAM_MEMORY_HARD_LIMIT_BYTES,
+            String.valueOf(memory) + "M");
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_PARAM_MEMORY_SOFT_LIMIT_BYTES,
+            String.valueOf((int) (memory * 0.9)) + "M");
+    verify(mockCGroupsHandler, times(1))
+        .updateCGroupParam(CGroupsHandler.CGroupController.MEMORY, id,
+            CGroupsHandler.CGROUP_PARAM_MEMORY_SWAPPINESS, String.valueOf(0));
+    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());
+  }
+}