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 2012/01/11 23:28:44 UTC

svn commit: r1230289 [1/3] - in /incubator/mesos/trunk: ./ include/mesos/ src/ src/common/ src/detector/ src/examples/ src/examples/java/ src/examples/python/ src/exec/ src/java/jni/ src/java/src/org/apache/mesos/ src/log/ src/master/ src/messages/ src...

Author: benh
Date: Wed Jan 11 22:28:41 2012
New Revision: 1230289

URL: http://svn.apache.org/viewvc?rev=1230289&view=rev
Log:
Merged code from within Twitter (adds attribute support, adds/fixes some license headers, and numerous other small bug fixes).

Removed:
    incubator/mesos/trunk/src/log/cache.hpp
Modified:
    incubator/mesos/trunk/configure.ac
    incubator/mesos/trunk/include/mesos/executor.hpp
    incubator/mesos/trunk/include/mesos/mesos.proto
    incubator/mesos/trunk/src/Makefile.in
    incubator/mesos/trunk/src/common/json.hpp
    incubator/mesos/trunk/src/common/lambda.hpp
    incubator/mesos/trunk/src/common/multihashmap.hpp
    incubator/mesos/trunk/src/common/process_utils.hpp
    incubator/mesos/trunk/src/common/resources.cpp
    incubator/mesos/trunk/src/common/resources.hpp
    incubator/mesos/trunk/src/common/seconds.hpp
    incubator/mesos/trunk/src/common/strings.hpp
    incubator/mesos/trunk/src/common/try.hpp
    incubator/mesos/trunk/src/common/utils.cpp
    incubator/mesos/trunk/src/common/utils.hpp
    incubator/mesos/trunk/src/common/uuid.hpp
    incubator/mesos/trunk/src/detector/detector.cpp
    incubator/mesos/trunk/src/examples/cpp_test_framework.cpp
    incubator/mesos/trunk/src/examples/java/TestExceptionFramework.java
    incubator/mesos/trunk/src/examples/java/TestExecutor.java
    incubator/mesos/trunk/src/examples/java/TestFramework.java
    incubator/mesos/trunk/src/examples/java/TestMultipleExecutorsFramework.java
    incubator/mesos/trunk/src/examples/long_lived_framework.cpp
    incubator/mesos/trunk/src/examples/memhog.cpp
    incubator/mesos/trunk/src/examples/python/test_framework.py
    incubator/mesos/trunk/src/examples/scheduled_memhog.cpp
    incubator/mesos/trunk/src/exec/exec.cpp
    incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp
    incubator/mesos/trunk/src/java/jni/org_apache_mesos_MesosExecutorDriver.cpp
    incubator/mesos/trunk/src/java/src/org/apache/mesos/Executor.java
    incubator/mesos/trunk/src/java/src/org/apache/mesos/ExecutorDriver.java
    incubator/mesos/trunk/src/java/src/org/apache/mesos/Log.java
    incubator/mesos/trunk/src/java/src/org/apache/mesos/MesosExecutorDriver.java
    incubator/mesos/trunk/src/java/src/org/apache/mesos/MesosSchedulerDriver.java
    incubator/mesos/trunk/src/java/src/org/apache/mesos/Scheduler.java
    incubator/mesos/trunk/src/java/src/org/apache/mesos/SchedulerDriver.java
    incubator/mesos/trunk/src/log/coordinator.cpp
    incubator/mesos/trunk/src/log/coordinator.hpp
    incubator/mesos/trunk/src/log/log.cpp
    incubator/mesos/trunk/src/log/log.hpp
    incubator/mesos/trunk/src/log/main.cpp
    incubator/mesos/trunk/src/log/network.hpp
    incubator/mesos/trunk/src/log/replica.cpp
    incubator/mesos/trunk/src/log/replica.hpp
    incubator/mesos/trunk/src/master/constants.hpp
    incubator/mesos/trunk/src/master/frameworks_manager.cpp
    incubator/mesos/trunk/src/master/frameworks_manager.hpp
    incubator/mesos/trunk/src/master/http.cpp
    incubator/mesos/trunk/src/master/http.hpp
    incubator/mesos/trunk/src/master/master.cpp
    incubator/mesos/trunk/src/master/master.hpp
    incubator/mesos/trunk/src/master/simple_allocator.cpp
    incubator/mesos/trunk/src/messages/log.hpp
    incubator/mesos/trunk/src/messages/log.proto
    incubator/mesos/trunk/src/messages/messages.proto
    incubator/mesos/trunk/src/python/native/mesos_executor_driver_impl.cpp
    incubator/mesos/trunk/src/python/native/mesos_executor_driver_impl.hpp
    incubator/mesos/trunk/src/python/src/mesos.py
    incubator/mesos/trunk/src/sched/sched.cpp
    incubator/mesos/trunk/src/slave/constants.hpp
    incubator/mesos/trunk/src/slave/http.cpp
    incubator/mesos/trunk/src/slave/http.hpp
    incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp
    incubator/mesos/trunk/src/slave/slave.cpp
    incubator/mesos/trunk/src/slave/slave.hpp
    incubator/mesos/trunk/src/tests/Makefile.in
    incubator/mesos/trunk/src/tests/base_zookeeper_test.cpp
    incubator/mesos/trunk/src/tests/base_zookeeper_test.hpp
    incubator/mesos/trunk/src/tests/exception_tests.cpp
    incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp
    incubator/mesos/trunk/src/tests/jvm.cpp
    incubator/mesos/trunk/src/tests/jvm.hpp
    incubator/mesos/trunk/src/tests/killtree_tests.cpp
    incubator/mesos/trunk/src/tests/log_tests.cpp
    incubator/mesos/trunk/src/tests/master_tests.cpp
    incubator/mesos/trunk/src/tests/multihashmap_tests.cpp
    incubator/mesos/trunk/src/tests/process_spawn.cpp
    incubator/mesos/trunk/src/tests/protobuf_io_tests.cpp
    incubator/mesos/trunk/src/tests/resource_offers_tests.cpp
    incubator/mesos/trunk/src/tests/resources_tests.cpp
    incubator/mesos/trunk/src/tests/strings_tests.cpp
    incubator/mesos/trunk/src/tests/url_processor_tests.cpp
    incubator/mesos/trunk/src/tests/utils.hpp
    incubator/mesos/trunk/src/tests/utils_tests.cpp
    incubator/mesos/trunk/src/tests/zookeeper_server.cpp
    incubator/mesos/trunk/src/tests/zookeeper_server.hpp
    incubator/mesos/trunk/src/tests/zookeeper_server_tests.cpp
    incubator/mesos/trunk/src/tests/zookeeper_tests.cpp
    incubator/mesos/trunk/src/webui/master/framework.tpl
    incubator/mesos/trunk/src/webui/static/stylesheet.css
    incubator/mesos/trunk/src/zookeeper/zookeeper.cpp
    incubator/mesos/trunk/src/zookeeper/zookeeper.hpp

Modified: incubator/mesos/trunk/configure.ac
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/configure.ac?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/configure.ac (original)
+++ incubator/mesos/trunk/configure.ac Wed Jan 11 22:28:41 2012
@@ -68,6 +68,7 @@ case "${target_os}" in
     echo Setting up build environment for ${target_cpu} ${target_os}
     echo ===========================================================
     OS_NAME=linux
+    LIBS="-lrt"
     ;;
   darwin*)
     echo ===========================================================

Modified: incubator/mesos/trunk/include/mesos/executor.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/include/mesos/executor.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/include/mesos/executor.hpp (original)
+++ incubator/mesos/trunk/include/mesos/executor.hpp Wed Jan 11 22:28:41 2012
@@ -72,7 +72,7 @@ public:
 
   // Lifecycle methods.
   virtual Status start() = 0;
-  virtual Status stop(bool failover = false) = 0;
+  virtual Status stop() = 0;
   virtual Status abort() = 0;
   virtual Status join() = 0;
   virtual Status run() = 0; // Start and then join driver.
@@ -98,7 +98,7 @@ public:
 
   // Lifecycle methods
   virtual Status start();
-  virtual Status stop(bool failover = false);
+  virtual Status stop();
   virtual Status abort();
   virtual Status join();
   virtual Status run(); // Start and then join driver

Modified: incubator/mesos/trunk/include/mesos/mesos.proto
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/include/mesos/mesos.proto?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/include/mesos/mesos.proto (original)
+++ incubator/mesos/trunk/include/mesos/mesos.proto Wed Jan 11 22:28:41 2012
@@ -60,19 +60,12 @@ message Params {
 }
 
 
