You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by tn...@apache.org on 2014/11/20 23:10:40 UTC

[1/2] mesos git commit: Match future dispatch messages with type info.

Repository: mesos
Updated Branches:
  refs/heads/master 67ade096a -> 2eaa7d361


Match future dispatch messages with type info.

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


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

Branch: refs/heads/master
Commit: b977eae0096bc9b7da16ef68fffd8215d1cc477f
Parents: 67ade09
Author: Timothy Chen <tn...@apache.org>
Authored: Fri Nov 14 17:53:54 2014 -0800
Committer: Timothy Chen <tn...@apache.org>
Committed: Thu Nov 20 14:02:08 2014 -0800

----------------------------------------------------------------------
 .../include/process/c++11/dispatch.hpp          | 23 +++++-----------
 .../libprocess/include/process/dispatch.hpp     | 29 +++++++-------------
 3rdparty/libprocess/include/process/event.hpp   | 16 ++++-------
 3rdparty/libprocess/include/process/gmock.hpp   | 15 ++++++++--
 3rdparty/libprocess/src/process.cpp             |  4 +--
 5 files changed, 37 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b977eae0/3rdparty/libprocess/include/process/c++11/dispatch.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/c++11/dispatch.hpp b/3rdparty/libprocess/include/process/c++11/dispatch.hpp
index 76da282..99de0e9 100644
--- a/3rdparty/libprocess/include/process/c++11/dispatch.hpp
+++ b/3rdparty/libprocess/include/process/c++11/dispatch.hpp
@@ -51,16 +51,7 @@ namespace internal {
 void dispatch(
     const UPID& pid,
     const std::shared_ptr<std::function<void(ProcessBase*)>>& f,
-    const std::string& method = std::string());
-
-
-// Canonicalizes a pointer to a member function (i.e., method) into a
-// bytes representation for comparison (e.g., in tests).
-template <typename Method>
-std::string canonicalize(Method method)
-{
-  return std::string(reinterpret_cast<const char*>(&method), sizeof(method));
-}
+    const Option<const std::type_info*>& functionType = None());
 
 } // namespace internal {
 
@@ -87,7 +78,7 @@ void dispatch(
             (t->*method)();
           }));
 
-  internal::dispatch(pid, f, internal::canonicalize(method));
+  internal::dispatch(pid, f, &typeid(method));
 }
 
 template <typename T>
@@ -127,7 +118,7 @@ void dispatch(
               (t->*method)(ENUM_PARAMS(N, a));                          \
             }));                                                        \
                                                                         \
-    internal::dispatch(pid, f, internal::canonicalize(method));         \
+    internal::dispatch(pid, f, &typeid(method));                        \
   }                                                                     \
                                                                         \
   template <typename T,                                                 \
@@ -174,7 +165,7 @@ Future<R> dispatch(
             promise->associate((t->*method)());
           }));
 
-  internal::dispatch(pid, f, internal::canonicalize(method));
+  internal::dispatch(pid, f, &typeid(method));
 
   return promise->future();
 }
@@ -216,7 +207,7 @@ Future<R> dispatch(
               promise->associate((t->*method)(ENUM_PARAMS(N, a)));      \
             }));                                                        \
                                                                         \
-    internal::dispatch(pid, f, internal::canonicalize(method));         \
+    internal::dispatch(pid, f, &typeid(method));                        \
                                                                         \
     return promise->future();                                           \
   }                                                                     \
@@ -267,7 +258,7 @@ Future<R> dispatch(
             promise->set((t->*method)());
           }));
 
-  internal::dispatch(pid, f, internal::canonicalize(method));
+  internal::dispatch(pid, f, &typeid(method));
 
   return promise->future();
 }
@@ -309,7 +300,7 @@ Future<R> dispatch(
               promise->set((t->*method)(ENUM_PARAMS(N, a)));            \
             }));                                                        \
                                                                         \
-    internal::dispatch(pid, f, internal::canonicalize(method));         \
+    internal::dispatch(pid, f, &typeid(method));                        \
                                                                         \
     return promise->future();                                           \
   }                                                                     \

http://git-wip-us.apache.org/repos/asf/mesos/blob/b977eae0/3rdparty/libprocess/include/process/dispatch.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/dispatch.hpp b/3rdparty/libprocess/include/process/dispatch.hpp
index bceda2a..4ec07d6 100644
--- a/3rdparty/libprocess/include/process/dispatch.hpp
+++ b/3rdparty/libprocess/include/process/dispatch.hpp
@@ -54,7 +54,7 @@ namespace internal {
 void dispatch(
     const UPID& pid,
     const memory::shared_ptr<lambda::function<void(ProcessBase*)> >& f,
-    const std::string& method = std::string());
+    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
@@ -101,15 +101,6 @@ void rdispatcher(
   promise->set((*thunk)(t));
 }
 
-
-// Canonicalizes a pointer to a member function (i.e., method) into a
-// bytes representation for comparison (e.g., in tests).
-template <typename Method>
-std::string canonicalize(Method method)
-{
-  return std::string(reinterpret_cast<const char*>(&method), sizeof(method));
-}
-
 } // namespace internal {
 
 
