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 [3/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...

Modified: incubator/mesos/trunk/src/tests/log_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/log_tests.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/log_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/log_tests.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 <gmock/gmock.h>
 
 #include <set>
@@ -10,7 +28,6 @@
 #include "common/type_utils.hpp"
 #include "common/utils.hpp"
 
-#include "log/cache.hpp"
 #include "log/coordinator.hpp"
 #include "log/log.hpp"
 #include "log/replica.hpp"
@@ -31,49 +48,6 @@ using testing::Eq;
 using testing::Return;
 
 
-TEST(LogTest, Cache)
-{
-  Cache<uint64_t, std::string> cache(10);
-
-  for (int i = 0; i < 10; i++) {
-    cache.put(i, utils::stringify(i));
-  }
-
-  for (int i = 0; i < 10; i++) {
-    Option<std::string> option = cache.get(i);
-    ASSERT_TRUE(option.isSome());
-    EXPECT_EQ(utils::stringify(i), option.get());
-  }
-
-  Option<std::string> option = Option<std::string>::none();
-
-  option = cache.get(1);
-  ASSERT_TRUE(option.isSome());
-  EXPECT_EQ("1", option.get());
-
-  cache.put(10, "10");
-
-  option = cache.get(0);
-  EXPECT_TRUE(option.isNone());
-
-  option = cache.get(10);
-  ASSERT_TRUE(option.isSome());
-  EXPECT_EQ("10", option.get());
-
-  option = cache.get(1);
-  ASSERT_TRUE(option.isSome());
-  EXPECT_EQ("1", option.get());
-
-  cache.put(11, "11");
-
-  option = cache.get(1);
-  EXPECT_TRUE(option.isSome());
-
-  option = cache.get(2);
-  EXPECT_TRUE(option.isNone());
-}
-
-
 TEST(ReplicaTest, Promise)
 {
   const std::string path = utils::os::getcwd() + "/.log";
@@ -304,7 +278,7 @@ TEST(CoordinatorTest, Elect)
   Coordinator coord(2, &replica1, &network);
 
   {
-    Result<uint64_t> result = coord.elect();
+    Result<uint64_t> result = coord.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(0, result.get());
   }
@@ -343,7 +317,7 @@ TEST(CoordinatorTest, AppendRead)
   Coordinator coord(2, &replica1, &network);
 
   {
-    Result<uint64_t> result = coord.elect();
+    Result<uint64_t> result = coord.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(0, result.get());
   }
@@ -351,7 +325,7 @@ TEST(CoordinatorTest, AppendRead)
   uint64_t position;
 
   {
-    Result<uint64_t> result2 = coord.append("hello world");
+    Result<uint64_t> result2 = coord.append("hello world", Timeout(1.0));
     ASSERT_TRUE(result2.isSome());
     position = result2.get();
     EXPECT_EQ(1, position);
@@ -392,7 +366,7 @@ TEST(CoordinatorTest, AppendReadError)
   Coordinator coord(2, &replica1, &network);
 
   {
-    Result<uint64_t> result = coord.elect();
+    Result<uint64_t> result = coord.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(0, result.get());
   }
@@ -400,7 +374,7 @@ TEST(CoordinatorTest, AppendReadError)
   uint64_t position;
 
   {
-    Result<uint64_t> result2 = coord.append("hello world");
+    Result<uint64_t> result2 = coord.append("hello world", Timeout(1.0));
     ASSERT_TRUE(result2.isSome());
     position = result2.get();
     EXPECT_EQ(1, position);
@@ -450,7 +424,7 @@ TEST(CoordinatorTest, DISABLED_ElectNoQu
   Coordinator coord(2, &replica, &network);
 
   {
-    Result<uint64_t> result = coord.elect();
+    Result<uint64_t> result = coord.elect(Timeout(1.0));
     ASSERT_TRUE(result.isNone());
   }
 
@@ -477,7 +451,7 @@ TEST(CoordinatorTest, DISABLED_AppendNoQ
   Coordinator coord(2, &replica1, &network);
 
   {
-    Result<uint64_t> result = coord.elect();
+    Result<uint64_t> result = coord.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(0, result.get());
   }
@@ -485,7 +459,7 @@ TEST(CoordinatorTest, DISABLED_AppendNoQ
   network.remove(replica1.pid());
 
   {
-    Result<uint64_t> result = coord.append("hello world");
+    Result<uint64_t> result = coord.append("hello world", Timeout(1.0));
     ASSERT_TRUE(result.isNone());
   }
 
@@ -513,7 +487,7 @@ TEST(CoordinatorTest, Failover)
   Coordinator coord1(2, &replica1, &network1);
 
   {
-    Result<uint64_t> result = coord1.elect();
+    Result<uint64_t> result = coord1.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(0, result.get());
   }
@@ -521,7 +495,7 @@ TEST(CoordinatorTest, Failover)
   uint64_t position;
 
   {
-    Result<uint64_t> result = coord1.append("hello world");
+    Result<uint64_t> result = coord1.append("hello world", Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     position = result.get();
     EXPECT_EQ(1, position);
@@ -535,7 +509,7 @@ TEST(CoordinatorTest, Failover)
   Coordinator coord2(2, &replica2, &network2);
 
   {
-    Result<uint64_t> result = coord2.elect();
+    Result<uint64_t> result = coord2.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(position, result.get());
   }
@@ -575,7 +549,7 @@ TEST(CoordinatorTest, Demoted)
   Coordinator coord1(2, &replica1, &network1);
 
   {
-    Result<uint64_t> result = coord1.elect();
+    Result<uint64_t> result = coord1.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(0, result.get());
   }
@@ -583,7 +557,7 @@ TEST(CoordinatorTest, Demoted)
   uint64_t position;
 
   {
-    Result<uint64_t> result = coord1.append("hello world");
+    Result<uint64_t> result = coord1.append("hello world", Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     position = result.get();
     EXPECT_EQ(1, position);
@@ -597,19 +571,19 @@ TEST(CoordinatorTest, Demoted)
   Coordinator coord2(2, &replica2, &network2);
 
   {
-    Result<uint64_t> result = coord2.elect();
+    Result<uint64_t> result = coord2.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(position, result.get());
   }
 
   {
-    Result<uint64_t> result = coord1.append("hello moto");
+    Result<uint64_t> result = coord1.append("hello moto", Timeout(1.0));
     ASSERT_TRUE(result.isError());
     EXPECT_EQ("Coordinator demoted", result.error());
   }
 
   {
-    Result<uint64_t> result = coord2.append("hello hello");
+    Result<uint64_t> result = coord2.append("hello hello", Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     position = result.get();
     EXPECT_EQ(2, position);
@@ -652,7 +626,7 @@ TEST(CoordinatorTest, Fill)
   Coordinator coord1(2, &replica1, &network1);
 
   {
-    Result<uint64_t> result = coord1.elect();
+    Result<uint64_t> result = coord1.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(0, result.get());
   }
@@ -660,7 +634,7 @@ TEST(CoordinatorTest, Fill)
   uint64_t position;
 
   {
-    Result<uint64_t> result = coord1.append("hello world");
+    Result<uint64_t> result = coord1.append("hello world", Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     position = result.get();
     EXPECT_EQ(1, position);
@@ -676,9 +650,9 @@ TEST(CoordinatorTest, Fill)
   Coordinator coord2(2, &replica3, &network2);
 
   {
-    Result<uint64_t> result = coord2.elect();
+    Result<uint64_t> result = coord2.elect(Timeout(1.0));
     ASSERT_TRUE(result.isNone());
-    result = coord2.elect();
+    result = coord2.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(position, result.get());
   }
@@ -730,7 +704,7 @@ TEST(CoordinatorTest, NotLearnedFill)
   Coordinator coord1(2, &replica1, &network1);
 
   {
-    Result<uint64_t> result = coord1.elect();
+    Result<uint64_t> result = coord1.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(0, result.get());
   }
@@ -738,7 +712,7 @@ TEST(CoordinatorTest, NotLearnedFill)
   uint64_t position;
 
   {
-    Result<uint64_t> result = coord1.append("hello world");
+    Result<uint64_t> result = coord1.append("hello world", Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     position = result.get();
     EXPECT_EQ(1, position);
@@ -754,9 +728,9 @@ TEST(CoordinatorTest, NotLearnedFill)
   Coordinator coord2(2, &replica3, &network2);
 
   {
-    Result<uint64_t> result = coord2.elect();
+    Result<uint64_t> result = coord2.elect(Timeout(1.0));
     ASSERT_TRUE(result.isNone());
-    result = coord2.elect();
+    result = coord2.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(position, result.get());
   }
@@ -799,13 +773,14 @@ TEST(CoordinatorTest, MultipleAppends)
   Coordinator coord(2, &replica1, &network);
 
   {
-    Result<uint64_t> result = coord.elect();
+    Result<uint64_t> result = coord.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(0, result.get());
   }
 
   for (uint64_t position = 1; position <= 10; position++) {
-    Result<uint64_t> result = coord.append(utils::stringify(position));
+    Result<uint64_t> result =
+      coord.append(utils::stringify(position), Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(position, result.get());
   }
@@ -857,13 +832,14 @@ TEST(CoordinatorTest, MultipleAppendsNot
   Coordinator coord1(2, &replica1, &network1);
 
   {
-    Result<uint64_t> result = coord1.elect();
+    Result<uint64_t> result = coord1.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(0, result.get());
   }
 
   for (uint64_t position = 1; position <= 10; position++) {
-    Result<uint64_t> result = coord1.append(utils::stringify(position));
+    Result<uint64_t> result =
+      coord1.append(utils::stringify(position), Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(position, result.get());
   }
@@ -878,9 +854,9 @@ TEST(CoordinatorTest, MultipleAppendsNot
   Coordinator coord2(2, &replica3, &network2);
 
   {
-    Result<uint64_t> result = coord2.elect();
+    Result<uint64_t> result = coord2.elect(Timeout(1.0));
     ASSERT_TRUE(result.isNone());
-    result = coord2.elect();
+    result = coord2.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(10, result.get());
   }
@@ -924,19 +900,20 @@ TEST(CoordinatorTest, Truncate)
   Coordinator coord(2, &replica1, &network);
 
   {
-    Result<uint64_t> result = coord.elect();
+    Result<uint64_t> result = coord.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(0, result.get());
   }
 
   for (uint64_t position = 1; position <= 10; position++) {
-    Result<uint64_t> result = coord.append(utils::stringify(position));
+    Result<uint64_t> result =
+      coord.append(utils::stringify(position), Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(position, result.get());
   }
 
   {
-    Result<uint64_t> result = coord.truncate(7);
+    Result<uint64_t> result = coord.truncate(7, Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(11, result.get());
   }
@@ -995,19 +972,20 @@ TEST(CoordinatorTest, TruncateNotLearned
   Coordinator coord1(2, &replica1, &network1);
 
   {
-    Result<uint64_t> result = coord1.elect();
+    Result<uint64_t> result = coord1.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(0, result.get());
   }
 
   for (uint64_t position = 1; position <= 10; position++) {
-    Result<uint64_t> result = coord1.append(utils::stringify(position));
+    Result<uint64_t> result =
+      coord1.append(utils::stringify(position), Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(position, result.get());
   }
 
   {
-    Result<uint64_t> result = coord1.truncate(7);
+    Result<uint64_t> result = coord1.truncate(7, Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(11, result.get());
   }
@@ -1022,9 +1000,9 @@ TEST(CoordinatorTest, TruncateNotLearned
   Coordinator coord2(2, &replica3, &network2);
 
   {
-    Result<uint64_t> result = coord2.elect();
+    Result<uint64_t> result = coord2.elect(Timeout(1.0));
     ASSERT_TRUE(result.isNone());
-    result = coord2.elect();
+    result = coord2.elect(Timeout(1.0));
     ASSERT_TRUE(result.isSome());
     EXPECT_EQ(11, result.get());
   }
@@ -1071,16 +1049,16 @@ TEST(LogTest, WriteRead)
 
   Log log(2, path2, pids);
 
-  Log::Writer writer(&log);
+  Log::Writer writer(&log, seconds(1.0));
 
-  Result<Log::Position> position = writer.append("hello world");
+  Result<Log::Position> position = writer.append("hello world", seconds(1.0));
 
   ASSERT_TRUE(position.isSome());
 
   Log::Reader reader(&log);
 
   Result<std::list<Log::Entry> > entries =
-    reader.read(position.get(), position.get());
+    reader.read(position.get(), position.get(), seconds(1.0));
 
   ASSERT_TRUE(entries.isSome());
   ASSERT_EQ(1, entries.get().size());
@@ -1107,9 +1085,9 @@ TEST(LogTest, Position)
 
   Log log(2, path2, pids);
 
-  Log::Writer writer(&log);
+  Log::Writer writer(&log, seconds(1.0));
 
-  Result<Log::Position> position = writer.append("hello world");
+  Result<Log::Position> position = writer.append("hello world", seconds(1.0));
 
   ASSERT_TRUE(position.isSome());
 

Modified: incubator/mesos/trunk/src/tests/master_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/master_tests.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/master_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/master_tests.cpp Wed Jan 11 22:28:41 2012
@@ -129,13 +129,14 @@ TEST(MasterTest, TaskRunning)
   vector<TaskDescription> tasks;
   tasks.push_back(task);
 
-  EXPECT_CALL(isolationModule, resourcesChanged(_, _,
-        Resources(offers[0].resources())))
+  EXPECT_CALL(isolationModule,
+              resourcesChanged(_, _, Resources(offers[0].resources())))
     .WillOnce(Trigger(&resourcesChangedCall));
 
   driver.launchTasks(offers[0].id(), tasks);
 
   WAIT_UNTIL(statusUpdateCall);
+
   EXPECT_EQ(TASK_RUNNING, status.state());
 
   WAIT_UNTIL(resourcesChangedCall);

Modified: incubator/mesos/trunk/src/tests/multihashmap_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/multihashmap_tests.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/multihashmap_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/multihashmap_tests.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 <gtest/gtest.h>
 
 #include <string>

Modified: incubator/mesos/trunk/src/tests/process_spawn.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/process_spawn.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/process_spawn.cpp (original)
+++ incubator/mesos/trunk/src/tests/process_spawn.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 <iostream>
 #include <climits>
 #include <cstdio>

Modified: incubator/mesos/trunk/src/tests/protobuf_io_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/protobuf_io_tests.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/protobuf_io_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/protobuf_io_tests.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 <string>
 
 #include <gmock/gmock.h>

Modified: incubator/mesos/trunk/src/tests/resource_offers_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/resource_offers_tests.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/resource_offers_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/resource_offers_tests.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 <gmock/gmock.h>
 
 #include <mesos/executor.hpp>
@@ -64,8 +82,8 @@ TEST(ResourceOffersTest, ResourceOfferWi
   EXPECT_GE(10, offers.size());
 
   Resources resources(offers[0].resources());
-  EXPECT_EQ(2, resources.get("cpus", Resource::Scalar()).value());
-  EXPECT_EQ(1024, resources.get("mem", Resource::Scalar()).value());
+  EXPECT_EQ(2, resources.get("cpus", Value::Scalar()).value());
+  EXPECT_EQ(1024, resources.get("mem", Value::Scalar()).value());
 
   driver.stop();
   driver.join();
@@ -122,7 +140,7 @@ TEST(ResourceOffersTest, TaskUsesNoResou
   WAIT_UNTIL(statusUpdateCall);
 
   EXPECT_EQ(task.task_id(), status.task_id());
-  EXPECT_EQ(TASK_FAILED, status.state());
+  EXPECT_EQ(TASK_LOST, status.state());
   EXPECT_TRUE(status.has_message());
   EXPECT_EQ("Task uses no resources", status.message());
 
@@ -167,7 +185,7 @@ TEST(ResourceOffersTest, TaskUsesInvalid
 
   Resource* cpus = task.add_resources();
   cpus->set_name("cpus");
-  cpus->set_type(Resource::SCALAR);
+  cpus->set_type(Value::SCALAR);
   cpus->mutable_scalar()->set_value(0);
 
   vector<TaskDescription> tasks;
@@ -186,7 +204,7 @@ TEST(ResourceOffersTest, TaskUsesInvalid
   WAIT_UNTIL(statusUpdateCall);
 
   EXPECT_EQ(task.task_id(), status.task_id());
-  EXPECT_EQ(TASK_FAILED, status.state());
+  EXPECT_EQ(TASK_LOST, status.state());
   EXPECT_TRUE(status.has_message());
   EXPECT_EQ("Task uses invalid resources", status.message());
 
@@ -231,7 +249,7 @@ TEST(ResourceOffersTest, TaskUsesMoreRes
 
   Resource* cpus = task.add_resources();
   cpus->set_name("cpus");
-  cpus->set_type(Resource::SCALAR);
+  cpus->set_type(Value::SCALAR);
   cpus->mutable_scalar()->set_value(2.01);
 
   vector<TaskDescription> tasks;
@@ -250,7 +268,7 @@ TEST(ResourceOffersTest, TaskUsesMoreRes
   WAIT_UNTIL(statusUpdateCall);
 
   EXPECT_EQ(task.task_id(), status.task_id());
-  EXPECT_EQ(TASK_FAILED, status.state());
+  EXPECT_EQ(TASK_LOST, status.state());
   EXPECT_TRUE(status.has_message());
   EXPECT_EQ("Task uses more resources than offered", status.message());
 
@@ -355,12 +373,12 @@ TEST(ResourceOffersTest, ResourcesGetReo
 
   Resource* cpus = task.add_resources();
   cpus->set_name("cpus");
-  cpus->set_type(Resource::SCALAR);
+  cpus->set_type(Value::SCALAR);
   cpus->mutable_scalar()->set_value(0);
 
   Resource* mem = task.add_resources();
   mem->set_name("mem");
-  mem->set_type(Resource::SCALAR);
+  mem->set_type(Value::SCALAR);
   mem->mutable_scalar()->set_value(1 * Gigabyte);
 
   vector<TaskDescription> tasks;
@@ -379,7 +397,7 @@ TEST(ResourceOffersTest, ResourcesGetReo
   WAIT_UNTIL(sched1StatusUpdateCall);
 
   EXPECT_EQ(task.task_id(), status.task_id());
-  EXPECT_EQ(TASK_FAILED, status.state());
+  EXPECT_EQ(TASK_LOST, status.state());
   EXPECT_TRUE(status.has_message());
   EXPECT_EQ("Task uses invalid resources", status.message());
 

Modified: incubator/mesos/trunk/src/tests/resources_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/resources_tests.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/resources_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/resources_tests.cpp Wed Jan 11 22:28:41 2012
@@ -34,15 +34,15 @@ using std::string;
 TEST(ResourcesTest, Parsing)
 {
   Resource cpus = Resources::parse("cpus", "45.55");
-  ASSERT_EQ(Resource::SCALAR, cpus.type());
+  ASSERT_EQ(Value::SCALAR, cpus.type());
   EXPECT_EQ(45.55, cpus.scalar().value());
 
   Resource ports = Resources::parse("ports", "[10000-20000, 30000-50000]");
-  ASSERT_EQ(Resource::RANGES, ports.type());
+  ASSERT_EQ(Value::RANGES, ports.type());
   EXPECT_EQ(2, ports.ranges().range_size());
 
   Resource disks = Resources::parse("disks", "{sda1}");
-  ASSERT_EQ(Resource::SET, disks.type());
+  ASSERT_EQ(Value::SET, disks.type());
   ASSERT_EQ(1, disks.set().item_size());
   EXPECT_EQ("sda1", disks.set().item(0));
 
@@ -89,7 +89,7 @@ TEST(ResourcesTest, InitializedIsEmpty)
 TEST(ResourcesTest, BadResourcesNotAllocatable)
 {
   Resource cpus;
-  cpus.set_type(Resource::SCALAR);
+  cpus.set_type(Value::SCALAR);
   cpus.mutable_scalar()->set_value(1);
   Resources r;
   r += cpus;
@@ -159,14 +159,14 @@ TEST(ResourcesTest, ScalarAddition)
 
   Resources sum = r1 + r2;
   EXPECT_EQ(2, sum.size());
-  EXPECT_EQ(3, sum.get("cpus", Resource::Scalar()).value());
-  EXPECT_EQ(15, sum.get("mem", Resource::Scalar()).value());
+  EXPECT_EQ(3, sum.get("cpus", Value::Scalar()).value());
+  EXPECT_EQ(15, sum.get("mem", Value::Scalar()).value());
 
   Resources r = r1;
   r += r2;
   EXPECT_EQ(2, r.size());
-  EXPECT_EQ(3, r.get("cpus", Resource::Scalar()).value());
-  EXPECT_EQ(15, r.get("mem", Resource::Scalar()).value());
+  EXPECT_EQ(3, r.get("cpus", Value::Scalar()).value());
+  EXPECT_EQ(15, r.get("mem", Value::Scalar()).value());
 }
 
 
@@ -188,13 +188,13 @@ TEST(ResourcesTest, ScalarSubtraction)
 
   Resources diff = r1 - r2;
   EXPECT_EQ(2, diff.size());
-  EXPECT_EQ(49.5, diff.get("cpus", Resource::Scalar()).value());
-  EXPECT_EQ(3072, diff.get("mem", Resource::Scalar()).value());
+  EXPECT_EQ(49.5, diff.get("cpus", Value::Scalar()).value());
+  EXPECT_EQ(3072, diff.get("mem", Value::Scalar()).value());
 
   Resources r = r1;
   r -= r2;
-  EXPECT_EQ(49.5, diff.get("cpus", Resource::Scalar()).value());
-  EXPECT_EQ(3072, diff.get("mem", Resource::Scalar()).value());
+  EXPECT_EQ(49.5, diff.get("cpus", Value::Scalar()).value());
+  EXPECT_EQ(3072, diff.get("mem", Value::Scalar()).value());
 
   r = r1;
   r -= r1;
@@ -218,8 +218,15 @@ TEST(ResourcesTest, RangesEquals)
 
 TEST(ResourcesTest, RangesSubset)
 {
-  Resource ports1 = Resources::parse("ports", "[20000-40000]");
-  Resource ports2 = Resources::parse("ports", "[20000-40000, 50000-60000]");
+  Resource ports1 = Resources::parse("ports", "[2-2, 4-5]");
+  Resource ports2 = Resources::parse("ports", "[1-10]");
+  Resource ports3 = Resources::parse("ports", "[2-3]");
+  Resource ports4 = Resources::parse("ports", "[1-2, 4-6]");
+
+  EXPECT_EQ(2, ports1.ranges().range_size());
+  EXPECT_EQ(1, ports2.ranges().range_size());
+  EXPECT_EQ(1, ports3.ranges().range_size());
+  EXPECT_EQ(2, ports4.ranges().range_size());
 
   Resources r1;
   r1 += ports1;
@@ -227,10 +234,20 @@ TEST(ResourcesTest, RangesSubset)
   Resources r2;
   r2 += ports2;
 
-  EXPECT_EQ(1, r1.size());
-  EXPECT_EQ(1, r2.size());
+  Resources r3;
+  r3 += ports3;
+
+  Resources r4;
+  r4 += ports4;
+
   EXPECT_TRUE(r1 <= r2);
   EXPECT_FALSE(r2 <= r1);
+  EXPECT_FALSE(r1 <= r3);
+  EXPECT_FALSE(r3 <= r1);
+  EXPECT_TRUE(r3 <= r2);
+  EXPECT_FALSE(r2 <= r3);
+  EXPECT_TRUE(r1 <= r4);
+  EXPECT_TRUE(r4 <= r2);
 }
 
 
@@ -245,7 +262,7 @@ TEST(ResourcesTest, RangesAddition)
 
   EXPECT_EQ(1, r.size());
 
-  const Resource::Ranges& ranges = r.get("ports", Resource::Ranges());
+  const Value::Ranges& ranges = r.get("ports", Value::Ranges());
 
   EXPECT_EQ(1, ranges.range_size());
   EXPECT_EQ(10000, ranges.range(0).begin());
@@ -264,7 +281,7 @@ TEST(ResourcesTest, RangesSubtraction)
 
   EXPECT_EQ(1, r.size());
 
-  const Resource::Ranges& ranges = r.get("ports", Resource::Ranges());
+  const Value::Ranges& ranges = r.get("ports", Value::Ranges());
 
   EXPECT_EQ(1, ranges.range_size());
   EXPECT_EQ(20001, ranges.range(0).begin());
@@ -272,6 +289,83 @@ TEST(ResourcesTest, RangesSubtraction)
 }
 
 
+TEST(ResourcesTest, RangesSubtraction1)
+{
+  Resource ports1 = Resources::parse("ports", "[50000-60000]");
+  Resource ports2 = Resources::parse("ports", "[50000-50001]");
+
+  Resources r;
+  r += ports1;
+  r -= ports2;
+
+  EXPECT_EQ(1, r.size());
+
+  const Value::Ranges& ranges = r.get("ports", Value::Ranges());
+
+  EXPECT_EQ(1, ranges.range_size());
+  EXPECT_EQ(50002, ranges.range(0).begin());
+  EXPECT_EQ(60000, ranges.range(0).end());
+}
+
+
+TEST(ResourcesTest, RangesSubtraction2)
+{
+  Resource ports1 = Resources::parse("ports", "[50000-60000]");
+  Resource ports2 = Resources::parse("ports", "[50000-50000]");
+
+  Resources r;
+  r += ports1;
+  r -= ports2;
+
+  EXPECT_EQ(1, r.size());
+
+  const Value::Ranges& ranges = r.get("ports", Value::Ranges());
+
+  EXPECT_EQ(1, ranges.range_size());
+  EXPECT_EQ(50001, ranges.range(0).begin());
+  EXPECT_EQ(60000, ranges.range(0).end());
+}
+
+
+TEST(ResourcesTest, RangesSubtraction3)
+{
+  Resources resources = Resources::parse("ports:[50000-60000]");
+
+  Resources resourcesOffered = Resources::parse("");
+  Resources resourcesInUse = Resources::parse("ports:[50000-50001]");
+
+  Resources resourcesFree = resources - (resourcesOffered + resourcesInUse);
+
+  resourcesFree = resourcesFree.allocatable();
+
+  EXPECT_EQ(1, resourcesFree.size());
+
+  const Value::Ranges& ranges = resourcesFree.get("ports", Value::Ranges());
+
+  EXPECT_EQ(1, ranges.range_size());
+  EXPECT_EQ(50002, ranges.range(0).begin());
+  EXPECT_EQ(60000, ranges.range(0).end());
+}
+
+
+TEST(ResourcesTest, RangesSubtraction4)
+{
+  Resources resources = Resources::parse("ports:[50000-60000]");
+
+  Resources resourcesOffered;
+
+  resourcesOffered += resources;
+
+  resourcesOffered -= resources;
+
+  EXPECT_EQ(1, resourcesOffered.size());
+
+  const Value::Ranges& ranges = resourcesOffered.get("ports", Value::Ranges());
+
+  EXPECT_EQ(0, ranges.range_size());
+}
+
+
 TEST(ResourcesTest, SetEquals)
 {
   Resource disks = Resources::parse("disks", "{sda1}");
@@ -315,7 +409,7 @@ TEST(ResourcesTest, SetAddition)
 
   EXPECT_EQ(1, r.size());
 
-  const Resource::Set& set = r.get("disks", Resource::Set());
+  const Value::Set& set = r.get("disks", Value::Set());
 
   EXPECT_EQ(4, set.item_size());
 }
@@ -332,7 +426,7 @@ TEST(ResourcesTest, SetSubtraction)
 
   EXPECT_EQ(1, r.size());
 
-  const Resource::Set& set = r.get("disks", Resource::Set());
+  const Value::Set& set = r.get("disks", Value::Set());
 
   EXPECT_EQ(1, set.item_size());
   EXPECT_EQ("sda1", set.item(0));

Modified: incubator/mesos/trunk/src/tests/strings_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/strings_tests.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/strings_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/strings_tests.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 <gtest/gtest.h>
 
 #include "common/strings.hpp"

Modified: incubator/mesos/trunk/src/tests/url_processor_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/url_processor_tests.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/url_processor_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/url_processor_tests.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 <algorithm>
 
 #include <fstream>

Modified: incubator/mesos/trunk/src/tests/utils.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/utils.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/utils.hpp (original)
+++ incubator/mesos/trunk/src/tests/utils.hpp Wed Jan 11 22:28:41 2012
@@ -182,7 +182,7 @@ MATCHER_P3(MsgMatcher, name, from, to, "
  * using the message matcher (see above) as well as the MockFilter
  * (see above).
  */
-#define EXPECT_MSG(mockFilter, name, from, to)                \
+#define EXPECT_MSG(mockFilter, name, from, to)                  \
   EXPECT_CALL(mockFilter, filter(MsgMatcher(name, from, to)))
 
 
@@ -228,7 +228,7 @@ ACTION_P(SendStatusUpdate, state)
     int sleeps = 0;                                                     \
     do {                                                                \
       __sync_synchronize();                                             \
-      if ((trigger).value)                                                \
+      if ((trigger).value)                                              \
         break;                                                          \
       usleep(10);                                                       \
       if (sleeps++ >= 200000) {                                         \

Modified: incubator/mesos/trunk/src/tests/utils_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/utils_tests.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/utils_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/utils_tests.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 <gtest/gtest.h>
 
 #include "common/foreach.hpp"

Modified: incubator/mesos/trunk/src/tests/zookeeper_server.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_server.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/zookeeper_server.cpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_server.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 <stdarg.h>
 

Modified: incubator/mesos/trunk/src/tests/zookeeper_server.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_server.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/zookeeper_server.hpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_server.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 __TESTING_ZOO_KEEPER_SERVER_HPP__
 #define __TESTING_ZOO_KEEPER_SERVER_HPP__
 

Modified: incubator/mesos/trunk/src/tests/zookeeper_server_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_server_tests.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/zookeeper_server_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_server_tests.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 <glog/logging.h>
 
 #include <gtest/gtest.h>

Modified: incubator/mesos/trunk/src/tests/zookeeper_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_tests.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/zookeeper_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_tests.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 <string>
 
 #include <gtest/gtest.h>

Modified: incubator/mesos/trunk/src/webui/master/framework.tpl
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/master/framework.tpl?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/master/framework.tpl (original)
+++ incubator/mesos/trunk/src/webui/master/framework.tpl Wed Jan 11 22:28:41 2012
@@ -44,7 +44,7 @@
 <p>
   Name: {{framework['name']}}<br />
   User: {{framework['user']}}<br />
-  Connected: {{format_time(framework['registered_time'])}}<br />
+  Registered: {{format_time(framework['registered_time'])}}<br />
   Executor: {{framework['executor_uri']}}<br />
   Running Tasks: {{len(framework['tasks'])}}<br />
   CPUs: {{framework['resources']['cpus']}}<br />
@@ -60,7 +60,7 @@
     <th class="lists">ID</th>
     <th class="lists">Name</th>
     <th class="lists">State</th>
-    <th class="lists">Running On Slave</th>
+    <th class="lists">Slave</th>
   </tr>
   % for i in range(len(framework['tasks'])):
   %   task = framework['tasks'][i]

Modified: incubator/mesos/trunk/src/webui/static/stylesheet.css
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/webui/static/stylesheet.css?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/webui/static/stylesheet.css (original)
+++ incubator/mesos/trunk/src/webui/static/stylesheet.css 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/zookeeper/zookeeper.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/zookeeper/zookeeper.cpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/zookeeper/zookeeper.cpp (original)
+++ incubator/mesos/trunk/src/zookeeper/zookeeper.cpp Wed Jan 11 22:28:41 2012
@@ -576,13 +576,13 @@ int64_t ZooKeeper::getSessionId()
 }
 
 
-int ZooKeeper::authenticate(const string& username, const string& password)
+int ZooKeeper::authenticate(const string& scheme, const string& credentials)
 {
 #ifndef USE_THREADED_ZOOKEEPER
   return process::call(impl->self(), &ZooKeeperImpl::authenticate,
-                       cref(username), cref(password));
+                       cref(scheme), cref(credentials));
 #else
-  Promise<int> promise = impl->authenticate(username, password);
+  Promise<int> promise = impl->authenticate(scheme, credentials);
   return promise.future().get();
 #endif // USE_THREADED_ZOOKEEPER
 }

Modified: incubator/mesos/trunk/src/zookeeper/zookeeper.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/zookeeper/zookeeper.hpp?rev=1230289&r1=1230288&r2=1230289&view=diff
==============================================================================
--- incubator/mesos/trunk/src/zookeeper/zookeeper.hpp (original)
+++ incubator/mesos/trunk/src/zookeeper/zookeeper.hpp Wed Jan 11 22:28:41 2012
@@ -17,11 +17,7 @@
  */
 
 /**
- * ZooKeeper C++ API. Originally created to be used with the
- * Libprocess library (http://www.eecs.berkeley.edu/~benh/libprocess),
- * these C++ classes could also use a different underlying
- * implementation, such as calling the synchronous functions exposed
- * by the ZooKeeper C++ API.
+ * ZooKeeper C++ API.
  *
  * To provide for varying underlying implementations the pimpl idiom
  * (also known as the compiler-firewall, bridge pattern, etc) was used