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/10/27 00:07:24 UTC

svn commit: r1402695 - in /incubator/mesos/trunk/src: ./ launcher/ tests/ tests/external/ExamplesTest/ tests/external/SampleFrameworks/

Author: benh
Date: Fri Oct 26 22:07:22 2012
New Revision: 1402695

URL: http://svn.apache.org/viewvc?rev=1402695&view=rev
Log:
Added tests::Flags (so they can be accessed by other tests) and used
flags.verbose to indicated when "external" tests should print out to
stdout/stderr (no longer creating test_output).

Also mixed into this commit was updating everything to be in the
'tests' namespace instead of the 'test' namespace and
s/SampleFrameworks/ExamplesTest/g.

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

Added:
    incubator/mesos/trunk/src/tests/examples_tests.cpp
      - copied, changed from r1402694, incubator/mesos/trunk/src/tests/sample_frameworks_tests.cpp
    incubator/mesos/trunk/src/tests/external.cpp
    incubator/mesos/trunk/src/tests/external.hpp
    incubator/mesos/trunk/src/tests/external/ExamplesTest/
    incubator/mesos/trunk/src/tests/external/ExamplesTest/CppFramework.sh
      - copied, changed from r1402694, incubator/mesos/trunk/src/tests/external/SampleFrameworks/CppFramework.sh
    incubator/mesos/trunk/src/tests/external/ExamplesTest/JavaExceptionFramework.sh
      - copied, changed from r1402694, incubator/mesos/trunk/src/tests/external/SampleFrameworks/JavaExceptionFramework.sh
    incubator/mesos/trunk/src/tests/external/ExamplesTest/JavaFramework.sh
      - copied, changed from r1402694, incubator/mesos/trunk/src/tests/external/SampleFrameworks/JavaFramework.sh
    incubator/mesos/trunk/src/tests/external/ExamplesTest/NoExecutorFramework.sh
      - copied, changed from r1402694, incubator/mesos/trunk/src/tests/external/SampleFrameworks/NoExecutorFramework.sh
    incubator/mesos/trunk/src/tests/external/ExamplesTest/PythonFramework.sh
      - copied, changed from r1402694, incubator/mesos/trunk/src/tests/external/SampleFrameworks/PythonFramework.sh
    incubator/mesos/trunk/src/tests/flags.hpp
Removed:
    incubator/mesos/trunk/src/tests/external/SampleFrameworks/CppFramework.sh
    incubator/mesos/trunk/src/tests/external/SampleFrameworks/JavaExceptionFramework.sh
    incubator/mesos/trunk/src/tests/external/SampleFrameworks/JavaFramework.sh
    incubator/mesos/trunk/src/tests/external/SampleFrameworks/NoExecutorFramework.sh
    incubator/mesos/trunk/src/tests/external/SampleFrameworks/PythonFramework.sh
    incubator/mesos/trunk/src/tests/external_test.hpp
    incubator/mesos/trunk/src/tests/external_tests.cpp
    incubator/mesos/trunk/src/tests/sample_frameworks_tests.cpp
Modified:
    incubator/mesos/trunk/src/Makefile.am
    incubator/mesos/trunk/src/launcher/executor.cpp
    incubator/mesos/trunk/src/tests/allocator_tests.cpp
    incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp
    incubator/mesos/trunk/src/tests/cgroups_isolation_tests.cpp
    incubator/mesos/trunk/src/tests/configurator_tests.cpp
    incubator/mesos/trunk/src/tests/exception_tests.cpp
    incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp
    incubator/mesos/trunk/src/tests/files_tests.cpp
    incubator/mesos/trunk/src/tests/gc_tests.cpp
    incubator/mesos/trunk/src/tests/killtree_tests.cpp
    incubator/mesos/trunk/src/tests/log_tests.cpp
    incubator/mesos/trunk/src/tests/logging_tests.cpp
    incubator/mesos/trunk/src/tests/lxc_isolation_tests.cpp
    incubator/mesos/trunk/src/tests/main.cpp
    incubator/mesos/trunk/src/tests/master_detector_tests.cpp
    incubator/mesos/trunk/src/tests/master_tests.cpp
    incubator/mesos/trunk/src/tests/resource_offers_tests.cpp
    incubator/mesos/trunk/src/tests/slave_state_tests.cpp
    incubator/mesos/trunk/src/tests/state_tests.cpp
    incubator/mesos/trunk/src/tests/utils.cpp
    incubator/mesos/trunk/src/tests/utils.hpp
    incubator/mesos/trunk/src/tests/zookeeper_test.cpp
    incubator/mesos/trunk/src/tests/zookeeper_test.hpp
    incubator/mesos/trunk/src/tests/zookeeper_test_server.cpp
    incubator/mesos/trunk/src/tests/zookeeper_test_server.hpp
    incubator/mesos/trunk/src/tests/zookeeper_tests.cpp

Modified: incubator/mesos/trunk/src/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/Makefile.am?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/Makefile.am (original)
+++ incubator/mesos/trunk/src/Makefile.am Fri Oct 26 22:07:22 2012
@@ -211,8 +211,8 @@ libmesos_no_third_party_la_SOURCES += co
 	slave/paths.hpp slave/state.hpp					\
 	slave/process_based_isolation_module.hpp slave/reaper.hpp	\
 	slave/slave.hpp slave/solaris_project_isolation_module.hpp	\