@@ -139,7 +130,7 @@ std::string canonicalize(Method method)
 //                        lambda::_1,
 //                        thunk)));
 //
-//   internal::dispatch(pid, dispatcher, internal::canonicalize(method));
+//   internal::dispatch(pid, dispatcher, &typeid(method));
 // }
 
 template <typename T>
@@ -157,7 +148,7 @@ void dispatch(
                        lambda::_1,
                        thunk)));
 
-  internal::dispatch(pid, dispatcher, internal::canonicalize(method));
+  internal::dispatch(pid, dispatcher, &typeid(method));
 }
 
 template <typename T>
@@ -197,7 +188,7 @@ void dispatch(
                          lambda::_1,                                    \
                          thunk)));                                      \
                                                                         \
-    internal::dispatch(pid, dispatcher, internal::canonicalize(method)); \
+    internal::dispatch(pid, dispatcher, &typeid(method));               \
   }                                                                     \
                                                                         \
   template <typename T,                                                 \
@@ -249,7 +240,7 @@ void dispatch(
 //                        lambda::_1,
 //                        thunk, promise)));
 //
-//   internal::dispatch(pid, dispatcher, internal::canonicalize(method));
+//   internal::dispatch(pid, dispatcher, &typeid(method));
 //
 //   return future;
 // }
@@ -272,7 +263,7 @@ Future<R> dispatch(
                        lambda::_1,
                        thunk, promise)));
 
-  internal::dispatch(pid, dispatcher, internal::canonicalize(method));
+  internal::dispatch(pid, dispatcher, &typeid(method));
 
   return future;
 }
@@ -318,7 +309,7 @@ Future<R> dispatch(
                          lambda::_1,                                    \
                          thunk, promise)));                             \
                                                                         \
-    internal::dispatch(pid, dispatcher, internal::canonicalize(method)); \
+    internal::dispatch(pid, dispatcher, &typeid(method));               \
                                                                         \
     return future;                                                      \
   }                                                                     \
@@ -374,7 +365,7 @@ Future<R> dispatch(
 //                        lambda::_1,
 //                        thunk, promise)));
 //
-//   internal::dispatch(pid, dispatcher, internal::canonicalize(method));
+//   internal::dispatch(pid, dispatcher, &typeid(method));
 //
 //   return future;
 // }
@@ -397,7 +388,7 @@ Future<R> dispatch(
                        lambda::_1,
                        thunk, promise)));
 
-  internal::dispatch(pid, dispatcher, internal::canonicalize(method));
+  internal::dispatch(pid, dispatcher, &typeid(method));
 
   return future;
 }
@@ -443,7 +434,7 @@ Future<R> dispatch(
                          lambda::_1,                                    \
                          thunk, promise)));                             \
                                                                         \
-    internal::dispatch(pid, dispatcher, internal::canonicalize(method)); \
+    internal::dispatch(pid, dispatcher, &typeid(method));               \
                                                                         \
     return future;                                                      \
   }                                                                     \

http://git-wip-us.apache.org/repos/asf/mesos/blob/b977eae0/3rdparty/libprocess/include/process/event.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/event.hpp b/3rdparty/libprocess/include/process/event.hpp
index 294e215..6392332 100644
--- a/3rdparty/libprocess/include/process/event.hpp
+++ b/3rdparty/libprocess/include/process/event.hpp
@@ -130,10 +130,10 @@ struct DispatchEvent : Event
   DispatchEvent(
       const UPID& _pid,
       const memory::shared_ptr<lambda::function<void(ProcessBase*)> >& _f,
-      const std::string& _method)
+      const Option<const std::type_info*>& _functionType)
     : pid(_pid),
       f(_f),
-      method(_method)
+      functionType(_functionType)
   {}
 
   virtual void visit(EventVisitor* visitor) const
@@ -145,15 +145,9 @@ struct DispatchEvent : Event
   const UPID pid;
 
   // Function to get invoked as a result of this dispatch event.
-  const memory::shared_ptr<lambda::function<void(ProcessBase*)> > f;
-
-  // Canonical "byte" representation of a pointer to a member function
-  // (i.e., method) encapsulated in the above function (or empty if
-  // not applicable). Note that we use a byte representation because a
-  // pointer to a member function is not actually a pointer, but
-  // instead a POD.
-  // TODO(benh): Perform canonicalization lazily.
-  const std::string method;
+  const memory::shared_ptr<lambda::function<void(ProcessBase*)>> f;
+
+  const Option<const std::type_info*> functionType;
 
 private:
   // Not copyable, not assignable.

http://git-wip-us.apache.org/repos/asf/mesos/blob/b977eae0/3rdparty/libprocess/include/process/gmock.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/include/process/gmock.hpp b/3rdparty/libprocess/include/process/gmock.hpp
index d6f2fc8..c5c861d 100644
--- a/3rdparty/libprocess/include/process/gmock.hpp
+++ b/3rdparty/libprocess/include/process/gmock.hpp
@@ -39,6 +39,17 @@
 #define DROP_MESSAGE(name, from, to)            \
   process::FutureMessage(name, from, to, true)
 
