You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2015/04/26 21:43:35 UTC

[02/10] mesos git commit: Removed pre-C++11 codepath in libprocess.

http://git-wip-us.apache.org/repos/asf/mesos/blob/5ab8ff81/3rdparty/libprocess/include/process/deferred.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/deferred.hpp b/3rdparty/libprocess/include/process/deferred.hpp
index 8bdc692..352205b 100644
--- a/3rdparty/libprocess/include/process/deferred.hpp
+++ b/3rdparty/libprocess/include/process/deferred.hpp
@@ -1,160 +1,270 @@
-#if __cplusplus >= 201103L
-#include <process/c++11/deferred.hpp>
-#else
 #ifndef __PROCESS_DEFERRED_HPP__
 #define __PROCESS_DEFERRED_HPP__
 
-#include <tr1/functional>
+#include <functional>
 
-#include <process/future.hpp>
+#include <process/dispatch.hpp>
 #include <process/pid.hpp>
 
 #include <stout/preprocessor.hpp>
 
 namespace process {
 
-// Forward declarations (removing these produces cryptic compiler
-// errors even though we are just using them to declare friends).
-class Executor;
-template <typename _F> struct _Defer;
-
-
 // Acts like a function call but runs within an asynchronous execution
 // context such as an Executor or a ProcessBase (enforced because only
 // an executor or the 'defer' routines are allowed to create them).
 template <typename F>
-struct Deferred : std::tr1::function<F>
+struct Deferred : std::function<F>
 {
 private:
-  // Only an Executor and the 'defer' routines can create these.
   friend class Executor;
 
-  template <typename _F> friend struct _Defer;
+  template <typename G> friend struct _Deferred;
+
+  // TODO(benh): Consider removing these in favor of having these
+  // functions return _Deferred.
+  template <typename T>
+  friend Deferred<void(void)>
+  defer(const PID<T>& pid, void (T::*method)(void));
+
+  template <typename R, typename T>
+  friend Deferred<Future<R>(void)>
+  defer(const PID<T>& pid, Future<R> (T::*method)(void));
+
+  template <typename R, typename T>
+  friend Deferred<Future<R>(void)>
+  defer(const PID<T>& pid, R (T::*method)(void));
+
+  /*implicit*/ Deferred(const std::function<F>& f) : std::function<F>(f) {}
+};
+
+
+// We need an intermeidate "deferred" type because when constructing a
+// Deferred we won't always know the underlying function type (for
+// example, if we're being passed a std::bind or a lambda). A lambda
+// won't always implicitly convert to a std::function so instead we
+// hold onto the functor type F and let the compiler invoke the
+// necessary cast operator (below) when it actually has determined
+// what type is needed. This is similar in nature to how std::bind
+// works with it's intermediate _Bind type (which the pre-C++11
+// implementation relied on).
+template <typename F>
+struct _Deferred
+{
+  operator Deferred<void()> () const
+  {
+    // The 'pid' differentiates an already dispatched functor versus
+    // one which still needs to be dispatched (which is done
+    // below). We have to delay wrapping the dispatch (for example, in
+    // defer.hpp) as long as possible because we don't always know
+    // what type the functor is or is going to be cast to (i.e., a
+    // std::bind might can be cast to functions that do or do not take
+    // arguments which will just be dropped when invoking the
+    // underlying bound function).
+    if (pid.isNone()) {
+      return std::function<void()>(f);
+    }
+
+    // We need to explicitly copy the members otherwise we'll
+    // implicitly copy 'this' which might not exist at invocation.
+    Option<UPID> pid_ = pid;
+    F f_ = f;
+
+    return std::function<void()>(
+        [=] () {
+          dispatch(pid_.get(), std::function<void()>(f_));
+        });
+  }
+
+  operator std::function<void()> () const
+  {
+    if (pid.isNone()) {
+      return std::function<void()>(f);
+    }
+
+    Option<UPID> pid_ = pid;
+    F f_ = f;
+
+    return std::function<void()>(
+        [=] () {
+          dispatch(pid_.get(), std::function<void()>(f_));
+        });
+  }
+
+  template <typename R>
+  operator Deferred<R()> () const
+  {
+    if (pid.isNone()) {
+      return std::function<R()>(f);
+    }
+
+    Option<UPID> pid_ = pid;
+    F f_ = f;
+
+    return std::function<R()>(
+        [=] () {
+          return dispatch(pid_.get(), std::function<R()>(f_));
+        });
+  }
+
+  template <typename R>
+  operator std::function<R()> () const
+  {
+    if (pid.isNone()) {
+      return std::function<R()>(f);
+    }
 
-  friend Deferred<void(void)> defer(
-      const UPID& pid,
-      const std::tr1::function<void(void)>& f);
+    Option<UPID> pid_ = pid;
+    F f_ = f;
 
-  friend Deferred<void(void)> defer(
-      const std::tr1::function<void(void)>& f);
+    return std::function<R()>(
+        [=] () {
+          return dispatch(pid_.get(), std::function<R()>(f_));
+        });
+  }
 
+  // Due to a bug (http://gcc.gnu.org/bugzilla/show_bug.cgi?id=41933)
+  // with variadic templates and lambdas, we still need to do
+  // preprocessor expansions. In addition, due to a bug with clang (or
+  // libc++) we can't use std::bind with a std::function so we have to
+  // explicitely use the std::function<R(P...)>::operator() (see
+  // http://stackoverflow.com/questions/20097616/stdbind-to-a-stdfunction-crashes-with-clang).
 #define TEMPLATE(Z, N, DATA)                                            \
-  template <ENUM_PARAMS(N, typename A)>                                 \
-  friend Deferred<void(ENUM_PARAMS(N, A))> defer(                       \
-      const UPID& pid,                                                  \
-      const std::tr1::function<void(ENUM_PARAMS(N, A))>& f);            \
+  template <ENUM_PARAMS(N, typename P)>                                 \
+  operator Deferred<void(ENUM_PARAMS(N, P))> () const                   \
+  {                                                                     \
+    if (pid.isNone()) {                                                 \
+      return std::function<void(ENUM_PARAMS(N, P))>(f);                 \
+    }                                                                   \
+                                                                        \
+    Option<UPID> pid_ = pid;                                            \
+    F f_ = f;                                                           \
                                                                         \
-  template <ENUM_PARAMS(N, typename A)>                                 \
-  friend Deferred<void(ENUM_PARAMS(N, A))> defer(                       \
-      const std::tr1::function<void(ENUM_PARAMS(N, A))>& f);            \
+    return std::function<void(ENUM_PARAMS(N, P))>(                      \
+        [=] (ENUM_BINARY_PARAMS(N, P, p)) {                             \
+          std::function<void()> f__([=] () {                            \
+            f_(ENUM_PARAMS(N, p));                                      \
+          });                                                           \
+          dispatch(pid_.get(), f__);                                    \
+        });                                                             \
+  }                                                                     \
                                                                         \
-  template <typename R, ENUM_PARAMS(N, typename A)>                     \
-  friend Deferred<Future<R>(ENUM_PARAMS(N, A))> defer(                  \
-      const UPID& pid,                                                  \
-      const std::tr1::function<Future<R>(ENUM_PARAMS(N, A))>& f);       \
+  template <ENUM_PARAMS(N, typename P)>                                 \
+  operator std::function<void(ENUM_PARAMS(N, P))> () const              \
+  {                                                                     \
+    if (pid.isNone()) {                                                 \
+      return std::function<void(ENUM_PARAMS(N, P))>(f);                 \
+    }                                                                   \
                                                                         \
-  template <typename R, ENUM_PARAMS(N, typename A)>                     \
-  friend Deferred<Future<R>(ENUM_PARAMS(N, A))> defer(                  \
-      const std::tr1::function<Future<R>(ENUM_PARAMS(N, A))>& f);
+    Option<UPID> pid_ = pid;                                            \
+    F f_ = f;                                                           \
+                                                                        \
+    return std::function<void(ENUM_PARAMS(N, P))>(                      \
+        [=] (ENUM_BINARY_PARAMS(N, P, p)) {                             \
+          std::function<void()> f__([=] () {                            \
+            f_(ENUM_PARAMS(N, p));                                      \
+          });                                                           \
+          dispatch(pid_.get(), f__);                                    \
+        });                                                             \
+  }
 
   REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
 #undef TEMPLATE
 
-  /*implicit*/ Deferred(const std::tr1::function<F>& f)
-    : std::tr1::function<F>(f) {}
-};
-
-
-// The result of invoking the 'defer' routines is actually an internal
-// type, effectively just a wrapper around the result of invoking
-// 'std::tr1::bind'. However, we want the result of bind to be
-// castable to a 'Deferred' but we don't want anyone to be able to
-// create a 'Deferred' so we use a level-of-indirection via this type.
-template <typename F>
-struct _Defer : std::tr1::_Bind<F>
-{
-  template <typename _F>
-  operator Deferred<_F> ()
-  {
-    return Deferred<_F>(std::tr1::function<_F>(*this));
+#define TEMPLATE(Z, N, DATA)                                            \
+  template <typename R, ENUM_PARAMS(N, typename P)>                     \
+  operator Deferred<R(ENUM_PARAMS(N, P))> () const                      \
+  {                                                                     \
+    if (pid.isNone()) {                                                 \
+      return std::function<R(ENUM_PARAMS(N, P))>(f);                    \
+    }                                                                   \
+                                                                        \
+    Option<UPID> pid_ = pid;                                            \
+    F f_ = f;                                                           \
+                                                                        \
+    return std::function<R(ENUM_PARAMS(N, P))>(                         \
+        [=] (ENUM_BINARY_PARAMS(N, P, p)) {                             \
+          std::function<R()> f__([=] () {                               \
+            return f_(ENUM_PARAMS(N, p));                               \
+          });                                                           \
+          return dispatch(pid_.get(), f__);                             \
+        });                                                             \
+  }                                                                     \
+                                                                        \
+  template <typename R, ENUM_PARAMS(N, typename P)>                     \
+  operator std::function<R(ENUM_PARAMS(N, P))> () const                 \
+  {                                                                     \
+    if (pid.isNone()) {                                                 \
+      return std::function<R(ENUM_PARAMS(N, P))>(f);                    \
+    }                                                                   \
+                                                                        \
+    Option<UPID> pid_ = pid;                                            \
+    F f_ = f;                                                           \
+                                                                        \
+    return std::function<R(ENUM_PARAMS(N, P))>(                         \
+        [=] (ENUM_BINARY_PARAMS(N, P, p)) {                             \
+          std::function<R()> f__([=] () {                               \
+            return f_(ENUM_PARAMS(N, p));                               \
+          });                                                           \
+          return dispatch(pid_.get(), f__);                             \
+        });                                                             \
   }
 
+  REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
+#undef TEMPLATE
+
 private:
   friend class Executor;
 
-  template <typename T>
-  friend _Defer<void(*(PID<T>, void (T::*)(void)))
-                (const PID<T>&, void (T::*)(void))>
-  defer(const PID<T>& pid, void (T::*method)(void));
+  template <typename G>
+  friend _Deferred<G> defer(const UPID& pid, G&& g);
 
 #define TEMPLATE(Z, N, DATA)                                            \
   template <typename T,                                                 \
             ENUM_PARAMS(N, typename P),                                 \
             ENUM_PARAMS(N, typename A)>                                 \
-  friend _Defer<void(*(PID<T>,                                          \
-                       void (T::*)(ENUM_PARAMS(N, P)),                  \
-                       ENUM_PARAMS(N, A)))                              \
-                (const PID<T>&,                                         \
-                 void (T::*)(ENUM_PARAMS(N, P)),                        \
-                 ENUM_PARAMS(N, P))>                                    \
-  defer(const PID<T>& pid,                                              \
-        void (T::*method)(ENUM_PARAMS(N, P)),                           \
-        ENUM_BINARY_PARAMS(N, A, a));
+  friend auto defer(const PID<T>& pid,                                  \
+             void (T::*method)(ENUM_PARAMS(N, P)),                      \
+             ENUM_BINARY_PARAMS(N, A, a))                               \
+    -> _Deferred<decltype(std::bind(&std::function<void(ENUM_PARAMS(N, P))>::operator(), std::function<void(ENUM_PARAMS(N, P))>(), ENUM_PARAMS(N, a)))>; // NOLINT(whitespace/line_length)
 
   REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
 #undef TEMPLATE
 
-  template <typename R, typename T>
-  friend _Defer<Future<R>(*(PID<T>, Future<R> (T::*)(void)))(
-      const PID<T>&, Future<R> (T::*)(void))>
-  defer(const PID<T>& pid, Future<R> (T::*method)(void));
-
 #define TEMPLATE(Z, N, DATA)                                            \
   template <typename R,                                                 \
             typename T,                                                 \
             ENUM_PARAMS(N, typename P),                                 \
             ENUM_PARAMS(N, typename A)>                                 \
-  friend _Defer<Future<R>(*(PID<T>,                                     \
-                            Future<R> (T::*)(ENUM_PARAMS(N, P)),        \
-                            ENUM_PARAMS(N, A)))                         \
-                (const PID<T>&,                                         \
-                 Future<R> (T::*)(ENUM_PARAMS(N, P)),                   \
-                 ENUM_PARAMS(N, P))>                                    \
-  defer(const PID<T>& pid,                                              \
-        Future<R> (T::*method)(ENUM_PARAMS(N, P)),                      \
-        ENUM_BINARY_PARAMS(N, A, a));
+  friend auto defer(const PID<T>& pid,                                  \
+             Future<R> (T::*method)(ENUM_PARAMS(N, P)),                 \
+             ENUM_BINARY_PARAMS(N, A, a))                               \
+    -> _Deferred<decltype(std::bind(&std::function<Future<R>(ENUM_PARAMS(N, P))>::operator(), std::function<Future<R>(ENUM_PARAMS(N, P))>(), ENUM_PARAMS(N, a)))>; // NOLINT(whitespace/line_length)
 
   REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
 #undef TEMPLATE
 
-  template <typename R, typename T>
-  friend _Defer<Future<R>(*(PID<T>, R (T::*)(void)))(
-      const PID<T>&, R (T::*)(void))>
-  defer(const PID<T>& pid, R (T::*method)(void));
-
 #define TEMPLATE(Z, N, DATA)                                            \
   template <typename R,                                                 \
             typename T,                                                 \
             ENUM_PARAMS(N, typename P),                                 \
             ENUM_PARAMS(N, typename A)>                                 \
-  friend _Defer<Future<R>(*(PID<T>,                                     \
-                            R (T::*)(ENUM_PARAMS(N, P)),                \
-                            ENUM_PARAMS(N, A)))                         \
-                (const PID<T>&,                                         \
-                 R (T::*)(ENUM_PARAMS(N, P)),                           \
-                 ENUM_PARAMS(N, P))>                                    \
-  defer(const PID<T>& pid,                                              \
-        R (T::*method)(ENUM_PARAMS(N, P)),                              \
-        ENUM_BINARY_PARAMS(N, A, a));
+  friend auto defer(const PID<T>& pid,                                  \
+             R (T::*method)(ENUM_PARAMS(N, P)),                         \
+             ENUM_BINARY_PARAMS(N, A, a))                               \
+    -> _Deferred<decltype(std::bind(&std::function<Future<R>(ENUM_PARAMS(N, P))>::operator(), std::function<Future<R>(ENUM_PARAMS(N, P))>(), ENUM_PARAMS(N, a)))>; // NOLINT(whitespace/line_length)
 
   REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
 #undef TEMPLATE
 
-  /*implicit*/ _Defer(const std::tr1::_Bind<F>& b)
-    : std::tr1::_Bind<F>(b) {}
+  _Deferred(const UPID& pid, F f) : pid(pid), f(f) {}
+
+  /*implicit*/ _Deferred(F f) : f(f) {}
+
+  Option<UPID> pid;
+  F f;
 };
 
 } // namespace process {
 
 #endif // __PROCESS_DEFERRED_HPP__
-#endif // __cplusplus >= 201103L

http://git-wip-us.apache.org/repos/asf/mesos/blob/5ab8ff81/3rdparty/libprocess/include/process/delay.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/delay.hpp b/3rdparty/libprocess/include/process/delay.hpp
index be2d36b..5818e83 100644
--- a/3rdparty/libprocess/include/process/delay.hpp
+++ b/3rdparty/libprocess/include/process/delay.hpp
@@ -1,6 +1,3 @@
-#if __cplusplus >= 201103L
-#include <process/c++11/delay.hpp>
-#else
 #ifndef __PROCESS_DELAY_HPP__
 #define __PROCESS_DELAY_HPP__
 
@@ -9,8 +6,6 @@
 #include <process/timer.hpp>
 
 #include <stout/duration.hpp>
-#include <stout/lambda.hpp>
-#include <stout/memory.hpp>
 #include <stout/preprocessor.hpp>
 
 namespace process {
@@ -25,23 +20,9 @@ Timer delay(const Duration& duration,
             const PID<T>& pid,
             void (T::*method)())
 {
-  memory::shared_ptr<lambda::function<void(T*)> > thunk(
-      new lambda::function<void(T*)>(
-          lambda::bind(method, lambda::_1)));
-
-  memory::shared_ptr<lambda::function<void(ProcessBase*)> > dispatcher(
-      new lambda::function<void(ProcessBase*)>(
-          lambda::bind(&internal::vdispatcher<T>,
-                       lambda::_1,
-                       thunk)));
-
-  lambda::function<void(void)> dispatch =
-    lambda::bind(internal::dispatch,
-                 pid,
-                 dispatcher,
-                 &typeid(method));
-
-  return Clock::timer(duration, dispatch);
+  return Clock::timer(duration, [=] () {
+    dispatch(pid, method);
+  });
 }
 
 
@@ -72,23 +53,9 @@ Timer delay(const Duration& duration,
               void (T::*method)(ENUM_PARAMS(N, P)),                     \
               ENUM_BINARY_PARAMS(N, A, a))                              \
   {                                                                     \
-    memory::shared_ptr<lambda::function<void(T*)> > thunk(              \
-        new lambda::function<void(T*)>(                                 \
-            lambda::bind(method, lambda::_1, ENUM_PARAMS(N, a))));      \
-                                                                        \
-    memory::shared_ptr<lambda::function<void(ProcessBase*)> > dispatcher( \
-        new lambda::function<void(ProcessBase*)>(                       \
-            lambda::bind(&internal::vdispatcher<T>,                     \
-                         lambda::_1,                                    \
-                         thunk)));                                      \
-                                                                        \
-    lambda::function<void(void)> dispatch =                             \
-      lambda::bind(internal::dispatch,                                  \
-                   pid,                                                 \
-                   dispatcher,                                          \
-                   &typeid(method));                                    \
-                                                                        \
-    return Clock::timer(duration, dispatch);                            \
+    return Clock::timer(duration, [=] () {                              \
+      dispatch(pid, method, ENUM_PARAMS(N, a));                         \
+    });                                                                 \
   }                                                                     \
                                                                         \
   template <typename T,                                                 \
@@ -119,4 +86,3 @@ Timer delay(const Duration& duration,
 } // namespace process {
 
 #endif // __PROCESS_DELAY_HPP__
-#endif // __cplusplus >= 201103L

http://git-wip-us.apache.org/repos/asf/mesos/blob/5ab8ff81/3rdparty/libprocess/include/process/dispatch.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/dispatch.hpp b/3rdparty/libprocess/include/process/dispatch.hpp
index 4ec07d6..99de0e9 100644
--- a/3rdparty/libprocess/include/process/dispatch.hpp
+++ b/3rdparty/libprocess/include/process/dispatch.hpp
@@ -1,15 +1,12 @@
-#if __cplusplus >= 201103L
-#include <process/c++11/dispatch.hpp>
-#else
 #ifndef __PROCESS_DISPATCH_HPP__
 #define __PROCESS_DISPATCH_HPP__
 
+#include <functional>
+#include <memory> // TODO(benh): Replace shared_ptr with unique_ptr.
 #include <string>
 
 #include <process/process.hpp>
 
-#include <stout/lambda.hpp>
-#include <stout/memory.hpp> // TODO(benh): Replace shared_ptr with unique_ptr.
 #include <stout/preprocessor.hpp>
 
 namespace process {
@@ -53,54 +50,9 @@ namespace internal {
 // will probably change in the future to unique_ptr (or a variant).
 void dispatch(
     const UPID& pid,
-    const memory::shared_ptr<lambda::function<void(ProcessBase*)> >& f,
+    const std::shared_ptr<std::function<void(ProcessBase*)>>& f,
     const Option<const std::type_info*>& functionType = None());
 
-// For each return type (void, future, value) there is a dispatcher
-// function which should complete the picture. Given the process
-// argument these routines downcast the process to the correct subtype
-// and invoke the thunk using the subtype as the argument
-// (receiver). Note that we must use dynamic_cast because we permit a
-// process to use multiple inheritance (e.g., to expose multiple
-// callback interfaces).
-
-template <typename T>
-void vdispatcher(
-    ProcessBase* process,
-    memory::shared_ptr<lambda::function<void(T*)> > thunk)
-{
-  assert(process != NULL);
-  T* t = dynamic_cast<T*>(process);
-  CHECK_NOTNULL(t);
-  (*thunk)(t);
-}
-
-
-template <typename R, typename T>
-void pdispatcher(
-    ProcessBase* process,
-    memory::shared_ptr<lambda::function<Future<R>(T*)> > thunk,
-    memory::shared_ptr<Promise<R> > promise)
-{
-  CHECK_NOTNULL(process);
-  T* t = dynamic_cast<T*>(process);
-  CHECK_NOTNULL(t);
-  promise->associate((*thunk)(t));
-}
-
-
-template <typename R, typename T>
-void rdispatcher(
-    ProcessBase* process,
-    memory::shared_ptr<lambda::function<R(T*)> > thunk,
-    memory::shared_ptr<Promise<R> > promise)
-{
-  CHECK_NOTNULL(process);
-  T* t = dynamic_cast<T*>(process);
-  CHECK_NOTNULL(t);
-  promise->set((*thunk)(t));
-}
-
 } // namespace internal {
 
 
@@ -111,50 +63,28 @@ void rdispatcher(
 // would shorten these definitions even more.
 //
 // First, definitions of dispatch for methods returning void:
-//
-// template <typename T, typename ...P>
-// void dispatch(
-//     const PID<T>& pid,
-//     void (T::*method)(P...),
-//     P... p)
-// {
-//   memory::shared_ptr<lambda::function<void(T*)> > thunk(
-//       new lambda::function<void(T*)>(
-//           lambda::bind(method,
-//                        lambda::_1,
-//                        std::forward<P>(p)...)));
-//
-//   memory::shared_ptr<lambda::function<void(ProcessBase*)> > dispatcher(
-//       new lambda::function<void(ProcessBase*)>(
-//           lambda::bind(&internal::vdispatcher<T>,
-//                        lambda::_1,
-//                        thunk)));
-//
-//   internal::dispatch(pid, dispatcher, &typeid(method));
-// }
 
 template <typename T>
 void dispatch(
     const PID<T>& pid,
-    void (T::*method)(void))
+    void (T::*method)())
 {
-  memory::shared_ptr<lambda::function<void(T*)> > thunk(
-      new lambda::function<void(T*)>(
-          lambda::bind(method, lambda::_1)));
-
-  memory::shared_ptr<lambda::function<void(ProcessBase*)> > dispatcher(
-      new lambda::function<void(ProcessBase*)>(
-          lambda::bind(&internal::vdispatcher<T>,
-                       lambda::_1,
-                       thunk)));
-
-  internal::dispatch(pid, dispatcher, &typeid(method));
+  std::shared_ptr<std::function<void(ProcessBase*)>> f(
+      new std::function<void(ProcessBase*)>(
+          [=] (ProcessBase* process) {
+            assert(process != NULL);
+            T* t = dynamic_cast<T*>(process);
+            assert(t != NULL);
+            (t->*method)();
+          }));
+
+  internal::dispatch(pid, f, &typeid(method));
 }
 
 template <typename T>
 void dispatch(
     const Process<T>& process,
-    void (T::*method)(void))
+    void (T::*method)())
 {
   dispatch(process.self(), method);
 }
@@ -162,11 +92,14 @@ void dispatch(
 template <typename T>
 void dispatch(
     const Process<T>* process,
-    void (T::*method)(void))
+    void (T::*method)())
 {
   dispatch(process->self(), method);
 }
 
+// Due to a bug (http://gcc.gnu.org/bugzilla/show_bug.cgi?id=41933)
+// with variadic templates and lambdas, we still need to do
+// preprocessor expansions.
 #define TEMPLATE(Z, N, DATA)                                            \
   template <typename T,                                                 \
             ENUM_PARAMS(N, typename P),                                 \
@@ -176,19 +109,16 @@ void dispatch(
       void (T::*method)(ENUM_PARAMS(N, P)),                             \
       ENUM_BINARY_PARAMS(N, A, a))                                      \
   {                                                                     \
-    memory::shared_ptr<lambda::function<void(T*)> > thunk(              \
-        new lambda::function<void(T*)>(                                 \
-            lambda::bind(method,                                        \
-                         lambda::_1,                                    \
-                         ENUM_PARAMS(N, a))));                          \
+    std::shared_ptr<std::function<void(ProcessBase*)>> f(               \
+        new std::function<void(ProcessBase*)>(                          \
+            [=] (ProcessBase* process) {                                \
+              assert(process != NULL);                                  \
+              T* t = dynamic_cast<T*>(process);                         \
+              assert(t != NULL);                                        \
+              (t->*method)(ENUM_PARAMS(N, a));                          \
+            }));                                                        \
                                                                         \
-    memory::shared_ptr<lambda::function<void(ProcessBase*)> > dispatcher( \
-        new lambda::function<void(ProcessBase*)>(                       \
-            lambda::bind(&internal::vdispatcher<T>,                     \
-                         lambda::_1,                                    \
-                         thunk)));                                      \
-                                                                        \
-    internal::dispatch(pid, dispatcher, &typeid(method));               \
+    internal::dispatch(pid, f, &typeid(method));                        \
   }                                                                     \
                                                                         \
   template <typename T,                                                 \
@@ -218,54 +148,26 @@ void dispatch(
 
 
 // Next, definitions of methods returning a future:
-//
-// template <typename R, typename T, typename ...P>
-// Future<R> dispatch(
-//     const PID<T>& pid,
-//     Future<R> (T::*method)(P...),
-//     P... p)
-// {
-//   memory::shared_ptr<lambda::function<Future<R>(T*)> > thunk(
-//       new lambda::function<Future<R>(T*)>(
-//           lambda::bind(method,
-//                        lambda::_1,
-//                        std::forward<P>(p)...)));
-//
-//   memory::shared_ptr<Promise<R> > promise(new Promise<R>());
-//   Future<R> future = promise->future();
-//
-//   memory::shared_ptr<lambda::function<void(ProcessBase*)> > dispatcher(
-//       new lambda::function<void(ProcessBase*)>(
-//           lambda::bind(&internal::pdispatcher<R, T>,
-//                        lambda::_1,
-//                        thunk, promise)));
-//
-//   internal::dispatch(pid, dispatcher, &typeid(method));
-//
-//   return future;
-// }
 
 template <typename R, typename T>
 Future<R> dispatch(
     const PID<T>& pid,
-    Future<R> (T::*method)(void))
+    Future<R> (T::*method)())
 {
-  memory::shared_ptr<lambda::function<Future<R>(T*)> > thunk(
-      new lambda::function<Future<R>(T*)>(
-          lambda::bind(method, lambda::_1)));
-
-  memory::shared_ptr<Promise<R> > promise(new Promise<R>());
-  Future<R> future = promise->future();
+  std::shared_ptr<Promise<R>> promise(new Promise<R>());
 
-  memory::shared_ptr<lambda::function<void(ProcessBase*)> > dispatcher(
-      new lambda::function<void(ProcessBase*)>(
-          lambda::bind(&internal::pdispatcher<R, T>,
-                       lambda::_1,
-                       thunk, promise)));
+  std::shared_ptr<std::function<void(ProcessBase*)>> f(
+      new std::function<void(ProcessBase*)>(
+          [=] (ProcessBase* process) {
+            assert(process != NULL);
+            T* t = dynamic_cast<T*>(process);
+            assert(t != NULL);
+            promise->associate((t->*method)());
+          }));
 
-  internal::dispatch(pid, dispatcher, &typeid(method));
+  internal::dispatch(pid, f, &typeid(method));
 
-  return future;
+  return promise->future();
 }
 
 template <typename R, typename T>
@@ -294,24 +196,20 @@ Future<R> dispatch(
       Future<R> (T::*method)(ENUM_PARAMS(N, P)),                        \
       ENUM_BINARY_PARAMS(N, A, a))                                      \
   {                                                                     \
-    memory::shared_ptr<lambda::function<Future<R>(T*)> > thunk(         \
-        new lambda::function<Future<R>(T*)>(                            \
-            lambda::bind(method,                                        \
-                         lambda::_1,                                    \
-                         ENUM_PARAMS(N, a))));                          \
+    std::shared_ptr<Promise<R>> promise(new Promise<R>());              \
                                                                         \
-    memory::shared_ptr<Promise<R> > promise(new Promise<R>());          \
-    Future<R> future = promise->future();                               \
+    std::shared_ptr<std::function<void(ProcessBase*)>> f(               \
+        new std::function<void(ProcessBase*)>(                          \
+            [=] (ProcessBase* process) {                                \
+              assert(process != NULL);                                  \
+              T* t = dynamic_cast<T*>(process);                         \
+              assert(t != NULL);                                        \
+              promise->associate((t->*method)(ENUM_PARAMS(N, a)));      \
+            }));                                                        \
                                                                         \
-    memory::shared_ptr<lambda::function<void(ProcessBase*)> > dispatcher( \
-        new lambda::function<void(ProcessBase*)>(                       \
-            lambda::bind(&internal::pdispatcher<R, T>,                  \
-                         lambda::_1,                                    \
-                         thunk, promise)));                             \
+    internal::dispatch(pid, f, &typeid(method));                        \
                                                                         \
-    internal::dispatch(pid, dispatcher, &typeid(method));               \
-                                                                        \
-    return future;                                                      \
+    return promise->future();                                           \
   }                                                                     \
                                                                         \
   template <typename R,                                                 \
@@ -343,60 +241,32 @@ Future<R> dispatch(
 
 
 // Next, definitions of methods returning a value.
-//
-// template <typename R, typename T, typename ...P>
-// Future<R> dispatch(
-//     const PID<T>& pid,
-//     R (T::*method)(P...),
-//     P... p)
-// {
-//   memory::shared_ptr<lambda::function<R(T*)> > thunk(
-//       new lambda::function<R(T*)>(
-//           lambda::bind(method,
-//                        lambda::_1,
-//                        std::forward<P>(p)...)));
-//
-//   memory::shared_ptr<Promise<R> > promise(new Promise<R>());
-//   Future<R> future = promise->future();
-//
-//   memory::shared_ptr<lambda::function<void(ProcessBase*)> > dispatcher(
-//       new lambda::function<void(ProcessBase*)>(
-//           lambda::bind(&internal::rdispatcher<R, T>,
-//                        lambda::_1,
-//                        thunk, promise)));
-//
-//   internal::dispatch(pid, dispatcher, &typeid(method));
-//
-//   return future;
-// }
 
 template <typename R, typename T>
 Future<R> dispatch(
     const PID<T>& pid,
     R (T::*method)(void))
 {
-  memory::shared_ptr<lambda::function<R(T*)> > thunk(
-      new lambda::function<R(T*)>(
-          lambda::bind(method, lambda::_1)));
-
-  memory::shared_ptr<Promise<R> > promise(new Promise<R>());
-  Future<R> future = promise->future();
+  std::shared_ptr<Promise<R>> promise(new Promise<R>());
 
-  memory::shared_ptr<lambda::function<void(ProcessBase*)> > dispatcher(
-      new lambda::function<void(ProcessBase*)>(
-          lambda::bind(&internal::rdispatcher<R, T>,
-                       lambda::_1,
-                       thunk, promise)));
+  std::shared_ptr<std::function<void(ProcessBase*)>> f(
+      new std::function<void(ProcessBase*)>(
+          [=] (ProcessBase* process) {
+            assert(process != NULL);
+            T* t = dynamic_cast<T*>(process);
+            assert(t != NULL);
+            promise->set((t->*method)());
+          }));
 
-  internal::dispatch(pid, dispatcher, &typeid(method));
+  internal::dispatch(pid, f, &typeid(method));
 
-  return future;
+  return promise->future();
 }
 
 template <typename R, typename T>
 Future<R> dispatch(
     const Process<T>& process,
-    R (T::*method)(void))
+    R (T::*method)())
 {
   return dispatch(process.self(), method);
 }
@@ -404,7 +274,7 @@ Future<R> dispatch(
 template <typename R, typename T>
 Future<R> dispatch(
     const Process<T>* process,
-    R (T::*method)(void))
+    R (T::*method)())
 {
   return dispatch(process->self(), method);
 }
@@ -419,24 +289,20 @@ Future<R> dispatch(
       R (T::*method)(ENUM_PARAMS(N, P)),                                \
       ENUM_BINARY_PARAMS(N, A, a))                                      \
   {                                                                     \
-    memory::shared_ptr<lambda::function<R(T*)> > thunk(                 \
-        new lambda::function<R(T*)>(                                    \
-            lambda::bind(method,                                        \
-                         lambda::_1,                                    \
-                         ENUM_PARAMS(N, a))));                          \
+    std::shared_ptr<Promise<R>> promise(new Promise<R>());              \
                                                                         \
-    memory::shared_ptr<Promise<R> > promise(new Promise<R>());          \
-    Future<R> future = promise->future();                               \
+    std::shared_ptr<std::function<void(ProcessBase*)>> f(               \
+        new std::function<void(ProcessBase*)>(                          \
+            [=] (ProcessBase* process) {                                \
+              assert(process != NULL);                                  \
+              T* t = dynamic_cast<T*>(process);                         \
+              assert(t != NULL);                                        \
+              promise->set((t->*method)(ENUM_PARAMS(N, a)));            \
+            }));                                                        \
                                                                         \
-    memory::shared_ptr<lambda::function<void(ProcessBase*)> > dispatcher( \
-        new lambda::function<void(ProcessBase*)>(                       \
-            lambda::bind(&internal::rdispatcher<R, T>,                  \
-                         lambda::_1,                                    \
-                         thunk, promise)));                             \
+    internal::dispatch(pid, f, &typeid(method));                        \
                                                                         \
-    internal::dispatch(pid, dispatcher, &typeid(method));               \
-                                                                        \
-    return future;                                                      \
+    return promise->future();                                           \
   }                                                                     \
                                                                         \
   template <typename R,                                                 \
@@ -466,7 +332,58 @@ Future<R> dispatch(
   REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
 #undef TEMPLATE
 
+
+inline void dispatch(
+    const UPID& pid,
+    const std::function<void()>& f)
+{
+  std::shared_ptr<std::function<void(ProcessBase*)>> f_(
+      new std::function<void(ProcessBase*)>(
+          [=] (ProcessBase*) {
+            f();
+          }));
+
+  internal::dispatch(pid, f_);
+}
+
+
+template <typename R>
+Future<R> dispatch(
+    const UPID& pid,
+    const std::function<Future<R>()>& f)
+{
+  std::shared_ptr<Promise<R>> promise(new Promise<R>());
+
+  std::shared_ptr<std::function<void(ProcessBase*)>> f_(
+      new std::function<void(ProcessBase*)>(
+          [=] (ProcessBase*) {
+            promise->associate(f());
+          }));
+
+  internal::dispatch(pid, f_);
+
+  return promise->future();
+}
+
+
+template <typename R>
+Future<R> dispatch(
+    const UPID& pid,
+    const std::function<R()>& f)
+{
+  std::shared_ptr<Promise<R>> promise(new Promise<R>());
+
+  std::shared_ptr<std::function<void(ProcessBase*)>> f_(
+      new std::function<void(ProcessBase*)>(
+          [=] (ProcessBase*) {
+            promise->set(f());
+          }));
+
+  internal::dispatch(pid, f_);
+
+  return promise->future();
+}
+
 } // namespace process {
 
 #endif // __PROCESS_DISPATCH_HPP__
-#endif // __cplusplus >= 201103L

http://git-wip-us.apache.org/repos/asf/mesos/blob/5ab8ff81/3rdparty/libprocess/include/process/executor.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/executor.hpp b/3rdparty/libprocess/include/process/executor.hpp
index e28790e..157a1d2 100644
--- a/3rdparty/libprocess/include/process/executor.hpp
+++ b/3rdparty/libprocess/include/process/executor.hpp
@@ -1,61 +1,23 @@
-#if __cplusplus >= 201103L
-#include <process/c++11/executor.hpp>
-#else
 #ifndef __PROCESS_EXECUTOR_HPP__
 #define __PROCESS_EXECUTOR_HPP__
 
+#include <process/defer.hpp>
 #include <process/deferred.hpp>
-#include <process/dispatch.hpp>
 #include <process/id.hpp>
+#include <process/process.hpp>
 
-#include <stout/preprocessor.hpp>
 #include <stout/thread.hpp>
 
 namespace process {
 
-namespace internal {
-
-// Underlying "process" which handles invoking actual callbacks
-// created through an Executor.
-class ExecutorProcess : public Process<ExecutorProcess>
-{
-private:
-  friend class process::Executor;
-
-  ExecutorProcess() : ProcessBase(ID::generate("__executor__")) {}
-  virtual ~ExecutorProcess() {}
-
-  // Not copyable, not assignable.
-  ExecutorProcess(const ExecutorProcess&);
-  ExecutorProcess& operator = (const ExecutorProcess&);
-
-  // No arg invoke.
-  void invoke(const std::tr1::function<void(void)>& f) { f(); }
-
-  // Args invoke.
-#define TEMPLATE(Z, N, DATA)                                            \
-  template <ENUM_PARAMS(N, typename A)>                                 \
-  void CAT(invoke, N)(                                         \
-      const std::tr1::function<void(ENUM_PARAMS(N, A))>& f,    \
-      ENUM_BINARY_PARAMS(N, A, a))                             \
-  {                                                            \
-    f(ENUM_PARAMS(N, a));                                      \
-  }
-
-  REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
-#undef TEMPLATE
-};
-
-} // namespace internal {
-
-
 // Provides an abstraction that can take a standard function object
-// and convert it to a 'Deferred'. Each converted function object will
-// get invoked serially with respect to one another.
+// and defer it without needing a process. Each converted function
+// object will get execute serially with respect to one another when
+// invoked.
 class Executor
 {
 public:
-  Executor()
+  Executor() : process(ID::generate("__executor__"))
   {
     spawn(process);
   }
@@ -68,7 +30,7 @@ public:
 
   void stop()
   {
-    terminate(process);
+    terminate(&process);
 
     // TODO(benh): Note that this doesn't wait because that could
     // cause a deadlock ... thus, the semantics here are that no more
@@ -76,180 +38,19 @@ public:
     // be occuring concurrently.
   }
 
-  // We can't easily use 'std::tr1::_Placeholder<X>' when doing macro
-  // expansion via ENUM_BINARY_PARAMS because compilers don't like it
-  // when you try and concatenate '<' 'N' '>'. Thus, we typedef them.
-private:
-#define TEMPLATE(Z, N, DATA)                            \
-  typedef std::tr1::_Placeholder<INC(N)> _ ## N;
-
-  REPEAT(10, TEMPLATE, _)
-#undef TEMPLATE
-
-public:
-  // We provide wrappers for all standard function objects.
-  Deferred<void(void)> defer(
-      const std::tr1::function<void(void)>& f)
-  {
-    return Deferred<void(void)>(
-        std::tr1::bind(
-            &Executor::dispatcher,
-            process.self(), f));
-  }
-
-#define TEMPLATE(Z, N, DATA)                                            \
-  template <ENUM_PARAMS(N, typename A)>                                 \
-  Deferred<void(ENUM_PARAMS(N, A))> defer(                              \
-      const std::tr1::function<void(ENUM_PARAMS(N, A))>& f)             \
-  {                                                                     \
-    return Deferred<void(ENUM_PARAMS(N, A))>(                           \
-        std::tr1::bind(                                                 \
-            &Executor::CAT(dispatcher, N)<ENUM_PARAMS(N, A)>,           \
-            process.self(), f,                                          \
-            ENUM_BINARY_PARAMS(N, _, () INTERCEPT)));                   \
-  }
-
-  REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
-#undef TEMPLATE
-
-  // Unfortunately, it is currently difficult to "forward" type
-  // information from one result to another, so we must explicilty
-  // define wrappers for all std::tr1::bind results. First we start
-  // with the non-member std::tr1::bind results.
-  Deferred<void(void)> defer(
-      const std::tr1::_Bind<void(*(void))(void)>& b)
+  template <typename F>
+  _Deferred<F> defer(F&& f)
   {
-    return defer(std::tr1::function<void(void)>(b));
-  }
-
-#define TEMPLATE(Z, N, DATA)                                            \
-  template <ENUM_PARAMS(N, typename A)>                                 \
-  Deferred<void(ENUM_PARAMS(N, A))> defer(                              \
-      const std::tr1::_Bind<                                            \
-      void(*(ENUM_PARAMS(N, _)))                                        \
-      (ENUM_PARAMS(N, A))>& b)                                          \
-  {                                                                     \
-    return defer(std::tr1::function<void(ENUM_PARAMS(N, A))>(b));       \
+    return _Deferred<F>(process.self(), std::forward<F>(f));
   }
 
-  REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
-#undef TEMPLATE
-
-  // Now the member std::tr1::bind results:
-  // 1. Non-const member (function), non-const pointer (receiver).
-  // 2. Const member, non-const pointer.
-  // 3. Const member, const pointer.
-  // 4. Non-const member, non-const reference.
-  // 5. Const member, non-const reference.
-  // 6. Const member, const reference.
-  // 7. Non-const member, value.
-  // 8. Const member, value.
-#define TEMPLATE(Z, N, DATA)                                            \
-  template <typename T ENUM_TRAILING_PARAMS(N, typename A)>             \
-  Deferred<void(ENUM_PARAMS(N, A))> defer(                              \
-      const std::tr1::_Bind<std::tr1::_Mem_fn<                          \
-      void(T::*)(ENUM_PARAMS(N, A))>                                    \
-      (T* ENUM_TRAILING_PARAMS(N, _))>& b)                              \
-  {                                                                     \
-    return defer(std::tr1::function<void(ENUM_PARAMS(N, A))>(b));       \
-  }                                                                     \
-                                                                        \
-  template <typename T ENUM_TRAILING_PARAMS(N, typename A)>             \
-  Deferred<void(ENUM_PARAMS(N, A))> defer(                              \
-      const std::tr1::_Bind<std::tr1::_Mem_fn<                          \
-      void(T::*)(ENUM_PARAMS(N, A)) const>                              \
-      (T* ENUM_TRAILING_PARAMS(N, _))>& b)                              \
-  {                                                                     \
-    return defer(std::tr1::function<void(ENUM_PARAMS(N, A))>(b));       \
-  }                                                                     \
-                                                                        \
-  template <typename T ENUM_TRAILING_PARAMS(N, typename A)>             \
-  Deferred<void(ENUM_PARAMS(N, A))> defer(                              \
-      const std::tr1::_Bind<std::tr1::_Mem_fn<                          \
-      void(T::*)(ENUM_PARAMS(N, A)) const>                              \
-      (const T* ENUM_TRAILING_PARAMS(N, _))>& b)                        \
-  {                                                                     \
-    return defer(std::tr1::function<void(ENUM_PARAMS(N, A))>(b));       \
-  }                                                                     \
-                                                                        \
-  template <typename T ENUM_TRAILING_PARAMS(N, typename A)>             \
-  Deferred<void(ENUM_PARAMS(N, A))> defer(                              \
-      const std::tr1::_Bind<std::tr1::_Mem_fn<                          \
-      void(T::*)(ENUM_PARAMS(N, A))>                                    \
-      (std::tr1::reference_wrapper<T> ENUM_TRAILING_PARAMS(N, _))>& b)  \
-  {                                                                     \
-    return defer(std::tr1::function<void(ENUM_PARAMS(N, A))>(b));       \
-  }                                                                     \
-                                                                        \
-  template <typename T ENUM_TRAILING_PARAMS(N, typename A)>             \
-  Deferred<void(ENUM_PARAMS(N, A))> defer(                              \
-      const std::tr1::_Bind<std::tr1::_Mem_fn<                          \
-      void(T::*)(ENUM_PARAMS(N, A)) const>                              \
-      (std::tr1::reference_wrapper<T> ENUM_TRAILING_PARAMS(N, _))>& b)  \
-  {                                                                     \
-    return defer(std::tr1::function<void(ENUM_PARAMS(N, A))>(b));       \
-  }                                                                     \
-                                                                        \
-  template <typename T ENUM_TRAILING_PARAMS(N, typename A)>             \
-  Deferred<void(ENUM_PARAMS(N, A))> defer(                              \
-      const std::tr1::_Bind<std::tr1::_Mem_fn<                          \
-      void(T::*)(ENUM_PARAMS(N, A)) const>                              \
-      (std::tr1::reference_wrapper<const T> ENUM_TRAILING_PARAMS(N, _))>& b) \
-  {                                                                     \
-    return defer(std::tr1::function<void(ENUM_PARAMS(N, A))>(b));       \
-  }                                                                     \
-                                                                        \
-  template <typename T ENUM_TRAILING_PARAMS(N, typename A)>             \
-  Deferred<void(ENUM_PARAMS(N, A))> defer(                              \
-      const std::tr1::_Bind<std::tr1::_Mem_fn<                          \
-      void(T::*)(ENUM_PARAMS(N, A))>                                    \
-      (T ENUM_TRAILING_PARAMS(N, _))>& b)                               \
-  {                                                                     \
-    return defer(std::tr1::function<void(ENUM_PARAMS(N, A))>(b));       \
-  }                                                                     \
-                                                                        \
-  template <typename T ENUM_TRAILING_PARAMS(N, typename A)>             \
-  Deferred<void(ENUM_PARAMS(N, A))> defer(                              \
-      const std::tr1::_Bind<std::tr1::_Mem_fn<                          \
-      void(T::*)(ENUM_PARAMS(N, A)) const>                              \
-      (T ENUM_TRAILING_PARAMS(N, _))>& b)                               \
-  {                                                                     \
-    return defer(std::tr1::function<void(ENUM_PARAMS(N, A))>(b));       \
-  }
-
-  REPEAT(11, TEMPLATE, _) // No args and args A0 -> A9.
-#undef TEMPLATE
 
 private:
   // Not copyable, not assignable.
   Executor(const Executor&);
   Executor& operator = (const Executor&);
 
-  static void dispatcher(
-      const PID<internal::ExecutorProcess>& pid,
-      const std::tr1::function<void(void)>& f)
-  {
-    // TODO(benh): Why not just use internal::dispatch?
-    dispatch(pid, &internal::ExecutorProcess::invoke, f);
-  }
-
-#define TEMPLATE(Z, N, DATA)                                            \
-  template <ENUM_PARAMS(N, typename A)>                                 \
-  static void CAT(dispatcher, N)(                                       \
-      const PID<internal::ExecutorProcess>& pid,                        \
-      const std::tr1::function<void(ENUM_PARAMS(N, A))>& f,             \
-      ENUM_BINARY_PARAMS(N, A, a))                                      \
-  {                                                                     \
-    dispatch(                                                           \
-        pid,                                                            \
-        &internal::ExecutorProcess::CAT(invoke, N)<ENUM_PARAMS(N, A)>,  \
-        f, ENUM_PARAMS(N, a));                                          \
-  }
-
-  REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
-#undef TEMPLATE
-
-  internal::ExecutorProcess process;
+  ProcessBase process;
 };
 
 
@@ -265,4 +66,3 @@ extern ThreadLocal<Executor>* _executor_;
 } // namespace process {
 
 #endif // __PROCESS_EXECUTOR_HPP__
-#endif // __cplusplus >= 201103L

http://git-wip-us.apache.org/repos/asf/mesos/blob/5ab8ff81/3rdparty/libprocess/include/process/future.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/future.hpp b/3rdparty/libprocess/include/process/future.hpp
index 530e153..9d301d7 100644
--- a/3rdparty/libprocess/include/process/future.hpp
+++ b/3rdparty/libprocess/include/process/future.hpp
@@ -7,17 +7,11 @@
 #include <iostream>
 #include <list>
 #include <set>
-#if  __cplusplus >= 201103L
 #include <type_traits>
-#endif // __cplusplus >= 201103L
 #include <vector>
 
 #include <glog/logging.h>
 
-#if  __cplusplus < 201103L
-#include <boost/type_traits.hpp>
-#endif // __cplusplus < 201103L
-
 #include <process/clock.hpp>
 #include <process/internal.hpp>
 #include <process/latch.hpp>
@@ -41,13 +35,8 @@ namespace process {
 template <typename _F>
 struct _Defer;
 
-template <typename F>
-struct Deferred;
-
-#if  __cplusplus >= 201103L
 template <typename G>
 struct _Deferred;
-#endif // __cplusplus >= 201103L
 
 template <typename T>
 class Future;
@@ -152,7 +141,6 @@ public:
   typedef lambda::function<void(void)> DiscardedCallback;
   typedef lambda::function<void(const Future<T>&)> AnyCallback;
 
-#if __cplusplus >= 201103L
   // Installs callbacks for the specified events and returns a const
   // reference to 'this' in order to easily support chaining.
   const Future<T>& onDiscard(DiscardCallback&& callback) const;
@@ -297,17 +285,6 @@ public:
     return onAny(std::forward<F>(f), Prefer());
   }
 
-#else // __cplusplus >= 201103L
-
-  // Installs callbacks for the specified events and returns a const
-  // reference to 'this' in order to easily support chaining.
-  const Future<T>& onDiscard(const DiscardCallback& callback) const;
-  const Future<T>& onReady(const ReadyCallback& callback) const;
-  const Future<T>& onFailed(const FailedCallback& callback) const;
-  const Future<T>& onDiscarded(const DiscardedCallback& callback) const;
-  const Future<T>& onAny(const AnyCallback& callback) const;
-#endif // __cplusplus >= 201103L
-
   // Installs callbacks that get executed when this future is ready
   // and associates the result of the callback with the future that is
   // returned to the caller (which may be of a different type).
@@ -329,13 +306,6 @@ public:
     return then(lambda::function<X(const T&)>(lambda::bind(f)));
   }
 
-  template <typename X>
-  Future<X> then(const Deferred<Future<X>(T)>& f) const
-  {
-    return then(lambda::function<Future<X>(const T&)>(f));
-  }
-
-#if __cplusplus >= 201103L
 private:
   template <typename F, typename X = typename internal::unwrap<typename std::result_of<F(const T&)>::type>::type> // NOLINT(whitespace/line_length)
   Future<X> then(_Deferred<F>&& f, Prefer) const
@@ -371,103 +341,6 @@ public:
     return then(std::forward<F>(f), Prefer());
   }
 
