You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by vi...@apache.org on 2016/02/09 00:33:44 UTC

[1/4] mesos git commit: Fixed flakiness in SlaveRecoveryTest/0.CleanupHTTPExecutor.

Repository: mesos
Updated Branches:
  refs/heads/master 2c1808685 -> 67017f136


Fixed flakiness in SlaveRecoveryTest/0.CleanupHTTPExecutor.

This change fixes the flakiness in this test. The issue was a race between the
`connected` callback being called before we did `process::spawn` to start the
process.

The details of the race that lead to the failure are as follows:
- We started the executor library inside the constructor of `TestExecutor`. The
  callback function did `process::defer(self(), &Self::connected)`
- The `connected` callback can be invoked by the Executor library before we got
  a chance to actually invoke `process::spawn` on the `TestExecutor` process
itself. This in can turn lead to the `dispatch` being silently dropped.

This change now starts the library inside the `initialize` function that is
gurranteed to be called after `process::spawn` is invoked.

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


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

Branch: refs/heads/master
Commit: 1af86559c3cf47159aa00c289208bbec50fd6df9
Parents: 2c18086
Author: Anand Mazumdar <ma...@gmail.com>
Authored: Mon Feb 8 15:32:22 2016 -0800
Committer: Vinod Kone <vi...@gmail.com>
Committed: Mon Feb 8 15:32:22 2016 -0800

----------------------------------------------------------------------
 src/examples/test_http_executor.cpp | 22 +++++++++++++++-------
 1 file changed, 15 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/1af86559/src/examples/test_http_executor.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_http_executor.cpp b/src/examples/test_http_executor.cpp
index 4916e0e..55a427f 100644
--- a/src/examples/test_http_executor.cpp
+++ b/src/examples/test_http_executor.cpp
@@ -60,10 +60,6 @@ public:
   TestExecutor(const FrameworkID& _frameworkId, const ExecutorID& _executorId)
     : frameworkId(_frameworkId),
       executorId(_executorId),
-      mesos(mesos::ContentType::PROTOBUF,
-            process::defer(self(), &Self::connected),
-            process::defer(self(), &Self::disconnected),
-            process::defer(self(), &Self::received, lambda::_1)),
       state(DISCONNECTED) {}
 
   void connected()
@@ -97,7 +93,7 @@ public:
       subscribe->add_unacknowledged_tasks()->MergeFrom(task);
     }
 
-    mesos.send(call);
+    mesos->send(call);
 
     process::delay(Seconds(1), self(), &Self::doReliableRegistration);
   }
@@ -129,7 +125,7 @@ public:
     // Capture the status update.
     updates[uuid] = call.update();
 
-    mesos.send(call);
+    mesos->send(call);
   }
 
   void received(queue<Event> events)
@@ -196,10 +192,22 @@ public:
     }
   }
 
+protected:
+  virtual void initialize()
+  {
+    // We initialize the library here to ensure that callbacks are only invoked
+    // after the process has spawned.
+    mesos.reset(new Mesos(
+        mesos::ContentType::PROTOBUF,
+        process::defer(self(), &Self::connected),
+        process::defer(self(), &Self::disconnected),
+        process::defer(self(), &Self::received, lambda::_1)));
+  }
+
 private:
   const FrameworkID frameworkId;
   const ExecutorID executorId;