-// TODO(benh): Add support for attributes.
-// message Attribute {
-//   required string name = 1;
-//   required string value = 2;
-// }
-
-
-// TODO(benh): Add better support for resources.
-message Resource {
+message Value {
   enum Type {
     SCALAR = 0;
     RANGES = 1;
     SET = 2;
+    TEXT = 3;
   }
 
   message Scalar {
@@ -89,14 +82,39 @@ message Resource {
   }
 
   message Set {
-    repeated string item = 1; 
+    repeated string item = 1;
   }
 
+  message Text {
+    required string value = 1;
+  }
+
+  required Type type = 1;
+  optional Scalar scalar = 2;
+  optional Ranges ranges = 3;
+  optional Set set = 4;
+  optional Text text = 5;
+}
+
+
+// Attribute doesn't support arithmetic operations.
+message Attribute {
   required string name = 1;
-  required Type type = 2;
-  optional Scalar scalar = 3;
-  optional Ranges ranges = 4;
-  optional Set set = 5;
+  required Value.Type type = 2;
+  optional Value.Scalar scalar = 3;
+  optional Value.Ranges ranges = 4;
+  optional Value.Text text = 5;
+}
+
+
+// TODO(benh): Add better support for resources.
+// Resource support arithmetic operations.
+message Resource {
+  required string name = 1;
+  required Value.Type type = 2;
+  optional Value.Scalar scalar = 3;
+  optional Value.Ranges ranges = 4;
+  optional Value.Set set = 5;
 }
 
 
@@ -154,11 +172,12 @@ message FrameworkInfo {
 
 
 message SlaveInfo {
-  required string hostname = 1;
   // TODO(benh,andyk): Send bound ports, not just hostname.
+  required string hostname = 1;
   required string webui_hostname = 2;
   optional int32 webui_port = 4 [default = 8081];
   repeated Resource resources = 3;
+  repeated Attribute attributes = 5;
 }
 
 
@@ -175,6 +194,7 @@ message Offer {
   required string hostname = 4;
   repeated Resource resources = 5;
   repeated ExecutorID executor_ids = 6;
+  repeated Attribute attributes = 7;
 }
 
 

Modified: incubator/mesos/trunk/src/Makefile.in
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/Makefile.in?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/Makefile.in (original)
+++ incubator/mesos/trunk/src/Makefile.in Wed Jan 11 22:28:41 2012
@@ -144,7 +144,7 @@ WEBUI_OBJ = $(MASTER_WEBUI_OBJ) $(SLAVE_
 COMMON_OBJ = common/fatal.o common/lock.o detector/detector.o		\
 	     detector/url_processor.o configurator/configurator.o	\
 	     common/logging.o common/date_utils.o common/resources.o	\
-	     common/utils.o
+	     common/utils.o common/values.o common/attributes.o
 
 ifeq ($(WITH_ZOOKEEPER),1)
   COMMON_OBJ += zookeeper/zookeeper.o zookeeper/authentication.o	\
@@ -195,7 +195,7 @@ MESOS_EXEC_LIB = $(LIBDIR)/libmesos_exec
 
 # C/C++ shared library which contains both the scheduler and executor
 # libraries, in addition to libglog, libprocess, and libev.
-MESOS_LIB_OBJ = 
+MESOS_LIB_OBJ =
 ifeq ($(OS_NAME),darwin)
   MESOS_LIB = $(LIBDIR)/libmesos.dylib
 else

Modified: incubator/mesos/trunk/src/common/json.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/json.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/json.hpp (original)
+++ incubator/mesos/trunk/src/common/json.hpp Wed Jan 11 22:28:41 2012
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 #include <iostream>
 #include <list>
 #include <map>

Modified: incubator/mesos/trunk/src/common/lambda.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/lambda.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/lambda.hpp (original)
+++ incubator/mesos/trunk/src/common/lambda.hpp Wed Jan 11 22:28:41 2012
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 #ifndef __LAMBDA_HPP__
 #define __LAMBDA_HPP__
 

Modified: incubator/mesos/trunk/src/common/multihashmap.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/multihashmap.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/multihashmap.hpp (original)
+++ incubator/mesos/trunk/src/common/multihashmap.hpp Wed Jan 11 22:28:41 2012
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 #ifndef __MULTIHASHMAP_HPP__
 #define __MULTIHASHMAP_HPP__
 

Modified: incubator/mesos/trunk/src/common/process_utils.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/process_utils.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/process_utils.hpp (original)
+++ incubator/mesos/trunk/src/common/process_utils.hpp Wed Jan 11 22:28:41 2012
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 #ifndef __PROCESS_UTILS_HPP__
 #define __PROCESS_UTILS_HPP__
 

Modified: incubator/mesos/trunk/src/common/resources.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/resources.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/resources.cpp (original)
+++ incubator/mesos/trunk/src/common/resources.cpp Wed Jan 11 22:28:41 2012
@@ -21,11 +21,11 @@
 
 #include <glog/logging.h>
 
-#include <boost/lexical_cast.hpp>
-
 #include "common/foreach.hpp"
 #include "common/resources.hpp"
 #include "common/strings.hpp"
+#include "common/try.hpp"
+#include "common/values.hpp"
 
 
 using std::ostream;
@@ -33,51 +33,51 @@ using std::string;
 using std::vector;
 
 
-namespace mesos { 
+namespace mesos {
 
-bool operator == (const Resource::Scalar& left, const Resource::Scalar& right)
+bool operator == (const Value::Scalar& left, const Value::Scalar& right)
 {
   return left.value() == right.value();
 }
 
 
-bool operator <= (const Resource::Scalar& left, const Resource::Scalar& right)
+bool operator <= (const Value::Scalar& left, const Value::Scalar& right)
 {
   return left.value() <= right.value();
 }
 
 
-Resource::Scalar operator + (const Resource::Scalar& left, const Resource::Scalar& right)
+Value::Scalar operator + (const Value::Scalar& left, const Value::Scalar& right)
 {
-  Resource::Scalar result;
+  Value::Scalar result;
   result.set_value(left.value() + right.value());
   return result;
 }
 
-  
-Resource::Scalar operator - (const Resource::Scalar& left, const Resource::Scalar& right)
+
+Value::Scalar operator - (const Value::Scalar& left, const Value::Scalar& right)
 {
-  Resource::Scalar result;
+  Value::Scalar result;
   result.set_value(left.value() - right.value());
   return result;
 }
 
-  
-Resource::Scalar& operator += (Resource::Scalar& left, const Resource::Scalar& right)
+
+Value::Scalar& operator += (Value::Scalar& left, const Value::Scalar& right)
 {
   left.set_value(left.value() + right.value());
   return left;
 }
 
 
-Resource::Scalar& operator -= (Resource::Scalar& left, const Resource::Scalar& right)
+Value::Scalar& operator -= (Value::Scalar& left, const Value::Scalar& right)
 {
   left.set_value(left.value() - right.value());
   return left;
 }
 
 
-static void coalesce(Resource::Ranges* ranges, const Resource::Range& range)
+static void coalesce(Value::Ranges* ranges, const Value::Range& range)
 {
   // Note that we assume that ranges has already been coalesced.
 
@@ -109,18 +109,18 @@ static void coalesce(Resource::Ranges* r
       break;
     }
   }
-  
+
   if (!coalesced) {
     ranges->add_range()->MergeFrom(range);
   }
 }
 
 
-static void remove(Resource::Ranges* ranges, const Resource::Range& range)
+static void remove(Value::Ranges* ranges, const Value::Range& range)
 {
   // Note that we assume that ranges has already been coalesced.
 
-  Resource::Ranges result;
+  Value::Ranges result;
 
   for (int i = 0; i < ranges->range_size(); i++) {
     int64_t begin = ranges->range(i).begin();
@@ -128,25 +128,25 @@ static void remove(Resource::Ranges* ran
 
     if (begin == range.begin() && range.end() == end) {
       // Remove range from ranges, but keep everything else.
-      for (int j = i; j < ranges->range_size(); j++) {
+      for (int j = i + 1; j < ranges->range_size(); j++) {
         result.add_range()->MergeFrom(ranges->range(j));
       }
       break;
     } else if (begin <= range.begin() && range.end() < end) {
       // Shrink range in ranges.
-      Resource::Range* temp = result.add_range();
+      Value::Range* temp = result.add_range();
       temp->set_begin(range.end() + 1);
       temp->set_end(end);
       break;
     } else if (begin < range.begin() && range.end() >= end) {
       // Shrink end of range in ranges.
-      Resource::Range* temp = result.add_range();
+      Value::Range* temp = result.add_range();
       temp->set_begin(begin);
       temp->set_end(range.begin() - 1);
       break;
     } else if (begin < range.begin() && range.end() < end) {
       // Split range in ranges.
-      Resource::Range* temp = result.add_range();
+      Value::Range* temp = result.add_range();
       temp->set_begin(begin);
       temp->set_end(range.begin() - 1);
       temp = result.add_range();
@@ -160,7 +160,7 @@ static void remove(Resource::Ranges* ran
 }
 
 
-bool operator == (const Resource::Ranges& left, const Resource::Ranges& right)
+bool operator == (const Value::Ranges& left, const Value::Ranges& right)
 {
   if (left.range_size() == right.range_size()) {
     for (int i = 0; i < left.range_size(); i++) {
@@ -186,31 +186,30 @@ bool operator == (const Resource::Ranges
 }
 
 
-bool operator <= (const Resource::Ranges& left, const Resource::Ranges& right)
+bool operator <= (const Value::Ranges& left, const Value::Ranges& right)
 {
-  if (left.range_size() <= right.range_size()) {
-    for (int i = 0; i < left.range_size(); i++) {
-      // Make sure this range is a subset of every range in right.
+   for (int i = 0; i < left.range_size(); i++) {
+      // Make sure this range is a subset of a range in right.
+      bool matched = false;
       for (int j = 0; j < right.range_size(); j++) {
-        if ((left.range(i).begin() <= right.range(j).begin() &&
-             left.range(i).end() > right.range(j).end()) ||
-            (left.range(i).begin() < right.range(j).begin() &&
-             left.range(i).end() >= right.range(j).end())) {
-          return false;
+        if ((left.range(i).begin() >= right.range(j).begin() &&
+             left.range(i).end() <= right.range(j).end())) {
+          matched = true;
+          break;
         }
       }
+      if (!matched) {
+        return false;
+      }
     }
 
     return true;
-  }
-
-  return false;
 }
 
 
-Resource::Ranges operator + (const Resource::Ranges& left, const Resource::Ranges& right)
+Value::Ranges operator + (const Value::Ranges& left, const Value::Ranges& right)
 {
-  Resource::Ranges result;
+  Value::Ranges result;
 
   for (int i = 0; i < left.range_size(); i++) {
     coalesce(&result, left.range(i));
@@ -223,10 +222,10 @@ Resource::Ranges operator + (const Resou
   return result;
 }
 
-  
-Resource::Ranges operator - (const Resource::Ranges& left, const Resource::Ranges& right)
+
+Value::Ranges operator - (const Value::Ranges& left, const Value::Ranges& right)
 {
-  Resource::Ranges result;
+  Value::Ranges result;
 
   for (int i = 0; i < left.range_size(); i++) {
     coalesce(&result, left.range(i));
@@ -243,10 +242,10 @@ Resource::Ranges operator - (const Resou
   return result;
 }
 
-  
-Resource::Ranges& operator += (Resource::Ranges& left, const Resource::Ranges& right)
+
+Value::Ranges& operator += (Value::Ranges& left, const Value::Ranges& right)
 {
-  Resource::Ranges temp;
+  Value::Ranges temp;
 
   for (int i = 0; i < left.range_size(); i++) {
     coalesce(&temp, left.range(i));
@@ -262,9 +261,9 @@ Resource::Ranges& operator += (Resource:
 }
 
 
-Resource::Ranges& operator -= (Resource::Ranges& left, const Resource::Ranges& right)
+Value::Ranges& operator -= (Value::Ranges& left, const Value::Ranges& right)
 {
-  Resource::Ranges temp;
+  Value::Ranges temp;
 
   for (int i = 0; i < left.range_size(); i++) {
     coalesce(&temp, left.range(i));
@@ -284,7 +283,7 @@ Resource::Ranges& operator -= (Resource:
 }
 
 
-bool operator == (const Resource::Set& left, const Resource::Set& right)
+bool operator == (const Value::Set& left, const Value::Set& right)
 {
   if (left.item_size() == right.item_size()) {
     for (int i = 0; i < left.item_size(); i++) {
@@ -309,7 +308,7 @@ bool operator == (const Resource::Set& l
 }
 
 
-bool operator <= (const Resource::Set& left, const Resource::Set& right)
+bool operator <= (const Value::Set& left, const Value::Set& right)
 {
   if (left.item_size() <= right.item_size()) {
     for (int i = 0; i < left.item_size(); i++) {
@@ -334,9 +333,9 @@ bool operator <= (const Resource::Set& l
 }
 
 
-Resource::Set operator + (const Resource::Set& left, const Resource::Set& right)
+Value::Set operator + (const Value::Set& left, const Value::Set& right)
 {
-  Resource::Set result;
+  Value::Set result;
 
   for (int i = 0; i < left.item_size(); i++) {
     result.add_item(left.item(i));
@@ -360,10 +359,10 @@ Resource::Set operator + (const Resource
   return result;
 }
 
-  
-Resource::Set operator - (const Resource::Set& left, const Resource::Set& right)
+
+Value::Set operator - (const Value::Set& left, const Value::Set& right)
 {
-  Resource::Set result;
+  Value::Set result;
 
   // Look for the same item in right as we add left to result.
   for (int i = 0; i < left.item_size(); i++) {
@@ -383,8 +382,8 @@ Resource::Set operator - (const Resource
   return result;
 }
 
-  
-Resource::Set& operator += (Resource::Set& left, const Resource::Set& right)
+
+Value::Set& operator += (Value::Set& left, const Value::Set& right)
 {
   // A little bit of extra logic to avoid adding duplicates from right.
   for (int i = 0; i < right.item_size(); i++) {
@@ -405,7 +404,7 @@ Resource::Set& operator += (Resource::Se
 }
 
 
-Resource::Set& operator -= (Resource::Set& left, const Resource::Set& right)
+Value::Set& operator -= (Value::Set& left, const Value::Set& right)
 {
   // For each item in right check if it's in left and add it if not.
   for (int i = 0; i < right.item_size(); i++) {
@@ -429,11 +428,11 @@ Resource::Set& operator -= (Resource::Se
 bool operator == (const Resource& left, const Resource& right)
 {
   if (left.name() == right.name() && left.type() == right.type()) {
-    if (left.type() == Resource::SCALAR) {
+    if (left.type() == Value::SCALAR) {
       return left.scalar() == right.scalar();
-    } else if (left.type() == Resource::RANGES) {
+    } else if (left.type() == Value::RANGES) {
       return left.ranges() == right.ranges();
-    } else if (left.type() == Resource::SET) {
+    } else if (left.type() == Value::SET) {
       return left.set() == right.set();
     }
   }
@@ -445,11 +444,11 @@ bool operator == (const Resource& left, 
 bool operator <= (const Resource& left, const Resource& right)
 {
   if (left.name() == right.name() && left.type() == right.type()) {
-    if (left.type() == Resource::SCALAR) {
+    if (left.type() == Value::SCALAR) {
       return left.scalar() <= right.scalar();
-    } else if (left.type() == Resource::RANGES) {
+    } else if (left.type() == Value::RANGES) {
       return left.ranges() <= right.ranges();
-    } else if (left.type() == Resource::SET) {
+    } else if (left.type() == Value::SET) {
       return left.set() <= right.set();
     }
   }
@@ -463,55 +462,55 @@ Resource operator + (const Resource& lef
   Resource result = left;
 
   if (left.name() == right.name() && left.type() == right.type()) {
-    if (left.type() == Resource::SCALAR) {
+    if (left.type() == Value::SCALAR) {
       result.mutable_scalar()->MergeFrom(left.scalar() + right.scalar());
-    } else if (left.type() == Resource::RANGES) {
+    } else if (left.type() == Value::RANGES) {
       result.mutable_ranges()->Clear();
       result.mutable_ranges()->MergeFrom(left.ranges() + right.ranges());
-    } else if (left.type() == Resource::SET) {
+    } else if (left.type() == Value::SET) {
       result.mutable_set()->Clear();
       result.mutable_set()->MergeFrom(left.set() + right.set());
     }
   }
-  
+
   return result;
 }
 
-  
+
 Resource operator - (const Resource& left, const Resource& right)
 {
   Resource result = left;
 
   if (left.name() == right.name() && left.type() == right.type()) {
-    if (left.type() == Resource::SCALAR) {
+    if (left.type() == Value::SCALAR) {
       result.mutable_scalar()->MergeFrom(left.scalar() - right.scalar());
-    } else if (left.type() == Resource::RANGES) {
+    } else if (left.type() == Value::RANGES) {
       result.mutable_ranges()->Clear();
       result.mutable_ranges()->MergeFrom(left.ranges() - right.ranges());
-    } else if (left.type() == Resource::SET) {
+    } else if (left.type() == Value::SET) {
       result.mutable_set()->Clear();
       result.mutable_set()->MergeFrom(left.set() - right.set());
     }
   }
-  
+
   return result;
 }
 
-  
+
 Resource& operator += (Resource& left, const Resource& right)
 {
   if (left.name() == right.name() && left.type() == right.type()) {
-    if (left.type() == Resource::SCALAR) {
+    if (left.type() == Value::SCALAR) {
       left.mutable_scalar()->MergeFrom(left.scalar() + right.scalar());
-    } else if (left.type() == Resource::RANGES) {
+    } else if (left.type() == Value::RANGES) {
       left.mutable_ranges()->Clear();
       left.mutable_ranges()->MergeFrom(left.ranges() + right.ranges());
-    } else if (left.type() == Resource::SET) {
+    } else if (left.type() == Value::SET) {
       left.mutable_set()->Clear();
       left.mutable_set()->MergeFrom(left.set() + right.set());
     }
   }
-  
+
   return left;
 }
 
@@ -519,17 +518,17 @@ Resource& operator += (Resource& left, c
 Resource& operator -= (Resource& left, const Resource& right)
 {
   if (left.name() == right.name() && left.type() == right.type()) {
-    if (left.type() == Resource::SCALAR) {
+    if (left.type() == Value::SCALAR) {
       left.mutable_scalar()->MergeFrom(left.scalar() - right.scalar());
-    } else if (left.type() == Resource::RANGES) {
+    } else if (left.type() == Value::RANGES) {
       left.mutable_ranges()->Clear();
       left.mutable_ranges()->MergeFrom(left.ranges() - right.ranges());
-    } else if (left.type() == Resource::SET) {
+    } else if (left.type() == Value::SET) {
       left.mutable_set()->Clear();
       left.mutable_set()->MergeFrom(left.set() - right.set());
     }
   }
-  
+
   return left;
 }
 
@@ -537,9 +536,9 @@ Resource& operator -= (Resource& left, c
 ostream& operator << (ostream& stream, const Resource& resource)
 {
   stream << resource.name() << "=";
-  if (resource.type() == Resource::SCALAR) {
+  if (resource.type() == Value::SCALAR) {
     stream << resource.scalar().value();
-  } else if (resource.type() == Resource::RANGES) {
+  } else if (resource.type() == Value::RANGES) {
     stream << "[";
     for (int i = 0; i < resource.ranges().range_size(); i++) {
       stream << resource.ranges().range(i).begin()
@@ -550,7 +549,7 @@ ostream& operator << (ostream& stream, c
       }
     }
     stream << "]";
-  } else if (resource.type() == Resource::SET) {
+  } else if (resource.type() == Value::SET) {
     stream << "{";
     for (int i = 0; i < resource.set().item_size(); i++) {
       stream << resource.set().item(i);
@@ -567,82 +566,38 @@ ostream& operator << (ostream& stream, c
 
 namespace internal {
 
-Resource Resources::parse(const string& name, const string& value)
+Resource Resources::parse(const std::string& name, const std::string& text)
 {
   Resource resource;
-  resource.set_name(name);
-
-  // Remove any spaces from the value.
-  string temp;
-  foreach (const char c, value) {
-    if (c != ' ') {
-      temp += c;
-    }
-  }
-
-  size_t index = temp.find('[');
-  if (index == 0) {
-    // This is a ranges.
-    Resource::Ranges ranges;
-    const vector<string>& tokens = strings::split(temp, "[]-,\n");
-    if (tokens.size() % 2 != 0) {
-      LOG(FATAL) << "Error parsing value for " << name
-                 << ", expecting one or more \"ranges\"";
-    } else {
-      for (int i = 0; i < tokens.size(); i += 2) {
-        Resource::Range *range = ranges.add_range();
-
-        int j = i;
-        try {
-          range->set_begin(boost::lexical_cast<uint64_t>((tokens[j++])));
-          range->set_end(boost::lexical_cast<uint64_t>(tokens[j++]));
-        } catch (const boost::bad_lexical_cast&) {
-          LOG(FATAL) << "Error parsing value for " << name
-                     << ", expecting non-negative integers in '"
-                     << tokens[j - 1] << "'";
-        }
-      }
+  Try<Value> result = values::parse(text);
 
-      resource.set_type(Resource::RANGES);
-      resource.mutable_ranges()->MergeFrom(ranges);
-    }
-  } else if (index == string::npos) {
-    size_t index = temp.find('{');
-    if (index == 0) {
-      // This is a set.
-      Resource::Set set;
-      const vector<string>& tokens = strings::split(temp, "{},\n");
-      for (int i = 0; i < tokens.size(); i++) {
-        set.add_item(tokens[i]);
-      }
-
-      resource.set_type(Resource::SET);
-      resource.mutable_set()->MergeFrom(set);
-    } else if (index == string::npos) {
-      // This *should* be a scalar.
-      Resource::Scalar scalar;
-      try {
-        scalar.set_value(boost::lexical_cast<double>(temp));
-      } catch (const boost::bad_lexical_cast&) {
-        LOG(FATAL) << "Error parsing value for " << name
-                   << ", expecting a number from '" << temp << "'";
-      }
-
-      resource.set_type(Resource::SCALAR);
-      resource.mutable_scalar()->MergeFrom(scalar);
+  if (result.isError()) {
+    LOG(FATAL) << "Failed to parse resource " << name
+               << " text " << text
+               << " error " << result.error();
+  } else{
+    Value value = result.get();
+    resource.set_name(name);
+
+    if (value.type() == Value::RANGES) {
+      resource.set_type(Value::RANGES);
+      resource.mutable_ranges()->MergeFrom(value.ranges());
+    } else if (value.type() == Value::SET) {
+      resource.set_type(Value::SET);
+      resource.mutable_set()->MergeFrom(value.set());
+    } else if (value.type() == Value::SCALAR) {
+      resource.set_type(Value::SCALAR);
+      resource.mutable_scalar()->MergeFrom(value.scalar());
     } else {
-      LOG(FATAL) << "Error parsing value for " << name
-                 << ", bad '{' found";
+      LOG(FATAL) << "Bad type for resource " << name
+                 << " text " << text
+                 << " type " << value.type();
     }
-  } else {
-    LOG(FATAL) << "Error parsing value for " << name
-               << ", bad '[' found";
   }
 
   return resource;
 }
 
-
 Resources Resources::parse(const string& s)
 {
   // Tokenize and parse the value of "resources".

Modified: incubator/mesos/trunk/src/common/resources.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/resources.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/resources.hpp (original)
+++ incubator/mesos/trunk/src/common/resources.hpp Wed Jan 11 22:28:41 2012
@@ -63,28 +63,28 @@
 
 namespace mesos {
 
-bool operator == (const Resource::Scalar& left, const Resource::Scalar& right);
-bool operator <= (const Resource::Scalar& left, const Resource::Scalar& right);
-Resource::Scalar operator + (const Resource::Scalar& left, const Resource::Scalar& right);
-Resource::Scalar operator - (const Resource::Scalar& left, const Resource::Scalar& right);
-Resource::Scalar& operator += (Resource::Scalar& left, const Resource::Scalar& right);
-Resource::Scalar& operator -= (Resource::Scalar& left, const Resource::Scalar& right);
-
-
-bool operator == (const Resource::Ranges& left, const Resource::Ranges& right);
-bool operator <= (const Resource::Ranges& left, const Resource::Ranges& right);
-Resource::Ranges operator + (const Resource::Ranges& left, const Resource::Ranges& right);
-Resource::Ranges operator - (const Resource::Ranges& left, const Resource::Ranges& right);
-Resource::Ranges& operator += (Resource::Ranges& left, const Resource::Ranges& right);
-Resource::Ranges& operator -= (Resource::Ranges& left, const Resource::Ranges& right);
-
-
-bool operator == (const Resource::Set& left, const Resource::Set& right);
-bool operator <= (const Resource::Set& left, const Resource::Set& right);
-Resource::Set operator + (const Resource::Set& left, const Resource::Set& right);
-Resource::Set operator - (const Resource::Set& left, const Resource::Set& right);
-Resource::Set& operator += (Resource::Set& left, const Resource::Set& right);
-Resource::Set& operator -= (Resource::Set& left, const Resource::Set& right);
+bool operator == (const Value::Scalar& left, const Value::Scalar& right);
+bool operator <= (const Value::Scalar& left, const Value::Scalar& right);
+Value::Scalar operator + (const Value::Scalar& left, const Value::Scalar& right);
+Value::Scalar operator - (const Value::Scalar& left, const Value::Scalar& right);
+Value::Scalar& operator += (Value::Scalar& left, const Value::Scalar& right);
+Value::Scalar& operator -= (Value::Scalar& left, const Value::Scalar& right);
+
+
+bool operator == (const Value::Ranges& left, const Value::Ranges& right);
+bool operator <= (const Value::Ranges& left, const Value::Ranges& right);
+Value::Ranges operator + (const Value::Ranges& left, const Value::Ranges& right);
+Value::Ranges operator - (const Value::Ranges& left, const Value::Ranges& right);
+Value::Ranges& operator += (Value::Ranges& left, const Value::Ranges& right);
+Value::Ranges& operator -= (Value::Ranges& left, const Value::Ranges& right);
+
+
+bool operator == (const Value::Set& left, const Value::Set& right);
+bool operator <= (const Value::Set& left, const Value::Set& right);
+Value::Set operator + (const Value::Set& left, const Value::Set& right);
+Value::Set operator - (const Value::Set& left, const Value::Set& right);
+Value::Set& operator += (Value::Set& left, const Value::Set& right);
+Value::Set& operator -= (Value::Set& left, const Value::Set& right);
 
 
 bool operator == (const Resource& left, const Resource& right);
@@ -229,7 +229,8 @@ public:
     bool added = false;
 
     foreach (const Resource& resource, resources) {
-      if (resource.name() == that.name() && resource.type() == that.type()) {
+      if (resource.name() == that.name() &&
+	resource.type() == that.type()) {
         result.resources.Add()->MergeFrom(resource + that);
         added = true;
       } else {
@@ -249,7 +250,8 @@ public:
     Resources result;
 
     foreach (const Resource& resource, resources) {
-      if (resource.name() == that.name() && resource.type() == that.type()) {
+      if (resource.name() == that.name() &&
+	resource.type() == that.type()) {
         result.resources.Add()->MergeFrom(resource - that);
       } else {
         result.resources.Add()->MergeFrom(resource);
@@ -306,16 +308,19 @@ public:
     if (!resource.has_name() ||
         resource.name() == "" ||
         !resource.has_type() ||
-        !Resource::Type_IsValid(resource.type())) {
+        !Value::Type_IsValid(resource.type())) {
       return false;
     }
 
-    if (resource.type() == Resource::SCALAR) {
+    if (resource.type() == Value::SCALAR) {
       return resource.has_scalar();
-    } else if (resource.type() == Resource::RANGES) {
+    } else if (resource.type() == Value::RANGES) {
       return resource.has_ranges();
-    } else if (resource.type() == Resource::SET) {
+    } else if (resource.type() == Value::SET) {
       return resource.has_ranges();
+    } else if (resource.type() == Value::TEXT) {
+      // Resources doesn't support text.
+      return false;
     }
 
     return false;
@@ -324,16 +329,16 @@ public:
   static bool isAllocatable(const Resource& resource)
   {
     if (isValid(resource)) {
-      if (resource.type() == Resource::SCALAR) {
+      if (resource.type() == Value::SCALAR) {
         if (resource.scalar().value() <= 0) {
           return false;
         }
-      } else if (resource.type() == Resource::RANGES) {
+      } else if (resource.type() == Value::RANGES) {
         if (resource.ranges().range_size() == 0) {
           return false;
         } else {
           for (int i = 0; i < resource.ranges().range_size(); i++) {
-            const Resource::Range& range = resource.ranges().range(i);
+            const Value::Range& range = resource.ranges().range(i);
 
             // Ensure the range make sense (isn't inverted).
             if (range.begin() > range.end()) {
@@ -349,7 +354,7 @@ public:
             }
           }
         }
-      } else if (resource.type() == Resource::SET) {
+      } else if (resource.type() == Value::SET) {
         if (resource.set().item_size() == 0) {
           return false;
         } else {
@@ -378,13 +383,13 @@ private:
 
 
 template <>
-inline Resource::Scalar Resources::get(
+inline Value::Scalar Resources::get(
     const std::string& name,
-    const Resource::Scalar& scalar) const
+    const Value::Scalar& scalar) const
 {
   foreach (const Resource& resource, resources) {
     if (resource.name() == name &&
-        resource.type() == Resource::SCALAR) {
+        resource.type() == Value::SCALAR) {
       return resource.scalar();
     }
   }
@@ -394,13 +399,13 @@ inline Resource::Scalar Resources::get(
 
 
 template <>
-inline Resource::Ranges Resources::get(
+inline Value::Ranges Resources::get(
     const std::string& name,
-    const Resource::Ranges& ranges) const
+    const Value::Ranges& ranges) const
 {
   foreach (const Resource& resource, resources) {
     if (resource.name() == name &&
-        resource.type() == Resource::RANGES) {
+        resource.type() == Value::RANGES) {
       return resource.ranges();
     }
   }
@@ -410,13 +415,13 @@ inline Resource::Ranges Resources::get(
 
 
 template <>
-inline Resource::Set Resources::get(
+inline Value::Set Resources::get(
     const std::string& name,
-    const Resource::Set& set) const
+    const Value::Set& set) const
 {
   foreach (const Resource& resource, resources) {
     if (resource.name() == name &&
-        resource.type() == Resource::SET) {
+        resource.type() == Value::SET) {
       return resource.set();
     }
   }

Modified: incubator/mesos/trunk/src/common/seconds.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/seconds.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/seconds.hpp (original)
+++ incubator/mesos/trunk/src/common/seconds.hpp Wed Jan 11 22:28:41 2012
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 #ifndef __SECONDS_HPP__
 #define __SECONDS_HPP__
 

Modified: incubator/mesos/trunk/src/common/strings.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/strings.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/strings.hpp (original)
+++ incubator/mesos/trunk/src/common/strings.hpp Wed Jan 11 22:28:41 2012
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 #ifndef __STRINGS_HPP__
 #define __STRINGS_HPP__
 
@@ -133,6 +151,27 @@ inline std::map<std::string, std::vector
   return result;
 }
 
+
+inline bool checkBracketsMatching(
+    const std::string& s,
+    const char openBracket,
+    const char closeBracket)
+{
+  int count = 0;
+  for (int i = 0; i < s.length(); i++) {
+    if (s[i] == openBracket) {
+      count++;
+    } else if (s[i] == closeBracket) {
+      count--;
+    }
+    if (count < 0) {
+      return false;
+    }
+  }
+  return count == 0;
+}
+
+
 } // namespaces strings {
 
 #endif // __STRINGS_HPP__

Modified: incubator/mesos/trunk/src/common/try.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/try.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/try.hpp (original)
+++ incubator/mesos/trunk/src/common/try.hpp Wed Jan 11 22:28:41 2012
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 #ifndef __TRY_HPP__
 #define __TRY_HPP__
 

Modified: incubator/mesos/trunk/src/common/utils.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/utils.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/utils.cpp (original)
+++ incubator/mesos/trunk/src/common/utils.cpp Wed Jan 11 22:28:41 2012
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 #include <ftw.h>
 
 #include "common/utils.hpp"

Modified: incubator/mesos/trunk/src/common/utils.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/utils.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/utils.hpp (original)
+++ incubator/mesos/trunk/src/common/utils.hpp Wed Jan 11 22:28:41 2012
@@ -29,7 +29,6 @@
 #include <signal.h>
 #include <stdarg.h>
 #include <stddef.h>
-#include <stdio.h>
 #include <unistd.h>
 
 #include <google/protobuf/message.h>

Modified: incubator/mesos/trunk/src/common/uuid.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/uuid.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/uuid.hpp (original)
+++ incubator/mesos/trunk/src/common/uuid.hpp Wed Jan 11 22:28:41 2012
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 #ifndef __UUID_HPP__
 #define __UUID_HPP__
 

Modified: incubator/mesos/trunk/src/detector/detector.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/detector/detector.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/detector/detector.cpp (original)
+++ incubator/mesos/trunk/src/detector/detector.cpp Wed Jan 11 22:28:41 2012
@@ -31,6 +31,7 @@
 
 #ifdef WITH_ZOOKEEPER
 #include "zookeeper/zookeeper.hpp"
+#include "zookeeper/authentication.hpp"
 #endif
 
 #include "messages/messages.hpp"
@@ -121,6 +122,7 @@ private:
 
   const string servers;
   const pair<string, string>* credentials;
+  ACL_vector acl;
   const string znode;
   const UPID pid;
   bool contend;
@@ -342,6 +344,8 @@ void ZooKeeperMasterDetector::initialize
 
   credentials = _credentials;
 
+  acl = credentials != NULL ? zookeeper::EVERYONE_READ_CREATOR_ALL : ZOO_OPEN_ACL_UNSAFE;
+
   // Start up the ZooKeeper connection!
   zk = new ZooKeeper(servers, milliseconds(10000), this);
 }
@@ -357,19 +361,6 @@ ZooKeeperMasterDetector::~ZooKeeperMaste
 }
 
 
-static ACL _EVERYONE_READ_CREATOR_ALL_ACL[] = {
-    {ZOO_PERM_READ, ZOO_ANYONE_ID_UNSAFE},
-    {ZOO_PERM_ALL, ZOO_AUTH_IDS}
-};
-
-
-// An ACL that ensures we're the only authenticated user to mutate our nodes -
-// others are welcome to read.
-static ACL_vector EVERYONE_READ_CREATOR_ALL = {
-    2, _EVERYONE_READ_CREATOR_ALL_ACL
-};
-
-
 void ZooKeeperMasterDetector::connected()
 {
   LOG(INFO) << "Master detector connected to ZooKeeper ...";
@@ -379,7 +370,7 @@ void ZooKeeperMasterDetector::connected(
     std::string username = credentials->first;
     std::string password = credentials->second;
     LOG(INFO) << "Authenticating to ZooKeeper with " << username << ":XXXXX";
-    ret = zk->authenticate(username, password);
+    ret = zk->authenticate("digest", username + ":" + password);
     if (ret != ZOK) {
       fatal("Failed to authenticate with ZooKeeper (%s) at : %s",
             zk->message(ret), servers.c_str());
@@ -404,7 +395,7 @@ void ZooKeeperMasterDetector::connected(
     LOG(INFO) << "Trying to create znode '" << prefix << "' in ZooKeeper";
 
     // Create the node (even if it already exists).
-    ret = zk->create(prefix, "", EVERYONE_READ_CREATOR_ALL, 0, &result);
+    ret = zk->create(prefix, "", acl, 0, &result);
 
     if (ret != ZOK && ret != ZNODEEXISTS) {
       fatal("failed to create ZooKeeper znode! (%s)", zk->message(ret));
@@ -422,7 +413,7 @@ void ZooKeeperMasterDetector::connected(
 
   if (contend) {
     // We contend with the pid given in constructor.
-    ret = zk->create(znode + "/", pid, EVERYONE_READ_CREATOR_ALL,
+    ret = zk->create(znode + "/", pid, acl,
 		     ZOO_SEQUENCE | ZOO_EPHEMERAL, &result);
 
     if (ret != ZOK) {

Modified: incubator/mesos/trunk/src/examples/cpp_test_framework.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/examples/cpp_test_framework.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/examples/cpp_test_framework.cpp (original)
+++ incubator/mesos/trunk/src/examples/cpp_test_framework.cpp Wed Jan 11 22:28:41 2012
@@ -66,10 +66,10 @@ public:
       for (int i = 0; i < offer.resources_size(); i++) {
         const Resource& resource = offer.resources(i);
         if (resource.name() == "cpus" &&
-            resource.type() == Resource::SCALAR) {
+            resource.type() == Value::SCALAR) {
           cpus = resource.scalar().value();
         } else if (resource.name() == "mem" &&
-                   resource.type() == Resource::SCALAR) {
+                   resource.type() == Value::SCALAR) {
           mem = resource.scalar().value();
         }
       }
@@ -93,12 +93,12 @@ public:
 
         resource = task.add_resources();
         resource->set_name("cpus");
-        resource->set_type(Resource::SCALAR);
+        resource->set_type(Value::SCALAR);
         resource->mutable_scalar()->set_value(CPUS_PER_TASK);
 
         resource = task.add_resources();
         resource->set_name("mem");
-        resource->set_type(Resource::SCALAR);
+        resource->set_type(Value::SCALAR);
         resource->mutable_scalar()->set_value(MEM_PER_TASK);
 
         tasks.push_back(task);

Modified: incubator/mesos/trunk/src/examples/java/TestExceptionFramework.java
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/examples/java/TestExceptionFramework.java?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/examples/java/TestExceptionFramework.java (original)
+++ incubator/mesos/trunk/src/examples/java/TestExceptionFramework.java Wed Jan 11 22:28:41 2012
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.

Modified: incubator/mesos/trunk/src/examples/java/TestExecutor.java
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/examples/java/TestExecutor.java?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/examples/java/TestExecutor.java (original)
+++ incubator/mesos/trunk/src/examples/java/TestExecutor.java Wed Jan 11 22:28:41 2012
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.

Modified: incubator/mesos/trunk/src/examples/java/TestFramework.java
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/examples/java/TestFramework.java?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/examples/java/TestFramework.java (original)
+++ incubator/mesos/trunk/src/examples/java/TestFramework.java Wed Jan 11 22:28:41 2012
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -64,15 +64,15 @@ public class TestFramework {
             .setSlaveId(offer.getSlaveId())
             .addResources(Resource.newBuilder()
                           .setName("cpus")
-                          .setType(Resource.Type.SCALAR)
-                          .setScalar(Resource.Scalar.newBuilder()
+                          .setType(Value.Type.SCALAR)
+                          .setScalar(Value.Scalar.newBuilder()
                                      .setValue(1)
                                      .build())
                           .build())
             .addResources(Resource.newBuilder()
                           .setName("mem")
-                          .setType(Resource.Type.SCALAR)
-                          .setScalar(Resource.Scalar.newBuilder()
+                          .setType(Value.Type.SCALAR)
+                          .setScalar(Value.Scalar.newBuilder()
                                      .setValue(128)
                                      .build())
                           .build())

Modified: incubator/mesos/trunk/src/examples/java/TestMultipleExecutorsFramework.java
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/examples/java/TestMultipleExecutorsFramework.java?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/examples/java/TestMultipleExecutorsFramework.java (original)
+++ incubator/mesos/trunk/src/examples/java/TestMultipleExecutorsFramework.java Wed Jan 11 22:28:41 2012
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -70,15 +70,15 @@ public class TestMultipleExecutorsFramew
               .setSlaveId(offer.getSlaveId())
               .addResources(Resource.newBuilder()
                             .setName("cpus")
-                            .setType(Resource.Type.SCALAR)
-                            .setScalar(Resource.Scalar.newBuilder()
+                            .setType(Value.Type.SCALAR)
+                            .setScalar(Value.Scalar.newBuilder()
                                        .setValue(1)
                                        .build())
                             .build())
               .addResources(Resource.newBuilder()
                             .setName("mem")
-                            .setType(Resource.Type.SCALAR)
-                            .setScalar(Resource.Scalar.newBuilder()
+                            .setType(Value.Type.SCALAR)
+                            .setScalar(Value.Scalar.newBuilder()
                                        .setValue(128)
                                        .build())
                             .build())
@@ -109,15 +109,15 @@ public class TestMultipleExecutorsFramew
               .setSlaveId(offer.getSlaveId())
               .addResources(Resource.newBuilder()
                             .setName("cpus")
-                            .setType(Resource.Type.SCALAR)
-                            .setScalar(Resource.Scalar.newBuilder()
+                            .setType(Value.Type.SCALAR)
+                            .setScalar(Value.Scalar.newBuilder()
                                        .setValue(1)
                                        .build())
                             .build())
               .addResources(Resource.newBuilder()
                             .setName("mem")
-                            .setType(Resource.Type.SCALAR)
-                            .setScalar(Resource.Scalar.newBuilder()
+                            .setType(Value.Type.SCALAR)
+                            .setScalar(Value.Scalar.newBuilder()
                                        .setValue(128)
                                        .build())
                             .build())

Modified: incubator/mesos/trunk/src/examples/long_lived_framework.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/examples/long_lived_framework.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/examples/long_lived_framework.cpp (original)
+++ incubator/mesos/trunk/src/examples/long_lived_framework.cpp Wed Jan 11 22:28:41 2012
@@ -66,10 +66,10 @@ public:
       for (int i = 0; i < offer.resources_size(); i++) {
         const Resource& resource = offer.resources(i);
         if (resource.name() == "cpus" &&
-            resource.type() == Resource::SCALAR) {
+            resource.type() == Value::SCALAR) {
           cpus = resource.scalar().value();
         } else if (resource.name() == "mem" &&
-                   resource.type() == Resource::SCALAR) {
+                   resource.type() == Value::SCALAR) {
           mem = resource.scalar().value();
         }
       }
@@ -91,12 +91,12 @@ public:
 
         resource = task.add_resources();
         resource->set_name("cpus");
-        resource->set_type(Resource::SCALAR);
+        resource->set_type(Value::SCALAR);
         resource->mutable_scalar()->set_value(CPUS_PER_TASK);
 
         resource = task.add_resources();
         resource->set_name("mem");
-        resource->set_type(Resource::SCALAR);
+        resource->set_type(Value::SCALAR);
         resource->mutable_scalar()->set_value(MEM_PER_TASK);
 
         tasks.push_back(task);

Modified: incubator/mesos/trunk/src/examples/memhog.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/examples/memhog.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/examples/memhog.cpp (original)
+++ incubator/mesos/trunk/src/examples/memhog.cpp Wed Jan 11 22:28:41 2012
@@ -65,10 +65,10 @@ public:
       for (int i = 0; i < offer.resources_size(); i++) {
         const Resource& resource = offer.resources(i);
         if (resource.name() == "cpus" &&
-            resource.type() == Resource::SCALAR) {
+            resource.type() == Value::SCALAR) {
           cpus = resource.scalar().value();
         } else if (resource.name() == "mem" &&
-                   resource.type() == Resource::SCALAR) {
+                   resource.type() == Value::SCALAR) {
           mem = resource.scalar().value();
         }
       }
@@ -90,12 +90,12 @@ public:
 
         resource = task.add_resources();
         resource->set_name("cpus");
-        resource->set_type(Resource::SCALAR);
+        resource->set_type(Value::SCALAR);
         resource->mutable_scalar()->set_value(1);
 
         resource = task.add_resources();
         resource->set_name("mem");
-        resource->set_type(Resource::SCALAR);
+        resource->set_type(Value::SCALAR);
         resource->mutable_scalar()->set_value(memToRequest);
 
         ostringstream data;

Modified: incubator/mesos/trunk/src/examples/python/test_framework.py
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/examples/python/test_framework.py?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/examples/python/test_framework.py (original)
+++ incubator/mesos/trunk/src/examples/python/test_framework.py Wed Jan 11 22:28:41 2012
@@ -71,7 +71,7 @@ class MyScheduler(mesos.Scheduler):
       self.tasksFinished += 1
       if self.tasksFinished == TOTAL_TASKS:
         print "All tasks done, exiting"
-        driver.stop(False)
+        driver.stop()
 
 if __name__ == "__main__":
   print "Connecting to %s" % sys.argv[1]

Modified: incubator/mesos/trunk/src/examples/scheduled_memhog.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/examples/scheduled_memhog.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/examples/scheduled_memhog.cpp (original)
+++ incubator/mesos/trunk/src/examples/scheduled_memhog.cpp Wed Jan 11 22:28:41 2012
@@ -113,10 +113,10 @@ public:
       for (int i = 0; i < offer.resources_size(); i++) {
         const Resource& resource = offer.resources(i);
         if (resource.name() == "cpus" &&
-            resource.type() == Resource::SCALAR) {
+            resource.type() == Value::SCALAR) {
           cpus = resource.scalar().value();
         } else if (resource.name() == "mem" &&
-                   resource.type() == Resource::SCALAR) {
+                   resource.type() == Value::SCALAR) {
           mem = resource.scalar().value();
         }
       }
@@ -142,12 +142,12 @@ public:
 
         resource = task.add_resources();
         resource->set_name("cpus");
-        resource->set_type(Resource::SCALAR);
+        resource->set_type(Value::SCALAR);
         resource->mutable_scalar()->set_value(1);
 
         resource = task.add_resources();
         resource->set_name("mem");
-        resource->set_type(Resource::SCALAR);
+        resource->set_type(Value::SCALAR);
         resource->mutable_scalar()->set_value(tasks[taskId].memToRequest);
 
         ostringstream data;

Modified: incubator/mesos/trunk/src/exec/exec.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/exec/exec.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/exec/exec.cpp (original)
+++ incubator/mesos/trunk/src/exec/exec.cpp Wed Jan 11 22:28:41 2012
@@ -380,7 +380,7 @@ Status MesosExecutorDriver::start()
 }
 
 
-Status MesosExecutorDriver::stop(bool failover)
+Status MesosExecutorDriver::stop()
 {
   Lock lock(&mutex);
 
@@ -392,9 +392,7 @@ Status MesosExecutorDriver::stop(bool fa
 
   CHECK(process != NULL);
 
-  if (!failover) {
-    terminate(process);
-  }
+  terminate(process);
 
   state = STOPPED;
   pthread_cond_signal(&cond);

Modified: incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp (original)
+++ incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp Wed Jan 11 22:28:41 2012
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 #include <jni.h>
 
 #include "log/log.hpp"
@@ -95,10 +113,15 @@ extern "C" {
 /*
  * Class:     org_apache_mesos_Log_Reader
  * Method:    read
- * Signature: (Lorg/apache/mesos/Log/Position;Lorg/apache/mesos/Log/Position;)Ljava/util/List;
+ * Signature: (Lorg/apache/mesos/Log/Position;Lorg/apache/mesos/Log/Position;JLjava/util/concurrent/TimeUnit;)Ljava/util/List;
  */
 JNIEXPORT jobject JNICALL Java_org_apache_mesos_Log_00024Reader_read
-  (JNIEnv* env, jobject thiz, jobject jfrom, jobject jto)
+  (JNIEnv* env,
+   jobject thiz,
+   jobject jfrom,
+   jobject jto,
+   jlong jtimeout,
+   jobject junit)
 {
   // Read out __reader.
   jclass clazz = env->GetObjectClass(thiz);
@@ -115,7 +138,16 @@ JNIEXPORT jobject JNICALL Java_org_apach
   Log::Position from = log->position(identity(env, jfrom));
   Log::Position to = log->position(identity(env, jto));
 
-  Result<std::list<Log::Entry> > entries = reader->read(from, to);
+  clazz = env->GetObjectClass(junit);
+
+  // long seconds = unit.toSeconds(time);
+  jmethodID toSeconds = env->GetMethodID(clazz, "toSeconds", "(J)J");
+
+  jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
+
+  seconds timeout(jseconds);
+
+  Result<std::list<Log::Entry> > entries = reader->read(from, to, timeout);
 
   if (entries.isError()) {
     clazz = env->FindClass("org/apache/mesos/Log$OperationFailedException");
@@ -239,10 +271,10 @@ JNIEXPORT void JNICALL Java_org_apache_m
 /*
  * Class:     org_apache_mesos_Log_Writer
  * Method:    append
- * Signature: ([B)Lorg/apache/mesos/Log/Position;
+ * Signature: ([BJLjava/util/concurrent/TimeUnit;)Lorg/apache/mesos/Log/Position;
  */
 JNIEXPORT jobject JNICALL Java_org_apache_mesos_Log_00024Writer_append
-  (JNIEnv* env, jobject thiz, jbyteArray jdata)
+  (JNIEnv* env, jobject thiz, jbyteArray jdata, jlong jtimeout, jobject junit)
 {
   // Read out __writer.
   jclass clazz = env->GetObjectClass(thiz);
@@ -256,7 +288,16 @@ JNIEXPORT jobject JNICALL Java_org_apach
 
   std::string data((char*) temp, (size_t) length);
 
-  Result<Log::Position> position = writer->append(data);
+  clazz = env->GetObjectClass(junit);
+
+  // long seconds = unit.toSeconds(time);
+  jmethodID toSeconds = env->GetMethodID(clazz, "toSeconds", "(J)J");
+
+  jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
+
+  seconds timeout(jseconds);
+
+  Result<Log::Position> position = writer->append(data, timeout);
 
   env->ReleaseByteArrayElements(jdata, temp, 0);
 
@@ -281,10 +322,10 @@ JNIEXPORT jobject JNICALL Java_org_apach
 /*
  * Class:     org_apache_mesos_Log_Writer
  * Method:    truncate
- * Signature: (Lorg/apache/mesos/Log/Position;)Lorg/apache/mesos/Log/Position;
+ * Signature: (Lorg/apache/mesos/Log/Position;JLjava/util/concurrent/TimeUnit;)Lorg/apache/mesos/Log/Position;
  */
 JNIEXPORT jobject JNICALL Java_org_apache_mesos_Log_00024Writer_truncate
-  (JNIEnv* env, jobject thiz, jobject jto)
+  (JNIEnv* env, jobject thiz, jobject jto, jlong jtimeout, jobject junit)
 {
   // Read out __writer.
   jclass clazz = env->GetObjectClass(thiz);
@@ -294,15 +335,22 @@ JNIEXPORT jobject JNICALL Java_org_apach
   Log::Writer* writer = (Log::Writer*) env->GetLongField(thiz, __writer);
 
   // Also need __log.
-  clazz = env->GetObjectClass(thiz);
-
   jfieldID __log = env->GetFieldID(clazz, "__log", "J");
 
   Log* log = (Log*) env->GetLongField(thiz, __log);
 
   Log::Position to = log->position(identity(env, jto));
 
-  Result<Log::Position> position = writer->truncate(to);
+  clazz = env->GetObjectClass(junit);
+
+  // long seconds = unit.toSeconds(time);
+  jmethodID toSeconds = env->GetMethodID(clazz, "toSeconds", "(J)J");
+
+  jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
+
+  seconds timeout(jseconds);
+
+  Result<Log::Position> position = writer->truncate(to, timeout);
 
   if (position.isError()) {
     clazz = env->FindClass("org/apache/mesos/Log$WriterFailedException");
@@ -325,10 +373,15 @@ JNIEXPORT jobject JNICALL Java_org_apach
 /*
  * Class:     org_apache_mesos_Log_Writer
  * Method:    initialize
- * Signature: (Lorg/apache/mesos/Log;)V
+ * Signature: (Lorg/apache/mesos/Log;JLjava/util/concurrent/TimeUnit;I)V
  */
 JNIEXPORT void JNICALL Java_org_apache_mesos_Log_00024Writer_initialize
-  (JNIEnv* env, jobject thiz, jobject jlog)
+  (JNIEnv* env,
+   jobject thiz,
+   jobject jlog,
+   jlong jtimeout,
+   jobject junit,
+   jint jretries)
 {
   // Get log.__log out and store it.
   jclass clazz = env->GetObjectClass(jlog);
@@ -342,8 +395,21 @@ JNIEXPORT void JNICALL Java_org_apache_m
   __log = env->GetFieldID(clazz, "__log", "J");
   env->SetLongField(thiz, __log, (jlong) log);
 
+  clazz = env->GetObjectClass(junit);
+
+  // long seconds = unit.toSeconds(time);
+  jmethodID toSeconds = env->GetMethodID(clazz, "toSeconds", "(J)J");
+
+  jlong jseconds = env->CallLongMethod(junit, toSeconds, jtimeout);
+
+  seconds timeout(jseconds);
+
+  int retries = jretries;
+
   // Create the C++ Log::Writer and initialize the __writer variable.
-  Log::Writer* writer = new Log::Writer(log);
+  Log::Writer* writer = new Log::Writer(log, timeout, retries);
+
+  clazz = env->GetObjectClass(thiz);
 
   jfieldID __writer = env->GetFieldID(clazz, "__writer", "J");
   env->SetLongField(thiz, __writer, (jlong) writer);

Modified: incubator/mesos/trunk/src/java/jni/org_apache_mesos_MesosExecutorDriver.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/java/jni/org_apache_mesos_MesosExecutorDriver.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/java/jni/org_apache_mesos_MesosExecutorDriver.cpp (original)
+++ incubator/mesos/trunk/src/java/jni/org_apache_mesos_MesosExecutorDriver.cpp Wed Jan 11 22:28:41 2012
@@ -350,7 +350,7 @@ JNIEXPORT jobject JNICALL Java_org_apach
  * Signature: (Z)Lorg/apache/mesos/Protos/Status;
  */
 JNIEXPORT jobject JNICALL Java_org_apache_mesos_MesosExecutorDriver_stop
-  (JNIEnv* env, jobject thiz, jboolean failover)
+  (JNIEnv* env, jobject thiz)
 {
   jclass clazz = env->GetObjectClass(thiz);
 
@@ -358,7 +358,7 @@ JNIEXPORT jobject JNICALL Java_org_apach
   MesosExecutorDriver* driver =
     (MesosExecutorDriver*) env->GetLongField(thiz, __driver);
 
-  Status status = driver->stop(failover);
+  Status status = driver->stop();
 
   return convert<Status>(env, status);
 }

Modified: incubator/mesos/trunk/src/java/src/org/apache/mesos/Executor.java
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/java/src/org/apache/mesos/Executor.java?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/java/src/org/apache/mesos/Executor.java (original)
+++ incubator/mesos/trunk/src/java/src/org/apache/mesos/Executor.java Wed Jan 11 22:28:41 2012
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.

Modified: incubator/mesos/trunk/src/java/src/org/apache/mesos/ExecutorDriver.java
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/java/src/org/apache/mesos/ExecutorDriver.java?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/java/src/org/apache/mesos/ExecutorDriver.java (original)
+++ incubator/mesos/trunk/src/java/src/org/apache/mesos/ExecutorDriver.java Wed Jan 11 22:28:41 2012
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -33,7 +33,6 @@ public interface ExecutorDriver {
   // Lifecycle methods.
   public Status start();
   public Status stop();
-  public Status stop(boolean failover);
   public Status abort();
   public Status join();
   public Status run();

Modified: incubator/mesos/trunk/src/java/src/org/apache/mesos/Log.java
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/java/src/org/apache/mesos/Log.java?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/java/src/org/apache/mesos/Log.java (original)
+++ incubator/mesos/trunk/src/java/src/org/apache/mesos/Log.java Wed Jan 11 22:28:41 2012
@@ -1,3 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.mesos;
 
 import java.io.Closeable;
@@ -10,8 +28,9 @@ import java.util.concurrent.TimeoutExcep
 import java.util.concurrent.TimeUnit;
 
 /**
- * Provides access to a distributed append only log.  The log can be read from using a
- * {@link Log.Reader} and written to using a {@link Log.Writer}.
+ * Provides access to a distributed append only log. The log can be
+ * read from using a {@link Log.Reader} and written to using a {@link
+ * Log.Writer}.
  */
 public class Log {
   static {
@@ -19,8 +38,9 @@ public class Log {
   }
 
   /**
-   * An opaque identifier of a log entry's position within the log.  Can be used to inidicate
-   * {@link Reader#read read} ranges and {@link Writer#truncate truncation} locations.
+   * An opaque identifier of a log entry's position within the
+   * log. Can be used to inidicate {@link Reader#read read} ranges and
+   * {@link Writer#truncate truncation} locations.
    */
   public static class Position implements Comparable<Position> {
     @Override
@@ -64,7 +84,8 @@ public class Log {
   }
 
   /**
-   * Represents an opaque data entry in the {@link Log} with a {@link Position}.
+   * Represents an opaque data entry in the {@link Log} with a {@link
+   * Position}.
    */
   public static class Entry {
     public final Position position;
@@ -107,8 +128,9 @@ public class Log {
   }
 
   /**
-   * Provides read access to the {@link Log}.  This class is safe for use from multiple threads and
-   * for the life of the log regardless of any exceptions thrown from its methods.
+   * Provides read access to the {@link Log}. This class is safe for
+   * use from multiple threads and for the life of the log regardless
+   * of any exceptions thrown from its methods.
    */
   public static class Reader {
     public Reader(Log log) {
@@ -124,7 +146,10 @@ public class Log {
      * failure) and therefore it is currently impossible to tell these
      * two cases apart.
      */
-    public native List<Entry> read(Position from, Position to)
+    public native List<Entry> read(Position from,
+                                   Position to,
+                                   long timeout,
+                                   TimeUnit unit)
       throws TimeoutException, OperationFailedException;
 
     /**
@@ -140,6 +165,7 @@ public class Log {
     public native Position ending();
 
     protected native void initialize(Log log);
+
     protected native void finalize();
 
     private Log log; // Keeps the log from getting garbage collected.
@@ -148,13 +174,14 @@ public class Log {
   }
 
   /**
-   * Provides write access to the {@link Log}.  This class is not safe for use from multiple
-   * threads and instances should be thrown out after any {@link WriterFailedException} is thrown.
+   * Provides write access to the {@link Log}. This class is not safe
+   * for use from multiple threads and instances should be thrown out
+   * after any {@link WriterFailedException} is thrown.
    */
   public static class Writer {
-    public Writer(Log log) {
+    public Writer(Log log, long timeout, TimeUnit unit, int retries) {
       this.log = log;
-      initialize(log);
+      initialize(log, timeout, unit, retries);
     }
 
     /**
@@ -169,7 +196,7 @@ public class Log {
      * WriterFailedException to differentiate the need for a new
      * writer from a bad position, or a bad disk, etc.
      */
-    public native Position append(byte[] data)
+    public native Position append(byte[] data, long timeout, TimeUnit unit)
       throws TimeoutException, WriterFailedException;
 
     /**
@@ -184,10 +211,14 @@ public class Log {
      * WriterFailedException to differentiate the need for a new
      * writer from a bad position, or a bad disk, etc.
      */
-    public native Position truncate(Position to)
+    public native Position truncate(Position to, long timeout, TimeUnit unit)
       throws TimeoutException, WriterFailedException;
 
-    protected native void initialize(Log log);
+    protected native void initialize(Log log,
+                                     long timeout,
+                                     TimeUnit unit,
+                                     int retries);
+
     protected native void finalize();
 
     private Log log; // Keeps the log from getting garbage collected.

Modified: incubator/mesos/trunk/src/java/src/org/apache/mesos/MesosExecutorDriver.java
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/java/src/org/apache/mesos/MesosExecutorDriver.java?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/java/src/org/apache/mesos/MesosExecutorDriver.java (original)
+++ incubator/mesos/trunk/src/java/src/org/apache/mesos/MesosExecutorDriver.java Wed Jan 11 22:28:41 2012
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -40,10 +40,7 @@ public class MesosExecutorDriver impleme
 
   // Lifecycle methods.
   public native Status start();
-  public Status stop() {
-    return stop(false);
-  }
-  public native Status stop(boolean failover);
+  public native Status stop();
   public native Status abort();
   public native Status join();
 

Modified: incubator/mesos/trunk/src/java/src/org/apache/mesos/MesosSchedulerDriver.java
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/java/src/org/apache/mesos/MesosSchedulerDriver.java?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/java/src/org/apache/mesos/MesosSchedulerDriver.java (original)
+++ incubator/mesos/trunk/src/java/src/org/apache/mesos/MesosSchedulerDriver.java Wed Jan 11 22:28:41 2012
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.

Modified: incubator/mesos/trunk/src/java/src/org/apache/mesos/Scheduler.java
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/java/src/org/apache/mesos/Scheduler.java?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/java/src/org/apache/mesos/Scheduler.java (original)
+++ incubator/mesos/trunk/src/java/src/org/apache/mesos/Scheduler.java Wed Jan 11 22:28:41 2012
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.

Modified: incubator/mesos/trunk/src/java/src/org/apache/mesos/SchedulerDriver.java
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/java/src/org/apache/mesos/SchedulerDriver.java?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/java/src/org/apache/mesos/SchedulerDriver.java (original)
+++ incubator/mesos/trunk/src/java/src/org/apache/mesos/SchedulerDriver.java Wed Jan 11 22:28:41 2012
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- * 
+ *
  *     http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.