-#else // __cplusplus >= 201103L
-
-  // Helpers for the compiler to be able to forward std::tr1::bind results.
-  template <typename X>
-  Future<X> then(const std::tr1::_Bind<X(*(void))(void)>& b) const
-  {
-    return then(std::tr1::function<X(const T&)>(b));
-  }
-
-#define TEMPLATE(Z, N, DATA)                                            \
-  template <typename X,                                                 \
-            ENUM_PARAMS(N, typename P),                                 \
-            ENUM_PARAMS(N, typename A)>                                 \
-  Future<X> then(                                                       \
-      const std::tr1::_Bind<X(*(ENUM_PARAMS(N, A)))                     \
-      (ENUM_PARAMS(N, P))>& b) const                                    \
-  {                                                                     \
-    return then(std::tr1::function<X(const T&)>(b));                    \
-  }
-
-  REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
-#undef TEMPLATE
-
-  template <typename X>
-  Future<X> then(const std::tr1::_Bind<Future<X>(*(void))(void)>& b) const
-  {
-    return then(std::tr1::function<Future<X>(const T&)>(b));
-  }
-
-#define TEMPLATE(Z, N, DATA)                                            \
-  template <typename X,                                                 \
-            ENUM_PARAMS(N, typename P),                                 \
-            ENUM_PARAMS(N, typename A)>                                 \
-  Future<X> then(                                                       \
-      const std::tr1::_Bind<Future<X>(*(ENUM_PARAMS(N, A)))             \
-      (ENUM_PARAMS(N, P))>& b) const                                    \
-  {                                                                     \
-    return then(std::tr1::function<Future<X>(const T&)>(b));            \
-  }
-
-  REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
-#undef TEMPLATE
-
-  // Helpers for the compiler to be able to forward 'defer' results.
-  template <typename X, typename U>
-  Future<X> then(const _Defer<Future<X>(*(PID<U>, X(U::*)(void)))
-                 (const PID<U>&, X(U::*)(void))>& d) const
-  {
-    return then(std::tr1::function<Future<X>(const T&)>(d));
-  }
-
-#define TEMPLATE(Z, N, DATA)                                            \
-  template <typename X,                                                 \
-            typename U,                                                 \
-            ENUM_PARAMS(N, typename P),                                 \
-            ENUM_PARAMS(N, typename A)>                                 \
-  Future<X> then(                                                       \
-      const _Defer<Future<X>(*(PID<U>,                                  \
-                               X(U::*)(ENUM_PARAMS(N, P)),              \
-                               ENUM_PARAMS(N, A)))                      \
-      (const PID<U>&,                                                   \
-       X(U::*)(ENUM_PARAMS(N, P)),                                      \
-       ENUM_PARAMS(N, P))>& d) const                                    \
-  {                                                                     \
-    return then(std::tr1::function<Future<X>(const T&)>(d));            \
-  }
-
-  REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
-#undef TEMPLATE
-
-  template <typename X, typename U>
-  Future<X> then(const _Defer<Future<X>(*(PID<U>, Future<X>(U::*)(void)))
-                 (const PID<U>&, Future<X>(U::*)(void))>& d) const
-  {
-    return then(std::tr1::function<Future<X>(const T&)>(d));
-  }
-
-#define TEMPLATE(Z, N, DATA)                                            \
-  template <typename X,                                                 \
-            typename U,                                                 \
-            ENUM_PARAMS(N, typename P),                                 \
-            ENUM_PARAMS(N, typename A)>                                 \
-  Future<X> then(                                                       \
-      const _Defer<Future<X>(*(PID<U>,                                  \
-                               Future<X>(U::*)(ENUM_PARAMS(N, P)),      \
-                               ENUM_PARAMS(N, A)))                      \
-      (const PID<U>&,                                                   \
-       Future<X>(U::*)(ENUM_PARAMS(N, P)),                              \
-       ENUM_PARAMS(N, P))>& d) const                                    \
-  {                                                                     \
-    return then(std::tr1::function<Future<X>(const T&)>(d));            \
-  }
-
-  REPEAT_FROM_TO(1, 11, TEMPLATE, _) // Args A0 -> A9.
-#undef TEMPLATE
-#endif // __cplusplus >= 201103L
-
   // Installs callbacks that get executed if this future completes
   // because it failed.
   Future<T> repair(
@@ -848,7 +721,6 @@ Future<Future<T> > select(const std::set<Future<T> >& futures)
   promise->future().onDiscard(
       lambda::bind(&internal::discarded<Future<T> >, promise->future()));
 
-#if __cplusplus >= 201103L
   typename std::set<Future<T>>::iterator iterator;
   for (iterator = futures.begin(); iterator != futures.end(); ++iterator) {
     // NOTE: We can't use std::bind with a std::function with Clang
@@ -858,15 +730,6 @@ Future<Future<T> > select(const std::set<Future<T> >& futures)
       internal::select(future, promise);
     });
   }