-	slave/webui.hpp tests/environment.hpp tests/external_test.hpp	\
-	tests/zookeeper_test.hpp tests/utils.hpp			\
+	slave/webui.hpp tests/environment.hpp tests/external.hpp	\
+	tests/zookeeper_test.hpp tests/flags.hpp tests/utils.hpp	\
 	tests/zookeeper_test_server.hpp zookeeper/authentication.hpp	\
 	zookeeper/group.hpp zookeeper/watcher.hpp			\
 	zookeeper/zookeeper.hpp zookeeper/url.hpp
@@ -774,8 +774,8 @@ mesos_tests_SOURCES = tests/main.cpp tes
 	              tests/fault_tolerance_tests.cpp			\
 	              tests/files_tests.cpp tests/flags_tests.cpp	\
 	              tests/log_tests.cpp tests/resources_tests.cpp	\
-	              tests/external_tests.cpp				\
-	              tests/sample_frameworks_tests.cpp			\
+	              tests/external.cpp				\
+	              tests/examples_tests.cpp				\
 	              tests/configurator_tests.cpp			\
 	              tests/multihashmap_tests.cpp			\
 	              tests/protobuf_io_tests.cpp			\

Modified: incubator/mesos/trunk/src/launcher/executor.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/launcher/executor.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/launcher/executor.cpp (original)
+++ incubator/mesos/trunk/src/launcher/executor.cpp Fri Oct 26 22:07:22 2012
@@ -63,7 +63,7 @@ static void waiter(pid_t pid, const Task
 
   Try<string> message = WIFEXITED(status)
     ? strings::format("Command exited with status %d", WEXITSTATUS(status))
-    : strings::format("Command terminated with signal %s",
+    : strings::format("Command terminated with signal '%s'",
                       strsignal(WTERMSIG(status)));
 
   if (message.isSome()) {

Modified: incubator/mesos/trunk/src/tests/allocator_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/allocator_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/allocator_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/allocator_tests.cpp Fri Oct 26 22:07:22 2012
@@ -33,7 +33,7 @@
 
 using namespace mesos;
 using namespace mesos::internal;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using mesos::internal::master::AllocatorProcess;
 using mesos::internal::master::Master;

Modified: incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp Fri Oct 26 22:07:22 2012
@@ -28,7 +28,7 @@
 
 using namespace mesos;
 using namespace mesos::internal;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using mesos::internal::master::AllocatorProcess;
 using mesos::internal::master::Master;

Modified: incubator/mesos/trunk/src/tests/cgroups_isolation_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/cgroups_isolation_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/cgroups_isolation_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/cgroups_isolation_tests.cpp Fri Oct 26 22:07:22 2012
@@ -18,7 +18,7 @@
 
 #include <gtest/gtest.h>
 
-#include "tests/external_test.hpp"
+#include "tests/external.hpp"
 
 // Run the balloon framework under cgroups isolation.
 TEST_EXTERNAL(CgroupsIsolation, ROOT_CGROUPS_BalloonFramework)

Modified: incubator/mesos/trunk/src/tests/configurator_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/configurator_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/configurator_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/configurator_tests.cpp Fri Oct 26 22:07:22 2012
@@ -33,7 +33,7 @@ using std::string;
 
 using namespace mesos;
 using namespace mesos::internal;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 
 TEST(ConfiguratorTest, Environment)

Copied: incubator/mesos/trunk/src/tests/examples_tests.cpp (from r1402694, incubator/mesos/trunk/src/tests/sample_frameworks_tests.cpp)
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/examples_tests.cpp?p2=incubator/mesos/trunk/src/tests/examples_tests.cpp&p1=incubator/mesos/trunk/src/tests/sample_frameworks_tests.cpp&r1=1402694&r2=1402695&rev=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/sample_frameworks_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/examples_tests.cpp Fri Oct 26 22:07:22 2012
@@ -18,18 +18,18 @@
 
 #include <gtest/gtest.h>
 
-#include "tests/external_test.hpp"
+#include "tests/external.hpp"
 
 
 // Run each of the sample frameworks in local mode.
-TEST_EXTERNAL(SampleFrameworks, CppFramework)
-TEST_EXTERNAL(SampleFrameworks, NoExecutorFramework)
+TEST_EXTERNAL(ExamplesTest, CppFramework)
+TEST_EXTERNAL(ExamplesTest, NoExecutorFramework)
 
 #ifdef MESOS_HAS_JAVA
-  TEST_EXTERNAL(SampleFrameworks, JavaFramework)
-  TEST_EXTERNAL(SampleFrameworks, JavaExceptionFramework)
+  TEST_EXTERNAL(ExamplesTest, JavaFramework)
+  TEST_EXTERNAL(ExamplesTest, JavaExceptionFramework)
 #endif
 
 #ifdef MESOS_HAS_PYTHON
-  TEST_EXTERNAL(SampleFrameworks, PythonFramework)
+  TEST_EXTERNAL(ExamplesTest, PythonFramework)
 #endif

Modified: incubator/mesos/trunk/src/tests/exception_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/exception_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/exception_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/exception_tests.cpp Fri Oct 26 22:07:22 2012
@@ -34,7 +34,7 @@
 
 using namespace mesos;
 using namespace mesos::internal;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using mesos::internal::master::Master;
 

Added: incubator/mesos/trunk/src/tests/external.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/external.cpp?rev=1402695&view=auto
==============================================================================
--- incubator/mesos/trunk/src/tests/external.cpp (added)
+++ incubator/mesos/trunk/src/tests/external.cpp Fri Oct 26 22:07:22 2012
@@ -0,0 +1,129 @@
+/**
+ * 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 <signal.h> // For strsignal.
+#include <stdio.h> // For freopen.
+#include <string.h> // For strlen, strerror.
+
+#include <sys/wait.h> // For wait (and associated macros).
+
+#include <gtest/gtest.h>
+
+#include <string>
+
+#include <stout/os.hpp>
+#include <stout/path.hpp>
+#include <stout/strings.hpp>
+
+#include "tests/external.hpp"
+#include "tests/utils.hpp"
+
+using std::string;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+namespace external {
+
+void run(const char* testCase, const char* testName)
+{
+  // Adjust the test name to remove any 'DISABLED_' prefix (to make
+  // things easier to read). While this might seem alarming, if we are
+  // "running" a disabled test it must be the case that the test was
+  // explicitly enabled (e.g., via 'gtest_filter').
+  if (strings::startsWith(testName, "DISABLED_")) {
+    testName += strlen("DISABLED_");
+  }
+
+  // Fork a process to change directory and run the test.
+  pid_t pid;
+  if ((pid = fork()) == -1) {
+    FAIL() << "Failed to fork to launch external test";
+  }
+  if (pid) {
+    // In parent process.
+    int status;
+    while (wait(&status) != pid || WIFSTOPPED(status));
+    CHECK(WIFEXITED(status) || WIFSIGNALED(status));
+
+    if (WIFEXITED(status)) {
+      if (WEXITSTATUS(status) != 0) {
+        FAIL()
+          << strings::join("/", testCase, testName)
+          << " exited with status " << WEXITSTATUS(status);
+      }
+    } else {
+      FAIL()
+        << strings::join("/", testCase, testName)
+        << " terminated with signal '" << strsignal(WTERMSIG(status)) << "'";
+    }
+  } else {
+    // Create a temporary directory for the test.
+    const string& path =
+      path::join("/tmp", strings::join("_", testCase, testName, "XXXXXX"));
+    Try<string> directory = os::mkdtemp(path);
+
+    if (directory.isError()) {
+      std::cerr << "Failed to create temporary directory at '"
+                << path << "':" << directory.error() << std::endl;
+      abort();
+    }
+
+    // Run the test out of the temporary directory we created.
+    if (!os::chdir(directory.get())) {
+      std::cerr << "Failed to chdir into '" << path << "'" << std::endl;
+      abort();
+    }
+
+    // Redirect output to /dev/null unless the test is verbose.
+    if (!flags.verbose) {
+      if (freopen("/dev/null", "w", stdout) == NULL ||
+          freopen("/dev/null", "w", stderr) == NULL) {
+        std::cerr << "Failed to redirect stdout/stderr to /dev/null:"
+                  << strerror(errno) << std::endl;
+        abort();
+      }
+    }
+
+    // Set up the environment for executing the script.
+    os::setenv("MESOS_SOURCE_DIR", flags.source_dir);
+    os::setenv("MESOS_BUILD_DIR", flags.build_dir);
+    os::setenv("MESOS_WEBUI_DIR", path::join(flags.source_dir, "src", "webui"));
+    os::setenv("MESOS_LAUNCHER_DIR", path::join(flags.build_dir, "src"));
+
+    // Now execute the script.
+    const string& script =
+      path::join(flags.source_dir,
+                 "src",
+                 "tests",
+                 "external",
+                 testCase,
+                 testName) + ".sh";
+
+    execl(script.c_str(), script.c_str(), (char*) NULL);
+
+    std::cerr << "Failed to execute '" << script << "': "
+              << strerror(errno) << std::endl;
+    abort();
+  }
+}
+
+} // namespace external {
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {

Added: incubator/mesos/trunk/src/tests/external.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/external.hpp?rev=1402695&view=auto
==============================================================================
--- incubator/mesos/trunk/src/tests/external.hpp (added)
+++ incubator/mesos/trunk/src/tests/external.hpp Fri Oct 26 22:07:22 2012
@@ -0,0 +1,49 @@
+/**
+ * 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 __TESTS_EXTERNAL_HPP__
+#define __TESTS_EXTERNAL_HPP__
+
+#include <gtest/gtest.h>
+
+// Run an external test with the given name. The test is expected to
+// be located in src/tests/external/<testCase>/<testName>.sh. We
+// execute this script in temporary directory and pipe its output to
+// '/dev/null' unless the verbose option is specified. The "test"
+// passes if the script returns 0.
+#define TEST_EXTERNAL(testCase, testName)                               \
+  TEST(testCase, testName) {                                            \
+    mesos::internal::tests::external::run(#testCase, #testName);        \
+  }
+
+
+namespace mesos {
+namespace internal {
+namespace tests {
+namespace external {
+
+// Function called by TEST_EXTERNAL to execute external tests. See
+// explanation of parameters at definition of TEST_EXTERNAL.
+void run(const char* testCase, const char* testName);
+
+} // namespace external {
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __TESTS_EXTERNAL_HPP__

Copied: incubator/mesos/trunk/src/tests/external/ExamplesTest/CppFramework.sh (from r1402694, incubator/mesos/trunk/src/tests/external/SampleFrameworks/CppFramework.sh)
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/external/ExamplesTest/CppFramework.sh?p2=incubator/mesos/trunk/src/tests/external/ExamplesTest/CppFramework.sh&p1=incubator/mesos/trunk/src/tests/external/SampleFrameworks/CppFramework.sh&r1=1402694&r2=1402695&rev=1402695&view=diff
==============================================================================
    (empty)

Copied: incubator/mesos/trunk/src/tests/external/ExamplesTest/JavaExceptionFramework.sh (from r1402694, incubator/mesos/trunk/src/tests/external/SampleFrameworks/JavaExceptionFramework.sh)
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/external/ExamplesTest/JavaExceptionFramework.sh?p2=incubator/mesos/trunk/src/tests/external/ExamplesTest/JavaExceptionFramework.sh&p1=incubator/mesos/trunk/src/tests/external/SampleFrameworks/JavaExceptionFramework.sh&r1=1402694&r2=1402695&rev=1402695&view=diff
==============================================================================
    (empty)

Copied: incubator/mesos/trunk/src/tests/external/ExamplesTest/JavaFramework.sh (from r1402694, incubator/mesos/trunk/src/tests/external/SampleFrameworks/JavaFramework.sh)
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/external/ExamplesTest/JavaFramework.sh?p2=incubator/mesos/trunk/src/tests/external/ExamplesTest/JavaFramework.sh&p1=incubator/mesos/trunk/src/tests/external/SampleFrameworks/JavaFramework.sh&r1=1402694&r2=1402695&rev=1402695&view=diff
==============================================================================
    (empty)

Copied: incubator/mesos/trunk/src/tests/external/ExamplesTest/NoExecutorFramework.sh (from r1402694, incubator/mesos/trunk/src/tests/external/SampleFrameworks/NoExecutorFramework.sh)
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/external/ExamplesTest/NoExecutorFramework.sh?p2=incubator/mesos/trunk/src/tests/external/ExamplesTest/NoExecutorFramework.sh&p1=incubator/mesos/trunk/src/tests/external/SampleFrameworks/NoExecutorFramework.sh&r1=1402694&r2=1402695&rev=1402695&view=diff
==============================================================================
    (empty)

Copied: incubator/mesos/trunk/src/tests/external/ExamplesTest/PythonFramework.sh (from r1402694, incubator/mesos/trunk/src/tests/external/SampleFrameworks/PythonFramework.sh)
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/external/ExamplesTest/PythonFramework.sh?p2=incubator/mesos/trunk/src/tests/external/ExamplesTest/PythonFramework.sh&p1=incubator/mesos/trunk/src/tests/external/SampleFrameworks/PythonFramework.sh&r1=1402694&r2=1402695&rev=1402695&view=diff
==============================================================================
    (empty)

Modified: incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp Fri Oct 26 22:07:22 2012
@@ -38,7 +38,7 @@
 
 using namespace mesos;
 using namespace mesos::internal;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using mesos::internal::master::Master;
 

Modified: incubator/mesos/trunk/src/tests/files_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/files_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/files_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/files_tests.cpp Fri Oct 26 22:07:22 2012
@@ -33,7 +33,7 @@
 #include "tests/utils.hpp"
 
 using namespace mesos::internal;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using process::Future;
 

Added: incubator/mesos/trunk/src/tests/flags.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/flags.hpp?rev=1402695&view=auto
==============================================================================
--- incubator/mesos/trunk/src/tests/flags.hpp (added)
+++ incubator/mesos/trunk/src/tests/flags.hpp Fri Oct 26 22:07:22 2012
@@ -0,0 +1,72 @@
+/**
+ * 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 __TESTS_FLAGS_HPP__
+#define __TESTS_FLAGS_HPP__
+
+#include <string>
+
+#include <stout/os.hpp>
+
+#include "flags/flags.hpp"
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+class Flags : public virtual flags::FlagsBase
+{
+public:
+  Flags()
+  {
+    // We log to stderr by default, but when running tests we'd prefer
+    // less junk to fly by, so force one to specify the verbosity.
+    add(&Flags::verbose,
+        "verbose",
+        "Log all severity levels to stderr",
+        false);
+
+    // We determine the defaults for 'source_dir' and 'build_dir' from
+    // preprocessor definitions (at the time this comment was written
+    // these were set via '-DSOURCE_DIR=...' and '-DBUILD_DIR=...' in
+    // src/Makefile.am).
+    Try<std::string> path = os::realpath(SOURCE_DIR);
+    CHECK(path.isSome()) << path.error();
+    add(&Flags::source_dir,
+        "source_dir",
+        "Where to find the source directory",
+        path.get());
+
+    path = os::realpath(BUILD_DIR);
+    CHECK(path.isSome()) << path.error();
+    add(&Flags::build_dir,
+        "build_dir",
+        "Where to find the build directory",
+        path.get());
+  }
+
+  bool verbose;
+  std::string source_dir;
+  std::string build_dir;
+};
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __TESTS_FLAGS_HPP__

Modified: incubator/mesos/trunk/src/tests/gc_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/gc_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/gc_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/gc_tests.cpp Fri Oct 26 22:07:22 2012
@@ -45,7 +45,7 @@
 
 using namespace mesos;
 using namespace mesos::internal;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using mesos::internal::master::Master;
 

Modified: incubator/mesos/trunk/src/tests/killtree_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/killtree_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/killtree_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/killtree_tests.cpp Fri Oct 26 22:07:22 2012
@@ -18,7 +18,7 @@
 
 #include <gtest/gtest.h>
 
-#include "tests/external_test.hpp"
+#include "tests/external.hpp"
 
 
 // Run a number of tests for the LXC isolation module.

Modified: incubator/mesos/trunk/src/tests/log_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/log_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/log_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/log_tests.cpp Fri Oct 26 22:07:22 2012
@@ -40,7 +40,7 @@
 using namespace mesos;
 using namespace mesos::internal;
 using namespace mesos::internal::log;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using namespace process;
 

Modified: incubator/mesos/trunk/src/tests/logging_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/logging_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/logging_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/logging_tests.cpp Fri Oct 26 22:07:22 2012
@@ -27,7 +27,7 @@
 #include "tests/utils.hpp"
 
 using namespace mesos::internal;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using process::http::BadRequest;
 using process::http::OK;

Modified: incubator/mesos/trunk/src/tests/lxc_isolation_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/lxc_isolation_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/lxc_isolation_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/lxc_isolation_tests.cpp Fri Oct 26 22:07:22 2012
@@ -18,7 +18,7 @@
 
 #include <gtest/gtest.h>
 
-#include "tests/external_test.hpp"
+#include "tests/external.hpp"
 
 
 // Run a number of tests for the LXC isolation module.

Modified: incubator/mesos/trunk/src/tests/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/main.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/main.cpp (original)
+++ incubator/mesos/trunk/src/tests/main.cpp Fri Oct 26 22:07:22 2012
@@ -27,14 +27,13 @@
 #include "configurator/configuration.hpp"
 #include "configurator/configurator.hpp"
 
-#include "logging/flags.hpp"
 #include "logging/logging.hpp"
 
 #include "tests/environment.hpp"
 #include "tests/utils.hpp"
 
 using namespace mesos::internal;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using std::cerr;
 using std::endl;
@@ -54,16 +53,7 @@ int main(int argc, char** argv)
 {
   GOOGLE_PROTOBUF_VERIFY_VERSION;
 
-  flags::Flags<logging::Flags> flags;
-
-  // We log to stderr by default, but when running tests we'd prefer
-  // less junk to fly by, so force one to specify the verbosity.
-  bool verbose;
-
-  flags.add(&verbose,
-            "verbose",
-            "Log all severity levels to stderr",
-            false);
+  using mesos::internal::tests::flags; // Needed to disabmiguate.
 
   bool help;
   flags.add(&help,
@@ -94,7 +84,7 @@ int main(int argc, char** argv)
   process::initialize();
 
   // Be quiet by default!
-  if (!verbose) {
+  if (!flags.verbose) {
     flags.quiet = true;
   }
 
@@ -105,21 +95,10 @@ int main(int argc, char** argv)
   testing::InitGoogleTest(&argc, argv);
   testing::FLAGS_gtest_death_test_style = "threadsafe";
 
-  // Get the absolute path to the source (i.e., root) directory.
-  Try<string> path = os::realpath(SOURCE_DIR);
-  CHECK(path.isSome()) << "Error getting source directory " << path.error();
-  mesosSourceDirectory = path.get();
-
-  std::cout << "Source directory: " << mesosSourceDirectory << std::endl;
-
-  // Get absolute path to the build directory.
-  path = os::realpath(BUILD_DIR);
-  CHECK(path.isSome()) << "Error getting build directory " << path.error();
-  mesosBuildDirectory = path.get();
-
-  std::cout << "Build directory: " << mesosBuildDirectory << std::endl;
+  std::cout << "Source directory: " << flags.source_dir << std::endl;
+  std::cout << "Build directory: " << flags.build_dir << std::endl;
 
-  ::testing::AddGlobalTestEnvironment(new Environment());
+  testing::AddGlobalTestEnvironment(new Environment());
 
   return RUN_ALL_TESTS();
 }

Modified: incubator/mesos/trunk/src/tests/master_detector_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/master_detector_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/master_detector_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/master_detector_tests.cpp Fri Oct 26 22:07:22 2012
@@ -39,7 +39,7 @@
 
 using namespace mesos;
 using namespace mesos::internal;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using mesos::internal::master::Master;
 

Modified: incubator/mesos/trunk/src/tests/master_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/master_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/master_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/master_tests.cpp Fri Oct 26 22:07:22 2012
@@ -45,7 +45,7 @@
 
 using namespace mesos;
 using namespace mesos::internal;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using mesos::internal::master::FrameworksManager;
 using mesos::internal::master::FrameworksStorage;

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=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/resource_offers_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/resource_offers_tests.cpp Fri Oct 26 22:07:22 2012
@@ -37,7 +37,7 @@
 
 using namespace mesos;
 using namespace mesos::internal;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using mesos::internal::master::Master;
 

Modified: incubator/mesos/trunk/src/tests/slave_state_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/slave_state_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/slave_state_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/slave_state_tests.cpp Fri Oct 26 22:07:22 2012
@@ -18,8 +18,8 @@
 
 #include <gtest/gtest.h>
 
-#include "stout/os.hpp"
-#include "stout/strings.hpp"
+#include <stout/os.hpp>
+#include <stout/strings.hpp>
 
 #include "messages/messages.hpp"
 

Modified: incubator/mesos/trunk/src/tests/state_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/state_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/state_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/state_tests.cpp Fri Oct 26 22:07:22 2012
@@ -39,15 +39,15 @@
 #include "state/state.hpp"
 #include "state/zookeeper.hpp"
 
+#include "tests/utils.hpp"
 #ifdef MESOS_HAS_JAVA
 #include "tests/zookeeper_test.hpp"
 #endif
-#include "tests/utils.hpp"
 
 using namespace mesos;
 using namespace mesos::internal;
 using namespace mesos::internal::state;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using namespace process;
 
@@ -287,7 +287,7 @@ TEST_F(LevelDBStateTest, Names)
 
 
 #ifdef MESOS_HAS_JAVA
-class ZooKeeperStateTest : public mesos::internal::test::ZooKeeperTest
+class ZooKeeperStateTest : public ZooKeeperTest
 {
 public:
   ZooKeeperStateTest()

Modified: incubator/mesos/trunk/src/tests/utils.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/utils.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/utils.cpp (original)
+++ incubator/mesos/trunk/src/tests/utils.cpp Fri Oct 26 22:07:22 2012
@@ -22,17 +22,13 @@
 
 #include <gtest/gtest.h>
 
-#include "tests/utils.hpp"
+#include <stout/path.hpp>
 
-using namespace mesos::internal;
+#include "tests/flags.hpp"
+#include "tests/utils.hpp"
 
 using std::string;
 
-
-string test::mesosSourceDirectory;
-string test::mesosBuildDirectory;
-
-
 namespace {
 
 // Check that a test name contains only letters, numbers and underscores, to
@@ -46,18 +42,19 @@ bool isValidTestName(const char* name) {
   return true;
 }
 
-} // namespace
+} // namespace {
 
 
-/**
- * Create and clean up the work directory for a given test, and cd into it,
- * given the test's test case name and test name.
- * Test directories are placed in
- * <mesosBuildDir>/test_output/<testCase>/<testName>.
- */
-void test::enterTestDirectory(const char* testCase, const char* testName)
+namespace mesos {
+namespace internal {
+namespace tests {
+
+flags::Flags<logging::Flags, Flags> flags;
+
+
+void enterTestDirectory(const char* testCase, const char* testName)
 {
-  // Remove DISABLED_ prefix from test name if this is a disabled test
+  // Remove DISABLED_ prefix from test name if this is a disabled test.
   if (strncmp(testName, "DISABLED_", strlen("DISABLED_")) == 0)
     testName += strlen("DISABLED_");
   // Check that the test name is valid
@@ -65,9 +62,9 @@ void test::enterTestDirectory(const char
     FAIL() << "Invalid test name for external test (name should " 
            << "only contain alphanumeric and underscore characters)";
   }
-  // Make the work directory for this test
-  string workDir = mesosBuildDirectory + "/test_output/" + testCase + "/"
-    + testName;
+  // Make the work directory for this test.
+  string workDir =
+    path::join(flags.build_dir, "test_output", testCase, testName);
   string command = "rm -fr '" + workDir + "'";
   ASSERT_EQ(0, system(command.c_str())) << "Command failed: " << command;
   command = "mkdir -p '" + workDir + "'";
@@ -76,3 +73,7 @@ void test::enterTestDirectory(const char
   if (chdir(workDir.c_str()) != 0)
     FAIL() << "Could not chdir into " << workDir;
 }
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {

Modified: incubator/mesos/trunk/src/tests/utils.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/utils.hpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/utils.hpp (original)
+++ incubator/mesos/trunk/src/tests/utils.hpp Fri Oct 26 22:07:22 2012
@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 
-#ifndef __TESTING_UTILS_HPP__
-#define __TESTING_UTILS_HPP__
+#ifndef __TESTS_UTILS_HPP__
+#define __TESTS_UTILS_HPP__
 
 #include <unistd.h> // For usleep.
 
@@ -27,8 +27,6 @@
 #include <map>
 #include <string>
 
-#include <master/allocator.hpp>
-#include <master/master.hpp>
 #include <mesos/executor.hpp>
 #include <mesos/scheduler.hpp>
 
@@ -44,6 +42,9 @@
 
 #include "common/type_utils.hpp"
 
+#include "logging/logging.hpp"
+
+#include "master/allocator.hpp"
 #include "master/drf_sorter.hpp"
 #include "master/hierarchical_allocator_process.hpp"
 #include "master/master.hpp"
@@ -53,51 +54,38 @@
 #include "slave/isolation_module.hpp"
 #include "slave/slave.hpp"
 
+#include "tests/flags.hpp"
+
+// TODO(benh): Kill these since they pollute everything that includes this file.
 using ::testing::_;
 using ::testing::Invoke;
 
 namespace mesos {
 namespace internal {
-namespace test {
-
-/**
- * The location of the Mesos source directory.  Used by tests to locate
- * various frameworks and binaries.  Initialized in main.cpp.
- */
-extern std::string mesosSourceDirectory;
-
+namespace tests {
 
-/**
- * The location of the Mesos build directory. Used by tests to locate
- * frameworks and binaries.  Initialized in main.cpp.
- */
-extern std::string mesosBuildDirectory;
+// Flags used to run the tests.
+extern flags::Flags<logging::Flags, Flags> flags;
 
 
-/**
- * Create and clean up the work directory for a given test, and cd into it,
- * given the test's test case name and test name.
- * Test directories are placed in <mesosHome>/test_output/<testCase>/<testName>.
- */
+// Create and clean up the work directory for a given test, and cd into it,
+// given the test's test case name and test name.
+// Test directories are placed in <mesosHome>/test_output/<testCase>/<testName>.
 void enterTestDirectory(const char* testCase, const char* testName);
 
 
-/**
- * Macro for running a test in a work directory (using enterTestDirectory).
- * Used in a similar way to gtest's TEST macro (by adding a body in braces).
- */
-#define TEST_WITH_WORKDIR(testCase, testName) \
-  void runTestBody_##testCase##_##testName(); \
-  TEST(testCase, testName) {                  \
-    enterTestDirectory(#testCase, #testName); \
-    runTestBody_##testCase##_##testName(); \
-  } \
-  void runTestBody_##testCase##_##testName() /* User code block follows */
+// Macro for running a test in a work directory (using enterTestDirectory).
+// Used in a similar way to gtest's TEST macro (by adding a body in braces).
+#define TEST_WITH_WORKDIR(testCase, testName)   \
+  void runTestBody_##testCase##_##testName();   \
+  TEST(testCase, testName) {                    \
+    enterTestDirectory(#testCase, #testName);   \
+    runTestBody_##testCase##_##testName();      \
+  }                                             \
+  void runTestBody_##testCase##_##testName()
 
 
-/**
- * Macros to get/create (default) ExecutorInfos and FrameworkInfos.
- */
+// Macros to get/create (default) ExecutorInfos and FrameworkInfos.
 #define DEFAULT_EXECUTOR_INFO                                           \
       ({ ExecutorInfo executor;                                         \
         executor.mutable_executor_id()->set_value("default");           \
@@ -122,9 +110,7 @@ void enterTestDirectory(const char* test
       DEFAULT_EXECUTOR_INFO.executor_id()
 
 
-/**
- * Definition of a mock Scheduler to be used in tests with gmock.
- */
+// Definition of a mock Scheduler to be used in tests with gmock.
 class MockScheduler : public Scheduler
 {
 public:
@@ -226,9 +212,7 @@ ACTION(DeclineOffers)
 }
 
 
-/**
- * Definition of a mock Executor to be used in tests with gmock.
- */
+// Definition of a mock Executor to be used in tests with gmock.
 class MockExecutor : public Executor
 {
 public:
@@ -439,9 +423,7 @@ inline const ::testing::Matcher<const st
 }
 
 
-/**
- * Definition of a mock Filter so that messages can act as triggers.
- */
+// Definition of a mock Filter so that messages can act as triggers.
 class MockFilter : public process::Filter
 {
 public:
@@ -464,10 +446,8 @@ public:
 };
 
 
-/**
- * A message can be matched against in conjunction with the MockFilter
- * (see above) to perform specific actions based for messages.
- */
+// A message can be matched against in conjunction with the MockFilter
+// (see above) to perform specific actions based for messages.
 MATCHER_P3(MsgMatcher, name, from, to, "")
 {
   const process::MessageEvent& event = ::std::tr1::get<0>(arg);
@@ -477,11 +457,9 @@ MATCHER_P3(MsgMatcher, name, from, to, "
 }
 
 
-/**
- * This macro provides some syntactic sugar for matching messages
- * using the message matcher (see above) as well as the MockFilter
- * (see above). We should also add EXPECT_DISPATCH, EXPECT_HTTP, etc.
- */
+// This macro provides some syntactic sugar for matching messages
+// using the message matcher (see above) as well as the MockFilter
+// (see above). We should also add EXPECT_DISPATCH, EXPECT_HTTP, etc.
 #define EXPECT_MESSAGE(mockFilter, name, from, to)              \
   EXPECT_CALL(mockFilter, filter(testing::A<const process::MessageEvent&>())) \
     .With(MsgMatcher(name, from, to))
@@ -495,12 +473,10 @@ ACTION_TEMPLATE(SaveArgField,
 }
 
 
-/**
- * A trigger is an object that can be used to effectively block a test
- * from proceeding until some event has occured. A trigger can get set
- * using a gmock action (see below) and you can wait for a trigger to
- * occur using the WAIT_UNTIL macro below.
- */
+// A trigger is an object that can be used to effectively block a test
+// from proceeding until some event has occured. A trigger can get set
+// using a gmock action (see below) and you can wait for a trigger to
+// occur using the WAIT_UNTIL macro below.
 struct trigger
 {
   trigger() : value(false) {}
@@ -509,36 +485,28 @@ struct trigger
 };
 
 
-/**
- * Definition of the Trigger action to be used with gmock.
- */
+// Definition of the Trigger action to be used with gmock.
 ACTION_P(Trigger, trigger)
 {
   trigger->value = true;
 }
 
 
-/**
- * Definition of an 'increment' action to be used with gmock.
- */
+// Definition of an 'increment' action to be used with gmock.
 ACTION_P(Increment, variable)
 {
   *variable = *variable + 1;
 }
 
 
-/**
- * Definition of a 'decrement' action to be used with gmock.
- */
+// Definition of a 'decrement' action to be used with gmock.
 ACTION_P(Decrement, variable)
 {
   *variable = *variable - 1;
 }
 
 
-/**
- * Definition of the SendStatusUpdateFromTask action to be used with gmock.
- */
+// Definition of the SendStatusUpdateFromTask action to be used with gmock.
 ACTION_P(SendStatusUpdateFromTask, state)
 {
   TaskStatus status;
@@ -548,9 +516,7 @@ ACTION_P(SendStatusUpdateFromTask, state
 }
 
 
-/**
- * Definition of the SendStatusUpdateFromTaskID action to be used with gmock.
- */
+// Definition of the SendStatusUpdateFromTaskID action to be used with gmock.
 ACTION_P(SendStatusUpdateFromTaskID, state)
 {
   TaskStatus status;
@@ -560,9 +526,7 @@ ACTION_P(SendStatusUpdateFromTaskID, sta
 }
 
 
-/**
- * These macros can be used to wait until some expression evaluates to true.
- */
+// These macros can be used to wait until some expression evaluates to true.
 #define WAIT_FOR(expression, duration)                                  \
   do {                                                                  \
     unsigned int sleeps = 0;                                            \
@@ -663,7 +627,7 @@ private:
   process::PID<slave::Slave> slave;
 };
 
-} // namespace test {
+} // namespace tests {
 } // namespace internal {
 } // namespace mesos {
 
@@ -952,4 +916,4 @@ inline ::testing::AssertionResult Assert
 #define EXPECT_RESPONSE_HEADER_WILL_EQ(expected, key, actual)        \
   EXPECT_PRED_FORMAT3(AssertResponseHeaderWillEq, expected, key, actual)
 
-#endif // __TESTING_UTILS_HPP__
+#endif // __TESTS_UTILS_HPP__

Modified: incubator/mesos/trunk/src/tests/zookeeper_test.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_test.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/zookeeper_test.cpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_test.cpp Fri Oct 26 22:07:22 2012
@@ -38,7 +38,7 @@
 
 namespace mesos {
 namespace internal {
-namespace test {
+namespace tests {
 
 const Milliseconds ZooKeeperTest::NO_TIMEOUT(5000);
 
@@ -79,7 +79,7 @@ static void silenceClientLogs()
 void ZooKeeperTest::SetUpTestCase()
 {
   if (jvm == NULL) {
-    std::string zkHome = mesosBuildDirectory +
+    std::string zkHome = flags.build_dir +
       "/third_party/zookeeper-" ZOOKEEPER_VERSION;
 
     std::string classpath = "-Djava.class.path=" +
@@ -197,7 +197,7 @@ ZooKeeperTest::TestWatcher::awaitEvent(
   }
 }
 
-} // namespace test
-} // namespace internal
-} // namespace mesos
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {
 

Modified: incubator/mesos/trunk/src/tests/zookeeper_test.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_test.hpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/zookeeper_test.hpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_test.hpp Fri Oct 26 22:07:22 2012
@@ -35,7 +35,7 @@
 
 namespace mesos {
 namespace internal {
-namespace test {
+namespace tests {
 
 // A fixture for tests that need to interact with a ZooKeeper server
 // ensemble. Tests can access the in process ZooKeeperTestServer via
@@ -104,8 +104,8 @@ private:
   static Jvm* jvm;
 };
 
-} // namespace test
-} // namespace internal
-} // namespace mesos
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {
 
 #endif // __ZOOKEEPER_TEST_HPP__

Modified: incubator/mesos/trunk/src/tests/zookeeper_test_server.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_test_server.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/zookeeper_test_server.cpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_test_server.cpp Fri Oct 26 22:07:22 2012
@@ -32,7 +32,7 @@
 
 namespace mesos {
 namespace internal {
-namespace test {
+namespace tests {
 
 ZooKeeperTestServer::ZooKeeperTestServer(Jvm* _jvm)
   : jvm(_jvm),
@@ -220,7 +220,7 @@ void ZooKeeperTestServer::checkStarted()
   CHECK(port > 0) << "Illegal state, must call startNetwork first!";
 }
 
-} // namespace test
-} // namespace internal
-} // namespace mesos
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {
 

Modified: incubator/mesos/trunk/src/tests/zookeeper_test_server.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_test_server.hpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/zookeeper_test_server.hpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_test_server.hpp Fri Oct 26 22:07:22 2012
@@ -31,7 +31,7 @@
 
 namespace mesos {
 namespace internal {
-namespace test {
+namespace tests {
 
 // An in-process ZooKeeper server that can be manipulated to control
 // repeatable client tests. Sessions can be programmatically expired
@@ -108,8 +108,8 @@ private:
   void checkStarted() const;
 };
 
-} // namespace test
-} // namespace internal
-} // namespace mesos
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {
 
 #endif // __ZOOKEEPER_TEST_SERVER_HPP__

Modified: incubator/mesos/trunk/src/tests/zookeeper_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_tests.cpp?rev=1402695&r1=1402694&r2=1402695&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/zookeeper_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_tests.cpp Fri Oct 26 22:07:22 2012
@@ -41,7 +41,7 @@
 #include "zookeeper/group.hpp"
 
 using namespace mesos::internal;
-using namespace mesos::internal::test;
+using namespace mesos::internal::tests;
 
 using process::Clock;