You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by an...@apache.org on 2017/12/01 00:06:04 UTC

[10/10] mesos git commit: Windows: Added `Cpu` and `Mem` isolators.

Windows: Added `Cpu` and `Mem` isolators.

Instead of deriving from the POSIX isolators, we now have two real
Windows isolators that can be used together or separately. The `Cpu`
isolator enables a hard-cap CPU limit, and the `Mem` isolator enables a
hard-cap memory limit on the job object for the container.

These classes are separate derivations of `MesosIsolatorProcess`,
because introducing a `WindowsIsolatorProcess` base class would be
abstraction for the sole purpose of code deduplication.

Note that these isolators support nesting, and so must support empty
`cpu` or `mem` resources. When these are not provided, the corresponding
code to set the limits is simply not called.

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


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

Branch: refs/heads/master
Commit: 80ed0eda900ee53e14b00f8cd14d28077bde9367
Parents: 17564c7
Author: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Authored: Tue Aug 22 15:29:09 2017 -0700
Committer: Andrew Schwartzmeyer <an...@schwartzmeyer.com>
Committed: Thu Nov 30 15:54:53 2017 -0800

----------------------------------------------------------------------
 src/CMakeLists.txt                              |   6 +-
 src/Makefile.am                                 |   3 +-
 src/slave/containerizer/mesos/containerizer.cpp |   8 +-
 .../containerizer/mesos/isolators/windows.hpp   |  91 ---------
 .../mesos/isolators/windows/cpu.cpp             | 204 +++++++++++++++++++
 .../mesos/isolators/windows/cpu.hpp             |  78 +++++++
 .../mesos/isolators/windows/mem.cpp             | 196 ++++++++++++++++++
 .../mesos/isolators/windows/mem.hpp             |  79 +++++++
 src/slave/flags.cpp                             |   4 +-
 9 files changed, 572 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/80ed0eda/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 76ef6ca..15cda10 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -291,7 +291,11 @@ set(AGENT_SRC
   slave/containerizer/mesos/provisioner/docker/store.cpp
   slave/resource_estimators/noop.cpp)
 
