You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by jp...@apache.org on 2018/07/24 22:09:04 UTC

[1/2] mesos git commit: Updated XFS disk isolator docs.

Repository: mesos
Updated Branches:
  refs/heads/master 9047fdccc -> 66f0908fe


Updated XFS disk isolator docs.

Added project IDs reclaiming mechanism description.

Review: https://reviews.apache.org/r/68029/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/66f0908f
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/66f0908f
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/66f0908f

Branch: refs/heads/master
Commit: 66f0908fe49dc3808ebb4edf7bfb6c2401fff9b1
Parents: 15b3408
Author: Ilya Pronin <ip...@twopensource.com>
Authored: Tue Jul 24 12:50:45 2018 -0700
Committer: James Peach <jp...@apache.org>
Committed: Tue Jul 24 15:07:49 2018 -0700

----------------------------------------------------------------------
 docs/isolators/disk-xfs.md | 11 +++++++++++
 docs/monitoring.md         | 14 ++++++++++++++
 2 files changed, 25 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/66f0908f/docs/isolators/disk-xfs.md
----------------------------------------------------------------------
diff --git a/docs/isolators/disk-xfs.md b/docs/isolators/disk-xfs.md
index 96bb39e..53fdea3 100644
--- a/docs/isolators/disk-xfs.md
+++ b/docs/isolators/disk-xfs.md
@@ -39,6 +39,17 @@ to display the `fsxattr.projid` field. For example:
 
     $ xfs_io -r -c stat /mnt/mesos/
 
+Project IDs are not reclaimed until the sandboxes they were assigned to
+are garbage collected. The XFS Disk isolator periodically checks if
+sandboxes of terminated containers still exist and deallocates project
+IDs of the ones that were removed. Such checks are performed at
+intervals specified by the
+[`--disk_watch_interval`](configuration/agent.md#disk_watch_interval)
+flag. Current number of available project IDs and total number of
+project IDs used by the isolator can be tracked using
+`containerizer/mesos/disk/project_ids_free` and
+`containerizer/mesos/disk/project_ids_total` metrics.
+
 ## Killing containers
 
 The XFS Disk isolator flag `--xfs_kill_containers` will create container

http://git-wip-us.apache.org/repos/asf/mesos/blob/66f0908f/docs/monitoring.md
----------------------------------------------------------------------
diff --git a/docs/monitoring.md b/docs/monitoring.md
index 2f06e9b..336f3aa 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -1846,6 +1846,20 @@ containerizers.
 </tr>
 <tr>
   <td>
+  <code>containerizer/mesos/disk/project_ids_free</code>
+  </td>
+  <td>Number of free project IDs available to the XFS Disk isolator</td>
+  <td>Gauge</td>
+</tr>
+<tr>
+  <td>
+  <code>containerizer/mesos/disk/project_ids_total</code>
+  </td>
+  <td>Number of project IDs configured for the XFS Disk isolator</td>
+  <td>Gauge</td>
+</tr>
+<tr>
+  <td>
   <code>containerizer/mesos/provisioner/docker_store/image_pull_ms</code>
   </td>
   <td>Mesos containerizer docker image pull latency in ms </td>


[2/2] mesos git commit: Updated XFS disk isolator to reclaim project IDs after disk GC.

Posted by jp...@apache.org.
Updated XFS disk isolator to reclaim project IDs after disk GC.

Currently upon container destruction its project ID is unallocated by
the isolator and removed from the container work directory. However due
to API limitations we can't unset project IDs on symlinks that may exist
inside the directory. Because of that the project may still exist until
the container directory is garbage collected. If the project ID is
reused for a new container, any lingering symlinks that still have that
project ID will contribute to disk usage of the new container. Typically
symlinks don't take much space, but still this leads to inaccuracy in
disk space usage accounting.

This patch postpones project ID reclaiming until sandbox GC time. The
isolator periodically checks if sandboxes of terminated containers still
exist and deallocates project IDs of the ones that were removed. Checks
are performed every "disk_watch_interval". This mechanism can be
improved in the future if we introduce a way for the isolators to learn
about disk GCs.

Current number of available project IDs can be tracked with the new
"containerizer/mesos/disk/project_ids_free" and
"containerizer/mesos/disk/project_ids_total" metrics.

Review: https://reviews.apache.org/r/67914/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/15b3408c
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/15b3408c
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/15b3408c

Branch: refs/heads/master
Commit: 15b3408c11c8e5d3611c59646bf8fe7cd8234210
Parents: 9047fdc
Author: Ilya Pronin <ip...@twopensource.com>
Authored: Tue Jul 24 12:50:36 2018 -0700
Committer: James Peach <jp...@apache.org>
Committed: Tue Jul 24 15:07:49 2018 -0700

----------------------------------------------------------------------
 .../containerizer/mesos/isolators/xfs/disk.cpp  | 103 +++++++---
 .../containerizer/mesos/isolators/xfs/disk.hpp  |  21 +-
 src/tests/containerizer/xfs_quota_tests.cpp     | 196 ++++++++++++++++++-
 3 files changed, 294 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/15b3408c/src/slave/containerizer/mesos/isolators/xfs/disk.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/xfs/disk.cpp b/src/slave/containerizer/mesos/isolators/xfs/disk.cpp
index 25f52a4..783da04 100644
--- a/src/slave/containerizer/mesos/isolators/xfs/disk.cpp
+++ b/src/slave/containerizer/mesos/isolators/xfs/disk.cpp
@@ -23,9 +23,12 @@
 #include <process/id.hpp>
 #include <process/loop.hpp>
 
+#include <process/metrics/metrics.hpp>
+
 #include <stout/check.hpp>
 #include <stout/foreach.hpp>
 #include <stout/os.hpp>
+#include <stout/utils.hpp>
 
 #include <stout/os/stat.hpp>
 
@@ -172,7 +175,10 @@ Try<Isolator*> XfsDiskIsolatorProcess::create(const Flags& flags)
   return new MesosIsolator(Owned<MesosIsolatorProcess>(
       new XfsDiskIsolatorProcess(
           flags.container_disk_watch_interval,
-          quotaPolicy, flags.work_dir, totalProjectIds.get())));
+          quotaPolicy,
+          flags.work_dir,
+          totalProjectIds.get(),
+          flags.disk_watch_interval)));
 }
 
 