+// The mechanism of how we match method dispatches is done by
+// comparing std::type_info of the member function pointers. Because
+// of this, the method function pointer passed to either
+// FUTURE_DISPATCH or DROP_DISPATCH must match exactly the member
+// function that is passed to the dispatch method.
+// TODO(tnachen): In a situation where a base class has a virtual
+// function and that a derived class overrides, and if in unit tests
+// we want to verify it calls the exact derived member function, we
+// need to change how dispatch matching works. One possible way is to
+// move the dispatch matching logic at event dequeue time, as we then
+// have the actual Process the dispatch event is calling to.
 #define FUTURE_DISPATCH(pid, method)            \
   process::FutureDispatch(pid, method)
 
@@ -331,8 +342,8 @@ MATCHER_P2(DispatchMatcher, pid, method, "")
 {
   const DispatchEvent& event = ::std::tr1::get<0>(arg);
   return (testing::Matcher<UPID>(pid).Matches(event.pid) &&
-          testing::Matcher<std::string>(internal::canonicalize(method))
-          .Matches(event.method));
+          event.functionType.isSome() &&
+          *event.functionType.get() == typeid(method));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/b977eae0/3rdparty/libprocess/src/process.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/process.cpp b/3rdparty/libprocess/src/process.cpp
index 7a986d7..00cd89f 100644
--- a/3rdparty/libprocess/src/process.cpp
+++ b/3rdparty/libprocess/src/process.cpp
@@ -3137,11 +3137,11 @@ namespace internal {
 void dispatch(
     const UPID& pid,
     const memory::shared_ptr<lambda::function<void(ProcessBase*)> >& f,
-    const string& method)
+    const Option<const std::type_info*>& functionType)
 {
   process::initialize();
 
-  DispatchEvent* event = new DispatchEvent(pid, f, method);
+  DispatchEvent* event = new DispatchEvent(pid, f, functionType);
   process_manager->deliver(pid, event, __process__);
 }
 


[2/2] mesos git commit: Fixed future dispatch in tests to match function type.

Posted by tn...@apache.org.
Fixed future dispatch in tests to match function type.

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


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

Branch: refs/heads/master
Commit: 2eaa7d3611cf8202a89617541ab0a144242abfa3
Parents: b977eae
Author: Timothy Chen <tn...@apache.org>
Authored: Fri Nov 14 17:54:42 2014 -0800
Committer: Timothy Chen <tn...@apache.org>
Committed: Thu Nov 20 14:12:40 2014 -0800

----------------------------------------------------------------------
 src/tests/allocator_tests.cpp | 5 +++--
 src/tests/master_tests.cpp    | 2 +-
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2eaa7d36/src/tests/allocator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/allocator_tests.cpp b/src/tests/allocator_tests.cpp
index 58e15aa..1fcbb4a 100644
--- a/src/tests/allocator_tests.cpp
+++ b/src/tests/allocator_tests.cpp
@@ -45,6 +45,7 @@ using namespace mesos::internal;
 using namespace mesos::internal::tests;
 
 using mesos::internal::master::allocator::Allocator;
+using mesos::internal::master::allocator::AllocatorProcess;
 using mesos::internal::master::allocator::HierarchicalDRFAllocatorProcess;
 
 using mesos::internal::master::Master;
@@ -754,7 +755,7 @@ TEST_F(ReservationAllocatorTest, ResourcesReturned)
     .Times(2);
 
   Future<Nothing> slaveAdded1 = FUTURE_DISPATCH(
-      allocator.real, &HierarchicalDRFAllocatorProcess::slaveAdded);
+      allocator.real, &AllocatorProcess::slaveAdded);
 
   slave::Flags flags1 = CreateSlaveFlags();
   flags1.resources = Some("cpus(role1):1;mem(role1):200;cpus(role2):2;"
@@ -766,7 +767,7 @@ TEST_F(ReservationAllocatorTest, ResourcesReturned)
   AWAIT_READY(slaveAdded1);
 
   Future<Nothing> slaveAdded2 = FUTURE_DISPATCH(
-      allocator.real, &HierarchicalDRFAllocatorProcess::slaveAdded);
+      allocator.real, &AllocatorProcess::slaveAdded);
 
   // This slave's resources will never be offered to anyone,
   // because there is no framework with role3.

http://git-wip-us.apache.org/repos/asf/mesos/blob/2eaa7d36/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index 0ed02b3..065fc3e 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -2467,7 +2467,7 @@ TEST_F(MasterTest, ReleaseResourcesForTerminalTaskWithPendingUpdates)
   AWAIT_READY(__statusUpdate2);
 
   Future<Nothing> resourcesRecovered = FUTURE_DISPATCH(
-      _, &HierarchicalDRFAllocatorProcess::resourcesRecovered);
+      _, &AllocatorProcess::resourcesRecovered);
 
   // Advance the clock so that the status update manager resends
   // TASK_RUNNING update with 'latest_state' as TASK_FINISHED.