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 2011/06/05 11:07:14 UTC

svn commit: r1132248 - in /incubator/mesos/trunk: src/master/ src/messaging/ src/sched/ src/slave/ src/tests/ third_party/protobuf-2.3.0/ third_party/protobuf-2.3.0/src/

Author: benh
Date: Sun Jun  5 09:07:13 2011
New Revision: 1132248

URL: http://svn.apache.org/viewvc?rev=1132248&view=rev
Log:
Removed dependency on ReliableProcess, now we do status update acknowledgements manually.

Modified:
    incubator/mesos/trunk/src/master/master.cpp
    incubator/mesos/trunk/src/messaging/messages.hpp
    incubator/mesos/trunk/src/messaging/messages.proto
    incubator/mesos/trunk/src/sched/sched.cpp
    incubator/mesos/trunk/src/slave/slave.cpp
    incubator/mesos/trunk/src/slave/slave.hpp
    incubator/mesos/trunk/src/tests/master_test.cpp
    incubator/mesos/trunk/third_party/protobuf-2.3.0/Makefile.in
    incubator/mesos/trunk/third_party/protobuf-2.3.0/aclocal.m4
    incubator/mesos/trunk/third_party/protobuf-2.3.0/src/Makefile.in

Modified: incubator/mesos/trunk/src/master/master.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/master.cpp?rev=1132248&r1=1132247&r2=1132248&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/master.cpp (original)
+++ incubator/mesos/trunk/src/master/master.cpp Sun Jun  5 09:07:13 2011
@@ -596,6 +596,23 @@ void Master::operator () ()
       break;
     }
 
+    case F2M_STATUS_UPDATE_ACK: {
+      const Message<F2M_STATUS_UPDATE_ACK>& msg = message();
+
+      Framework *framework = lookupFramework(msg.framework_id());
+      if (framework != NULL) {
+        Slave *slave = lookupSlave(msg.slave_id());
+        if (slave != NULL) {
+          Message<M2S_STATUS_UPDATE_ACK> out;
+          out.mutable_framework_id()->MergeFrom(msg.framework_id());
+          out.mutable_slave_id()->MergeFrom(msg.slave_id());
+          out.mutable_task_id()->MergeFrom(msg.task_id());
+          send(slave->pid, out);
+        }
+      }
+      break;
+    }
+
     case S2M_REGISTER_SLAVE: {
       const Message<S2M_REGISTER_SLAVE>& msg = message();
 
@@ -667,7 +684,7 @@ void Master::operator () ()
 
       // TODO(benh|alig): We should put a timeout on how long we keep
       // tasks running that never have frameworks reregister that
-      // claim them.
+      // claim them.a
       break;
     }
 
@@ -702,20 +719,13 @@ void Master::operator () ()
           Message<M2F_STATUS_UPDATE> out;
           out.mutable_framework_id()->MergeFrom(msg.framework_id());
           out.mutable_status()->MergeFrom(status);
-          forward(framework->pid, out);
-
-          // No need to reprocess this message if already seen.
-          if (duplicate()) {
-            LOG(WARNING) << "Ignoring duplicate message with sequence: "
-                         << seq();
-            break;
-          }
+          send(framework->pid, out);
 
-          // Update the task state locally.
+          // Lookup the task and see if we need to update anything locally.
           Task *task = slave->lookupTask(msg.framework_id(), status.task_id());
           if (task != NULL) {
             task->set_state(status.state());
-            // Remove the task if it finished or failed.
+            // Remove the task if necessary.
             if (status.state() == TASK_FINISHED ||
                 status.state() == TASK_FAILED ||
                 status.state() == TASK_KILLED ||
@@ -723,16 +733,16 @@ void Master::operator () ()
               removeTask(task, TRR_TASK_ENDED);
             }
           } else {
-	    LOG(ERROR) << "Status update error: couldn't lookup "
-		       << "task " << status.task_id();
+	    LOG(WARNING) << "Status update error: couldn't lookup "
+                         << "task " << status.task_id();
 	  }
         } else {
-          LOG(ERROR) << "Status update error: couldn't lookup "
-                     << "framework " << msg.framework_id();
+          LOG(WARNING) << "Status update error: couldn't lookup "
+                       << "framework " << msg.framework_id();
         }
       } else {
-        LOG(ERROR) << "Status update error: couldn't lookup slave "
-                   << status.slave_id();
+        LOG(WARNING) << "Status update error: couldn't lookup slave "
+                     << status.slave_id();
       }
       break;
     }

Modified: incubator/mesos/trunk/src/messaging/messages.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/messaging/messages.hpp?rev=1132248&r1=1132247&r2=1132248&view=diff
==============================================================================
--- incubator/mesos/trunk/src/messaging/messages.hpp (original)
+++ incubator/mesos/trunk/src/messaging/messages.hpp Sun Jun  5 09:07:13 2011
@@ -6,7 +6,7 @@
 #include <string>
 
 #include <mesos.hpp>
-#include <reliable.hpp>
+#include <process.hpp>
 
 #include "messaging/messages.pb.h"
 