@@ -180,9 +186,11 @@ XfsDiskIsolatorProcess::XfsDiskIsolatorProcess(
     Duration _watchInterval,
     xfs::QuotaPolicy _quotaPolicy,
     const std::string& _workDir,
-    const IntervalSet<prid_t>& projectIds)
+    const IntervalSet<prid_t>& projectIds,
+    Duration _projectWatchInterval)
   : ProcessBase(process::ID::generate("xfs-disk-isolator")),
     watchInterval(_watchInterval),
+    projectWatchInterval(_projectWatchInterval),
     quotaPolicy(_quotaPolicy),
     workDir(_workDir),
     totalProjectIds(projectIds),
@@ -192,6 +200,9 @@ XfsDiskIsolatorProcess::XfsDiskIsolatorProcess(
   // configured project range.
 
   LOG(INFO) << "Allocating XFS project IDs from the range " << totalProjectIds;
+
+  metrics.project_ids_total = totalProjectIds.size();
+  metrics.project_ids_free = totalProjectIds.size();
 }
 
 
@@ -255,6 +266,12 @@ Future<Nothing> XfsDiskIsolatorProcess::recover(
     infos.put(containerId, Owned<Info>(new Info(sandbox, projectId.get())));
     freeProjectIds -= projectId.get();
 
+    // The operator could have changed the project ID range, so as per
+    // returnProjectId(), we should only count this if is is still in range.
+    if (totalProjectIds.contains(projectId.get())) {
+      --metrics.project_ids_free;
+    }
+
     // If this is a known orphan, the containerizer will send a cleanup call
     // later. If this is a live container, we will manage it. Otherwise, we have
     // to dispatch a cleanup ourselves.  Note that we don't wait for the result
@@ -476,36 +493,33 @@ Future<Nothing> XfsDiskIsolatorProcess::cleanup(const ContainerID& containerId)
 
   infos.erase(containerId);
 
-  LOG(INFO) << "Removing project ID " << projectId
-            << " from '" << directory << "'";
+  // Schedule the directory for project ID reclaiming.
+  //
+  // We don't reclaim project ID here but wait until sandbox GC time.
+  // This is because the sandbox can potentially contain symlinks,
+  // from which we can't remove the project ID due to kernel API
+  // limitations. Such symlinks would then contribute to disk usage
+  // of another container if the project ID was reused causing small
+  // inaccuracies in accounting.
+  scheduledProjects.put(projectId, directory);
+
+  LOG(INFO) << "Removing quota from project " << projectId
+            << " for '" << directory << "'";
 
   Try<Nothing> quotaStatus = xfs::clearProjectQuota(
       directory, projectId);
 
+  // Note that if we failed to clear the quota, we will still eventually
+  // reclaim the project ID. If there is a persistent error will the quota
+  // system, then we would ultimately fail to re-use that project ID since
+  // the quota update would fail.
   if (quotaStatus.isError()) {
     LOG(ERROR) << "Failed to clear quota for '"
                << directory << "': " << quotaStatus.error();
-  }
-
-  Try<Nothing> projectStatus = xfs::clearProjectId(directory);
-  if (projectStatus.isError()) {
-    LOG(ERROR) << "Failed to remove project ID "
-               << projectId
-               << " from '" << directory << "': "
-               << projectStatus.error();
-  }
-
-  // If we failed to remove the on-disk project ID we can't reclaim it
-  // because the quota would then be applied across two containers. This
-  // would be a project ID leak, but we could recover it at GC time if
-  // that was visible to isolators.
-  if (quotaStatus.isError() || projectStatus.isError()) {
-    freeProjectIds -= projectId;
     return Failure("Failed to cleanup '" + directory + "'");
-  } else {
-    returnProjectId(projectId);
-    return Nothing();
   }
+
+  return Nothing();
 }
 
 
