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/04/30 21:22:14 UTC

[1/2] mesos git commit: Added soft limit and kill to `disk/xfs`.

Repository: mesos
Updated Branches:
  refs/heads/master 175416694 -> f4345c1f1


Added soft limit and kill to `disk/xfs`.

Added a new flag `--xfs_kill_containers` to the disk/xfs
isolator. This will create a 10 MB buffer between the soft
and hard limits for a container. When the soft limit is
exceeded the container will subsequently be killed.

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


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

Branch: refs/heads/master
Commit: 9277c50d93e616c038d6a7bf6c153cd8ac04d629
Parents: 1754166
Author: Harold Dost <h....@criteo.com>
Authored: Wed Apr 25 08:46:30 2018 -0700
Committer: James Peach <jp...@apache.org>
Committed: Mon Apr 30 13:46:38 2018 -0700

----------------------------------------------------------------------
 .../containerizer/mesos/isolators/xfs/disk.cpp  | 133 +++++++++++++++----
 .../containerizer/mesos/isolators/xfs/disk.hpp  |  14 +-
 .../containerizer/mesos/isolators/xfs/utils.cpp |  48 +++++--
 .../containerizer/mesos/isolators/xfs/utils.hpp |  20 ++-
 src/slave/flags.cpp                             |   8 +-
 src/slave/flags.hpp                             |   1 +
 src/tests/containerizer/xfs_quota_tests.cpp     |   4 +-
 7 files changed, 180 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9277c50d/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 8d9f8f8..db11f9a 100644
--- a/src/slave/containerizer/mesos/isolators/xfs/disk.cpp
+++ b/src/slave/containerizer/mesos/isolators/xfs/disk.cpp
@@ -18,7 +18,10 @@
 
 #include <glog/logging.h>
 
+#include <process/after.hpp>
+#include <process/dispatch.hpp>
 #include <process/id.hpp>
+#include <process/loop.hpp>
 
 #include <stout/check.hpp>
 #include <stout/foreach.hpp>
@@ -26,6 +29,8 @@
 
 #include <stout/os/stat.hpp>
 
+#include "common/protobuf_utils.hpp"
+
 #include "slave/paths.hpp"
 
 using std::list;
@@ -36,7 +41,6 @@ using process::Future;
 using process::Owned;
 using process::PID;
 using process::Process;
-using process::Promise;
 
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
@@ -156,21 +160,28 @@ Try<Isolator*> XfsDiskIsolatorProcess::create(const Flags& flags)
     return Error(status->message);
   }
 
-  xfs::QuotaPolicy quotaPolicy =
-    flags.enforce_container_disk_quota ? xfs::QuotaPolicy::ENFORCING
-                                       : xfs::QuotaPolicy::ACCOUNTING;
+  xfs::QuotaPolicy quotaPolicy = xfs::QuotaPolicy::ACCOUNTING;
+
+  if (flags.enforce_container_disk_quota) {
+    quotaPolicy = flags.xfs_kill_containers
+      ? xfs::QuotaPolicy::ENFORCING_ACTIVE
+      : xfs::QuotaPolicy::ENFORCING_PASSIVE;
+  }
 
   return new MesosIsolator(Owned<MesosIsolatorProcess>(
       new XfsDiskIsolatorProcess(
+          flags.container_disk_watch_interval,
           quotaPolicy, flags.work_dir, totalProjectIds.get())));
 }
 
 
 XfsDiskIsolatorProcess::XfsDiskIsolatorProcess(
+    Duration _watchInterval,
     xfs::QuotaPolicy _quotaPolicy,
     const std::string& _workDir,
     const IntervalSet<prid_t>& projectIds)
   : ProcessBase(process::ID::generate("xfs-disk-isolator")),
+    watchInterval(_watchInterval),
     quotaPolicy(_quotaPolicy),
     workDir(_workDir),
     totalProjectIds(projectIds),
@@ -298,15 +309,20 @@ Future<Option<ContainerLaunchInfo>> XfsDiskIsolatorProcess::prepare(
 }
 
 