@@ -15,12 +15,12 @@ namespace mesos { namespace internal {
 
 
 // TODO(benh): Eliminate versioning once message ids become strings.
-const std::string MESOS_MESSAGING_VERSION = "1";
+const std::string MESOS_MESSAGING_VERSION = "2";
 
 
 enum MessageType {
   /* From framework to master. */
-  F2M_REGISTER_FRAMEWORK = RELIABLE_MSGID,
+  F2M_REGISTER_FRAMEWORK = PROCESS_MSGID,
   F2M_REREGISTER_FRAMEWORK,
   F2M_UNREGISTER_FRAMEWORK,
   F2M_RESOURCE_OFFER_REPLY,
@@ -104,7 +104,10 @@ enum MessageType {
   /* Generic. */
   TERMINATE,
 
-  MASTER_DETECTION_FAILURE, // TODO(benh): Put this in it's proper place.
+  // TODO(benh): Put these all in their right place.
+  MASTER_DETECTION_FAILURE, 
+  F2M_STATUS_UPDATE_ACK,
+  M2S_STATUS_UPDATE_ACK,
 
   MESOS_MSGID,
 };
@@ -150,13 +153,13 @@ private:
 };
 
 
-class MesosProcess : public ReliableProcess
+class MesosProcess : public Process
 {
 public:
   static void post(const PID &to, MSGID id)
   {
     const std::string &data = MESOS_MESSAGING_VERSION + "|";
-    ReliableProcess::post(to, id, data.data(), data.size());
+    Process::post(to, id, data.data(), data.size());
   }
 
   template <MSGID ID>
@@ -165,7 +168,7 @@ public:
     std::string data;
     msg.SerializeToString(&data);
     data = MESOS_MESSAGING_VERSION + "|" + data;
-    ReliableProcess::post(to, ID, data.data(), data.size());
+    Process::post(to, ID, data.data(), data.size());
   }
 
 protected:
@@ -177,7 +180,7 @@ protected:
   std::string body() const
   {
     size_t size;
-    const char *s = ReliableProcess::body(&size);
+    const char *s = Process::body(&size);
     const std::string data(s, size);
     size_t index = data.find('|');
     CHECK(index != std::string::npos);
@@ -187,7 +190,7 @@ protected:
   virtual void send(const PID &to, MSGID id)
   {
     const std::string &data = MESOS_MESSAGING_VERSION + "|";
-    ReliableProcess::send(to, id, data.data(), data.size());
+    Process::send(to, id, data.data(), data.size());
   }
 
   template <MSGID ID>
@@ -196,44 +199,17 @@ protected:
     std::string data;
     msg.SerializeToString(&data);
     data = MESOS_MESSAGING_VERSION + "|" + data;
-    ReliableProcess::send(to, ID, data.data(), data.size());
-  }
-
-  template <MSGID ID>
-  bool forward(const PID &to, const Message<ID> &msg)
-  {
-    std::string data;
-    msg.SerializeToString(&data);
-    data = MESOS_MESSAGING_VERSION + "|" + data;
-    ReliableProcess::forward(to, ID, data.data(), data.size());
-  }
-
-  template <MSGID ID>
-  int rsend(const PID &to, const Message<ID> &msg)
-  {
-    std::string data;
-    msg.SerializeToString(&data);
-    data = MESOS_MESSAGING_VERSION + "|" + data;
-    return ReliableProcess::rsend(to, ID, data.data(), data.size());
-  }
-
-  template <MSGID ID>
-  int rsend(const PID &via, const PID &to, const Message<ID> &msg)
-  {
-    std::string data;
-    msg.SerializeToString(&data);
-    data = MESOS_MESSAGING_VERSION + "|" + data;
-    return ReliableProcess::rsend(via, to, ID, data.data(), data.size());
+    Process::send(to, ID, data.data(), data.size());
   }
 
   virtual MSGID receive(double secs = 0)
   {
     bool indefinite = secs == 0;
     double now = elapsed();
-    MSGID id = ReliableProcess::receive(secs);
-    if (RELIABLE_MSGID < id && id < MESOS_MSGID) {
+    MSGID id = Process::receive(secs);
+    if (PROCESS_MSGID < id && id < MESOS_MSGID) {
       size_t size;
-      const char *s = ReliableProcess::body(&size);
+      const char *s = Process::body(&size);
       const std::string data(s, size);
       size_t index = data.find('|');
       if (index == std::string::npos ||
@@ -260,6 +236,7 @@ MESSAGE(F2M_RESOURCE_OFFER_REPLY, Resour
 MESSAGE(F2M_REVIVE_OFFERS, ReviveOffersMessage);
 MESSAGE(F2M_KILL_TASK, KillTaskMessage);
 MESSAGE(F2M_FRAMEWORK_MESSAGE, FrameworkMessageMessage);
+MESSAGE(F2M_STATUS_UPDATE_ACK, StatusUpdateAckMessage);
 
 MESSAGE(M2F_REGISTER_REPLY, FrameworkRegisteredMessage);
 MESSAGE(M2F_RESOURCE_OFFER, ResourceOfferMessage);
@@ -285,6 +262,7 @@ MESSAGE(M2S_KILL_TASK, KillTaskMessage);
 MESSAGE(M2S_KILL_FRAMEWORK, KillFrameworkMessage);
 MESSAGE(M2S_FRAMEWORK_MESSAGE, FrameworkMessageMessage);
 MESSAGE(M2S_UPDATE_FRAMEWORK, UpdateFrameworkMessage);
+MESSAGE(M2S_STATUS_UPDATE_ACK, StatusUpdateAckMessage);
 
 MESSAGE(E2S_REGISTER_EXECUTOR, RegisterExecutorMessage);
 MESSAGE(E2S_STATUS_UPDATE, StatusUpdateMessage);

Modified: incubator/mesos/trunk/src/messaging/messages.proto
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/messaging/messages.proto?rev=1132248&r1=1132247&r2=1132248&view=diff
==============================================================================
--- incubator/mesos/trunk/src/messaging/messages.proto (original)
+++ incubator/mesos/trunk/src/messaging/messages.proto Sun Jun  5 09:07:13 2011
@@ -99,6 +99,13 @@ message StatusUpdateMessage {
 }
 
 
+message StatusUpdateAckMessage {
+  required FrameworkID framework_id = 1;
+  required SlaveID slave_id = 2;
+  required TaskID task_id = 3;
+}
+
+
 message LostSlaveMessage {
   required SlaveID slave_id = 1;
 }

Modified: incubator/mesos/trunk/src/sched/sched.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/sched/sched.cpp?rev=1132248&r1=1132247&r2=1132248&view=diff
==============================================================================
--- incubator/mesos/trunk/src/sched/sched.cpp (original)
+++ incubator/mesos/trunk/src/sched/sched.cpp Sun Jun  5 09:07:13 2011
@@ -19,6 +19,7 @@
 
 #include <boost/bind.hpp>
 #include <boost/unordered_map.hpp>
+#include <boost/unordered_set.hpp>
 
 #include "configurator/configuration.hpp"
 
@@ -43,6 +44,7 @@ using std::vector;
 using boost::bind;
 using boost::cref;
 using boost::unordered_map;
+using boost::unordered_set;
 
 
 namespace mesos { namespace internal {
@@ -158,7 +160,6 @@ protected:
 
         VLOG(1) << "New master at " << msg.pid();
 
-        redirect(master, msg.pid());
         master = msg.pid();
         link(master);
 
@@ -239,12 +240,14 @@ protected:
 
         const TaskStatus &status = msg.status();
 
-        // Drop this if it's a duplicate.
-        if (duplicate()) {
-          VLOG(1) << "Received a duplicate status update for task "
-                  << status.task_id() << ", status = " << status.state();
-          break;
-        }
+        // TODO(benh): Note that this maybe a duplicate status update!
+        // Once we get support to try and have a more consistent view
+        // of what's running in the cluster, we'll just let this one
+        // slide. The alternative is possibly dealing with a scheduler
+        // failover and not correctly giving the scheduler it's status
+        // update, which seems worse than giving a status update
+        // multiple times (of course, if a scheduler re-uses a TaskID,
+        // that could be bad.
 
         // Stop any status update timers we might have had running.
         if (timers.count(status.task_id()) > 0) {
@@ -257,11 +260,15 @@ protected:
 
         invoke(bind(&Scheduler::statusUpdate, sched, driver, cref(status)));
 
-        // Acknowledge the message (we do this after we invoke the
-        // scheduler in case it causes a crash, since this way the
-        // message might get resent/routed after the scheduler comes
-        // back online).
-        ack();
+        // Acknowledge the message (we do this last, after we invoked
+        // the scheduler, if we did at all, in case it causes a crash,
+        // since this way the message might get resent/routed after
+        // the scheduler comes back online).
+        Message<F2M_STATUS_UPDATE_ACK> out;
+        out.mutable_framework_id()->MergeFrom(frameworkId);
+        out.mutable_slave_id()->MergeFrom(status.slave_id());
+        out.mutable_task_id()->MergeFrom(status.task_id());
+        send(master, out);
         break;
       }
 
@@ -425,6 +432,8 @@ private:
   unordered_map<OfferID, unordered_map<SlaveID, PID> > savedOffers;
   unordered_map<SlaveID, PID> savedSlavePids;
 
+  unordered_set<TaskID> tasks;
+
   // Timers to ensure we get a status update for each task we launch.
   unordered_map<TaskID, StatusUpdateTimer *> timers;
 };

Modified: incubator/mesos/trunk/src/slave/slave.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.cpp?rev=1132248&r1=1132247&r2=1132248&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.cpp (original)
+++ incubator/mesos/trunk/src/slave/slave.cpp Sun Jun  5 09:07:13 2011
@@ -163,13 +163,12 @@ void Slave::operator () ()
   isolationModule->initialize(this);
 
   while (true) {
-    switch (receive()) {
+    switch (receive(1)) {
       case NEW_MASTER_DETECTED: {
         const Message<NEW_MASTER_DETECTED>& msg = message();
 
 	LOG(INFO) << "New master at " << msg.pid();
 
-        redirect(master, msg.pid());
 	master = msg.pid();
 	link(master);
 
@@ -315,9 +314,10 @@ void Slave::operator () ()
             status->mutable_task_id()->MergeFrom(msg.task_id());
             status->mutable_slave_id()->MergeFrom(slaveId);
             status->set_state(TASK_LOST);
+            send(master, out);
 
-            int seq = rsend(master, framework->pid, out);
-            seqs[msg.framework_id()].insert(seq);
+            double deadline = elapsed() + STATUS_UPDATE_RETRY_TIMEOUT;
+            framework->statuses[deadline][status->task_id()] = *status;
           } else {
             // Otherwise, send a message to the executor and wait for
             // it to send us a status update.
@@ -337,9 +337,10 @@ void Slave::operator () ()
           status->mutable_task_id()->MergeFrom(msg.task_id());
           status->mutable_slave_id()->MergeFrom(slaveId);
           status->set_state(TASK_LOST);
+          send(master, out);
 
-          int seq = rsend(master, out);
-          seqs[msg.framework_id()].insert(seq);
+          double deadline = elapsed() + STATUS_UPDATE_RETRY_TIMEOUT;
+          framework->statuses[deadline][status->task_id()] = *status;
         }
         break;
       }
@@ -402,6 +403,24 @@ void Slave::operator () ()
         break;
       }
 
+      case M2S_STATUS_UPDATE_ACK: {
+        const Message<M2S_STATUS_UPDATE_ACK>& msg = message();
+
+        Framework* framework = getFramework(msg.framework_id());
+        if (framework != NULL) {
+          foreachpair (double deadline, _, framework->statuses) {
+            if (framework->statuses[deadline].count(msg.task_id()) > 0) {
+              LOG(INFO) << "Got acknowledgement of status update"
+                        << " for task " << msg.task_id()
+                        << " of framework " << framework->frameworkId;
+              framework->statuses[deadline].erase(msg.task_id());
+              break;
+            }
+          }
+        }
+        break;
+      }
+
       case E2S_REGISTER_EXECUTOR: {
         const Message<E2S_REGISTER_EXECUTOR>& msg = message();
 
@@ -476,13 +495,14 @@ void Slave::operator () ()
               isolationModule->resourcesChanged(framework, executor);
             }
 
-            // Reliably send message and save sequence number for
-            // canceling later.
+            // Send message and record the status for possible resending.
             Message<S2M_STATUS_UPDATE> out;
             out.mutable_framework_id()->MergeFrom(msg.framework_id());
             out.mutable_status()->MergeFrom(status);
-            int seq = rsend(master, framework->pid, out);
-            seqs[msg.framework_id()].insert(seq);
+            send(master, out);
+
+            double deadline = elapsed() + STATUS_UPDATE_RETRY_TIMEOUT;
+            framework->statuses[deadline][status.task_id()] = status;
           } else {
             LOG(WARNING) << "Status update error: couldn't lookup "
                          << "executor for framework " << msg.framework_id();
@@ -523,17 +543,6 @@ void Slave::operator () ()
         break;
       }
 
-      case PROCESS_EXIT: {
-        LOG(INFO) << "Process exited: " << from();
-
-        if (from() == master) {
-	  LOG(WARNING) << "Master disconnected! "
-		       << "Waiting for a new master to be elected.";
-	  // TODO(benh): After so long waiting for a master, commit suicide.
-	}
-        break;
-      }
-
       case M2S_SHUTDOWN: {
         LOG(INFO) << "Asked to shut down by master: " << from();
         foreachpaircopy (_, Framework *framework, frameworks) {
@@ -550,6 +559,37 @@ void Slave::operator () ()
         return;
       }
 
+      case PROCESS_EXIT: {
+        LOG(INFO) << "Process exited: " << from();
+
+        if (from() == master) {
+	  LOG(WARNING) << "Master disconnected! "
+		       << "Waiting for a new master to be elected.";
+	  // TODO(benh): After so long waiting for a master, commit suicide.
+	}
+        break;
+      }
+
+      case PROCESS_TIMEOUT: {
+        // Check and see if we should re-send any status updates.
+        foreachpair (_, Framework* framework, frameworks) {
+          foreachpair (double deadline, _, framework->statuses) {
+            if (deadline <= elapsed()) {
+              foreachpair (_, const TaskStatus& status, framework->statuses[deadline]) {
+                LOG(WARNING) << "Resending status update"
+                             << " for task " << status.task_id()
+                             << " of framework " << framework->frameworkId;
+                Message<S2M_STATUS_UPDATE> out;
+                out.mutable_framework_id()->MergeFrom(framework->frameworkId);
+                out.mutable_status()->MergeFrom(status);
+                send(master, out);
+              }
+            }
+          }
+        }
+        break;
+      }
+
       default: {
         LOG(ERROR) << "Received unknown message (" << msgid()
                    << ") from " << from();
@@ -600,13 +640,6 @@ void Slave::killFramework(Framework *fra
 {
   LOG(INFO) << "Cleaning up framework " << framework->frameworkId;
 
-  // Cancel sending any reliable messages for this framework.
-  foreach (int seq, seqs[framework->frameworkId]) {
-    cancel(seq);
-  }
-
-  seqs.erase(framework->frameworkId);
-
   // Shutdown all executors of this framework.
   foreachpaircopy (const ExecutorID& executorId, Executor* executor, framework->executors) {
     if (killExecutors) {

Modified: incubator/mesos/trunk/src/slave/slave.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.hpp?rev=1132248&r1=1132247&r2=1132248&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.hpp (original)
+++ incubator/mesos/trunk/src/slave/slave.hpp Sun Jun  5 09:07:13 2011
@@ -12,6 +12,7 @@
 #include <iostream>
 #include <list>
 #include <sstream>
+#include <set>
 #include <vector>
 
 #include <arpa/inet.h>
@@ -48,6 +49,9 @@ namespace mesos { namespace internal { n
 using foreach::_;
 
 
+const double STATUS_UPDATE_RETRY_TIMEOUT = 10;
+
+
 // Information describing an executor (goes away if executor crashes).
 struct Executor
 {
@@ -171,6 +175,7 @@ struct Framework
   PID pid;
 
   boost::unordered_map<ExecutorID, Executor*> executors;
+  boost::unordered_map<double, boost::unordered_map<TaskID, TaskStatus> > statuses;
 };
 
 
@@ -258,9 +263,6 @@ private:
   // Invariant: framework will exist if executor exists.
   boost::unordered_map<FrameworkID, Framework*> frameworks;
 
-  // Sequence numbers of reliable messages sent per framework.
-  boost::unordered_map<FrameworkID, boost::unordered_set<int> > seqs;
-
   IsolationModule *isolationModule;
   Heart* heart;
 };

Modified: incubator/mesos/trunk/src/tests/master_test.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/master_test.cpp?rev=1132248&r1=1132247&r2=1132248&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/master_test.cpp (original)
+++ incubator/mesos/trunk/src/tests/master_test.cpp Sun Jun  5 09:07:13 2011
@@ -25,6 +25,7 @@ using mesos::internal::master::Master;
 
 using mesos::internal::slave::Slave;
 using mesos::internal::slave::ProcessBasedIsolationModule;
+using mesos::internal::slave::STATUS_UPDATE_RETRY_TIMEOUT;
 
 using std::string;
 using std::map;
@@ -822,7 +823,7 @@ TEST(MasterTest, SchedulerFailoverStatus
 
   WAIT_UNTIL(registeredCall);
 
-  Clock::advance(RELIABLE_TIMEOUT);
+  Clock::advance(STATUS_UPDATE_RETRY_TIMEOUT);
 
   WAIT_UNTIL(statusUpdateCall);
 

Modified: incubator/mesos/trunk/third_party/protobuf-2.3.0/Makefile.in
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/protobuf-2.3.0/Makefile.in?rev=1132248&r1=1132247&r2=1132248&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/protobuf-2.3.0/Makefile.in (original)
+++ incubator/mesos/trunk/third_party/protobuf-2.3.0/Makefile.in Sun Jun  5 09:07:13 2011
@@ -1,8 +1,8 @@
-# Makefile.in generated by automake 1.10.1 from Makefile.am.
+# Makefile.in generated by automake 1.10 from Makefile.am.
 # @configure_input@
 
 # Copyright (C) 1994, 1995, 1996, 1997, 1998, 1999, 2000, 2001, 2002,
-# 2003, 2004, 2005, 2006, 2007, 2008  Free Software Foundation, Inc.
+# 2003, 2004, 2005, 2006  Free Software Foundation, Inc.
 # This Makefile.in is free software; the Free Software Foundation
 # gives unlimited permission to copy and/or distribute it,
 # with or without modifications, as long as this notice is preserved.
@@ -511,8 +511,8 @@ ID: $(HEADERS) $(SOURCES) $(LISP) $(TAGS
 	unique=`for i in $$list; do \
 	    if test -f "$$i"; then echo $$i; else echo $(srcdir)/$$i; fi; \
 	  done | \
-	  $(AWK) '{ files[$$0] = 1; nonemtpy = 1; } \
-	      END { if (nonempty) { for (i in files) print i; }; }'`; \
+	  $(AWK) '    { files[$$0] = 1; } \
+	       END { for (i in files) print i; }'`; \
 	mkid -fID $$unique
 tags: TAGS
 
@@ -537,8 +537,8 @@ TAGS: tags-recursive $(HEADERS) $(SOURCE
 	unique=`for i in $$list; do \
 	    if test -f "$$i"; then echo $$i; else echo $(srcdir)/$$i; fi; \
 	  done | \
-	  $(AWK) '{ files[$$0] = 1; nonempty = 1; } \
-	      END { if (nonempty) { for (i in files) print i; }; }'`; \
+	  $(AWK) '    { files[$$0] = 1; } \
+	       END { for (i in files) print i; }'`; \
 	if test -z "$(ETAGS_ARGS)$$tags$$unique"; then :; else \
 	  test -n "$$unique" || unique=$$empty_fix; \
 	  $(ETAGS) $(ETAGSFLAGS) $(AM_ETAGSFLAGS) $(ETAGS_ARGS) \
@@ -548,12 +548,13 @@ ctags: CTAGS
 CTAGS: ctags-recursive $(HEADERS) $(SOURCES) config.h.in $(TAGS_DEPENDENCIES) \
 		$(TAGS_FILES) $(LISP)
 	tags=; \
+	here=`pwd`; \
 	list='$(SOURCES) $(HEADERS) config.h.in $(LISP) $(TAGS_FILES)'; \
 	unique=`for i in $$list; do \
 	    if test -f "$$i"; then echo $$i; else echo $(srcdir)/$$i; fi; \
 	  done | \
-	  $(AWK) '{ files[$$0] = 1; nonempty = 1; } \
-	      END { if (nonempty) { for (i in files) print i; }; }'`; \
+	  $(AWK) '    { files[$$0] = 1; } \
+	       END { for (i in files) print i; }'`; \
 	test -z "$(CTAGS_ARGS)$$tags$$unique" \
 	  || $(CTAGS) $(CTAGSFLAGS) $(AM_CTAGSFLAGS) $(CTAGS_ARGS) \
 	     $$tags $$unique
@@ -624,10 +625,6 @@ dist-bzip2: distdir
 	tardir=$(distdir) && $(am__tar) | bzip2 -9 -c >$(distdir).tar.bz2
 	$(am__remove_distdir)
 
-dist-lzma: distdir
-	tardir=$(distdir) && $(am__tar) | lzma -9 -c >$(distdir).tar.lzma
-	$(am__remove_distdir)
-
 dist-tarZ: distdir
 	tardir=$(distdir) && $(am__tar) | compress -c >$(distdir).tar.Z
 	$(am__remove_distdir)
@@ -654,8 +651,6 @@ distcheck: dist
 	  GZIP=$(GZIP_ENV) gunzip -c $(distdir).tar.gz | $(am__untar) ;;\
 	*.tar.bz2*) \
 	  bunzip2 -c $(distdir).tar.bz2 | $(am__untar) ;;\
-	*.tar.lzma*) \
-	  unlzma -c $(distdir).tar.lzma | $(am__untar) ;;\
 	*.tar.Z*) \
 	  uncompress -c $(distdir).tar.Z | $(am__untar) ;;\
 	*.shar.gz*) \
@@ -812,16 +807,16 @@ uninstall-am: uninstall-pkgconfigDATA
 .PHONY: $(RECURSIVE_CLEAN_TARGETS) $(RECURSIVE_TARGETS) CTAGS GTAGS \
 	all all-am am--refresh check check-am check-local clean \
 	clean-generic clean-libtool clean-local ctags ctags-recursive \
-	dist dist-all dist-bzip2 dist-gzip dist-lzma dist-shar \
-	dist-tarZ dist-zip distcheck distclean distclean-generic \
-	distclean-hdr distclean-libtool distclean-tags distcleancheck \
-	distdir distuninstallcheck dvi dvi-am html html-am info \
-	info-am install install-am install-data install-data-am \
-	install-dvi install-dvi-am install-exec install-exec-am \
-	install-html install-html-am install-info install-info-am \
-	install-man install-pdf install-pdf-am install-pkgconfigDATA \
-	install-ps install-ps-am install-strip installcheck \
-	installcheck-am installdirs installdirs-am maintainer-clean \
+	dist dist-all dist-bzip2 dist-gzip dist-shar dist-tarZ \
+	dist-zip distcheck distclean distclean-generic distclean-hdr \
+	distclean-libtool distclean-tags distcleancheck distdir \
+	distuninstallcheck dvi dvi-am html html-am info info-am \
+	install install-am install-data install-data-am install-dvi \
+	install-dvi-am install-exec install-exec-am install-html \
+	install-html-am install-info install-info-am install-man \
+	install-pdf install-pdf-am install-pkgconfigDATA install-ps \
+	install-ps-am install-strip installcheck installcheck-am \
+	installdirs installdirs-am maintainer-clean \
 	maintainer-clean-generic mostlyclean mostlyclean-generic \
 	mostlyclean-libtool pdf pdf-am ps ps-am tags tags-recursive \
 	uninstall uninstall-am uninstall-pkgconfigDATA

Modified: incubator/mesos/trunk/third_party/protobuf-2.3.0/aclocal.m4
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/protobuf-2.3.0/aclocal.m4?rev=1132248&r1=1132247&r2=1132248&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/protobuf-2.3.0/aclocal.m4 (original)
+++ incubator/mesos/trunk/third_party/protobuf-2.3.0/aclocal.m4 Sun Jun  5 09:07:13 2011
@@ -1,7 +1,7 @@
-# generated automatically by aclocal 1.10.1 -*- Autoconf -*-
+# generated automatically by aclocal 1.10 -*- Autoconf -*-
 
 # Copyright (C) 1996, 1997, 1998, 1999, 2000, 2001, 2002, 2003, 2004,
-# 2005, 2006, 2007, 2008  Free Software Foundation, Inc.
+# 2005, 2006  Free Software Foundation, Inc.
 # This file is free software; the Free Software Foundation
 # gives unlimited permission to copy and/or distribute it,
 # with or without modifications, as long as this notice is preserved.
@@ -11,15 +11,12 @@
 # even the implied warranty of MERCHANTABILITY or FITNESS FOR A
 # PARTICULAR PURPOSE.
 
-m4_ifndef([AC_AUTOCONF_VERSION],
-  [m4_copy([m4_PACKAGE_VERSION], [AC_AUTOCONF_VERSION])])dnl
-m4_if(AC_AUTOCONF_VERSION, [2.61],,
-[m4_warning([this file was generated for autoconf 2.61.
-You have another version of autoconf.  It may work, but is not guaranteed to.
-If you have problems, you may need to regenerate the build system entirely.
-To do so, use the procedure documented by the package, typically `autoreconf'.])])
+m4_if(m4_PACKAGE_VERSION, [2.61],,
+[m4_fatal([this file was generated for autoconf 2.61.
+You have another version of autoconf.  If you want to use that,
+you should regenerate the build system entirely.], [63])])
 
-# Copyright (C) 2002, 2003, 2005, 2006, 2007  Free Software Foundation, Inc.
+# Copyright (C) 2002, 2003, 2005, 2006  Free Software Foundation, Inc.
 #
 # This file is free software; the Free Software Foundation
 # gives unlimited permission to copy and/or distribute it,
@@ -34,7 +31,7 @@ AC_DEFUN([AM_AUTOMAKE_VERSION],
 [am__api_version='1.10'
 dnl Some users find AM_AUTOMAKE_VERSION and mistake it for a way to
 dnl require some minimum version.  Point them to the right macro.
-m4_if([$1], [1.10.1], [],
+m4_if([$1], [1.10], [],
       [AC_FATAL([Do not call $0, use AM_INIT_AUTOMAKE([$1]).])])dnl
 ])
 
@@ -50,10 +47,8 @@ m4_define([_AM_AUTOCONF_VERSION], [])
 # Call AM_AUTOMAKE_VERSION and AM_AUTOMAKE_VERSION so they can be traced.
 # This function is AC_REQUIREd by AC_INIT_AUTOMAKE.
 AC_DEFUN([AM_SET_CURRENT_AUTOMAKE_VERSION],
-[AM_AUTOMAKE_VERSION([1.10.1])dnl
-m4_ifndef([AC_AUTOCONF_VERSION],
-  [m4_copy([m4_PACKAGE_VERSION], [AC_AUTOCONF_VERSION])])dnl
-_AM_AUTOCONF_VERSION(AC_AUTOCONF_VERSION)])
+[AM_AUTOMAKE_VERSION([1.10])dnl
+_AM_AUTOCONF_VERSION(m4_PACKAGE_VERSION)])
 
 # AM_AUX_DIR_EXPAND                                         -*- Autoconf -*-
 
@@ -325,7 +320,7 @@ AC_DEFUN([_AM_OUTPUT_DEPENDENCY_COMMANDS
   # each Makefile.in and add a new line on top of each file to say so.
   # Grep'ing the whole file is not good either: AIX grep has a line
   # limit of 2048, but all sed's we know have understand at least 4000.
-  if sed -n 's,^#.*generated by automake.*,X,p' "$mf" | grep X >/dev/null 2>&1; then
+  if sed 10q "$mf" | grep '^#.*generated by automake' > /dev/null 2>&1; then
     dirpart=`AS_DIRNAME("$mf")`
   else
     continue
@@ -373,13 +368,13 @@ AC_DEFUN([AM_OUTPUT_DEPENDENCY_COMMANDS]
 # Do all the work for Automake.                             -*- Autoconf -*-
 
 # Copyright (C) 1996, 1997, 1998, 1999, 2000, 2001, 2002, 2003, 2004,
-# 2005, 2006, 2008 Free Software Foundation, Inc.
+# 2005, 2006 Free Software Foundation, Inc.
 #
 # This file is free software; the Free Software Foundation
 # gives unlimited permission to copy and/or distribute it,
 # with or without modifications, as long as this notice is preserved.
 
-# serial 13
+# serial 12
 
 # This macro actually does too much.  Some checks are only needed if
 # your package does certain things.  But this isn't really a big deal.
@@ -484,17 +479,16 @@ AC_PROVIDE_IFELSE([AC_PROG_OBJC],
 # our stamp files there.
 AC_DEFUN([_AC_AM_CONFIG_HEADER_HOOK],
 [# Compute $1's index in $config_headers.
-_am_arg=$1
 _am_stamp_count=1
 for _am_header in $config_headers :; do
   case $_am_header in
-    $_am_arg | $_am_arg:* )
+    $1 | $1:* )
       break ;;
     * )
       _am_stamp_count=`expr $_am_stamp_count + 1` ;;
   esac
 done
-echo "timestamp for $_am_arg" >`AS_DIRNAME(["$_am_arg"])`/stamp-h[]$_am_stamp_count])
+echo "timestamp for $1" >`AS_DIRNAME([$1])`/stamp-h[]$_am_stamp_count])
 
 # Copyright (C) 2001, 2003, 2005  Free Software Foundation, Inc.
 #
@@ -766,7 +760,7 @@ AC_SUBST([INSTALL_STRIP_PROGRAM])])
 
 # _AM_SUBST_NOTMAKE(VARIABLE)
 # ---------------------------
-# Prevent Automake from outputting VARIABLE = @VARIABLE@ in Makefile.in.
+# Prevent Automake from outputing VARIABLE = @VARIABLE@ in Makefile.in.
 # This macro is traced by Automake.
 AC_DEFUN([_AM_SUBST_NOTMAKE])
 

Modified: incubator/mesos/trunk/third_party/protobuf-2.3.0/src/Makefile.in
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/protobuf-2.3.0/src/Makefile.in?rev=1132248&r1=1132247&r2=1132248&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/protobuf-2.3.0/src/Makefile.in (original)
+++ incubator/mesos/trunk/third_party/protobuf-2.3.0/src/Makefile.in Sun Jun  5 09:07:13 2011
@@ -1,8 +1,8 @@
-# Makefile.in generated by automake 1.10.1 from Makefile.am.
+# Makefile.in generated by automake 1.10 from Makefile.am.
 # @configure_input@
 
 # Copyright (C) 1994, 1995, 1996, 1997, 1998, 1999, 2000, 2001, 2002,
-# 2003, 2004, 2005, 2006, 2007, 2008  Free Software Foundation, Inc.
+# 2003, 2004, 2005, 2006  Free Software Foundation, Inc.
 # This Makefile.in is free software; the Free Software Foundation
 # gives unlimited permission to copy and/or distribute it,
 # with or without modifications, as long as this notice is preserved.
@@ -231,7 +231,7 @@ am__zcgzip_SOURCES_DIST = google/protobu
 zcgzip_OBJECTS = $(am_zcgzip_OBJECTS)
 @HAVE_ZLIB_TRUE@zcgzip_DEPENDENCIES = $(am__DEPENDENCIES_1) \
 @HAVE_ZLIB_TRUE@	libprotobuf.la
-DEFAULT_INCLUDES = -I.@am__isrc@ -I$(top_builddir)
+DEFAULT_INCLUDES = -I. -I$(top_builddir)@am__isrc@
 depcomp = $(SHELL) $(top_srcdir)/depcomp
 am__depfiles_maybe = depfiles
 CXXCOMPILE = $(CXX) $(DEFS) $(DEFAULT_INCLUDES) $(INCLUDES) \
@@ -771,9 +771,9 @@ $(srcdir)/Makefile.in:  $(srcdir)/Makefi
 	      exit 1;; \
 	  esac; \
 	done; \
-	echo ' cd $(top_srcdir) && $(AUTOMAKE) --gnu  src/Makefile'; \
+	echo ' cd $(top_srcdir) && $(AUTOMAKE) --foreign  src/Makefile'; \
 	cd $(top_srcdir) && \
-	  $(AUTOMAKE) --gnu  src/Makefile
+	  $(AUTOMAKE) --foreign  src/Makefile
 .PRECIOUS: Makefile
 Makefile: $(srcdir)/Makefile.in $(top_builddir)/config.status
 	@case '$?' in \
@@ -797,8 +797,8 @@ install-libLTLIBRARIES: $(lib_LTLIBRARIE
 	@list='$(lib_LTLIBRARIES)'; for p in $$list; do \
 	  if test -f $$p; then \
 	    f=$(am__strip_dir) \
-	    echo " $(LIBTOOL) $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) --mode=install $(libLTLIBRARIES_INSTALL) $(INSTALL_STRIP_FLAG) '$$p' '$(DESTDIR)$(libdir)/$$f'"; \
-	    $(LIBTOOL) $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) --mode=install $(libLTLIBRARIES_INSTALL) $(INSTALL_STRIP_FLAG) "$$p" "$(DESTDIR)$(libdir)/$$f"; \
+	    echo " $(LIBTOOL) --mode=install $(libLTLIBRARIES_INSTALL) $(INSTALL_STRIP_FLAG) '$$p' '$(DESTDIR)$(libdir)/$$f'"; \
+	    $(LIBTOOL) --mode=install $(libLTLIBRARIES_INSTALL) $(INSTALL_STRIP_FLAG) "$$p" "$(DESTDIR)$(libdir)/$$f"; \
 	  else :; fi; \
 	done
 
@@ -806,8 +806,8 @@ uninstall-libLTLIBRARIES:
 	@$(NORMAL_UNINSTALL)
 	@list='$(lib_LTLIBRARIES)'; for p in $$list; do \
 	  p=$(am__strip_dir) \
-	  echo " $(LIBTOOL) $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) --mode=uninstall rm -f '$(DESTDIR)$(libdir)/$$p'"; \
-	  $(LIBTOOL) $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) --mode=uninstall rm -f "$(DESTDIR)$(libdir)/$$p"; \
+	  echo " $(LIBTOOL) --mode=uninstall rm -f '$(DESTDIR)$(libdir)/$$p'"; \
+	  $(LIBTOOL) --mode=uninstall rm -f "$(DESTDIR)$(libdir)/$$p"; \
 	done
 
 clean-libLTLIBRARIES:
@@ -833,8 +833,8 @@ install-binPROGRAMS: $(bin_PROGRAMS)
 	     || test -f $$p1 \
 	  ; then \
 	    f=`echo "$$p1" | sed 's,^.*/,,;$(transform);s/$$/$(EXEEXT)/'`; \
-	   echo " $(INSTALL_PROGRAM_ENV) $(LIBTOOL) $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) --mode=install $(binPROGRAMS_INSTALL) '$$p' '$(DESTDIR)$(bindir)/$$f'"; \
-	   $(INSTALL_PROGRAM_ENV) $(LIBTOOL) $(AM_LIBTOOLFLAGS) $(LIBTOOLFLAGS) --mode=install $(binPROGRAMS_INSTALL) "$$p" "$(DESTDIR)$(bindir)/$$f" || exit 1; \
+	   echo " $(INSTALL_PROGRAM_ENV) $(LIBTOOL) --mode=install $(binPROGRAMS_INSTALL) '$$p' '$(DESTDIR)$(bindir)/$$f'"; \
+	   $(INSTALL_PROGRAM_ENV) $(LIBTOOL) --mode=install $(binPROGRAMS_INSTALL) "$$p" "$(DESTDIR)$(bindir)/$$f" || exit 1; \
 	  else :; fi; \
 	done
 
@@ -2489,8 +2489,8 @@ ID: $(HEADERS) $(SOURCES) $(LISP) $(TAGS
 	unique=`for i in $$list; do \
 	    if test -f "$$i"; then echo $$i; else echo $(srcdir)/$$i; fi; \
 	  done | \
-	  $(AWK) '{ files[$$0] = 1; nonemtpy = 1; } \
-	      END { if (nonempty) { for (i in files) print i; }; }'`; \
+	  $(AWK) '    { files[$$0] = 1; } \
+	       END { for (i in files) print i; }'`; \
 	mkid -fID $$unique
 tags: TAGS
 
@@ -2502,8 +2502,8 @@ TAGS:  $(HEADERS) $(SOURCES)  $(TAGS_DEP
 	unique=`for i in $$list; do \
 	    if test -f "$$i"; then echo $$i; else echo $(srcdir)/$$i; fi; \
 	  done | \
-	  $(AWK) '{ files[$$0] = 1; nonempty = 1; } \
-	      END { if (nonempty) { for (i in files) print i; }; }'`; \
+	  $(AWK) '    { files[$$0] = 1; } \
+	       END { for (i in files) print i; }'`; \
 	if test -z "$(ETAGS_ARGS)$$tags$$unique"; then :; else \
 	  test -n "$$unique" || unique=$$empty_fix; \
 	  $(ETAGS) $(ETAGSFLAGS) $(AM_ETAGSFLAGS) $(ETAGS_ARGS) \
@@ -2513,12 +2513,13 @@ ctags: CTAGS
 CTAGS:  $(HEADERS) $(SOURCES)  $(TAGS_DEPENDENCIES) \
 		$(TAGS_FILES) $(LISP)
 	tags=; \
+	here=`pwd`; \
 	list='$(SOURCES) $(HEADERS)  $(LISP) $(TAGS_FILES)'; \
 	unique=`for i in $$list; do \
 	    if test -f "$$i"; then echo $$i; else echo $(srcdir)/$$i; fi; \
 	  done | \
-	  $(AWK) '{ files[$$0] = 1; nonempty = 1; } \
-	      END { if (nonempty) { for (i in files) print i; }; }'`; \
+	  $(AWK) '    { files[$$0] = 1; } \
+	       END { for (i in files) print i; }'`; \
 	test -z "$(CTAGS_ARGS)$$tags$$unique" \
 	  || $(CTAGS) $(CTAGSFLAGS) $(AM_CTAGSFLAGS) $(CTAGS_ARGS) \
 	     $$tags $$unique