You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by dm...@apache.org on 2014/11/12 22:05:39 UTC

[2/2] mesos git commit: Replace the ip and port pairs from UPID class and process namespace with Node class.

Replace the ip and port pairs from UPID class and process namespace with Node class.

At the moment, the Node class is used to keep a mapping from a socket to the ip
& port pair in the process namespace. I want to propose to extend its use by
replacing the ip & port fields from the UPID class and process namespace with this
type.

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


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

Branch: refs/heads/master
Commit: 76bfb49301ff80a4535f47bce59d547ba867319f
Parents: f64562f
Author: Evelina Dumitrescu <ev...@gmail.com>
Authored: Wed Nov 12 12:56:54 2014 -0800
Committer: Dominic Hamon <dh...@twitter.com>
Committed: Wed Nov 12 12:58:02 2014 -0800

----------------------------------------------------------------------
 src/common/protobuf_utils.cpp                 |  6 +--
 src/master/master.cpp                         | 10 ++--
 src/sched/sched.cpp                           |  2 +-
 src/scheduler/scheduler.cpp                   |  2 +-
 src/slave/http.cpp                            |  2 +-
 src/slave/slave.cpp                           |  6 +--
 src/tests/fetcher_tests.cpp                   |  4 +-
 src/tests/files_tests.cpp                     |  8 +--
 src/tests/gc_tests.cpp                        |  6 +--
 src/tests/logging_tests.cpp                   |  3 +-
 src/tests/master_contender_detector_tests.cpp | 60 +++++++++++-----------
 src/tests/master_tests.cpp                    | 12 ++---
 src/tests/monitor_tests.cpp                   |  2 +-
 13 files changed, 61 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/76bfb493/src/common/protobuf_utils.cpp