@@ -518,6 +532,7 @@ Option<prid_t> XfsDiskIsolatorProcess::nextProjectId()
   prid_t projectId = freeProjectIds.begin()->lower();
 
   freeProjectIds -= projectId;
+  --metrics.project_ids_free;
   return projectId;
 }
 
@@ -530,6 +545,21 @@ void XfsDiskIsolatorProcess::returnProjectId(
   // and we recover a previous container from the old range.
   if (totalProjectIds.contains(projectId)) {
     freeProjectIds += projectId;
+    ++metrics.project_ids_free;
+  }
+}
+
+
+void XfsDiskIsolatorProcess::reclaimProjectIds()
+{
+  foreachpair (
+      prid_t projectId, const string& dir, utils::copy(scheduledProjects)) {
+    if (!os::exists(dir)) {
+      returnProjectId(projectId);
+      scheduledProjects.erase(projectId);
+      LOG(INFO) << "Reclaimed project ID " << projectId
+                << " from '" << dir << "'";
+    }
   }
 }
 
@@ -551,6 +581,33 @@ void XfsDiskIsolatorProcess::initialize()
           return process::Continue();
         });
   }
+
+  // Start a periodic check for which project IDs are currently in use.
+  process::loop(
+      self,
+      [=]() {
+        return process::after(projectWatchInterval);
+      },
+      [=](const Nothing&) -> process::ControlFlow<Nothing> {
+        reclaimProjectIds();
+        return process::Continue();
+      });
+}
+
+
+XfsDiskIsolatorProcess::Metrics::Metrics()
+  : project_ids_total("containerizer/mesos/disk/project_ids_total"),
+    project_ids_free("containerizer/mesos/disk/project_ids_free")
+{
+  process::metrics::add(project_ids_total);
+  process::metrics::add(project_ids_free);
+}
+
+
+XfsDiskIsolatorProcess::Metrics::~Metrics()
+{
+  process::metrics::remove(project_ids_free);
+  process::metrics::remove(project_ids_total);
 }
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/15b3408c/src/slave/containerizer/mesos/isolators/xfs/disk.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/xfs/disk.hpp b/src/slave/containerizer/mesos/isolators/xfs/disk.hpp
index 0891f77..38c467b 100644
--- a/src/slave/containerizer/mesos/isolators/xfs/disk.hpp
+++ b/src/slave/containerizer/mesos/isolators/xfs/disk.hpp
@@ -21,6 +21,8 @@
 
 #include <process/owned.hpp>
 
