You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2015/07/24 05:15:23 UTC

mesos git commit: Converted Limitation and ExecutorRunState structs into protobufs.

Repository: mesos
Updated Branches:
  refs/heads/master 341e23fc5 -> 30205f825


Converted Limitation and ExecutorRunState structs into protobufs.

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


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

Branch: refs/heads/master
Commit: 30205f82532d40fa993215a59f2f9e8174a5b9bf
Parents: 341e23f
Author: Kapil Arya <ka...@mesosphere.io>
Authored: Thu Jul 23 20:08:16 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Jul 23 20:15:06 2015 -0700

----------------------------------------------------------------------
 include/mesos/slave/isolator.hpp                | 55 ++-----------------
 include/mesos/slave/isolator.proto              | 58 ++++++++++++++++++++
 src/Makefile.am                                 | 41 ++++++++------
 src/common/protobuf_utils.cpp                   | 39 +++++++++++++
 src/common/protobuf_utils.hpp                   | 17 ++++++
 src/exec/exec.cpp                               |  2 +-
 src/slave/containerizer/isolator.cpp            |  2 +-
 .../isolators/cgroups/cpushare.cpp              |  6 +-
 .../isolators/cgroups/cpushare.hpp              |  4 +-
 .../containerizer/isolators/cgroups/mem.cpp     | 11 ++--
 .../containerizer/isolators/cgroups/mem.hpp     |  4 +-
 .../isolators/cgroups/perf_event.cpp            |  8 +--
 .../isolators/cgroups/perf_event.hpp            |  2 +-
 .../isolators/filesystem/posix.cpp              |  8 +--
 .../isolators/filesystem/posix.hpp              |  2 +-
 .../isolators/filesystem/shared.cpp             |  6 +-
 .../isolators/filesystem/shared.hpp             |  2 +-
 .../containerizer/isolators/namespaces/pid.cpp  |  8 +--
 .../containerizer/isolators/namespaces/pid.hpp  |  2 +-
 .../isolators/network/port_mapping.cpp          | 10 ++--
 .../isolators/network/port_mapping.hpp          |  2 +-
 src/slave/containerizer/isolators/posix.hpp     | 22 ++++----
 .../containerizer/isolators/posix/disk.cpp      | 16 +++---
 .../containerizer/isolators/posix/disk.hpp      | 11 ++--
 src/slave/containerizer/launcher.cpp            |  4 +-
 src/slave/containerizer/linux_launcher.cpp      |  4 +-
 src/slave/containerizer/mesos/containerizer.cpp | 37 +++++++------
 src/slave/containerizer/mesos/containerizer.hpp |  4 +-
 src/tests/containerizer_tests.cpp               |  6 +-
 src/tests/isolator.hpp                          |  4 +-
 30 files changed, 243 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/include/mesos/slave/isolator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/slave/isolator.hpp b/include/mesos/slave/isolator.hpp
index 85e38f5..8387efd 100644
--- a/include/mesos/slave/isolator.hpp
+++ b/include/mesos/slave/isolator.hpp
@@ -24,6 +24,9 @@
 
 #include <mesos/resources.hpp>
 
+// ONLY USEFUL AFTER RUNNING PROTOC.
+#include <mesos/slave/isolator.pb.h>
+
 #include <process/dispatch.hpp>
 #include <process/future.hpp>
 #include <process/owned.hpp>