-#else // __cplusplus >= 201103L
-  lambda::function<void(const Future<T>&)> select =
-    lambda::bind(&internal::select<T>, lambda::_1, promise);
-
-  typename std::set<Future<T> >::iterator iterator;
-  for (iterator = futures.begin(); iterator != futures.end(); ++iterator) {
-    (*iterator).onAny(lambda::bind(select, lambda::_1));
-  }
-#endif // __cplusplus >= 201103L
 
   return promise->future();
 }
@@ -1191,7 +1054,6 @@ const std::string& Future<T>::failure() const
 }
 
 
-#if __cplusplus >= 201103L
 template <typename T>
 const Future<T>& Future<T>::onDiscard(DiscardCallback&& callback) const
 {
@@ -1311,128 +1173,6 @@ const Future<T>& Future<T>::onAny(AnyCallback&& callback) const
   return *this;
 }
 
-#else // __cplusplus >= 201103L
-template <typename T>
-const Future<T>& Future<T>::onDiscard(const DiscardCallback& callback) const
-{
-  bool run = false;
-
-  internal::acquire(&data->lock);
-  {
-    if (data->discard) {
-      run = true;
-    } else if (data->state == PENDING) {
-      data->onDiscardCallbacks.push_back(callback);
-    }
-  }
-  internal::release(&data->lock);
-
-  // TODO(*): Invoke callback in another execution context.
-  if (run) {
-    callback();
-  }
-
-  return *this;
-}
-
-
-template <typename T>
-const Future<T>& Future<T>::onReady(const ReadyCallback& callback) const
-{
-  bool run = false;
-
-  internal::acquire(&data->lock);
-  {
-    if (data->state == READY) {
-      run = true;
-    } else if (data->state == PENDING) {
-      data->onReadyCallbacks.push_back(callback);
-    }
-  }
-  internal::release(&data->lock);
-
-  // TODO(*): Invoke callback in another execution context.
-  if (run) {
-    callback(*data->t);
-  }
-
-  return *this;
-}
-
-
-template <typename T>
-const Future<T>& Future<T>::onFailed(const FailedCallback& callback) const
-{
-  bool run = false;
-
-  internal::acquire(&data->lock);
-  {
-    if (data->state == FAILED) {
-      run = true;
-    } else if (data->state == PENDING) {
-      data->onFailedCallbacks.push_back(callback);
-    }
-  }
-  internal::release(&data->lock);
-
-  // TODO(*): Invoke callback in another execution context.
-  if (run) {
-    callback(*data->message);
-  }
-
-  return *this;
-}
-
-
-template <typename T>
-const Future<T>& Future<T>::onDiscarded(
-    const DiscardedCallback& callback) const
-{
-  bool run = false;
-
-  internal::acquire(&data->lock);
-  {
-    if (data->state == DISCARDED) {
-      run = true;
-    } else if (data->state == PENDING) {
-      data->onDiscardedCallbacks.push_back(callback);
-    }
-  }
-  internal::release(&data->lock);
-
-  // TODO(*): Invoke callback in another execution context.
-  if (run) {
-    callback();
-  }
-
-  return *this;
-}
-
-
-template <typename T>
-const Future<T>& Future<T>::onAny(const AnyCallback& callback) const
-{
-  bool run = false;
-
-  internal::acquire(&data->lock);
-  {
-    if (data->state == PENDING) {
-      data->onAnyCallbacks.push_back(callback);
-    } else {
-      run = true;
-    }
-  }
-  internal::release(&data->lock);
-
-  // TODO(*): Invoke callback in another execution context.
-  if (run) {
-    callback(*this);
-  }
-
-  return *this;
-}
-#endif // __cplusplus >= 201103L
-
 
 namespace internal {
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/5ab8ff81/3rdparty/libprocess/include/process/timeseries.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/timeseries.hpp b/3rdparty/libprocess/include/process/timeseries.hpp
index dbb2c7a..ec0ac67 100644
--- a/3rdparty/libprocess/include/process/timeseries.hpp
+++ b/3rdparty/libprocess/include/process/timeseries.hpp
@@ -215,18 +215,8 @@ private:
         index = 1;
       }
 