+#include <process/metrics/push_gauge.hpp>
+
 #include <stout/bytes.hpp>
 #include <stout/duration.hpp>
 #include <stout/hashmap.hpp>
@@ -76,7 +78,8 @@ private:
       Duration watchInterval,
       xfs::QuotaPolicy quotaPolicy,
       const std::string& workDir,
-      const IntervalSet<prid_t>& projectIds);
+      const IntervalSet<prid_t>& projectIds,
+      Duration projectWatchInterval);
 
   // Responsible for validating a container hasn't broken the soft limit.
   void check();
@@ -87,6 +90,10 @@ private:
   // Return this project ID to the unallocated pool.
   void returnProjectId(prid_t projectId);
 
+  // Check which project IDs are currently in use and deallocate the ones
+  // that are not.
+  void reclaimProjectIds();
+
   struct Info
   {
     explicit Info(const std::string& _directory, prid_t _projectId)
@@ -99,11 +106,23 @@ private:
   };
 
   const Duration watchInterval;
+  const Duration projectWatchInterval;
   xfs::QuotaPolicy quotaPolicy;
   const std::string workDir;
   const IntervalSet<prid_t> totalProjectIds;
   IntervalSet<prid_t> freeProjectIds;
   hashmap<ContainerID, process::Owned<Info>> infos;
+  hashmap<prid_t, std::string> scheduledProjects;
+
+  // Metrics used by the XFS disk isolator.
+  struct Metrics
+  {
+    Metrics();
+    ~Metrics();
+
+    process::metrics::PushGauge project_ids_total;
+    process::metrics::PushGauge project_ids_free;
+  } metrics;
 };
 
 } // namespace slave {

http://git-wip-us.apache.org/repos/asf/mesos/blob/15b3408c/src/tests/containerizer/xfs_quota_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/xfs_quota_tests.cpp b/src/tests/containerizer/xfs_quota_tests.cpp
index dc18a8a..59ec182 100644
--- a/src/tests/containerizer/xfs_quota_tests.cpp
+++ b/src/tests/containerizer/xfs_quota_tests.cpp
@@ -32,11 +32,14 @@
 #include <stout/os.hpp>
 #include <stout/path.hpp>
 
+#include "common/values.hpp"
+
 #include "linux/fs.hpp"
 
 #include "master/master.hpp"
 
 #include "slave/flags.hpp"
+#include "slave/gc_process.hpp"
 #include "slave/paths.hpp"
 #include "slave/slave.hpp"
 
@@ -63,11 +66,14 @@ using testing::Return;
 using mesos::internal::master::Master;
 
 using mesos::internal::slave::Fetcher;
+using mesos::internal::slave::GarbageCollectorProcess;
 using mesos::internal::slave::MesosContainerizer;
 using mesos::internal::slave::MesosContainerizerProcess;
 using mesos::internal::slave::Slave;
 using mesos::internal::slave::XfsDiskIsolatorProcess;
 