@@ -39,54 +42,6 @@ namespace slave {
 // Forward declaration.
 class IsolatorProcess;
 
-// Information when an executor is impacted by a resource limitation
-// and should be terminated. Intended to support resources like memory
-// where the Linux kernel may invoke the OOM killer, killing some/all
-// of a container's processes.
-struct Limitation
-{
-  Limitation(
-      const Resources& _resources,
-      const std::string& _message)
-    : resources(_resources),
-      message(_message) {}
-
-  // Resources that triggered the limitation.
-  // NOTE: 'Resources' is used here because the resource may span
-  // multiple roles (e.g. `"mem(*):1;mem(role):2"`).
-  Resources resources;
-
-  // Description of the limitation.
-  std::string message;
-};
-
-
-// This struct is derived from slave::state::RunState. It contains
-// only those fields that are needed by Isolators for recovering the
-// containers. The reason for not using RunState instead is to avoid
-// any dependency on RunState and in turn on internal protobufs.
-struct ExecutorRunState
-{
-  ExecutorRunState(
-      const ExecutorInfo& executorInfo_,
-      const ContainerID& id_,
-      pid_t pid_,
-      const std::string& directory_,
-      const Option<std::string>& rootfs_)
-    : executorInfo(executorInfo_),
-      id(id_),
-      pid(pid_),
-      directory(directory_),
-      rootfs(rootfs_) {}
-
-  ExecutorInfo executorInfo;
-  ContainerID id;        // Container id of the last executor run.
-  pid_t pid;             // Executor pid.
-  std::string directory; // Executor work directory.
-  Option<std::string> rootfs; // Optional container rootfs.
-};
-
-
 class Isolator
 {
 public:
@@ -130,7 +85,7 @@ public:
   // Watch the containerized executor and report if any resource
   // constraint impacts the container, e.g., the kernel killing some
   // processes.
-  process::Future<Limitation> watch(const ContainerID& containerId);
+  process::Future<ExecutorLimitation> watch(const ContainerID& containerId);
 
   // Update the resources allocated to the container.
   process::Future<Nothing> update(
@@ -175,7 +130,7 @@ public:
       const ContainerID& containerId,
       pid_t pid) = 0;
 
-  virtual process::Future<Limitation> watch(
+  virtual process::Future<ExecutorLimitation> watch(
       const ContainerID& containerId) = 0;
 
   virtual process::Future<Nothing> update(

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/include/mesos/slave/isolator.proto
----------------------------------------------------------------------
diff --git a/include/mesos/slave/isolator.proto b/include/mesos/slave/isolator.proto
new file mode 100644
index 0000000..07c1c1a
--- /dev/null
+++ b/include/mesos/slave/isolator.proto
@@ -0,0 +1,58 @@
+/**
+ * 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.
+ */
+
+import "mesos/mesos.proto";
+
+package mesos.slave;
+
+
+/**
+ * Information when an executor is impacted by a resource limitation
+ * and should be terminated. Intended to support resources like memory
+ * where the Linux kernel may invoke the OOM killer, killing some/all
+ * of a container's processes.
+ */
+message ExecutorLimitation
+{
+  // Resources that triggered the limitation.
+  // NOTE: 'Resources' is used here because the resource may span
+  // multiple roles (e.g. `"mem(*):1;mem(role):2"`).
+  repeated Resource resources = 1;
+
+  // Description of the limitation.
+  optional string message = 2;
+}
+
+
+/**
+ * This message is derived from slave::state::RunState. It contains
+ * only those fields that are needed by Isolators for recovering the
+ * containers. The reason for not using RunState instead is to avoid
+ * any dependency on RunState and in turn on internal protobufs.
+ */
+message ExecutorRunState
+{
+  required ExecutorInfo executor_info = 1;
+
+  // Container id of the last executor run.
+  required ContainerID container_id = 2;
+
+  required uint64 pid = 3;            // Executor pid.
+  required string directory = 4;      // Executor work directory.
+  optional string rootfs = 5;         // Optional container rootfs.
+}

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 489ddb4..9f2d7e3 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -136,26 +136,29 @@ endif
 
 MESOS_PROTO = $(top_srcdir)/include/mesos/mesos.proto
 
-ALLOCATOR_PROTO =							\
-  $(top_srcdir)/include/mesos/master/allocator.proto
-
 AUTHENTICATION_PROTO =							\
   $(top_srcdir)/include/mesos/authentication/authentication.proto
 
 CONTAINERIZER_PROTO =							\
   $(top_srcdir)/include/mesos/containerizer/containerizer.proto
 
+EXECUTOR_PROTO =							\
+  $(top_srcdir)/include/mesos/executor/executor.proto
+
 FETCHER_PROTO =								\
   $(top_srcdir)/include/mesos/fetcher/fetcher.proto
 
+ALLOCATOR_PROTO =							\
+  $(top_srcdir)/include/mesos/master/allocator.proto
+
 MODULE_PROTO =								\
   $(top_srcdir)/include/mesos/module/module.proto
 
 SCHEDULER_PROTO =							\
   $(top_srcdir)/include/mesos/scheduler/scheduler.proto
 
-EXECUTOR_PROTO =							\
-  $(top_srcdir)/include/mesos/executor/executor.proto
+ISOLATOR_PROTO =							\
+  $(top_srcdir)/include/mesos/slave/isolator.proto
 
 OVERSUBSCRIPTION_PROTO =						\
   $(top_srcdir)/include/mesos/slave/oversubscription.proto
@@ -167,6 +170,8 @@ CXX_PROTOS =								\
   ../include/mesos/authentication/authentication.pb.h			\
   containerizer/containerizer.pb.cc					\
   ../include/mesos/containerizer/containerizer.pb.h			\
+  executor/executor.pb.cc						\
+  ../include/mesos/executor/executor.pb.h				\
   fetcher/fetcher.pb.cc							\
   ../include/mesos/fetcher/fetcher.pb.h					\
   master/allocator.pb.cc						\
@@ -175,8 +180,8 @@ CXX_PROTOS =								\
   ../include/mesos/module/module.pb.h					\
   scheduler/scheduler.pb.cc						\
   ../include/mesos/scheduler/scheduler.pb.h				\
-  executor/executor.pb.cc						\
-  ../include/mesos/executor/executor.pb.h				\
+  slave/isolator.pb.cc							\
+  ../include/mesos/slave/isolator.pb.h					\
   slave/oversubscription.pb.cc						\
   ../include/mesos/slave/oversubscription.pb.h
 
@@ -236,6 +241,12 @@ containerizer/%.pb.cc ../include/mesos/containerizer/%.pb.h: $(CONTAINERIZER_PRO
 	$(PROTOC) $(PROTOCFLAGS) --cpp_out=../include $^
 	mv ../include/mesos/containerizer/*.pb.cc $(@D)
 
+executor/%.pb.cc ../include/mesos/executor/%.pb.h: $(EXECUTOR_PROTO)
+	$(MKDIR_P) $(@D)
+	$(MKDIR_P) ../include/mesos/executor
+	$(PROTOC) $(PROTOCFLAGS) --cpp_out=../include $^
+	mv ../include/mesos/executor/*.pb.cc $(@D)
+
 fetcher/%.pb.cc ../include/mesos/fetcher/%.pb.h: $(FETCHER_PROTO)
 	$(MKDIR_P) $(@D)
 	$(MKDIR_P) ../include/mesos/fetcher
@@ -260,17 +271,11 @@ scheduler/%.pb.cc ../include/mesos/scheduler/%.pb.h: $(SCHEDULER_PROTO)
 	$(PROTOC) $(PROTOCFLAGS) --cpp_out=../include $^
 	mv ../include/mesos/scheduler/*.pb.cc $(@D)
 
-executor/%.pb.cc ../include/mesos/executor/%.pb.h: $(EXECUTOR_PROTO)
-	$(MKDIR_P) $(@D)
-	$(MKDIR_P) ../include/mesos/executor
-	$(PROTOC) $(PROTOCFLAGS) --cpp_out=../include $^
-	mv ../include/mesos/executor/*.pb.cc $(@D)
-
-slave/%.pb.cc ../include/mesos/slave/%.pb.h: $(OVERSUBSCRIPTION_PROTO)
+slave/%.pb.cc ../include/mesos/slave/%.pb.h: $(top_srcdir)/include/mesos/slave/%.proto
 	$(MKDIR_P) $(@D)
 	$(MKDIR_P) ../include/mesos/slave
 	$(PROTOC) $(PROTOCFLAGS) --cpp_out=../include $^
-	mv ../include/mesos/slave/*.pb.cc $(@D)
+	mv ../include/mesos/slave/$(*F).pb.cc $(@D)
 
 %.pb.cc %.pb.h: %.proto
 	$(MKDIR_P) $(@D)
@@ -505,12 +510,15 @@ slavedir = $(pkgincludedir)/slave
 
 slave_HEADERS =								\
   $(top_srcdir)/include/mesos/slave/isolator.hpp			\
+  $(top_srcdir)/include/mesos/slave/isolator.proto			\
   $(top_srcdir)/include/mesos/slave/oversubscription.hpp		\
   $(top_srcdir)/include/mesos/slave/oversubscription.proto		\
   $(top_srcdir)/include/mesos/slave/qos_controller.hpp			\
   $(top_srcdir)/include/mesos/slave/resource_estimator.hpp
 
-nodist_slave_HEADERS = ../include/mesos/slave/oversubscription.pb.h
+nodist_slave_HEADERS =							\
+  ../include/mesos/slave/isolator.pb.h					\
+  ../include/mesos/slave/oversubscription.pb.h
 
 if OS_LINUX
   libmesos_no_3rdparty_la_SOURCES += linux/cgroups.cpp
@@ -786,6 +794,7 @@ libmesos_la_SOURCES =							\
   $(MODULE_PROTO)							\
   $(SCHEDULER_PROTO)							\
   $(EXECUTOR_PROTO)							\
+  $(ISOLATOR_PROTO)							\
   $(OVERSUBSCRIPTION_PROTO)
 
 libmesos_la_LDFLAGS = -release $(PACKAGE_VERSION)

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/common/protobuf_utils.cpp
----------------------------------------------------------------------
diff --git a/src/common/protobuf_utils.cpp b/src/common/protobuf_utils.cpp
index e0f82b5..4cfbda4 100644
--- a/src/common/protobuf_utils.cpp
+++ b/src/common/protobuf_utils.cpp
@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 
+#include <mesos/slave/isolator.hpp>
+
 #include <mesos/type_utils.hpp>
 
 #include <process/clock.hpp>
@@ -29,6 +31,9 @@
 
 using std::string;
 
+using mesos::slave::ExecutorLimitation;
+using mesos::slave::ExecutorRunState;
+
 namespace mesos {
 namespace internal {
 namespace protobuf {
@@ -199,6 +204,40 @@ Label createLabel(const std::string& key, const std::string& value)
   return label;
 }
 
+namespace slave {
+
+ExecutorLimitation createExecutorLimitation(
+    const Resources& resources,
+    const std::string& message)
+{
+  ExecutorLimitation limitation;
+  foreach (Resource resource, resources) {
+    limitation.add_resources()->CopyFrom(resource);
+  }
+  limitation.set_message(message);
+  return limitation;
+}
+
+
+ExecutorRunState createExecutorRunState(
+    const ExecutorInfo& executorInfo,
+    const ContainerID& container_id,
+    pid_t pid,
+    const std::string& directory,
+    const Option<std::string>& rootfs)
+{
+  ExecutorRunState state;
+  state.mutable_executor_info()->CopyFrom(executorInfo);
+  state.mutable_container_id()->CopyFrom(container_id);
+  state.set_pid(pid);
+  state.set_directory(directory);
+  if (rootfs.isSome()) {
+    state.set_rootfs(rootfs.get());
+  }
+  return state;
+}
+
+} // namespace slave {
 } // namespace protobuf {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/common/protobuf_utils.hpp
----------------------------------------------------------------------
diff --git a/src/common/protobuf_utils.hpp b/src/common/protobuf_utils.hpp
index 2e827a0..22046ba 100644
--- a/src/common/protobuf_utils.hpp
+++ b/src/common/protobuf_utils.hpp
@@ -21,6 +21,8 @@
 
 #include <string>
 
+#include <mesos/slave/isolator.hpp>
+
 #include <stout/ip.hpp>
 #include <stout/option.hpp>
 #include <stout/uuid.hpp>
@@ -73,6 +75,21 @@ MasterInfo createMasterInfo(const process::UPID& pid);
 
 Label createLabel(const std::string& key, const std::string& value);
 
+namespace slave {
+
+mesos::slave::ExecutorLimitation createExecutorLimitation(
+    const Resources& resources,
+    const std::string& message);
+
+
+mesos::slave::ExecutorRunState createExecutorRunState(
+    const ExecutorInfo& executorInfo,
+    const ContainerID& id,
+    pid_t pid,
+    const std::string& directory,
+    const Option<std::string>& rootfs);
+
+} // namespace slave {
 } // namespace protobuf {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/exec/exec.cpp
----------------------------------------------------------------------
diff --git a/src/exec/exec.cpp b/src/exec/exec.cpp
index a1ae074..54ef622 100644
--- a/src/exec/exec.cpp
+++ b/src/exec/exec.cpp
@@ -686,7 +686,7 @@ Status MesosExecutorDriver::start()
     value = os::getenv("MESOS_CHECKPOINT");
     checkpoint = value.isSome() && value.get() == "1";
 
-    Duration recoveryTimeout = slave::RECOVERY_TIMEOUT;
+    Duration recoveryTimeout = RECOVERY_TIMEOUT;
 
     // Get the recovery timeout if checkpointing is enabled.
     if (checkpoint) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolator.cpp b/src/slave/containerizer/isolator.cpp
index 278824c..ec14b20 100644
--- a/src/slave/containerizer/isolator.cpp
+++ b/src/slave/containerizer/isolator.cpp
@@ -84,7 +84,7 @@ Future<Nothing> Isolator::isolate(
 }
 
 
-Future<Limitation> Isolator::watch(const ContainerID& containerId)
+Future<ExecutorLimitation> Isolator::watch(const ContainerID& containerId)
 {
   return dispatch(process.get(), &IsolatorProcess::watch, containerId);
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/cgroups/cpushare.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/cpushare.cpp b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
index 750bef9..b1ebdad 100644
--- a/src/slave/containerizer/isolators/cgroups/cpushare.cpp
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
@@ -53,10 +53,10 @@ using std::set;
 using std::string;
 using std::vector;
 
+using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
-using mesos::slave::Limitation;
 
 namespace mesos {
 namespace internal {
@@ -175,7 +175,7 @@ Future<Nothing> CgroupsCpushareIsolatorProcess::recover(
     const hashset<ContainerID>& orphans)
 {
   foreach (const ExecutorRunState& state, states) {
-    const ContainerID& containerId = state.id;
+    const ContainerID& containerId = state.container_id();
     const string cgroup = path::join(flags.cgroups_root, containerId.value());
 
     Try<bool> exists = cgroups::exists(hierarchies["cpu"], cgroup);
@@ -338,7 +338,7 @@ Future<Nothing> CgroupsCpushareIsolatorProcess::isolate(
 }
 
 
-Future<Limitation> CgroupsCpushareIsolatorProcess::watch(
+Future<ExecutorLimitation> CgroupsCpushareIsolatorProcess::watch(
     const ContainerID& containerId)
 {
   if (!infos.contains(containerId)) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/cgroups/cpushare.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/cpushare.hpp b/src/slave/containerizer/isolators/cgroups/cpushare.hpp
index 2118c97..4fa9015 100644
--- a/src/slave/containerizer/isolators/cgroups/cpushare.hpp
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.hpp
@@ -59,7 +59,7 @@ public:
       const ContainerID& containerId,
       pid_t pid);
 
-  virtual process::Future<mesos::slave::Limitation> watch(
+  virtual process::Future<mesos::slave::ExecutorLimitation> watch(
       const ContainerID& containerId);
 
   virtual process::Future<Nothing> update(
@@ -92,7 +92,7 @@ private:
     Option<pid_t> pid;
     Option<Resources> resources;
 
-    process::Promise<mesos::slave::Limitation> limitation;
+    process::Promise<mesos::slave::ExecutorLimitation> limitation;
   };
 
   const Flags flags;

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/cgroups/mem.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/mem.cpp b/src/slave/containerizer/isolators/cgroups/mem.cpp
index b0e343f..919e0f7 100644
--- a/src/slave/containerizer/isolators/cgroups/mem.cpp
+++ b/src/slave/containerizer/isolators/cgroups/mem.cpp
@@ -41,6 +41,8 @@
 #include <stout/stringify.hpp>
 #include <stout/try.hpp>
 
+#include "common/protobuf_utils.hpp"
+
 #include "linux/cgroups.hpp"
 
 #include "slave/containerizer/isolators/cgroups/mem.hpp"
@@ -56,10 +58,10 @@ using std::set;
 using std::string;
 using std::vector;
 
+using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
-using mesos::slave::Limitation;
 
 namespace mesos {
 namespace internal {
@@ -163,7 +165,7 @@ Future<Nothing> CgroupsMemIsolatorProcess::recover(
     const hashset<ContainerID>& orphans)
 {
   foreach (const ExecutorRunState& state, states) {
-    const ContainerID& containerId = state.id;
+    const ContainerID& containerId = state.container_id();
     const string cgroup = path::join(flags.cgroups_root, containerId.value());
 
     Try<bool> exists = cgroups::exists(hierarchy, cgroup);
@@ -315,7 +317,7 @@ Future<Nothing> CgroupsMemIsolatorProcess::isolate(
 }
 
 
-Future<Limitation> CgroupsMemIsolatorProcess::watch(
+Future<ExecutorLimitation> CgroupsMemIsolatorProcess::watch(
     const ContainerID& containerId)
 {
   if (!infos.contains(containerId)) {
@@ -692,7 +694,8 @@ void CgroupsMemIsolatorProcess::oom(const ContainerID& containerId)
       stringify(usage.isSome() ? usage.get().megabytes() : 0),
       "*").get();
 
-  info->limitation.set(Limitation(mem, message.str()));
+  info->limitation.set(protobuf::slave::createExecutorLimitation(
+        mem, message.str()));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/cgroups/mem.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/mem.hpp b/src/slave/containerizer/isolators/cgroups/mem.hpp
index dc75201..c198c83 100644
--- a/src/slave/containerizer/isolators/cgroups/mem.hpp
+++ b/src/slave/containerizer/isolators/cgroups/mem.hpp
@@ -59,7 +59,7 @@ public:
       const ContainerID& containerId,
       pid_t pid);
 
-  virtual process::Future<mesos::slave::Limitation> watch(
+  virtual process::Future<mesos::slave::ExecutorLimitation> watch(
       const ContainerID& containerId);
 
   virtual process::Future<Nothing> update(
@@ -97,7 +97,7 @@ private:
     const std::string cgroup;
     Option<pid_t> pid;
 
-    process::Promise<mesos::slave::Limitation> limitation;
+    process::Promise<mesos::slave::ExecutorLimitation> limitation;
 
     // Used to cancel the OOM listening.
     process::Future<Nothing> oomNotifier;

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/cgroups/perf_event.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/perf_event.cpp b/src/slave/containerizer/isolators/cgroups/perf_event.cpp
index 512df3b..367cb43 100644
--- a/src/slave/containerizer/isolators/cgroups/perf_event.cpp
+++ b/src/slave/containerizer/isolators/cgroups/perf_event.cpp
@@ -58,10 +58,10 @@ using std::set;
 using std::string;
 using std::vector;
 
+using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
-using mesos::slave::Limitation;
 
 namespace mesos {
 namespace internal {
@@ -148,7 +148,7 @@ Future<Nothing> CgroupsPerfEventIsolatorProcess::recover(
     const hashset<ContainerID>& orphans)
 {
   foreach (const ExecutorRunState& state, states) {
-    const ContainerID& containerId = state.id;
+    const ContainerID& containerId = state.container_id();
     const string cgroup = path::join(flags.cgroups_root, containerId.value());
 
     Try<bool> exists = cgroups::exists(hierarchy, cgroup);
@@ -298,11 +298,11 @@ Future<Nothing> CgroupsPerfEventIsolatorProcess::isolate(
 }
 
 
-Future<Limitation> CgroupsPerfEventIsolatorProcess::watch(
+Future<ExecutorLimitation> CgroupsPerfEventIsolatorProcess::watch(
     const ContainerID& containerId)
 {
   // No resources are limited.
-  return Future<Limitation>();
+  return Future<ExecutorLimitation>();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/cgroups/perf_event.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/perf_event.hpp b/src/slave/containerizer/isolators/cgroups/perf_event.hpp
index 560cd03..243cf5a 100644
--- a/src/slave/containerizer/isolators/cgroups/perf_event.hpp
+++ b/src/slave/containerizer/isolators/cgroups/perf_event.hpp
@@ -57,7 +57,7 @@ public:
       const ContainerID& containerId,
       pid_t pid);
 
-  virtual process::Future<mesos::slave::Limitation> watch(
+  virtual process::Future<mesos::slave::ExecutorLimitation> watch(
       const ContainerID& containerId);
 
   virtual process::Future<Nothing> update(

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/filesystem/posix.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/posix.cpp b/src/slave/containerizer/isolators/filesystem/posix.cpp
index 0c56627..72d2738 100644
--- a/src/slave/containerizer/isolators/filesystem/posix.cpp
+++ b/src/slave/containerizer/isolators/filesystem/posix.cpp
@@ -35,10 +35,10 @@ using namespace process;
 using std::list;
 using std::string;
 
+using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
-using mesos::slave::Limitation;
 
 namespace mesos {
 namespace internal {
@@ -66,7 +66,7 @@ Future<Nothing> PosixFilesystemIsolatorProcess::recover(
     const hashset<ContainerID>& orphans)
 {
   foreach (const ExecutorRunState& state, states) {
-    infos.put(state.id, Owned<Info>(new Info(state.directory)));
+    infos.put(state.container_id(), Owned<Info>(new Info(state.directory())));
   }
 
   return Nothing();
@@ -106,11 +106,11 @@ Future<Nothing> PosixFilesystemIsolatorProcess::isolate(
 }
 
 
-Future<Limitation> PosixFilesystemIsolatorProcess::watch(
+Future<ExecutorLimitation> PosixFilesystemIsolatorProcess::watch(
     const ContainerID& containerId)
 {
   // No-op.
-  return Future<Limitation>();
+  return Future<ExecutorLimitation>();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/filesystem/posix.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/posix.hpp b/src/slave/containerizer/isolators/filesystem/posix.hpp
index 16ba26f..d44023e 100644
--- a/src/slave/containerizer/isolators/filesystem/posix.hpp
+++ b/src/slave/containerizer/isolators/filesystem/posix.hpp
@@ -49,7 +49,7 @@ public:
       const ContainerID& containerId,
       pid_t pid);
 
-  virtual process::Future<mesos::slave::Limitation> watch(
+  virtual process::Future<mesos::slave::ExecutorLimitation> watch(
       const ContainerID& containerId);
 
   virtual process::Future<Nothing> update(

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/filesystem/shared.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/shared.cpp b/src/slave/containerizer/isolators/filesystem/shared.cpp
index 24f3074..f90045e 100644
--- a/src/slave/containerizer/isolators/filesystem/shared.cpp
+++ b/src/slave/containerizer/isolators/filesystem/shared.cpp
@@ -32,10 +32,10 @@ namespace mesos {
 namespace internal {
 namespace slave {
 
+using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
-using mesos::slave::Limitation;
 
 SharedFilesystemIsolatorProcess::SharedFilesystemIsolatorProcess(
     const Flags& _flags)
@@ -236,12 +236,12 @@ Future<Nothing> SharedFilesystemIsolatorProcess::isolate(
 }
 
 
-Future<Limitation> SharedFilesystemIsolatorProcess::watch(
+Future<ExecutorLimitation> SharedFilesystemIsolatorProcess::watch(
     const ContainerID& containerId)
 {
   // No-op, for now.
 
-  return Future<Limitation>();
+  return Future<ExecutorLimitation>();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/filesystem/shared.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/shared.hpp b/src/slave/containerizer/isolators/filesystem/shared.hpp
index 4d7d9a9..4a5dcc3 100644
--- a/src/slave/containerizer/isolators/filesystem/shared.hpp
+++ b/src/slave/containerizer/isolators/filesystem/shared.hpp
@@ -56,7 +56,7 @@ public:
       const ContainerID& containerId,
       pid_t pid);
 
-  virtual process::Future<mesos::slave::Limitation> watch(
+  virtual process::Future<mesos::slave::ExecutorLimitation> watch(
       const ContainerID& containerId);
 
   virtual process::Future<Nothing> update(

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/namespaces/pid.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/namespaces/pid.cpp b/src/slave/containerizer/isolators/namespaces/pid.cpp
index 5de0791..4241fa7 100644
--- a/src/slave/containerizer/isolators/namespaces/pid.cpp
+++ b/src/slave/containerizer/isolators/namespaces/pid.cpp
@@ -39,10 +39,10 @@ using std::list;
 using std::set;
 using std::string;
 
+using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
-using mesos::slave::Limitation;
 
 namespace mesos {
 namespace internal {
@@ -133,7 +133,7 @@ Future<Nothing> NamespacesPidIsolatorProcess::recover(
 {
   hashset<ContainerID> recovered;
   foreach (const ExecutorRunState& state, states) {
-    recovered.insert(state.id);
+    recovered.insert(state.container_id());
   }
 
   // Clean up any unknown orphaned bind mounts and empty files. Known
@@ -220,10 +220,10 @@ Future<Nothing> NamespacesPidIsolatorProcess::isolate(
 }
 
 
-Future<Limitation> NamespacesPidIsolatorProcess::watch(
+Future<ExecutorLimitation> NamespacesPidIsolatorProcess::watch(
     const ContainerID& containerId)
 {
-  return Future<Limitation>();
+  return Future<ExecutorLimitation>();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/namespaces/pid.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/namespaces/pid.hpp b/src/slave/containerizer/isolators/namespaces/pid.hpp
index 9cda3fd..702f331 100644
--- a/src/slave/containerizer/isolators/namespaces/pid.hpp
+++ b/src/slave/containerizer/isolators/namespaces/pid.hpp
@@ -73,7 +73,7 @@ public:
       const ContainerID& containerId,
       pid_t pid);
 
-  virtual process::Future<mesos::slave::Limitation> watch(
+  virtual process::Future<mesos::slave::ExecutorLimitation> watch(
       const ContainerID& containerId);
 
   virtual process::Future<Nothing> update(

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/network/port_mapping.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.cpp b/src/slave/containerizer/isolators/network/port_mapping.cpp
index a7757f2..98c6faa 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.cpp
@@ -117,10 +117,10 @@ namespace mesos {
 namespace internal {
 namespace slave {
 
+using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
-using mesos::slave::Limitation;
 
 // The minimum number of ephemeral ports a container should have.
 static const uint16_t MIN_EPHEMERAL_PORTS_SIZE = 16;
@@ -1805,8 +1805,8 @@ Future<Nothing> PortMappingIsolatorProcess::recover(
 
   // Now, actually recover the isolator from slave's state.
   foreach (const ExecutorRunState& state, states) {
-    const ContainerID& containerId = state.id;
-    pid_t pid = state.pid;
+    const ContainerID& containerId = state.id();
+    pid_t pid = state.pid();
 
     VLOG(1) << "Recovering network isolator for container "
             << containerId << " with pid " << pid;
@@ -2495,7 +2495,7 @@ Future<Nothing> PortMappingIsolatorProcess::isolate(
 }
 
 
-Future<Limitation> PortMappingIsolatorProcess::watch(
+Future<ExecutorLimitation> PortMappingIsolatorProcess::watch(
     const ContainerID& containerId)
 {
   if (unmanaged.contains(containerId)) {
@@ -2506,7 +2506,7 @@ Future<Limitation> PortMappingIsolatorProcess::watch(
 
   // Currently, we always return a pending future because limitation
   // is never reached.
-  return Future<Limitation>();
+  return Future<ExecutorLimitation>();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/network/port_mapping.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.hpp b/src/slave/containerizer/isolators/network/port_mapping.hpp
index 6b5cf62..6ffd729 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.hpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.hpp
@@ -169,7 +169,7 @@ public:
       const ContainerID& containerId,
       pid_t pid);
 
-  virtual process::Future<mesos::slave::Limitation> watch(
+  virtual process::Future<mesos::slave::ExecutorLimitation> watch(
       const ContainerID& containerId);
 
   virtual process::Future<Nothing> update(

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/posix.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/posix.hpp b/src/slave/containerizer/isolators/posix.hpp
index 271061e..6ddab7d 100644
--- a/src/slave/containerizer/isolators/posix.hpp
+++ b/src/slave/containerizer/isolators/posix.hpp
@@ -48,15 +48,15 @@ public:
     foreach (const mesos::slave::ExecutorRunState& run, state) {
       // This should (almost) never occur: see comment in
       // PosixLauncher::recover().
-      if (pids.contains(run.id)) {
+      if (pids.contains(run.container_id())) {
         return process::Failure("Container already recovered");
       }
 
-      pids.put(run.id, run.pid);
+      pids.put(run.container_id(), run.pid());
 
-      process::Owned<process::Promise<mesos::slave::Limitation>> promise(
-          new process::Promise<mesos::slave::Limitation>());
-      promises.put(run.id, promise);
+      process::Owned<process::Promise<mesos::slave::ExecutorLimitation>>
+        promise(new process::Promise<mesos::slave::ExecutorLimitation>());
+      promises.put(run.container_id(), promise);
     }
 
     return Nothing();
@@ -74,8 +74,8 @@ public:
                               " has already been prepared");
     }
 
-    process::Owned<process::Promise<mesos::slave::Limitation>> promise(
-        new process::Promise<mesos::slave::Limitation>());
+    process::Owned<process::Promise<mesos::slave::ExecutorLimitation>> promise(
+        new process::Promise<mesos::slave::ExecutorLimitation>());
     promises.put(containerId, promise);
 
     return None();
@@ -94,7 +94,7 @@ public:
     return Nothing();
   }
 
-  virtual process::Future<mesos::slave::Limitation> watch(
+  virtual process::Future<mesos::slave::ExecutorLimitation> watch(
       const ContainerID& containerId)
   {
     if (!promises.contains(containerId)) {
@@ -133,9 +133,9 @@ public:
 
 protected:
   hashmap<ContainerID, pid_t> pids;
-  hashmap<
-      ContainerID,
-      process::Owned<process::Promise<mesos::slave::Limitation>>> promises;
+  hashmap<ContainerID,
+          process::Owned<process::Promise<mesos::slave::ExecutorLimitation>>>
+    promises;
 };
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/posix/disk.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/posix/disk.cpp b/src/slave/containerizer/isolators/posix/disk.cpp
index b2f995c..238f179 100644
--- a/src/slave/containerizer/isolators/posix/disk.cpp
+++ b/src/slave/containerizer/isolators/posix/disk.cpp
@@ -44,6 +44,8 @@
 #include <stout/os/exists.hpp>
 #include <stout/os/killtree.hpp>
 
+#include "common/protobuf_utils.hpp"
+
 #include "slave/containerizer/isolators/posix/disk.hpp"
 
 using namespace process;
@@ -57,10 +59,10 @@ namespace mesos {
 namespace internal {
 namespace slave {
 
+using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
-using mesos::slave::Limitation;
 
 Try<Isolator*> PosixDiskIsolatorProcess::create(const Flags& flags)
 {
@@ -91,10 +93,10 @@ Future<Nothing> PosixDiskIsolatorProcess::recover(
   foreach (const ExecutorRunState& state, states) {
     // Since we checkpoint the executor after we create its working
     // directory, the working directory should definitely exist.
-    CHECK(os::exists(state.directory))
-      << "Executor work directory " << state.directory << " doesn't exist";
+    CHECK(os::exists(state.directory()))
+      << "Executor work directory " << state.directory() << " doesn't exist";
 
-    infos.put(state.id, Owned<Info>(new Info(state.directory)));
+    infos.put(state.container_id(), Owned<Info>(new Info(state.directory())));
   }
 
   return Nothing();
@@ -130,7 +132,7 @@ Future<Nothing> PosixDiskIsolatorProcess::isolate(
 }
 
 
-Future<Limitation> PosixDiskIsolatorProcess::watch(
+Future<ExecutorLimitation> PosixDiskIsolatorProcess::watch(
     const ContainerID& containerId)
 {
   if (!infos.contains(containerId)) {
@@ -246,8 +248,8 @@ void PosixDiskIsolatorProcess::_collect(
       CHECK_SOME(quota);
 
       if (future.get() > quota.get()) {
-        info->limitation.set(Limitation(
-            info->paths[path].quota,
+        info->limitation.set(protobuf::slave::createExecutorLimitation(
+            Resources(info->paths[path].quota),
             "Disk usage (" + stringify(future.get()) +
             ") exceeds quota (" + stringify(quota.get()) + ")"));
       }

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/isolators/posix/disk.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/posix/disk.hpp b/src/slave/containerizer/isolators/posix/disk.hpp
index 5dfa815..fdf24a1 100644
--- a/src/slave/containerizer/isolators/posix/disk.hpp
+++ b/src/slave/containerizer/isolators/posix/disk.hpp
@@ -58,9 +58,10 @@ private:
 
 
 // This isolator monitors the disk usage for containers, and reports
-// Limitation when a container exceeds its disk quota. This leverages
-// the DiskUsageCollector to ensure that we don't induce too much CPU
-// usage and disk caching effects from running 'du' too often.
+// ExecutorLimitation when a container exceeds its disk quota. This
+// leverages the DiskUsageCollector to ensure that we don't induce too
+// much CPU usage and disk caching effects from running 'du' too
+// often.
 //
 // NOTE: Currently all containers are processed in the same queue,
 // which means that when a container starts, it could take many disk
@@ -92,7 +93,7 @@ public:
       const ContainerID& containerId,
       pid_t pid);
 
-  virtual process::Future<mesos::slave::Limitation> watch(
+  virtual process::Future<mesos::slave::ExecutorLimitation> watch(
       const ContainerID& containerId);
 
   virtual process::Future<Nothing> update(
@@ -124,7 +125,7 @@ private:
     // to collect disk usage for disk resources without DiskInfo.
     const std::string directory;
 
-    process::Promise<mesos::slave::Limitation> limitation;
+    process::Promise<mesos::slave::ExecutorLimitation> limitation;
 
     // The keys of the hashmaps contain the executor working directory
     // above, and optionally paths of volumes used by the container.

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/launcher.cpp b/src/slave/containerizer/launcher.cpp
index 24df1ca..ecb3330 100644
--- a/src/slave/containerizer/launcher.cpp
+++ b/src/slave/containerizer/launcher.cpp
@@ -53,8 +53,8 @@ Future<hashset<ContainerID>> PosixLauncher::recover(
     const list<ExecutorRunState>& states)
 {
   foreach (const ExecutorRunState& state, states) {
-    const ContainerID& containerId = state.id;
-    pid_t pid = state.pid;
+    const ContainerID& containerId = state.container_id();
+    pid_t pid = state.pid();
 
     if (pids.containsValue(pid)) {
       // This should (almost) never occur. There is the possibility

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/linux_launcher.cpp b/src/slave/containerizer/linux_launcher.cpp
index 790e392..ed2e881 100644
--- a/src/slave/containerizer/linux_launcher.cpp
+++ b/src/slave/containerizer/linux_launcher.cpp
@@ -115,8 +115,8 @@ Future<hashset<ContainerID>> LinuxLauncher::recover(
   hashset<string> recovered;
 
   foreach (const ExecutorRunState& state, states) {
-    const ContainerID& containerId = state.id;
-    pid_t pid = state.pid;
+    const ContainerID& containerId = state.container_id();
+    pid_t pid = state.pid();
 
     if (pids.containsValue(pid)) {
       // This should (almost) never occur. There is the possibility

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 609620c..c21e925 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -31,6 +31,8 @@
 #include <stout/os.hpp>
 #include <stout/path.hpp>
 
+#include "common/protobuf_utils.hpp"
+
 #include "module/manager.hpp"
 
 #include "slave/paths.hpp"
@@ -82,10 +84,10 @@ namespace slave {
 
 using mesos::modules::ModuleManager;
 
+using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
-using mesos::slave::Limitation;
 
 using state::SlaveState;
 using state::FrameworkState;
@@ -415,12 +417,13 @@ Future<Nothing> MesosContainerizerProcess::recover(
 
         CHECK(os::exists(directory));
 
-        ExecutorRunState executorRunState(
-            executorInfo,
-            run.get().id.get(),
-            run.get().forkedPid.get(),
-            directory,
-            run.get().rootfs);
+        ExecutorRunState executorRunState =
+          protobuf::slave::createExecutorRunState(
+              executorInfo,
+              run.get().id.get(),
+              run.get().forkedPid.get(),
+              directory,
+              run.get().rootfs);
 
         recoverable.push_back(executorRunState);
       }
@@ -460,24 +463,26 @@ Future<Nothing> MesosContainerizerProcess::__recover(
     const hashset<ContainerID>& orphans)
 {
   foreach (const ExecutorRunState& run, recovered) {
-    const ContainerID& containerId = run.id;
+    const ContainerID& containerId = run.container_id();
 
     Container* container = new Container();
 
-    Future<Option<int>> status = process::reap(run.pid);
+    Future<Option<int>> status = process::reap(run.pid());
     status.onAny(defer(self(), &Self::reaped, containerId));
     container->status = status;
 
-    container->directory = run.directory;
+    container->directory = run.directory();
 
-    container->rootfs = run.rootfs;
+    if (run.has_rootfs()) {
+      container->rootfs = run.rootfs();
+    }
 
     // We only checkpoint the containerizer pid after the container
     // successfully launched, therefore we can assume checkpointed
     // containers should be running after recover.
     container->state = RUNNING;
 
-    container->executorInfo = run.executorInfo;
+    container->executorInfo = run.executor_info();
 
     containers_[containerId] = Owned<Container>(container);
 
@@ -1318,8 +1323,8 @@ void MesosContainerizerProcess::_____destroy(
   // exit.
   if (!killed && container->limitations.size() > 0) {
     string message_;
-    foreach (const Limitation& limitation, container->limitations) {
-      message_ += limitation.message;
+    foreach (const ExecutorLimitation& limitation, container->limitations) {
+      message_ += limitation.message();
     }
     message = strings::trim(message_);
   } else if (!killed && message.isNone()) {
@@ -1362,7 +1367,7 @@ void MesosContainerizerProcess::reaped(const ContainerID& containerId)
 
 void MesosContainerizerProcess::limited(
     const ContainerID& containerId,
-    const Future<Limitation>& future)
+    const Future<ExecutorLimitation>& future)
 {
   if (!containers_.contains(containerId) ||
       containers_[containerId]->state == DESTROYING) {
@@ -1371,7 +1376,7 @@ void MesosContainerizerProcess::limited(
 
   if (future.isReady()) {
     LOG(INFO) << "Container " << containerId << " has reached its limit for"
-              << " resource " << future.get().resources
+              << " resource " << future.get().resources()
               << " and will be terminated";
 
     containers_[containerId]->limitations.push_back(future.get());

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index f6c580d..5155362 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -262,7 +262,7 @@ private:
   // processes. This will trigger container destruction.
   void limited(
       const ContainerID& containerId,
-      const process::Future<mesos::slave::Limitation>& future);
+      const process::Future<mesos::slave::ExecutorLimitation>& future);
 
   // Call back for when the executor exits. This will trigger container
   // destroy.
@@ -311,7 +311,7 @@ private:
 
     // We keep track of any limitations received from each isolator so we can
     // determine the cause of an executor termination.
-    std::vector<mesos::slave::Limitation> limitations;
+    std::vector<mesos::slave::ExecutorLimitation> limitations;
 
     // We keep track of the resources for each container so we can set the
     // ResourceStatistics limits in usage().

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/tests/containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer_tests.cpp b/src/tests/containerizer_tests.cpp
index 88c46e7..0b13381 100644
--- a/src/tests/containerizer_tests.cpp
+++ b/src/tests/containerizer_tests.cpp
@@ -62,10 +62,10 @@ using mesos::internal::slave::state::FrameworkState;
 using mesos::internal::slave::state::RunState;
 using mesos::internal::slave::state::SlaveState;
 
+using mesos::slave::ExecutorLimitation;
 using mesos::slave::ExecutorRunState;
 using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
-using mesos::slave::Limitation;
 
 using std::list;
 using std::map;
@@ -444,7 +444,7 @@ public:
 
   MOCK_METHOD1(
       watch,
-      Future<Limitation>(const ContainerID&));
+      Future<mesos::slave::ExecutorLimitation>(const ContainerID&));
 
   MOCK_METHOD2(
       update,
@@ -458,7 +458,7 @@ public:
       cleanup,
       Future<Nothing>(const ContainerID&));
 
-  Promise<Limitation> watchPromise;
+  Promise<mesos::slave::ExecutorLimitation> watchPromise;
 };
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/30205f82/src/tests/isolator.hpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator.hpp b/src/tests/isolator.hpp
index fd6aec7..671b021 100644
--- a/src/tests/isolator.hpp
+++ b/src/tests/isolator.hpp
@@ -61,7 +61,7 @@ public:
 
   MOCK_METHOD1(
       watch,
-      process::Future<mesos::slave::Limitation>(const ContainerID&));
+      process::Future<mesos::slave::ExecutorLimitation>(const ContainerID&));
 
   MOCK_METHOD2(
       update,
@@ -92,7 +92,7 @@ private:
 
   const Option<CommandInfo> commandInfo;
 
-  process::Promise<mesos::slave::Limitation> promise;
+  process::Promise<mesos::slave::ExecutorLimitation> promise;
 };
 
 } // namespace tests {