-#if __cplusplus >= 201103L
       next = values.erase(next);
       next++; // Skip one element.
-#else
-      // Store the next deletion candidate.
-      typename std::map<Time, T>::iterator copy = next;
-      copy++; // Skip every other element.
-      copy++; // Next deletion candidate.
-
-      values.erase(next);
-      next = copy;
-#endif
       index = index.get() + 1;
     }
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/5ab8ff81/3rdparty/libprocess/src/io.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/io.cpp b/3rdparty/libprocess/src/io.cpp
index 3afa11a..1ec85e3 100644
--- a/3rdparty/libprocess/src/io.cpp
+++ b/3rdparty/libprocess/src/io.cpp
@@ -232,7 +232,6 @@ Future<size_t> write(int fd, void* data, size_t size)
 
 namespace internal {
 
-#if __cplusplus >= 201103L
 Future<string> _read(
     int fd,
     const memory::shared_ptr<string>& buffer,
@@ -248,45 +247,8 @@ Future<string> _read(
       return _read(fd, buffer, data, length);
     });
 }
-#else
-// Forward declataion.
-Future<string> _read(
-    int fd,
-    const memory::shared_ptr<string>& buffer,
-    const boost::shared_array<char>& data,
-    size_t length);
-
-
-Future<string> __read(
-    size_t size,
-    int fd,
-    const memory::shared_ptr<string>& buffer,
-    const boost::shared_array<char>& data,
-    size_t length)
-{
-  if (size == 0) { // EOF.
-    return string(*buffer);
-  }
-
-  buffer->append(data.get(), size);
-
-  return _read(fd, buffer, data, length);
-}
-
-
-Future<string> _read(
-    int fd,
-    const memory::shared_ptr<string>& buffer,
-    const boost::shared_array<char>& data,
-    size_t length)
-{
-  return io::read(fd, data.get(), length)
-    .then(lambda::bind(&__read, lambda::_1, fd, buffer, data, length));
-}
-#endif // __cplusplus >= 201103L
 
 
-#if __cplusplus >= 201103L
 Future<Nothing> _write(
     int fd,
     Owned<string> data,
@@ -300,39 +262,8 @@ Future<Nothing> _write(
       return _write(fd, data, index + length);
     });
 }