+using mesos::internal::values::rangesToIntervalSet;
+
 using mesos::master::detector::MasterDetector;
 
 namespace mesos {
@@ -963,14 +969,18 @@ TEST_F(ROOT_XFS_QuotaTest, NoCheckpointRecovery)
   // One sandbox and one symlink.
   ASSERT_EQ(2u, sandboxes->size());
 
-  // Scan the remaining sandboxes and make sure that no projects are assigned.
+  // Scan the remaining sandboxes and check that project ID is still assigned
+  // but quota is unset.
   foreach (const string& sandbox, sandboxes.get()) {
     // Skip the "latest" symlink.
     if (os::stat::islink(sandbox)) {
       continue;
     }
 
-    EXPECT_NONE(xfs::getProjectId(sandbox));
+    Result<prid_t> projectId = xfs::getProjectId(sandbox);
+    ASSERT_SOME(projectId);
+
+    EXPECT_NONE(xfs::getProjectQuota(sandbox, projectId.get()));
   }
 
   driver.stop();
@@ -1188,6 +1198,188 @@ TEST_F(ROOT_XFS_QuotaTest, RecoverOldContainers)
 }
 
 
+// Verify that XFS project IDs are reclaimed when sandbox directories they were
+// set on are garbage collected.
+TEST_F(ROOT_XFS_QuotaTest, ProjectIdReclaiming)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.gc_delay = Seconds(10);
+  flags.disk_watch_interval = Seconds(10);
+
+  Try<Resource> projects =
+    Resources::parse("projects", flags.xfs_project_range, "*");
+  ASSERT_SOME(projects);
+  ASSERT_EQ(Value::RANGES, projects->type());
+  Try<IntervalSet<prid_t>> totalProjectIds =
+    rangesToIntervalSet<prid_t>(projects->ranges());
+  ASSERT_SOME(totalProjectIds);
+
+  Fetcher fetcher(flags);
+
+  Try<MesosContainerizer*> _containerizer = MesosContainerizer::create(
+      flags, true, &fetcher);
+  ASSERT_SOME(_containerizer);
+  Owned<MesosContainerizer> containerizer(_containerizer.get());
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(
+      detector.get(), containerizer.get(), flags);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(_, _, _));
+
+  Future<vector<Offer>> offers1;
+  Future<vector<Offer>> offers2;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers1))
+    .WillOnce(FutureArg<1>(&offers2))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  JSON::Object metrics = Metrics();
+  EXPECT_EQ(totalProjectIds->size(),
+            metrics.values["containerizer/mesos/disk/project_ids_total"]);
+  EXPECT_EQ(totalProjectIds->size(),
+            metrics.values["containerizer/mesos/disk/project_ids_free"]);
+
+  AWAIT_READY(offers1);
+  ASSERT_FALSE(offers1->empty());
+  Offer offer = offers1->at(0);
+
+  TaskInfo task = createTask(
+      offer.slave_id(),
+      Resources::parse("cpus:1;mem:128;disk:2").get(),
+      "dd if=/dev/zero of=file bs=1048576 count=1 && sleep 1000");
+
+  Future<TaskStatus> startingStatus;
+  Future<TaskStatus> runningStatus;
+  Future<TaskStatus> exitStatus;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&startingStatus))
+    .WillOnce(FutureArg<1>(&runningStatus))
+    .WillOnce(FutureArg<1>(&exitStatus))
+    .WillRepeatedly(Return());
+
+  driver.launchTasks(offer.id(), {task});
+
+  AWAIT_READY(startingStatus);
+  EXPECT_EQ(task.task_id(), startingStatus->task_id());
+  EXPECT_EQ(TASK_STARTING, startingStatus->state());
+
+  AWAIT_READY(runningStatus);
+  EXPECT_EQ(task.task_id(), runningStatus->task_id());
+  EXPECT_EQ(TASK_RUNNING, runningStatus->state());
+
+  metrics = Metrics();
+  EXPECT_EQ(totalProjectIds->size(),
+            metrics.values["containerizer/mesos/disk/project_ids_total"]);
+  EXPECT_EQ(totalProjectIds->size() - 1,
+            metrics.values["containerizer/mesos/disk/project_ids_free"]);
+
+  Future<Nothing> schedule =
+    FUTURE_DISPATCH(_, &GarbageCollectorProcess::schedule);
+
+  driver.killTask(task.task_id());
+  AWAIT_READY(exitStatus);
+  EXPECT_EQ(TASK_KILLED, exitStatus->state());
+
+  AWAIT_READY(schedule);
+
+  Try<list<string>> sandboxes = getSandboxes();
+  ASSERT_SOME(sandboxes);
+  ASSERT_EQ(2u, sandboxes->size());
+
+  // Scan the remaining sandboxes and check that project ID is still assigned
+  // but quota is unset.
+  Option<prid_t> usedProjectId;
+  foreach (const string& sandbox, sandboxes.get()) {
+    if (!os::stat::islink(sandbox)) {
+      Result<prid_t> projectId = xfs::getProjectId(sandbox);
+      ASSERT_SOME(projectId);
+      usedProjectId = projectId.get();
+
+      EXPECT_NONE(xfs::getProjectQuota(sandbox, projectId.get()));
+    }
+  }
+  ASSERT_SOME(usedProjectId);
+
+  // Advance the clock to trigger sandbox GC and project ID usage check.
+  Clock::pause();
+  Clock::advance(flags.gc_delay);
+  Clock::settle();
+  Clock::advance(flags.disk_watch_interval);
+  Clock::settle();
+  Clock::resume();
+
+  // Check that the sandbox was GCed.
+  sandboxes = getSandboxes();
+  ASSERT_SOME(sandboxes);
+  ASSERT_TRUE(sandboxes->empty());
+
+  AWAIT_READY(offers2);
+  ASSERT_FALSE(offers2->empty());
+  offer = offers2->at(0);
+
+  metrics = Metrics();
+  EXPECT_EQ(totalProjectIds->size(),
+            metrics.values["containerizer/mesos/disk/project_ids_total"]);
+  EXPECT_EQ(totalProjectIds->size(),
+            metrics.values["containerizer/mesos/disk/project_ids_free"]);
+
+  task = createTask(
+      offer.slave_id(),
+      Resources::parse("cpus:1;mem:128;disk:2").get(),
+      "dd if=/dev/zero of=file bs=1048576 count=1 && sleep 1000");
+
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&startingStatus))
+    .WillOnce(FutureArg<1>(&runningStatus))
+    .WillOnce(FutureArg<1>(&exitStatus))
+    .WillRepeatedly(Return());
+
+  driver.launchTasks(offer.id(), {task});
+
+  AWAIT_READY(startingStatus);
+  EXPECT_EQ(task.task_id(), startingStatus->task_id());
+  EXPECT_EQ(TASK_STARTING, startingStatus->state());
+
+  AWAIT_READY(runningStatus);
+  EXPECT_EQ(task.task_id(), runningStatus->task_id());
+  EXPECT_EQ(TASK_RUNNING, runningStatus->state());
+
+  // Scan the sandboxes and check that the project ID was reused.
+  sandboxes = getSandboxes();
+  ASSERT_SOME(sandboxes);
+  EXPECT_EQ(2u, sandboxes->size());
+  foreach (const string& sandbox, sandboxes.get()) {
+    // Skip the "latest" symlink.
+    if (!os::stat::islink(sandbox)) {
+      EXPECT_SOME_EQ(usedProjectId.get(), xfs::getProjectId(sandbox));
+    }
+  }
+
+  metrics = Metrics();
+  EXPECT_EQ(totalProjectIds->size() - 1,
+            metrics.values["containerizer/mesos/disk/project_ids_free"]);
+
+  driver.killTask(task.task_id());
+  AWAIT_READY(exitStatus);
+  EXPECT_EQ(TASK_KILLED, exitStatus->state());
+
+  driver.stop();
+  driver.join();
+}
+
+
 TEST_F(ROOT_XFS_QuotaTest, IsolatorFlags)
 {
   slave::Flags flags;