-Future<Nothing> XfsDiskIsolatorProcess::isolate(
-    const ContainerID& containerId,
-    pid_t pid)
+Future<ContainerLimitation> XfsDiskIsolatorProcess::watch(
+    const ContainerID& containerId)
 {
-  if (!infos.contains(containerId)) {
-    return Failure("Unknown container");
+  if (infos.contains(containerId)) {
+    return infos[containerId]->limitation.future();
   }
 
-  return Nothing();
+  // Any container that did not have a project ID assigned when
+  // we recovered it won't be tracked. This will happend when the
+  // isolator is first enabled, since we didn't get a chance to
+  // assign project IDs to existing containers. We don't want to
+  // cause those containers to fail, so we just ignore them.
+  LOG(WARNING) << "Ignoring watch for unknown container " << containerId;
+  return Future<ContainerLimitation>();
 }
 
 
@@ -320,8 +336,8 @@ Future<Nothing> XfsDiskIsolatorProcess::update(
   }
 
   const Owned<Info>& info = infos[containerId];
-
   Option<Bytes> needed = getDiskResource(resources);
+
   if (needed.isNone()) {
     // TODO(jpeach) If there's no disk resource attached, we should set the
     // minimum quota (1 block), since a zero quota would be unconstrained.
@@ -342,9 +358,20 @@ Future<Nothing> XfsDiskIsolatorProcess::update(
       break;
     }
 
-    case xfs::QuotaPolicy::ENFORCING: {
+    case xfs::QuotaPolicy::ENFORCING_ACTIVE:
+    case xfs::QuotaPolicy::ENFORCING_PASSIVE: {
+      Bytes hardLimit = needed.get();
+
+      // The purpose behind adding to the hard limit is so that the soft
+      // limit can be exceeded thereby allowing us to check if the limit
+      // has been reached without allowing the process to allocate too
+      // much beyond the desired limit.
+      if (quotaPolicy == xfs::QuotaPolicy::ENFORCING_ACTIVE) {
+        hardLimit += Megabytes(10);
+      }
+
       Try<Nothing> status = xfs::setProjectQuota(
-          info->directory, info->projectId, needed.get());
+          info->directory, info->projectId, needed.get(), hardLimit);
 
       if (status.isError()) {
         return Failure("Failed to update quota for project " +
@@ -353,7 +380,7 @@ Future<Nothing> XfsDiskIsolatorProcess::update(
 
       LOG(INFO) << "Set quota on container " << containerId
                 << " for project " << info->projectId
-                << " to " << needed.get();
+                << " to " << needed.get() << "/" << hardLimit;
 
       break;
     }
@@ -364,6 +391,41 @@ Future<Nothing> XfsDiskIsolatorProcess::update(
 }
 
 
+void XfsDiskIsolatorProcess::check()
+{
+  CHECK(quotaPolicy == xfs::QuotaPolicy::ENFORCING_ACTIVE);
+
+  foreachpair(const ContainerID& containerId, const Owned<Info>& info, infos) {
+    Result<xfs::QuotaInfo> quotaInfo = xfs::getProjectQuota(
+        info->directory, info->projectId);
+
+    if (quotaInfo.isError()) {
+      LOG(WARNING) << "Failed to check disk usage for container '"
+                   << containerId  << "': " << quotaInfo.error();
+
+      continue;
+    }
+
+    // If the soft limit is exceeded the container should be killed.
+    if (quotaInfo->used > quotaInfo->softLimit) {
+      Resource resource;
+      resource.set_name("disk");
+      resource.set_type(Value::SCALAR);
+      resource.mutable_scalar()->set_value(
+        quotaInfo->used.bytes() / Bytes::MEGABYTES);
+
+      info->limitation.set(
+          protobuf::slave::createContainerLimitation(
+              Resources(resource),
+              "Disk usage (" + stringify(quotaInfo->used) +
+              ") exceeds quota (" +
+              stringify(quotaInfo->softLimit) + ")",
+              TaskStatus::REASON_CONTAINER_LIMITATION_DISK));
+    }
+  }
+}
+
+
 Future<ResourceStatistics> XfsDiskIsolatorProcess::usage(
     const ContainerID& containerId)
 {
@@ -408,26 +470,27 @@ Future<Nothing> XfsDiskIsolatorProcess::cleanup(const ContainerID& containerId)
 
   // Take a copy of the Info we are removing so that we can use it
   // to construct the Failure message if necessary.
-  const Info info = *infos[containerId];
+  const std::string directory = infos[containerId]->directory;
+  const prid_t projectId = infos[containerId]->projectId;
 
   infos.erase(containerId);
 
-  LOG(INFO) << "Removing project ID " << info.projectId
-            << " from '" << info.directory << "'";
+  LOG(INFO) << "Removing project ID " << projectId
+            << " from '" << directory << "'";
 
   Try<Nothing> quotaStatus = xfs::clearProjectQuota(
-      info.directory, info.projectId);
+      directory, projectId);
 
   if (quotaStatus.isError()) {
     LOG(ERROR) << "Failed to clear quota for '"
-               << info.directory << "': " << quotaStatus.error();
+               << directory << "': " << quotaStatus.error();
   }
 
-  Try<Nothing> projectStatus = xfs::clearProjectId(info.directory);
+  Try<Nothing> projectStatus = xfs::clearProjectId(directory);
   if (projectStatus.isError()) {
     LOG(ERROR) << "Failed to remove project ID "
-               << info.projectId
-               << " from '" << info.directory << "': "
+               << projectId
+               << " from '" << directory << "': "
                << projectStatus.error();
   }
 
@@ -436,10 +499,10 @@ Future<Nothing> XfsDiskIsolatorProcess::cleanup(const ContainerID& containerId)
   // 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 -= info.projectId;
-    return Failure("Failed to cleanup '" + info.directory + "'");
+    freeProjectIds -= projectId;
+    return Failure("Failed to cleanup '" + directory + "'");
   } else {
-    returnProjectId(info.projectId);
+    returnProjectId(projectId);
     return Nothing();
   }
 }
@@ -469,6 +532,26 @@ void XfsDiskIsolatorProcess::returnProjectId(
   }
 }
 
+
+void XfsDiskIsolatorProcess::initialize()
+{
+  process::PID<XfsDiskIsolatorProcess> self(this);
+
+  if (quotaPolicy == xfs::QuotaPolicy::ENFORCING_ACTIVE) {
+    // Start a loop to periodically check for containers
+    // breaking the soft limit.
+    process::loop(
+        self,
+        [=]() {
+          return process::after(watchInterval);
+        },
+        [=](const Nothing&) -> process::ControlFlow<Nothing> {
+          check();
+          return process::Continue();
+        });
+  }
+}
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9277c50d/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 07e68a7..9a5ca8b 100644
--- a/src/slave/containerizer/mesos/isolators/xfs/disk.hpp
+++ b/src/slave/containerizer/mesos/isolators/xfs/disk.hpp
@@ -55,9 +55,8 @@ public:
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig);
 
-  virtual process::Future<Nothing> isolate(
-      const ContainerID& containerId,
-      pid_t pid);
+  virtual process::Future<mesos::slave::ContainerLimitation> watch(
+      const ContainerID& containerId);
 
   virtual process::Future<Nothing> update(
       const ContainerID& containerId,
@@ -69,12 +68,19 @@ public:
   virtual process::Future<Nothing> cleanup(
       const ContainerID& containerId);
 
+protected:
+  virtual void initialize();
+
 private:
   XfsDiskIsolatorProcess(
+      Duration watchInterval,
       xfs::QuotaPolicy quotaPolicy,
       const std::string& workDir,
       const IntervalSet<prid_t>& projectIds);
 
+  // Responsible for validating a container hasn't broken the soft limit.
+  void check();
+
   // Take the next project ID from the unallocated pool.
   Option<prid_t> nextProjectId();
 
@@ -89,8 +95,10 @@ private:
     const std::string directory;
     Bytes quota;
     const prid_t projectId;
+    process::Promise<mesos::slave::ContainerLimitation> limitation;
   };
 
+  const Duration watchInterval;
   xfs::QuotaPolicy quotaPolicy;
   const std::string workDir;
   const IntervalSet<prid_t> totalProjectIds;

http://git-wip-us.apache.org/repos/asf/mesos/blob/9277c50d/src/slave/containerizer/mesos/isolators/xfs/utils.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/xfs/utils.cpp b/src/slave/containerizer/mesos/isolators/xfs/utils.cpp
index 2708524..cc14254 100644
--- a/src/slave/containerizer/mesos/isolators/xfs/utils.cpp
+++ b/src/slave/containerizer/mesos/isolators/xfs/utils.cpp
@@ -134,7 +134,8 @@ namespace internal {
 static Try<Nothing> setProjectQuota(
     const string& path,
     prid_t projectId,
-    Bytes limit)
+    Bytes softLimit,
+    Bytes hardLimit)
 {
   Try<string> devname = getDeviceForPath(path);
   if (devname.isError()) {
@@ -148,14 +149,10 @@ static Try<Nothing> setProjectQuota(
   // Specify that we are setting a project quota for this ID.
   quota.d_id = projectId;
   quota.d_flags = FS_PROJ_QUOTA;
-
-  // Set both the hard and the soft limit to the same quota. Functionally
-  // all we need is the hard limit. The soft limit has no effect when it
-  // is the same as the hard limit but we set it for explicitness.
   quota.d_fieldmask = FS_DQ_BSOFT | FS_DQ_BHARD;
 
-  quota.d_blk_hardlimit = BasicBlocks(limit).blocks();
-  quota.d_blk_softlimit = BasicBlocks(limit).blocks();
+  quota.d_blk_hardlimit = BasicBlocks(hardLimit).blocks();
+  quota.d_blk_softlimit = BasicBlocks(softLimit).blocks();
 
   if (::quotactl(QCMD(Q_XSETQLIM, PRJQUOTA),
                  devname.get().c_str(),
@@ -246,7 +243,8 @@ Result<QuotaInfo> getProjectQuota(
   }
 
   QuotaInfo info;
-  info.limit = BasicBlocks(quota.d_blk_hardlimit).bytes();
+  info.softLimit = BasicBlocks(quota.d_blk_softlimit).bytes();
+  info.hardLimit = BasicBlocks(quota.d_blk_hardlimit).bytes();
   info.used = BasicBlocks(quota.d_bcount).bytes();
 
   return info;
@@ -256,7 +254,31 @@ Result<QuotaInfo> getProjectQuota(
 Try<Nothing> setProjectQuota(
     const string& path,
     prid_t projectId,
-    Bytes limit)
+    Bytes softLimit,
+    Bytes hardLimit)
+{
+  if (projectId == NON_PROJECT_ID) {
+    return nonProjectError();
+  }
+
+  // A 0 limit deletes the quota record. If that's desired, the
+  // caller should use clearProjectQuota().
+  if (hardLimit == 0) {
+    return Error("Quota hard limit must be greater than 0");
+  }
+
+  if (softLimit == 0) {
+    return Error("Quota soft limit must be greater than 0");
+  }
+
+  return internal::setProjectQuota(path, projectId, softLimit, hardLimit);
+}
+
+
+Try<Nothing> setProjectQuota(
+    const string& path,
+    prid_t projectId,
+    Bytes hardLimit)
 {
   if (projectId == NON_PROJECT_ID) {
     return nonProjectError();
@@ -264,11 +286,11 @@ Try<Nothing> setProjectQuota(
 
   // A 0 limit deletes the quota record. If that's desired, the
   // caller should use clearProjectQuota().
-  if (limit == 0) {
-    return Error( "Quota limit must be greater than 0");
+  if (hardLimit == 0) {
+    return Error("Quota limit must be greater than 0");
   }
 
-  return internal::setProjectQuota(path, projectId, limit);
+  return internal::setProjectQuota(path, projectId, hardLimit, hardLimit);
 }
 
 
@@ -280,7 +302,7 @@ Try<Nothing> clearProjectQuota(
     return nonProjectError();
   }
 
-  return internal::setProjectQuota(path, projectId, Bytes(0));
+  return internal::setProjectQuota(path, projectId, Bytes(0), Bytes(0));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/9277c50d/src/slave/containerizer/mesos/isolators/xfs/utils.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/xfs/utils.hpp b/src/slave/containerizer/mesos/isolators/xfs/utils.hpp
index e034133..e269eb5 100644
--- a/src/slave/containerizer/mesos/isolators/xfs/utils.hpp
+++ b/src/slave/containerizer/mesos/isolators/xfs/utils.hpp
@@ -32,7 +32,8 @@ namespace xfs {
 
 struct QuotaInfo
 {
-  Bytes limit;
+  Bytes softLimit;
+  Bytes hardLimit;
   Bytes used;
 };
 
@@ -70,14 +71,18 @@ private:
 
 
 enum class QuotaPolicy {
-  ENFORCING,
   ACCOUNTING,
+  ENFORCING_ACTIVE,
+  ENFORCING_PASSIVE
 };
 
 
 inline bool operator==(const QuotaInfo& left, const QuotaInfo& right)
 {
-  return left.limit == right.limit && left.used == right.used;
+  return
+    left.hardLimit == right.hardLimit &&
+    left.softLimit == right.softLimit &&
+    left.used == right.used;
 }
 
 
@@ -101,7 +106,14 @@ Result<QuotaInfo> getProjectQuota(
 Try<Nothing> setProjectQuota(
     const std::string& path,
     prid_t projectId,
-    Bytes limit);
+    Bytes softLimit,
+    Bytes hardLimit);
+
+
+Try<Nothing> setProjectQuota(
+    const std::string& path,
+    prid_t projectId,
+    Bytes hardLimit);
 
 
 Try<Nothing> clearProjectQuota(

http://git-wip-us.apache.org/repos/asf/mesos/blob/9277c50d/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index 0ee4f65..02e1a8b 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -1131,7 +1131,7 @@ mesos::internal::slave::Flags::Flags()
   add(&Flags::container_disk_watch_interval,
       "container_disk_watch_interval",
       "The interval between disk quota checks for containers. This flag is\n"
-      "used by the `disk/du` isolator.",
+      "used by the `disk/du` and `disk/xfs` isolators.",
       Seconds(15));
 
   // TODO(jieyu): Consider enabling this flag by default. Remember
@@ -1310,6 +1310,12 @@ mesos::internal::slave::Flags::Flags()
       "xfs_project_range",
       "The ranges of XFS project IDs to use for tracking directory quotas",
       "[5000-10000]");
+
+  add(&Flags::xfs_kill_containers,
+      "xfs_kill_containers",
+      "Whether the `disk/xfs` isolator should detect and terminate\n"
+      "containers that exceed their allocated disk quota.",
+      false);
 #endif
 
   add(&Flags::http_command_executor,

http://git-wip-us.apache.org/repos/asf/mesos/blob/9277c50d/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 4195d3b..a839591 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -178,6 +178,7 @@ public:
   Option<std::string> master_detector;
 #if ENABLE_XFS_DISK_ISOLATOR
   std::string xfs_project_range;
+  bool xfs_kill_containers;
 #endif
   bool http_command_executor;
   Option<SlaveCapabilities> agent_features;

http://git-wip-us.apache.org/repos/asf/mesos/blob/9277c50d/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 64c3e1c..75500ed 100644
--- a/src/tests/containerizer/xfs_quota_tests.cpp
+++ b/src/tests/containerizer/xfs_quota_tests.cpp
@@ -77,7 +77,7 @@ static QuotaInfo makeQuotaInfo(
     Bytes limit,
     Bytes used)
 {
-  return {limit, used};
+  return {limit, limit, used};
 }
 
 
@@ -267,7 +267,7 @@ TEST_F(ROOT_XFS_QuotaTest, QuotaGetSet)
   Result<QuotaInfo> info = getProjectQuota(root, projectId);
   ASSERT_SOME(info);
 
-  EXPECT_EQ(limit, info->limit);
+  EXPECT_EQ(limit, info->hardLimit);
   EXPECT_EQ(Bytes(0), info->used);
 
   EXPECT_SOME(clearProjectQuota(root, projectId));


[2/2] mesos git commit: Added test for `disk/xfs` container limitation.

Posted by jp...@apache.org.
Added test for `disk/xfs` container limitation.

Added a `disk/xfs` test to verify that the `xfs_kill_containers`
will terminate a container that exceeds its disk quota.

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


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

Branch: refs/heads/master
Commit: f4345c1f1ed24fc09df121881265392865c92694
Parents: 9277c50
Author: Harold Dost <h....@criteo.com>
Authored: Wed Apr 25 08:46:53 2018 -0700
Committer: James Peach <jp...@apache.org>
Committed: Mon Apr 30 13:46:48 2018 -0700

----------------------------------------------------------------------
 src/tests/containerizer/xfs_quota_tests.cpp | 98 ++++++++++++++++++++++++
 1 file changed, 98 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f4345c1f/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 75500ed..ec9d86d 100644
--- a/src/tests/containerizer/xfs_quota_tests.cpp
+++ b/src/tests/containerizer/xfs_quota_tests.cpp
@@ -149,6 +149,12 @@ public:
       fs::unmount(mountPoint.get(), MNT_FORCE | MNT_DETACH);
     }
 
+    // Make sure we resume the clock so that we can wait on the
+    // `losetup` process.
+    if (Clock::paused()) {
+      Clock::resume();
+    }
+
     // Make a best effort to tear everything down. We don't make any assertions
     // here because even if something goes wrong we still want to clean up as
     // much as we can.
@@ -524,6 +530,98 @@ TEST_F(ROOT_XFS_QuotaTest, DiskUsageExceedsQuotaNoEnforce)
 }
 
 
+// Verify that when the `xfs_kill_containers` flag is enabled, tasks that
+// exceed their disk quota are killed with the correct container limitation.
+TEST_F(ROOT_XFS_QuotaTest, DiskUsageExceedsQuotaWithKill)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags flags = CreateSlaveFlags();
+
+  // Enable killing containers on disk quota violations.
+  flags.xfs_kill_containers = true;
+
+  // Tune the watch interval down so that the isolator will detect
+  // the quota violation as soon as possible.
+  flags.container_disk_watch_interval = Milliseconds(1);
+
+  Try<Owned<cluster::Slave>> slave =
+    StartSlave(detector.get(), flags);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  ASSERT_FALSE(offers->empty());
+
+  const Offer& offer = offers.get()[0];
+
+  // Create a task which requests 1MB disk, but actually uses 2MB. This
+  // waits a long time to ensure that the task lives long enough for the
+  // isolator to impose a container limitation.
+  TaskInfo task = createTask(
+      offer.slave_id(),
+      Resources::parse("cpus:1;mem:128;disk:1").get(),
+      "dd if=/dev/zero of=file bs=1048576 count=2 && sleep 100000");
+
+  Future<TaskStatus> startingStatus;
+  Future<TaskStatus> runningStatus;
+  Future<TaskStatus> killedStatus;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&startingStatus))
+    .WillOnce(FutureArg<1>(&runningStatus))
+    .WillOnce(FutureArg<1>(&killedStatus));
+
+  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());
+
+  AWAIT_READY(killedStatus);
+  EXPECT_EQ(task.task_id(), killedStatus->task_id());
+  EXPECT_EQ(TASK_FAILED, killedStatus->state());
+
+  EXPECT_EQ(TaskStatus::SOURCE_SLAVE, killedStatus->source());
+  EXPECT_EQ(
+      TaskStatus::REASON_CONTAINER_LIMITATION_DISK, killedStatus->reason());
+
+  ASSERT_TRUE(killedStatus->has_limitation())
+    << JSON::protobuf(killedStatus.get());
+
+  Resources limit = Resources(killedStatus->limitation().resources());
+
+  // Expect that we were limited on a single disk resource that represents
+  // the amount of disk that the task consumed. Note that while the task
+  // used up to 2MB, the executor logs might use more so we have to check
+  // for >= 2MB in this expectation.
+  EXPECT_EQ(1u, limit.size());
+  ASSERT_SOME(limit.disk());
+  EXPECT_GE(limit.disk().get(), Megabytes(2));
+
+  driver.stop();
+  driver.join();
+}
+
+
 // Verify that we can get accurate resource statistics from the XFS
 // disk isolator.
 TEST_F(ROOT_XFS_QuotaTest, ResourceStatistics)