-#else
-// Forward declaration.
-Future<Nothing> _write(
-    int fd,
-    Owned<string> data,
-    size_t index);
-
-
-Future<Nothing> __write(
-    int fd,
-    Owned<string> data,
-    size_t index,
-    size_t length)
-{
-  if (index + length == data->size()) {
-    return Nothing();
-  }
-  return _write(fd, data, index + length);
-}
-
-
-Future<Nothing> _write(
-    int fd,
-    Owned<string> data,
-    size_t index)
-{
-  return io::write(fd, (void*) (data->data() + index), data->size() - index)
-    .then(lambda::bind(&__write, fd, data, index, lambda::_1));
-}
-#endif // __cplusplus >= 201103L
 
 
-#if __cplusplus >= 201103L
 void _splice(
     int from,
     int to,
@@ -378,93 +309,6 @@ void _splice(
     .onFailed([=] (const string& message) { promise->fail(message); })
     .onDiscarded([=] () { promise->discard(); });
 }
-#else
-// Forward declarations.
-void __splice(
-    int from,
-    int to,
-    size_t chunk,
-    boost::shared_array<char> data,
-    memory::shared_ptr<Promise<Nothing> > promise,
-    size_t size);
-
-void ___splice(
-    memory::shared_ptr<Promise<Nothing> > promise,
-    const string& message);
-
-void ____splice(
-    memory::shared_ptr<Promise<Nothing> > promise);
-
-
-void _splice(
-    int from,
-    int to,
-    size_t chunk,
-    boost::shared_array<char> data,
-    memory::shared_ptr<Promise<Nothing> > promise)
-{
-  // Stop splicing if a discard occured on our future.
-  if (promise->future().hasDiscard()) {
-    // TODO(benh): Consider returning the number of bytes already
-    // spliced on discarded, or a failure. Same for the 'onDiscarded'
-    // callbacks below.
-    promise->discard();
-    return;
-  }
-
-  Future<size_t> read = io::read(from, data.get(), chunk);
-
-  // Stop reading (or potentially indefinitely polling) if a discard
-  // occurs on our future.
-  promise->future().onDiscard(
-      lambda::bind(&process::internal::discard<size_t>,
-                   WeakFuture<size_t>(read)));
-
-  read
-    .onReady(
-        lambda::bind(&__splice, from, to, chunk, data, promise, lambda::_1))
-    .onFailed(lambda::bind(&___splice, promise, lambda::_1))
-    .onDiscarded(lambda::bind(&____splice, promise));
-}
-
-
-void __splice(
-    int from,
-    int to,
-    size_t chunk,
-    boost::shared_array<char> data,
-    memory::shared_ptr<Promise<Nothing> > promise,
-    size_t size)
-{
-  if (size == 0) { // EOF.
-    promise->set(Nothing());
-  } else {
-    // Note that we always try and complete the write, even if a
-    // discard has occured on our future, in order to provide
-    // semantics where everything read is written. The promise will
-    // eventually be discarded in the next read.
-    io::write(to, string(data.get(), size))
-      .onReady(lambda::bind(&_splice, from, to, chunk, data, promise))
-      .onFailed(lambda::bind(&___splice, promise, lambda::_1))
-      .onDiscarded(lambda::bind(&____splice, promise));
-  }
-}
-
-
-void ___splice(
-    memory::shared_ptr<Promise<Nothing> > promise,
-    const string& message)
-{
-  promise->fail(message);
-}
-
-
-void ____splice(
-    memory::shared_ptr<Promise<Nothing> > promise)
-{
-  promise->discard();
-}
-#endif // __cplusplus >= 201103L
 
 
 Future<Nothing> splice(int from, int to, size_t chunk)

http://git-wip-us.apache.org/repos/asf/mesos/blob/5ab8ff81/3rdparty/libprocess/src/tests/process_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/tests/process_tests.cpp b/3rdparty/libprocess/src/tests/process_tests.cpp
index eb38edc..7c2cb4b 100644
--- a/3rdparty/libprocess/src/tests/process_tests.cpp
+++ b/3rdparty/libprocess/src/tests/process_tests.cpp
@@ -711,26 +711,12 @@ TEST(Process, defer3)
   volatile bool bool2 = false;
 
   Deferred<void(bool)> set1 =
-#if __cplusplus >= 201103L
     defer([&bool1] (bool b) { bool1 = b; });
-#else // __cplusplus >= 201103L
-    defer(std::tr1::function<void(bool)>(
-              std::tr1::bind(&set<volatile bool>,
-                             &bool1,
-                             std::tr1::placeholders::_1)));
-#endif // __cplusplus >= 201103L
 
   set1(true);
 
   Deferred<void(bool)> set2 =
-#if __cplusplus >= 201103L
     defer([&bool2] (bool b) { bool2 = b; });
-#else // __cplusplus >= 201103L
-    defer(std::tr1::function<void(bool)>(
-              std::tr1::bind(&set<volatile bool>,
-                             &bool2,
-                             std::tr1::placeholders::_1)));
-#endif // __cplusplus >= 201103L
 
   set2(true);
 
@@ -1686,7 +1672,6 @@ TEST(Process, provide)
 }
 
 
-#if __cplusplus >= 201103L
 int baz(string s) { return 42; }
 
 Future<int> bam(string s) { return 42; }
@@ -1829,7 +1814,6 @@ TEST(Process, defers)
   Future<int> future13 = Future<string>().then(
       defer(functor));
 }
-#endif // __cplusplus >= 201103L
 
 
 TEST(Future, FromTry)