-  Mesos mesos;
+  process::Owned<Mesos> mesos;
   enum State
   {
     CONNECTED,


[4/4] mesos git commit: Re-enabled test ExamplesTest.EventCallFramework.

Posted by vi...@apache.org.
Re-enabled test ExamplesTest.EventCallFramework.

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


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

Branch: refs/heads/master
Commit: 67017f136b9820c493b9d861ccb7fc9e463dfa92
Parents: 5e0f787
Author: Anand Mazumdar <ma...@gmail.com>
Authored: Mon Feb 8 15:33:25 2016 -0800
Committer: Vinod Kone <vi...@gmail.com>
Committed: Mon Feb 8 15:33:25 2016 -0800

----------------------------------------------------------------------
 src/tests/examples_tests.cpp | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/67017f13/src/tests/examples_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/examples_tests.cpp b/src/tests/examples_tests.cpp
index a9b685b..9b8b30f 100644
--- a/src/tests/examples_tests.cpp
+++ b/src/tests/examples_tests.cpp
@@ -24,8 +24,7 @@ TEST_SCRIPT(ExamplesTest, TestFramework, "test_framework_test.sh")
 TEST_SCRIPT(ExamplesTest, NoExecutorFramework, "no_executor_framework_test.sh")
 
 
-// Temporarily disabled this test due to MESOS-3273.
-TEST_SCRIPT(DISABLED_ExamplesTest, EventCallFramework,
+TEST_SCRIPT(ExamplesTest, EventCallFramework,
             "event_call_framework_test.sh")
 
 


[3/4] mesos git commit: Replaced naked pointer with Owned.

Posted by vi...@apache.org.
Replaced naked pointer with Owned.

This trivial change replaces the old C style pointer with `process::Owned`.

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


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

Branch: refs/heads/master
Commit: 5e0f787fee8ad810ed6580cecdc2dc92644944ed
Parents: d68d342
Author: Anand Mazumdar <ma...@gmail.com>
Authored: Mon Feb 8 15:33:17 2016 -0800
Committer: Vinod Kone <vi...@gmail.com>
Committed: Mon Feb 8 15:33:17 2016 -0800

----------------------------------------------------------------------
 src/examples/event_call_framework.cpp | 9 ++++-----
 1 file changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/5e0f787f/src/examples/event_call_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/event_call_framework.cpp b/src/examples/event_call_framework.cpp
index fed8fca..d07d05d 100644
--- a/src/examples/event_call_framework.cpp
+++ b/src/examples/event_call_framework.cpp
@@ -437,12 +437,11 @@ int main(int argc, char** argv)
 
   framework.set_principal(value.get());
 
-  EventCallScheduler* scheduler;
-  scheduler = new EventCallScheduler(framework, executor, master.get());
+  process::Owned<EventCallScheduler> scheduler(
+      new EventCallScheduler(framework, executor, master.get()));
 
-  process::spawn(scheduler);
-  process::wait(scheduler);
-  delete scheduler;
+  process::spawn(scheduler.get());
+  process::wait(scheduler.get());
 
   return EXIT_SUCCESS;
 }


[2/4] mesos git commit: Fixed flakiness in ExamplesTest.EventCallFramework.

Posted by vi...@apache.org.
Fixed flakiness in ExamplesTest.EventCallFramework.

The root cause is similar to r43285.

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


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

Branch: refs/heads/master
Commit: d68d3429fcfd351cd3f0907bc798bb3139ed0903
Parents: 1af8655
Author: Anand Mazumdar <ma...@gmail.com>
Authored: Mon Feb 8 15:33:05 2016 -0800
Committer: Vinod Kone <vi...@gmail.com>
Committed: Mon Feb 8 15:33:05 2016 -0800

----------------------------------------------------------------------
 src/examples/event_call_framework.cpp | 41 +++++++++++++++++-------------
 1 file changed, 24 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d68d3429/src/examples/event_call_framework.cpp
----------------------------------------------------------------------
diff --git a/src/examples/event_call_framework.cpp b/src/examples/event_call_framework.cpp
index 8cfe1a0..fed8fca 100644
--- a/src/examples/event_call_framework.cpp
+++ b/src/examples/event_call_framework.cpp
@@ -25,6 +25,7 @@
 #include <mesos/v1/scheduler.hpp>
 
 #include <process/delay.hpp>
+#include <process/owned.hpp>
 #include <process/process.hpp>
 #include <process/protobuf.hpp>
 
@@ -69,14 +70,10 @@ class EventCallScheduler : public process::Process<EventCallScheduler>
 public:
   EventCallScheduler(const FrameworkInfo& _framework,
                      const ExecutorInfo& _executor,
-                     const string& master)
+                     const string& _master)
     : framework(_framework),
       executor(_executor),
-      mesos(master,
-            mesos::ContentType::PROTOBUF,
-            process::defer(self(), &Self::connected),
-            process::defer(self(), &Self::disconnected),
-            process::defer(self(), &Self::received, lambda::_1)),
+      master(_master),
       state(INITIALIZING),
       tasksLaunched(0),
       tasksFinished(0),
@@ -84,15 +81,11 @@ public:
 
   EventCallScheduler(const FrameworkInfo& _framework,
                      const ExecutorInfo& _executor,
-                     const string& master,
+                     const string& _master,
                      const Credential& credential)
     : framework(_framework),
       executor(_executor),
-      mesos(master,
-            mesos::ContentType::PROTOBUF,
-            process::defer(self(), &Self::connected),
-            process::defer(self(), &Self::disconnected),
-            process::defer(self(), &Self::received, lambda::_1)),
+      master(_master),
       state(INITIALIZING),
       tasksLaunched(0),
       tasksFinished(0),
@@ -196,6 +189,19 @@ public:
     }
   }
 
+protected:
+virtual void initialize()
+{
+  // We initialize the library here to ensure that callbacks are only invoked
+  // after the process has spawned.
+  mesos.reset(new scheduler::Mesos(
+      master,
+      mesos::ContentType::PROTOBUF,
+      process::defer(self(), &Self::connected),
+      process::defer(self(), &Self::disconnected),
+      process::defer(self(), &Self::received, lambda::_1)));
+}
+
 private:
   void resourceOffers(const vector<Offer>& offers)
   {
@@ -252,7 +258,7 @@ private:
         operation->mutable_launch()->add_task_infos()->CopyFrom(taskInfo);
       }
 
-      mesos.send(call);
+      mesos->send(call);
     }
   }
 
@@ -276,7 +282,7 @@ private:
       ack->mutable_task_id ()->CopyFrom(status.task_id ());
       ack->set_uuid(status.uuid());
 
-      mesos.send(call);
+      mesos->send(call);
     }
 
     if (status.state() == TASK_FINISHED) {
@@ -317,7 +323,7 @@ private:
       subscribe->set_force(true);
     }
 
-    mesos.send(call);
+    mesos->send(call);
 
     process::delay(Seconds(1),
                    self(),
@@ -331,12 +337,13 @@ private:
     call.mutable_framework_id()->CopyFrom(framework.id());
     call.set_type(Call::TEARDOWN);
 
-    mesos.send(call);
+    mesos->send(call);
   }
 
   FrameworkInfo framework;
   const ExecutorInfo executor;
-  scheduler::Mesos mesos;
+  const string master;
+  process::Owned<scheduler::Mesos> mesos;
 
   enum State
   {