-if (NOT WIN32)
+if (WIN32)
+  list(APPEND AGENT_SRC
+    slave/containerizer/mesos/isolators/windows/cpu.cpp
+    slave/containerizer/mesos/isolators/windows/mem.cpp)
+else ()
   list(APPEND AGENT_SRC
     slave/containerizer/mesos/isolators/environment_secret.cpp
     slave/containerizer/mesos/isolators/docker/volume/driver.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/80ed0eda/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 4a3b728..3444388 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1200,7 +1200,8 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   slave/containerizer/mesos/isolators/network/cni/paths.hpp		\
   slave/containerizer/mesos/isolators/network/cni/spec.hpp		\
   slave/containerizer/mesos/isolators/volume/sandbox_path.hpp		\
-  slave/containerizer/mesos/isolators/windows.hpp			\
+  slave/containerizer/mesos/isolators/windows/cpu.hpp			\
+  slave/containerizer/mesos/isolators/windows/mem.hpp			\
   slave/containerizer/mesos/provisioner/backend.hpp			\
   slave/containerizer/mesos/provisioner/constants.hpp			\
   slave/containerizer/mesos/provisioner/paths.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/80ed0eda/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 7f3b86d..44e8f7a 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -77,7 +77,8 @@
 #include "slave/containerizer/mesos/provisioner/provisioner.hpp"
 
 #ifdef __WINDOWS__
-#include "slave/containerizer/mesos/isolators/windows.hpp"
+#include "slave/containerizer/mesos/isolators/windows/cpu.hpp"
+#include "slave/containerizer/mesos/isolators/windows/mem.hpp"
 #include "slave/containerizer/mesos/isolators/filesystem/windows.hpp"
 #endif // __WINDOWS__
 
@@ -181,7 +182,8 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     LOG(WARNING) << "The 'process' isolation flag is deprecated, "
                  << "please update your flags to "
                  << "'--isolation=posix/cpu,posix/mem' (or "
-                 << "'--isolation=windows/cpu' if you are on Windows).";
+                 << "'--isolation=windows/cpu,windows/mem' "
+                 << "if you are on Windows).";
 
     isolations->erase("process");
 
@@ -190,6 +192,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     isolations->insert("posix/mem");
 #else
     isolations->insert("windows/cpu");
+    isolations->insert("windows/mem");
 #endif // __WINDOWS__
   } else if (flags.isolation == "cgroups") {
     LOG(WARNING) << "The 'cgroups' isolation flag is deprecated, "
@@ -388,6 +391,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
 #endif // ENABLE_XFS_DISK_ISOLATOR
 #else
     {"windows/cpu", &WindowsCpuIsolatorProcess::create},
+    {"windows/mem", &WindowsMemIsolatorProcess::create},
 #endif // __WINDOWS__
 
 #ifdef __linux__

http://git-wip-us.apache.org/repos/asf/mesos/blob/80ed0eda/src/slave/containerizer/mesos/isolators/windows.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/windows.hpp b/src/slave/containerizer/mesos/isolators/windows.hpp
deleted file mode 100644
index b0621a5..0000000
--- a/src/slave/containerizer/mesos/isolators/windows.hpp
+++ /dev/null
@@ -1,91 +0,0 @@
-// 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.
-
-#ifndef __WINDOWS_ISOLATOR_HPP__
-#define __WINDOWS_ISOLATOR_HPP__
-
-#include <process/future.hpp>
-#include <process/id.hpp>
-
-#include "slave/flags.hpp"
-
-#include "slave/containerizer/mesos/isolator.hpp"
-#include "slave/containerizer/mesos/isolators/posix.hpp"
-
-namespace mesos {
-namespace internal {
-namespace slave {
-
-// A basic MesosIsolatorProcess that keeps track of the pid but
-// doesn't do any resource isolation. Subclasses must implement
-// usage() for their appropriate resource(s).
-//
-// TODO(hausdorff): (MESOS-5462) For now the Windows isolators are essentially
-// direct copies of their POSIX counterparts. In the future, we expect to
-// refactor the POSIX classes into platform-independent base class, with
-// Windows and POSIX implementations. For now, we leave the Windows
-// implementations as inheriting from the POSIX implementations.
-class WindowsIsolatorProcess : public PosixIsolatorProcess {};
-
-
-// TODO(hausdorff): (MESOS-5462) For now the Windows isolators are essentially
-// direct copies of their POSIX counterparts. In the future, we expect to
-// refactor the POSIX classes into platform-independent base class, with
-// Windows and POSIX implementations. For now, we leave the Windows
-// implementations as inheriting from the POSIX implementations.
-class WindowsCpuIsolatorProcess : public PosixCpuIsolatorProcess
-{
-public:
-  static Try<mesos::slave::Isolator*> create(const Flags& flags)
-  {
-    process::Owned<MesosIsolatorProcess> process(
-        new WindowsCpuIsolatorProcess());
-
-    return new MesosIsolator(process);
-  }
-
-private:
-  WindowsCpuIsolatorProcess()
-    : ProcessBase(process::ID::generate("windows-cpu-isolator")) {}
-};
-
-
-// TODO(hausdorff): (MESOS-5462) For now the Windows isolators are essentially
-// direct copies of their POSIX counterparts. In the future, we expect to
-// refactor the POSIX classes into platform-independent base class, with
-// Windows and POSIX implementations. For now, we leave the Windows
-// implementations as inheriting from the POSIX implementations.
-class WindowsMemIsolatorProcess : public PosixMemIsolatorProcess
-{
-public:
-  static Try<mesos::slave::Isolator*> create(const Flags& flags)
-  {
-    process::Owned<MesosIsolatorProcess> process(
-        new WindowsMemIsolatorProcess());
-
-    return new MesosIsolator(process);
-  }
-
-private:
-  WindowsMemIsolatorProcess()
-    : ProcessBase(process::ID::generate("windows-mem-isolator")) {}
-};
-
-} // namespace slave {
-} // namespace internal {
-} // namespace mesos {
-
-#endif // __WINDOWS_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/80ed0eda/src/slave/containerizer/mesos/isolators/windows/cpu.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/windows/cpu.cpp b/src/slave/containerizer/mesos/isolators/windows/cpu.cpp
new file mode 100644
index 0000000..e9f4dac
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/windows/cpu.cpp
@@ -0,0 +1,204 @@
+// 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.
+
+#include <algorithm>
+#include <list>
+
+#include <process/future.hpp>
+#include <process/id.hpp>
+
+#include <stout/duration.hpp>
+#include <stout/hashset.hpp>
+#include <stout/nothing.hpp>
+#include <stout/stringify.hpp>
+#include <stout/try.hpp>
+#include <stout/windows.hpp>
+
+#include <stout/windows/os.hpp>
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolators/windows/cpu.hpp"
+
+using process::Failure;
+using process::Future;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// Reasonable minimum constraints.
+constexpr double MIN_CPU = 0.001;
+
+
+bool WindowsCpuIsolatorProcess::supportsNesting() { return true; }
+
+
+// When recovering, this ensures that our ContainerID -> PID mapping is
+// recreated.
+Future<Nothing> WindowsCpuIsolatorProcess::recover(
+    const std::list<mesos::slave::ContainerState>& state,
+    const hashset<ContainerID>& orphans)
+{
+  foreach (const mesos::slave::ContainerState& run, state) {
+    // This should (almost) never occur: see comment in
+    // SubprocessLauncher::recover().
+    if (pids.contains(run.container_id())) {
+      return Failure("Container already recovered");
+    }
+
+    pids.put(run.container_id(), run.pid());
+  }
+
+  return Nothing();
+}
+
+
+process::Future<Option<mesos::slave::ContainerLaunchInfo>>
+WindowsCpuIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const mesos::slave::ContainerConfig& containerConfig)
+{
+  if (cpuLimits.contains(containerId)) {
+    return Failure("Container already prepared: " + stringify(containerId));
+  }
+
+  Resources resources{containerConfig.resources()};
+  if (resources.cpus().isSome()) {
+    // Save the limit information so that `isolate` can set the limit
+    // immediately.
+    cpuLimits[containerId] = std::max(resources.cpus().get(), MIN_CPU);
+  }
+
+  return None();
+}
+
+
+// This is called when the actual container is launched, and hence has a PID.
+// It creates the ContainerID -> PID mapping, and sets the initial CPU limit.
+Future<Nothing> WindowsCpuIsolatorProcess::isolate(
+    const ContainerID& containerId, pid_t pid)
+{
+  if (pids.contains(containerId)) {
+    return Failure("Container already isolated: " + stringify(containerId));
+  }
+
+  pids.put(containerId, pid);
+
+  if (cpuLimits.contains(containerId)) {
+    Try<Nothing> set =
+      os::set_job_cpu_limit(pids[containerId], cpuLimits[containerId]);
+    if (set.isError()) {
+      return Failure(
+          "Failed to update container '" + stringify(containerId) +
+          "': " + set.error());
+    }
+  }
+
+  return Nothing();
+}
+
+
+Future<Nothing> WindowsCpuIsolatorProcess::cleanup(
+    const ContainerID& containerId)
+{
+  if (!pids.contains(containerId)) {
+    VLOG(1) << "Ignoring cleanup request for unknown container " << containerId;
+
+    return Nothing();
+  }
+
+  pids.erase(containerId);
+  cpuLimits.erase(containerId);
+
+  return Nothing();
+}
+
+
+Try<mesos::slave::Isolator*> WindowsCpuIsolatorProcess::create(
+    const Flags& flags)
+{
+  process::Owned<MesosIsolatorProcess> process(new WindowsCpuIsolatorProcess());
+
+  return new MesosIsolator(process);
+}
+
+
+Future<Nothing> WindowsCpuIsolatorProcess::update(
+    const ContainerID& containerId, const Resources& resources)
+{
+  if (containerId.has_parent()) {
+    return Failure("Not supported for nested containers");
+  }
+
+  if (!pids.contains(containerId)) {
+    return Failure("Unknown container: " + stringify(containerId));
+  }
+
+  if (resources.cpus().isNone()) {
+    return Failure(
+        "Failed to update container '" + stringify(containerId) +
+        "': No cpus resource given");
+  }
+
+  cpuLimits[containerId] = std::max(resources.cpus().get(), MIN_CPU);
+  Try<Nothing> set =
+    os::set_job_cpu_limit(pids[containerId], cpuLimits[containerId]);
+  if (set.isError()) {
+    return Failure(
+        "Failed to update container '" + stringify(containerId) +
+        "': " + set.error());
+  }
+
+  return Nothing();
+}
+
+
+Future<ResourceStatistics> WindowsCpuIsolatorProcess::usage(
+    const ContainerID& containerId)
+{
+  if (!pids.contains(containerId)) {
+    LOG(WARNING) << "No resource usage for unknown container '" << containerId
+                 << "'";
+
+    return ResourceStatistics();
+  }
+
+  ResourceStatistics result;
+
+  result.set_timestamp(process::Clock::now().secs());
+
+  const Try<JOBOBJECT_BASIC_ACCOUNTING_INFORMATION> info =
+    os::get_job_info(pids[containerId]);
+  if (info.isError()) {
+    return result;
+  }
+
+  result.set_processes(info.get().ActiveProcesses);
+
+  // The reported time fields are in 100-nanosecond ticks.
+  result.set_cpus_user_time_secs(
+      Nanoseconds(info.get().TotalUserTime.QuadPart * 100).secs());
+
+  result.set_cpus_system_time_secs(
+      Nanoseconds(info.get().TotalKernelTime.QuadPart * 100).secs());
+
+  return result;
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/80ed0eda/src/slave/containerizer/mesos/isolators/windows/cpu.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/windows/cpu.hpp b/src/slave/containerizer/mesos/isolators/windows/cpu.hpp
new file mode 100644
index 0000000..8672b59
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/windows/cpu.hpp
@@ -0,0 +1,78 @@
+// 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.
+
+#ifndef __WINDOWS_CPU_ISOLATOR_HPP__
+#define __WINDOWS_CPU_ISOLATOR_HPP__
+
+#include <list>
+
+#include <process/future.hpp>
+#include <process/id.hpp>
+
+#include <stout/hashmap.hpp>
+#include <stout/hashset.hpp>
+#include <stout/nothing.hpp>
+#include <stout/try.hpp>
+#include <stout/windows.hpp>
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolator.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class WindowsCpuIsolatorProcess final : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(const Flags& flags);
+
+  bool supportsNesting() override;
+
+  process::Future<Nothing> recover(
+      const std::list<mesos::slave::ContainerState>& state,
+      const hashset<ContainerID>& orphans) override;
+
+  process::Future<Option<mesos::slave::ContainerLaunchInfo>> prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig) override;
+
+  process::Future<Nothing> isolate(
+      const ContainerID& containerId, pid_t pid) override;
+
+  process::Future<Nothing> update(
+      const ContainerID& containerId, const Resources& resources) override;
+
+  process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId) override;
+
+  process::Future<Nothing> cleanup(const ContainerID& containerId) override;
+
+private:
+  hashmap<ContainerID, pid_t> pids;
+  hashmap<ContainerID, double> cpuLimits;
+
+  WindowsCpuIsolatorProcess()
+    : ProcessBase(process::ID::generate("windows-cpu-isolator"))
+  {}
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __WINDOWS_CPU_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/80ed0eda/src/slave/containerizer/mesos/isolators/windows/mem.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/windows/mem.cpp b/src/slave/containerizer/mesos/isolators/windows/mem.cpp
new file mode 100644
index 0000000..5abcb4e
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/windows/mem.cpp
@@ -0,0 +1,196 @@
+// 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.
+
+#include <algorithm>
+#include <list>
+
+#include <process/future.hpp>
+#include <process/id.hpp>
+
+#include <stout/bytes.hpp>
+#include <stout/duration.hpp>
+#include <stout/hashset.hpp>
+#include <stout/nothing.hpp>
+#include <stout/stringify.hpp>
+#include <stout/try.hpp>
+#include <stout/windows.hpp>
+
+#include <stout/windows/os.hpp>
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolators/windows/mem.hpp"
+
+using process::Failure;
+using process::Future;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// Reasonable minimum constraints.
+constexpr Bytes MIN_MEM = Megabytes(32);
+
+
+bool WindowsMemIsolatorProcess::supportsNesting() { return true; }
+
+
+// When recovering, this ensures that our ContainerID -> PID mapping is
+// recreated.
+Future<Nothing> WindowsMemIsolatorProcess::recover(
+    const std::list<mesos::slave::ContainerState>& state,
+    const hashset<ContainerID>& orphans)
+{
+  foreach (const mesos::slave::ContainerState& run, state) {
+    // This should (almost) never occur: see comment in
+    // SubprocessLauncher::recover().
+    if (pids.contains(run.container_id())) {
+      return Failure("Container already recovered");
+    }
+
+    pids.put(run.container_id(), run.pid());
+  }
+
+  return Nothing();
+}
+
+
+process::Future<Option<mesos::slave::ContainerLaunchInfo>>
+WindowsMemIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const mesos::slave::ContainerConfig& containerConfig)
+{
+  if (memLimits.contains(containerId)) {
+    return Failure("Container already prepared: " + stringify(containerId));
+  }
+
+  Resources resources{containerConfig.resources()};
+
+  if (resources.mem().isSome()) {
+    // Save the limit information so that `isolate` can set the limit
+    // immediately.
+    memLimits[containerId] = std::max(resources.mem().get(), MIN_MEM);
+  }
+
+  return None();
+}
+
+
+// This is called when the actual container is launched, and hence has a PID.
+// It creates the ContainerID -> PID mapping, and sets the initial memory limit.
+Future<Nothing> WindowsMemIsolatorProcess::isolate(
+    const ContainerID& containerId, pid_t pid)
+{
+  if (pids.contains(containerId)) {
+    return Failure("Container already isolated: " + stringify(containerId));
+  }
+
+  pids.put(containerId, pid);
+
+  if (memLimits.contains(containerId)) {
+    Try<Nothing> set =
+      os::set_job_mem_limit(pids[containerId], memLimits[containerId]);
+    if (set.isError()) {
+      return Failure(
+          "Failed to isolate container '" + stringify(containerId) +
+          "': " + set.error());
+    }
+  }
+
+  return Nothing();
+}
+
+
+Future<Nothing> WindowsMemIsolatorProcess::cleanup(
+    const ContainerID& containerId)
+{
+  if (!pids.contains(containerId)) {
+    VLOG(1) << "Ignoring cleanup request for unknown container " << containerId;
+
+    return Nothing();
+  }
+
+  pids.erase(containerId);
+  memLimits.erase(containerId);
+
+  return Nothing();
+}
+
+
+Try<mesos::slave::Isolator*> WindowsMemIsolatorProcess::create(
+    const Flags& flags)
+{
+  process::Owned<MesosIsolatorProcess> process(new WindowsMemIsolatorProcess());
+
+  return new MesosIsolator(process);
+}
+
+
+Future<Nothing> WindowsMemIsolatorProcess::update(
+    const ContainerID& containerId, const Resources& resources)
+{
+  if (containerId.has_parent()) {
+    return Failure("Not supported for nested containers");
+  }
+
+  if (!pids.contains(containerId)) {
+    return Failure("Unknown container: " + stringify(containerId));
+  }
+
+  if (resources.mem().isNone()) {
+    return Failure(
+        "Failed to update container '" + stringify(containerId) +
+        "': No mem resource given");
+  }
+
+  memLimits[containerId] = std::max(resources.mem().get(), MIN_MEM);
+  Try<Nothing> set =
+    os::set_job_mem_limit(pids[containerId], memLimits[containerId]);
+  if (set.isError()) {
+    return Failure(
+        "Failed to update container '" + stringify(containerId) +
+        "': " + set.error());
+  }
+
+  return Nothing();
+}
+
+
+Future<ResourceStatistics> WindowsMemIsolatorProcess::usage(
+    const ContainerID& containerId)
+{
+  if (!pids.contains(containerId)) {
+    LOG(WARNING) << "No resource usage for unknown container '" << containerId
+                 << "'";
+
+    return ResourceStatistics();
+  }
+
+  ResourceStatistics result;
+
+  result.set_timestamp(process::Clock::now().secs());
+
+  const Try<Bytes> mem_total_bytes = os::get_job_mem(pids[containerId]);
+  if (mem_total_bytes.isSome()) {
+    result.set_mem_total_bytes(mem_total_bytes.get().bytes());
+  }
+
+  return result;
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/80ed0eda/src/slave/containerizer/mesos/isolators/windows/mem.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/windows/mem.hpp b/src/slave/containerizer/mesos/isolators/windows/mem.hpp
new file mode 100644
index 0000000..24426f7
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/windows/mem.hpp
@@ -0,0 +1,79 @@
+// 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.
+
+#ifndef __WINDOWS_MEM_ISOLATOR_HPP__
+#define __WINDOWS_MEM_ISOLATOR_HPP__
+
+#include <list>
+
+#include <process/future.hpp>
+#include <process/id.hpp>
+
+#include <stout/bytes.hpp>
+#include <stout/hashmap.hpp>
+#include <stout/hashset.hpp>
+#include <stout/nothing.hpp>
+#include <stout/try.hpp>
+#include <stout/windows.hpp>
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolator.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class WindowsMemIsolatorProcess final : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(const Flags& flags);
+
+  bool supportsNesting() override;
+
+  process::Future<Nothing> recover(
+      const std::list<mesos::slave::ContainerState>& state,
+      const hashset<ContainerID>& orphans) override;
+
+  process::Future<Option<mesos::slave::ContainerLaunchInfo>> prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig) override;
+
+  process::Future<Nothing> isolate(
+      const ContainerID& containerId, pid_t pid) override;
+
+  process::Future<Nothing> update(
+      const ContainerID& containerId, const Resources& resources) override;
+
+  process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId) override;
+
+  process::Future<Nothing> cleanup(const ContainerID& containerId) override;
+
+private:
+  hashmap<ContainerID, pid_t> pids;
+  hashmap<ContainerID, Bytes> memLimits;
+
+  WindowsMemIsolatorProcess()
+    : ProcessBase(process::ID::generate("windows-mem-isolator"))
+  {}
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __WINDOWS_MEM_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/80ed0eda/src/slave/flags.cpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.cpp b/src/slave/flags.cpp
index 7998d9b..34edf4c 100644
--- a/src/slave/flags.cpp
+++ b/src/slave/flags.cpp
@@ -114,7 +114,7 @@ mesos::internal::slave::Flags::Flags()
   add(&Flags::isolation,
       "isolation",
       "Isolation mechanisms to use, e.g., `posix/cpu,posix/mem` (or \n"
-      "`windows/cpu` if you are on Windows), or\n"
+      "`windows/cpu,windows/mem` if you are on Windows), or\n"
       "`cgroups/cpu,cgroups/mem`, or `network/port_mapping`\n"
       "(configure with flag: `--with-network-isolator` to enable),\n"
       "or `gpu/nvidia` for nvidia specific gpu isolation,\n"
@@ -124,7 +124,7 @@ mesos::internal::slave::Flags::Flags()
 #ifndef __WINDOWS__
       "posix/cpu,posix/mem"
 #else
-      "windows/cpu"
+      "windows/cpu,windows/mem"
 #endif // __WINDOWS__
       );