----------------------------------------------------------------------
diff --git a/src/common/protobuf_utils.cpp b/src/common/protobuf_utils.cpp
index 33ce782..e3e430d 100644
--- a/src/common/protobuf_utils.cpp
+++ b/src/common/protobuf_utils.cpp
@@ -111,11 +111,11 @@ MasterInfo createMasterInfo(const process::UPID& pid)
 {
   MasterInfo info;
   info.set_id(stringify(pid) + "-" + UUID::random().toString());
-  info.set_ip(pid.ip);
-  info.set_port(pid.port);
+  info.set_ip(pid.node.ip);
+  info.set_port(pid.node.port);
   info.set_pid(pid);
 
-  Try<std::string> hostname = net::getHostname(pid.ip);
+  Try<std::string> hostname = net::getHostname(pid.node.ip);
   if (hostname.isSome()) {
     info.set_hostname(hostname.get());
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/76bfb493/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 83fecba..fbf6375 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -282,20 +282,20 @@ Master::Master(
   // address and port from self() and the OS PID.
   Try<string> id =
     strings::format("%s-%u-%u-%d", DateUtils::currentDate(),
-                    self().ip, self().port, getpid());
+                    self().node.ip, self().node.port, getpid());
 
   CHECK(!id.isError()) << id.error();
 
   info_.set_id(id.get());
-  info_.set_ip(self().ip);
-  info_.set_port(self().port);
+  info_.set_ip(self().node.ip);
+  info_.set_port(self().node.port);
   info_.set_pid(self());
 
   // Determine our hostname or use the hostname provided.
   string hostname;
 
   if (flags.hostname.isNone()) {
-    Try<string> result = net::getHostname(self().ip);
+    Try<string> result = net::getHostname(self().node.ip);
 
     if (result.isError()) {
       LOG(FATAL) << "Failed to get hostname: " << result.error();
@@ -318,7 +318,7 @@ void Master::initialize()
   LOG(INFO) << "Master " << info_.id() << " (" << info_.hostname() << ")"
             << " started on " << string(self()).substr(7);
 
-  if (stringify(net::IP(ntohl(self().ip))) == "127.0.0.1") {
+  if (stringify(net::IP(ntohl(self().node.ip))) == "127.0.0.1") {
     LOG(WARNING) << "\n**************************************************\n"
                  << "Master bound to loopback interface!"
                  << " Cannot communicate with remote schedulers or slaves."

http://git-wip-us.apache.org/repos/asf/mesos/blob/76bfb493/src/sched/sched.cpp
----------------------------------------------------------------------
diff --git a/src/sched/sched.cpp b/src/sched/sched.cpp
index 4981dfb..d662182 100644
--- a/src/sched/sched.cpp
+++ b/src/sched/sched.cpp
@@ -1100,7 +1100,7 @@ void MesosSchedulerDriver::initialize() {
   // Initialize libprocess.
   process::initialize(schedulerId);
 
-  if (stringify(net::IP(ntohl(process::ip()))) == "127.0.0.1") {
+  if (stringify(net::IP(ntohl(process::node().ip))) == "127.0.0.1") {
     LOG(WARNING) << "\n**************************************************\n"
                  << "Scheduler driver bound to loopback interface!"
                  << " Cannot communicate with remote master(s)."

http://git-wip-us.apache.org/repos/asf/mesos/blob/76bfb493/src/scheduler/scheduler.cpp
----------------------------------------------------------------------
diff --git a/src/scheduler/scheduler.cpp b/src/scheduler/scheduler.cpp
index cbb982a..ff6ff11 100644
--- a/src/scheduler/scheduler.cpp
+++ b/src/scheduler/scheduler.cpp
@@ -127,7 +127,7 @@ public:
     // want to use flags to initialize libprocess).
     process::initialize();
 
-    if (stringify(net::IP(ntohl(self().ip))) == "127.0.0.1") {
+    if (stringify(net::IP(ntohl(self().node.ip))) == "127.0.0.1") {
       LOG(WARNING) << "\n**************************************************\n"
                    << "Scheduler driver bound to loopback interface!"
                    << " Cannot communicate with remote master(s)."

http://git-wip-us.apache.org/repos/asf/mesos/blob/76bfb493/src/slave/http.cpp
----------------------------------------------------------------------
diff --git a/src/slave/http.cpp b/src/slave/http.cpp
index 965d78c..d1cf8a6 100644
--- a/src/slave/http.cpp
+++ b/src/slave/http.cpp
@@ -361,7 +361,7 @@ Future<Response> Slave::Http::state(const Request& request)
   object.values["lost_tasks"] = slave->stats.tasks[TASK_LOST];
 
   if (slave->master.isSome()) {
-    Try<string> masterHostname = net::getHostname(slave->master.get().ip);
+    Try<string> masterHostname = net::getHostname(slave->master.get().node.ip);
     if (masterHostname.isSome()) {
       object.values["master_hostname"] = masterHostname.get();
     }

http://git-wip-us.apache.org/repos/asf/mesos/blob/76bfb493/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 99fd055..275081c 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -171,7 +171,7 @@ void Slave::initialize()
 {
   LOG(INFO) << "Slave started on " << string(self()).substr(6);
 
-  if (stringify(net::IP(ntohl(self().ip))) == "127.0.0.1") {
+  if (stringify(net::IP(ntohl(self().node.ip))) == "127.0.0.1") {
     LOG(WARNING) << "\n**************************************************\n"
                  << "Slave bound to loopback interface!"
                  << " Cannot communicate with remote master(s)."
@@ -302,7 +302,7 @@ void Slave::initialize()
   string hostname;
 
   if (flags.hostname.isNone()) {
-    Try<string> result = net::getHostname(self().ip);
+    Try<string> result = net::getHostname(self().node.ip);
 
     if (result.isError()) {
       LOG(FATAL) << "Failed to get hostname: " << result.error();
@@ -315,7 +315,7 @@ void Slave::initialize()
 
   // Initialize slave info.
   info.set_hostname(hostname);
-  info.set_port(self().port);
+  info.set_port(self().node.port);
   info.mutable_resources()->CopyFrom(resources.get());
   info.mutable_attributes()->CopyFrom(attributes);
   info.set_checkpoint(flags.checkpoint);

http://git-wip-us.apache.org/repos/asf/mesos/blob/76bfb493/src/tests/fetcher_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/fetcher_tests.cpp b/src/tests/fetcher_tests.cpp
index a01eec1..19aee31 100644
--- a/src/tests/fetcher_tests.cpp
+++ b/src/tests/fetcher_tests.cpp
@@ -331,8 +331,8 @@ TEST_F(FetcherTest, OSNetUriTest)
 
   spawn(process);
 
-  string url = "http://" + net::getHostname(process.self().ip).get() +
-                ":" + stringify(process.self().port) + "/help";
+  string url = "http://" + net::getHostname(process.self().node.ip).get() +
+                ":" + stringify(process.self().node.port) + "/help";
 
   string localFile = path::join(os::getcwd(), "help");
   EXPECT_FALSE(os::exists(localFile));

http://git-wip-us.apache.org/repos/asf/mesos/blob/76bfb493/src/tests/files_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/files_tests.cpp b/src/tests/files_tests.cpp
index a696aa2..9ad6db5 100644
--- a/src/tests/files_tests.cpp
+++ b/src/tests/files_tests.cpp
@@ -84,7 +84,7 @@ TEST_F(FilesTest, DetachTest)
 TEST_F(FilesTest, ReadTest)
 {
   Files files;
-  process::UPID upid("files", process::ip(), process::port());
+  process::UPID upid("files", process::node());
 
   Future<Response> response =
     process::http::get(upid, "read.json");
@@ -138,7 +138,7 @@ TEST_F(FilesTest, ReadTest)
 TEST_F(FilesTest, ResolveTest)
 {
   Files files;
-  process::UPID upid("files", process::ip(), process::port());
+  process::UPID upid("files", process::node());
 
   // Test the directory / file resolution.
   ASSERT_SOME(os::mkdir("1/2"));
@@ -214,7 +214,7 @@ TEST_F(FilesTest, ResolveTest)
 TEST_F(FilesTest, BrowseTest)
 {
   Files files;
-  process::UPID upid("files", process::ip(), process::port());
+  process::UPID upid("files", process::node());
 
   ASSERT_SOME(os::mkdir("1/2"));
   ASSERT_SOME(os::mkdir("1/3"));
@@ -267,7 +267,7 @@ TEST_F(FilesTest, BrowseTest)
 TEST_F(FilesTest, DownloadTest)
 {
   Files files;
-  process::UPID upid("files", process::ip(), process::port());
+  process::UPID upid("files", process::node());
 
   // This is a one-pixel black gif image.
   const unsigned char gifData[] = {

http://git-wip-us.apache.org/repos/asf/mesos/blob/76bfb493/src/tests/gc_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/gc_tests.cpp b/src/tests/gc_tests.cpp
index f7747e2..8618ae1 100644
--- a/src/tests/gc_tests.cpp
+++ b/src/tests/gc_tests.cpp
@@ -458,7 +458,7 @@ TEST_F(GarbageCollectorIntegrationTest, ExitedFramework)
 
   ASSERT_FALSE(os::exists(frameworkDir));
 
-  process::UPID filesUpid("files", process::ip(), process::port());
+  process::UPID filesUpid("files", process::node());
   AWAIT_EXPECT_RESPONSE_STATUS_EQ(
       process::http::NotFound().status,
       process::http::get(filesUpid, "browse.json", "path=" + frameworkDir));
@@ -559,7 +559,7 @@ TEST_F(GarbageCollectorIntegrationTest, ExitedExecutor)
   // Executor's directory should be gc'ed by now.
   ASSERT_FALSE(os::exists(executorDir));
 
-  process::UPID files("files", process::ip(), process::port());
+  process::UPID files("files", process::node());
   AWAIT_EXPECT_RESPONSE_STATUS_EQ(
       process::http::NotFound().status,
       process::http::get(files, "browse.json", "path=" + executorDir));
@@ -674,7 +674,7 @@ TEST_F(GarbageCollectorIntegrationTest, DiskUsage)
   // Executor's directory should be gc'ed by now.
   ASSERT_FALSE(os::exists(executorDir));
 
-  process::UPID files("files", process::ip(), process::port());
+  process::UPID files("files", process::node());
   AWAIT_EXPECT_RESPONSE_STATUS_EQ(
       process::http::NotFound().status,
       process::http::get(files, "browse.json", "path=" + executorDir));

http://git-wip-us.apache.org/repos/asf/mesos/blob/76bfb493/src/tests/logging_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/logging_tests.cpp b/src/tests/logging_tests.cpp
index 58e9b33..22a785e 100644
--- a/src/tests/logging_tests.cpp
+++ b/src/tests/logging_tests.cpp
@@ -37,8 +37,7 @@ TEST(LoggingTest, Toggle)
 {
   process::PID<> pid;
   pid.id = "logging";
-  pid.ip = process::ip();
-  pid.port = process::port();
+  pid.node = process::node();
 
   process::Future<Response> response = process::http::get(pid, "toggle");
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/76bfb493/src/tests/master_contender_detector_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_contender_detector_tests.cpp b/src/tests/master_contender_detector_tests.cpp
index 8f7fb18..d847a30 100644
--- a/src/tests/master_contender_detector_tests.cpp
+++ b/src/tests/master_contender_detector_tests.cpp
@@ -130,8 +130,8 @@ TEST_F(MasterContenderDetectorTest, File)
 TEST(BasicMasterContenderDetectorTest, Contender)
 {
   PID<Master> master;
-  master.ip = 10000000;
-  master.port = 10000;
+  master.node.ip = 10000000;
+  master.node.port = 10000;
 
   MasterContender* contender = new StandaloneMasterContender();
 
@@ -155,8 +155,8 @@ TEST(BasicMasterContenderDetectorTest, Contender)
 TEST(BasicMasterContenderDetectorTest, Detector)
 {
   PID<Master> master;
-  master.ip = 10000000;
-  master.port = 10000;
+  master.node.ip = 10000000;
+  master.node.port = 10000;
 
   StandaloneMasterDetector detector;
 
@@ -199,8 +199,8 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, MasterContender)
   ZooKeeperMasterContender* contender = new ZooKeeperMasterContender(group);
 
   PID<Master> pid;
-  pid.ip = 10000000;
-  pid.port = 10000;
+  pid.node.ip = 10000000;
+  pid.node.port = 10000;
   MasterInfo master = internal::protobuf::createMasterInfo(pid);
 
   contender->initialize(master);
@@ -257,8 +257,8 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, ContenderPendingElection)
   ZooKeeperMasterContender contender(url.get());
 
   PID<Master> pid;
-  pid.ip = 10000000;
-  pid.port = 10000;
+  pid.node.ip = 10000000;
+  pid.node.port = 10000;
   MasterInfo master = internal::protobuf::createMasterInfo(pid);
 
   contender.initialize(master);
@@ -311,8 +311,8 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, MasterContenders)
     new ZooKeeperMasterContender(url.get());
 
   PID<Master> pid1;
-  pid1.ip = 10000000;
-  pid1.port = 10000;
+  pid1.node.ip = 10000000;
+  pid1.node.port = 10000;
   MasterInfo master1 = internal::protobuf::createMasterInfo(pid1);
 
   contender1->initialize(master1);
@@ -329,8 +329,8 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, MasterContenders)
   ZooKeeperMasterContender contender2(url.get());
 
   PID<Master> pid2;
-  pid2.ip = 10000001;
-  pid2.port = 10001;
+  pid2.node.ip = 10000001;
+  pid2.node.port = 10001;
   MasterInfo master2 = internal::protobuf::createMasterInfo(pid2);
 
   contender2.initialize(master2);
@@ -368,8 +368,8 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, NonRetryableFrrors)
   AWAIT_READY(group1.join("data"));
 
   PID<Master> pid;
-  pid.ip = 10000000;
-  pid.port = 10000;
+  pid.node.ip = 10000000;
+  pid.node.port = 10000;
   MasterInfo master = internal::protobuf::createMasterInfo(pid);
 
   // group2's password is wrong and operations on it will fail.
@@ -432,8 +432,8 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, ContenderDetectorShutdownNetwork)
   ZooKeeperMasterContender contender(url.get());
 
   PID<Master> pid;
-  pid.ip = 10000000;
-  pid.port = 10000;
+  pid.node.ip = 10000000;
+  pid.node.port = 10000;
   MasterInfo master = internal::protobuf::createMasterInfo(pid);
 
   contender.initialize(master);
@@ -508,8 +508,8 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, MasterDetectorTimedoutSession)
   ZooKeeperMasterContender leaderContender(leaderGroup);
 
   PID<Master> pid;
-  pid.ip = 10000000;
-  pid.port = 10000;
+  pid.node.ip = 10000000;
+  pid.node.port = 10000;
   MasterInfo leader = internal::protobuf::createMasterInfo(pid);
 
   leaderContender.initialize(leader);
@@ -529,8 +529,8 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, MasterDetectorTimedoutSession)
   ZooKeeperMasterContender followerContender(followerGroup);
 
   PID<Master> pid2;
-  pid2.ip = 10000001;
-  pid2.port = 10001;
+  pid2.node.ip = 10000001;
+  pid2.node.port = 10001;
   MasterInfo follower = internal::protobuf::createMasterInfo(pid2);
 
   followerContender.initialize(follower);
@@ -618,8 +618,8 @@ TEST_F(ZooKeeperMasterContenderDetectorTest,
   ASSERT_SOME(url);
 
   PID<Master> pid;
-  pid.ip = 10000000;
-  pid.port = 10000;
+  pid.node.ip = 10000000;
+  pid.node.port = 10000;
   MasterInfo leader = internal::protobuf::createMasterInfo(pid);
 
   // Create the group instance so we can expire its session.
@@ -647,8 +647,8 @@ TEST_F(ZooKeeperMasterContenderDetectorTest,
 
   // Simulate a following master.
   PID<Master> pid2;
-  pid2.ip = 10000001;
-  pid2.port = 10001;
+  pid2.node.ip = 10000001;
+  pid2.node.port = 10001;
   MasterInfo follower = internal::protobuf::createMasterInfo(pid2);
 
   ZooKeeperMasterDetector followerDetector(url.get());
@@ -694,8 +694,8 @@ TEST_F(ZooKeeperMasterContenderDetectorTest, MasterDetectorExpireSlaveZKSession)
   ASSERT_SOME(url);
 
   PID<Master> pid;
-  pid.ip = 10000000;
-  pid.port = 10000;
+  pid.node.ip = 10000000;
+  pid.node.port = 10000;
   MasterInfo master = internal::protobuf::createMasterInfo(pid);
 
   ZooKeeperMasterContender masterContender(url.get());
@@ -755,8 +755,8 @@ TEST_F(ZooKeeperMasterContenderDetectorTest,
   ZooKeeperMasterDetector leaderDetector(leaderGroup);
 
   PID<Master> pid;
-  pid.ip = 10000000;
-  pid.port = 10000;
+  pid.node.ip = 10000000;
+  pid.node.port = 10000;
   MasterInfo leader = internal::protobuf::createMasterInfo(pid);
 
   leaderContender.initialize(leader);
@@ -775,8 +775,8 @@ TEST_F(ZooKeeperMasterContenderDetectorTest,
   ZooKeeperMasterDetector followerDetector(followerGroup);
 
   PID<Master> pid2;
-  pid2.ip = 10000001;
-  pid2.port = 10001;
+  pid2.node.ip = 10000001;
+  pid2.node.port = 10001;
   MasterInfo follower = internal::protobuf::createMasterInfo(pid2);
 
   followerContender.initialize(follower);

http://git-wip-us.apache.org/repos/asf/mesos/blob/76bfb493/src/tests/master_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_tests.cpp b/src/tests/master_tests.cpp
index a6d1a4a..66423a9 100644
--- a/src/tests/master_tests.cpp
+++ b/src/tests/master_tests.cpp
@@ -964,8 +964,8 @@ TEST_F(MasterTest, MasterInfo)
   driver.start();
 
   AWAIT_READY(masterInfo);
-  EXPECT_EQ(master.get().port, masterInfo.get().port());
-  EXPECT_EQ(master.get().ip, masterInfo.get().ip());
+  EXPECT_EQ(master.get().node.port, masterInfo.get().port());
+  EXPECT_EQ(master.get().node.ip, masterInfo.get().ip());
 
   driver.stop();
   driver.join();
@@ -1022,8 +1022,8 @@ TEST_F(MasterTest, MasterInfoOnReElection)
   AWAIT_READY(disconnected);
 
   AWAIT_READY(masterInfo);
-  EXPECT_EQ(master.get().port, masterInfo.get().port());
-  EXPECT_EQ(master.get().ip, masterInfo.get().ip());
+  EXPECT_EQ(master.get().node.port, masterInfo.get().port());
+  EXPECT_EQ(master.get().node.ip, masterInfo.get().ip());
 
   // The re-registered framework should get offers.
   AWAIT_READY(resourceOffers2);
@@ -2138,8 +2138,8 @@ TEST_F(MasterTest, MaxExecutorsPerSlave)
   driver.start();
 
   AWAIT_READY(masterInfo);
-  EXPECT_EQ(master.get().port, masterInfo.get().port());
-  EXPECT_EQ(master.get().ip, masterInfo.get().ip());
+  EXPECT_EQ(master.get().node.port, masterInfo.get().port());
+  EXPECT_EQ(master.get().node.ip, masterInfo.get().ip());
 
   driver.stop();
   driver.join();

http://git-wip-us.apache.org/repos/asf/mesos/blob/76bfb493/src/tests/monitor_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/monitor_tests.cpp b/src/tests/monitor_tests.cpp
index 4b950e1..3b02619 100644
--- a/src/tests/monitor_tests.cpp
+++ b/src/tests/monitor_tests.cpp
@@ -219,7 +219,7 @@ TEST(MonitorTest, Statistics)
   // Now wait for ResouorceMonitorProcess::watch to finish.
   process::Clock::settle();
 
-  process::UPID upid("monitor", process::ip(), process::port());
+  process::UPID upid("monitor", process::node());
 
   // Request the statistics, this will ask the isolator.
   Future<Response> response = process::http::get(upid, "statistics.json");