Browse Source

[master] Merge branch 'trac2205'

JINMEI Tatuya 12 years ago
parent
commit
f0fc85c74e

+ 1 - 0
src/bin/auth/Makefile.am

@@ -55,6 +55,7 @@ b10_auth_SOURCES += auth_config.cc auth_config.h
 b10_auth_SOURCES += command.cc command.h
 b10_auth_SOURCES += command.cc command.h
 b10_auth_SOURCES += common.h common.cc
 b10_auth_SOURCES += common.h common.cc
 b10_auth_SOURCES += statistics.cc statistics.h
 b10_auth_SOURCES += statistics.cc statistics.h
+b10_auth_SOURCES += datasrc_clients_mgr.h
 b10_auth_SOURCES += datasrc_config.h datasrc_config.cc
 b10_auth_SOURCES += datasrc_config.h datasrc_config.cc
 b10_auth_SOURCES += main.cc
 b10_auth_SOURCES += main.cc
 
 

+ 6 - 0
src/bin/auth/auth_log.cc

@@ -21,6 +21,12 @@ namespace auth {
 
 
 isc::log::Logger auth_logger("auth");
 isc::log::Logger auth_logger("auth");
 
 
+const int DBG_AUTH_START = DBGLVL_START_SHUT;
+const int DBG_AUTH_SHUT = DBGLVL_START_SHUT;
+const int DBG_AUTH_OPS = DBGLVL_COMMAND;
+const int DBG_AUTH_DETAIL = DBGLVL_TRACE_BASIC;
+const int DBG_AUTH_MESSAGES = DBGLVL_TRACE_DETAIL_DATA;
+
 } // namespace auth
 } // namespace auth
 } // namespace isc
 } // namespace isc
 
 

+ 5 - 5
src/bin/auth/auth_log.h

@@ -28,21 +28,21 @@ namespace auth {
 /// output.
 /// output.
 
 
 // Debug messages indicating normal startup are logged at this debug level.
 // Debug messages indicating normal startup are logged at this debug level.
-const int DBG_AUTH_START = DBGLVL_START_SHUT;
+extern const int DBG_AUTH_START;
 // Debug messages upon shutdown
 // Debug messages upon shutdown
-const int DBG_AUTH_SHUT = DBGLVL_START_SHUT;
+extern const int DBG_AUTH_SHUT;
 
 
 // Debug level used to log setting information (such as configuration changes).
 // Debug level used to log setting information (such as configuration changes).
-const int DBG_AUTH_OPS = DBGLVL_COMMAND;
+extern const int DBG_AUTH_OPS;
 
 
 // Trace detailed operations, including errors raised when processing invalid
 // Trace detailed operations, including errors raised when processing invalid
 // packets.  (These are not logged at severities of WARN or higher for fear
 // packets.  (These are not logged at severities of WARN or higher for fear
 // that a set of deliberately invalid packets set to the authoritative server
 // that a set of deliberately invalid packets set to the authoritative server
 // could overwhelm the logging.)
 // could overwhelm the logging.)
-const int DBG_AUTH_DETAIL = DBGLVL_TRACE_BASIC;
+extern const int DBG_AUTH_DETAIL;
 
 
 // This level is used to log the contents of packets received and sent.
 // This level is used to log the contents of packets received and sent.
-const int DBG_AUTH_MESSAGES = DBGLVL_TRACE_DETAIL_DATA;
+extern const int DBG_AUTH_MESSAGES;
 
 
 /// Define the logger for the "auth" module part of b10-auth.  We could define
 /// Define the logger for the "auth" module part of b10-auth.  We could define
 /// a logger in each file, but we would want to define a common name to avoid
 /// a logger in each file, but we would want to define a common name to avoid

+ 41 - 0
src/bin/auth/auth_messages.mes

@@ -57,6 +57,47 @@ At attempt to update the configuration the server with information
 from the configuration database has failed, the reason being given in
 from the configuration database has failed, the reason being given in
 the message.
 the message.
 
 
+% AUTH_DATASRC_CLIENTS_BUILDER_COMMAND data source builder received command: %1
+A debug message, showing when the separate thread for maintaining data
+source clients receives a command from the manager.
+
+% AUTH_DATASRC_CLIENTS_BUILDER_FAILED data source builder thread stopped due to an exception: %1
+The separate thread for maintaining data source clients has been
+terminated due to some uncaught exception.  The manager cannot always
+catch this condition in timely fashion, and there is no way to recover
+from this situation except for restarting the entire server.  So this
+message needs to be carefully watched, and should it occur the auth
+server needs to be restarted by hand.
+
+% AUTH_DATASRC_CLIENTS_BUILDER_FAILED_UNEXPECTED data source builder thread stopped due to an unexpected exception
+This is similar to AUTH_DATASRC_CLIENTS_BUILDER_FAILED, but the
+exception type is even more unexpected.  This may rather indicate some
+run time failure than program errors, but in any case the server needs
+to be restarted by hand.
+
+% AUTH_DATASRC_CLIENTS_BUILDER_STARTED data source builder thread started
+A separate thread for maintaining data source clients has been started.
+
+% AUTH_DATASRC_CLIENTS_BUILDER_STOPPED data source builder thread stopped
+The separate thread for maintaining data source clients has been stopped.
+
+% AUTH_DATASRC_CLIENTS_SHUTDOWN_ERROR error on waiting for data source builder thread: %1
+This indicates that the separate thread for maintaining data source
+clients had been terminated due to an uncaught exception, and the
+manager notices that at its own termination.  There should have been
+AUTH_DATASRC_CLIENTS_BUILDER_FAILED or
+AUTH_DATASRC_CLIENTS_BUILDER_FAILED_UNEXPECTED error messages in past
+logs.  If this message appears, the maintenance of the data source
+clients hasn't been working properly for some time.
+
+% AUTH_DATASRC_CLIENTS_SHUTDOWN_UNEXPECTED_ERROR Unexpected error on waiting for data source builder thread
+Some exception happens while waiting for the termination of the
+separate thread for maintaining data source clients.  This shouldn't
+happen in normal conditions; it should be either fatal system level
+errors such as severe memory shortage or some internal bug.  If that
+happens, and if it's not in the middle of terminating b10-auth, it's
+probably better to stop and restart it.
+
 % AUTH_DATA_SOURCE data source database file: %1
 % AUTH_DATA_SOURCE data source database file: %1
 This is a debug message produced by the authoritative server when it accesses a
 This is a debug message produced by the authoritative server when it accesses a
 datebase data source, listing the file that is being accessed.
 datebase data source, listing the file that is being accessed.

+ 320 - 0
src/bin/auth/datasrc_clients_mgr.h

@@ -0,0 +1,320 @@
+// Copyright (C) 2012  Internet Systems Consortium, Inc. ("ISC")
+//
+// Permission to use, copy, modify, and/or distribute this software for any
+// purpose with or without fee is hereby granted, provided that the above
+// copyright notice and this permission notice appear in all copies.
+//
+// THE SOFTWARE IS PROVIDED "AS IS" AND ISC DISCLAIMS ALL WARRANTIES WITH
+// REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY
+// AND FITNESS.  IN NO EVENT SHALL ISC BE LIABLE FOR ANY SPECIAL, DIRECT,
+// INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM
+// LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE
+// OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR
+// PERFORMANCE OF THIS SOFTWARE.
+
+#ifndef DATASRC_CLIENTS_MGR_H
+#define DATASRC_CLIENTS_MGR_H 1
+
+#include <util/threads/thread.h>
+#include <util/threads/lock.h>
+
+#include <log/logger_support.h>
+#include <log/log_dbglevels.h>
+
+#include <cc/data.h>
+
+#include <auth/auth_log.h>
+
+#include <boost/array.hpp>
+#include <boost/bind.hpp>
+
+#include <list>
+#include <utility>
+
+namespace isc {
+namespace auth {
+
+namespace datasrc_clientmgr_internal {
+// This namespace is essentially private for DataSrcClientsMgr(Base) and
+// DataSrcClientsBuilder(Base).  This is exposed in the public header
+// only because these classes are templated (for testing purposes) and
+// class internal has to be defined here.
+
+/// \brief ID of commands from the DataSrcClientsMgr to DataSrcClientsBuilder.
+enum CommandID {
+    NOOP,         ///< Do nothing.  Only useful for tests; no argument
+    SHUTDOWN,     ///< Shutdown the builder; no argument
+    NUM_COMMANDS
+};
+
+/// \brief The data type passed from DataSrcClientsMgr to
+/// DataSrcClientsBuilder.
+///
+/// The first element of the pair is the command ID, and the second element
+/// is its argument.  If the command doesn't take an argument it should be
+/// a null pointer.
+typedef std::pair<CommandID, data::ConstElementPtr> Command;
+} // namespace datasrc_clientmgr_internal
+
+/// \brief Frontend to the manager object for data source clients.
+///
+/// This class provides interfaces for configuring and updating a set of
+/// data source clients "in the background".  The user of this class can
+/// assume any operation on this class can be done effectively non-blocking,
+/// not suspending any delay-sensitive operations such as DNS query
+/// processing.  The only exception is the time when this class object
+/// is destroyed (normally as a result of an implicit call to the destructor);
+/// in the current implementation it can take time depending on what is
+/// running "in the background" at the time of the call.
+///
+/// Internally, an object of this class invokes a separate thread to perform
+/// time consuming operations such as loading large zone data into memory,
+/// but such details are completely hidden from the user of this class.
+///
+/// This class is templated only so that we can test the class without
+/// involving actual threads or mutex.  Normal applications will only
+/// need one specific specialization that has a typedef of
+/// \c DataSrcClientsMgr.
+template <typename ThreadType, typename BuilderType, typename MutexType,
+          typename CondVarType>
+class DataSrcClientsMgrBase {
+public:
+    /// \brief Constructor.
+    ///
+    /// It internally invokes a separate thread and waits for further
+    /// operations from the user application.
+    ///
+    /// This method is basically exception free except in case of really
+    /// rare system-level errors.  When that happens the only reasonable
+    /// action that the application can take would be to terminate the program
+    /// in practice.
+    ///
+    /// \throw std::bad_alloc internal memory allocation failure.
+    /// \throw isc::Unexpected general unexpected system errors.
+    DataSrcClientsMgrBase() :
+        builder_(&command_queue_, &cond_, &queue_mutex_),
+        builder_thread_(boost::bind(&BuilderType::run, &builder_))
+    {}
+
+    /// \brief The destructor.
+    ///
+    /// It tells the internal thread to stop and waits for it completion.
+    /// In the current implementation, it can block for some unpredictably
+    /// long period depending on what the thread is doing at that time
+    /// (in future we may want to implement a rapid way of killing the thread
+    /// and/or provide a separate interface for waiting so that the application
+    /// can choose the timing).
+    ///
+    /// The waiting operation can result in an exception, but this method
+    /// catches any of them so this method itself is exception free.
+    ~DataSrcClientsMgrBase() {
+        // We share class member variables with the builder, which will be
+        // invalidated after the call to the destructor, so we need to make
+        // sure the builder thread is terminated.  Depending on the timing
+        // this could take a long time; if we don't want that to happen in
+        // this context, we may want to introduce a separate 'shutdown()'
+        // method.
+        // Also, since we don't want to propagate exceptions from a destructor,
+        // we catch any possible ones.  In fact the only really expected one
+        // is Thread::UncaughtException when the builder thread died due to
+        // an exception.  We specifically log it and just ignore others.
+        try {
+            sendCommand(datasrc_clientmgr_internal::SHUTDOWN,
+                        data::ConstElementPtr());
+            builder_thread_.wait();
+        } catch (const util::thread::Thread::UncaughtException& ex) {
+            // technically, logging this could throw, which will be propagated.
+            // But such an exception would be a fatal one anyway, so we
+            // simply let it go through.
+            LOG_ERROR(auth_logger, AUTH_DATASRC_CLIENTS_SHUTDOWN_ERROR).
+                arg(ex.what());
+        } catch (...) {
+            LOG_ERROR(auth_logger,
+                      AUTH_DATASRC_CLIENTS_SHUTDOWN_UNEXPECTED_ERROR);
+        }
+
+        cleanup();              // see below
+    }
+
+private:
+    // This is expected to be called at the end of the destructor.  It
+    // actually does nothing, but provides a customization point for
+    // specialized class for tests so that the tests can inspect the last
+    // state of the class.
+    void cleanup() {}
+
+    void sendCommand(datasrc_clientmgr_internal::CommandID command,
+                     data::ConstElementPtr arg)
+    {
+        {
+            typename MutexType::Locker locker(queue_mutex_);
+            command_queue_.push_back(
+                datasrc_clientmgr_internal::Command(command, arg));
+        }
+        cond_.signal();
+    }
+
+    //
+    // The following are shared with the builder.
+    //
+    // The list is used as a one-way queue: back-in, front-out
+    std::list<datasrc_clientmgr_internal::Command> command_queue_;
+    CondVarType cond_;          // condition variable for queue operations
+    MutexType queue_mutex_;     // mutex to protect the queue
+#ifdef notyet                   // until #2210 or #2212
+    boost::shared_ptr<DataSrcClientListMap> clients_map_;
+    MutexType map_mutex_;
+#endif
+
+    BuilderType builder_;
+    ThreadType builder_thread_; // for safety this should be placed last
+};
+
+namespace datasrc_clientmgr_internal {
+
+/// \brief A class that maintains a set of data source clients.
+///
+/// An object of this class is supposed to run on a dedicated thread, whose
+/// main function is a call to its \c run() method.  It runs in a loop
+/// waiting for commands from the manager and handles each command (including
+/// reloading a new version of zone data into memory or fully reconfiguration
+/// of specific set of data source clients).  When it receives a SHUTDOWN
+/// command, it exits from the loop, which will terminate the thread.
+///
+/// While this class is defined in a publicly visible namespace, it's
+/// essentially private to \c DataSrcClientsMgr.  Except for tests,
+/// applications should not directly access this class.
+///
+/// This class is templated so that we can test it without involving actual
+/// threads or locks.
+template <typename MutexType, typename CondVarType>
+class DataSrcClientsBuilderBase {
+public:
+    /// \brief Constructor.
+    ///
+    /// It simply sets up a local copy of shared data with the manager.
+    ///
+    /// Note: this will take actual set (map) of data source clients and
+    /// a mutex object for it in #2210 or #2212.
+    ///
+    /// \throw None
+    DataSrcClientsBuilderBase(std::list<Command>* command_queue,
+                              CondVarType* cond, MutexType* queue_mutex
+#ifdef notyet
+                              // In #2210 or #2212 we pass other data
+#endif
+        ) :
+        command_queue_(command_queue), cond_(cond), queue_mutex_(queue_mutex)
+    {}
+
+    /// \brief The main loop.
+    void run();
+
+    /// \brief Handle one command from the manager.
+    ///
+    /// This is a dedicated subroutine of run() and is essentially private,
+    /// but is defined as a separate public method so we can test each
+    /// command test individually.  In any case, this class itself is
+    /// generally considered private.
+    ///
+    /// \return true if the builder should keep running; false otherwise.
+    bool handleCommand(const Command& command);
+
+private:
+    // NOOP command handler.  We use this so tests can override it; the default
+    // implementation really does nothing.
+    void doNoop() {}
+
+    // The following are shared with the manager
+    std::list<Command>* command_queue_;
+    CondVarType* cond_;
+    MutexType* queue_mutex_;
+};
+
+// Shortcut typedef for normal use
+typedef DataSrcClientsBuilderBase<util::thread::Mutex, util::thread::CondVar>
+DataSrcClientsBuilder;
+
+template <typename MutexType, typename CondVarType>
+void
+DataSrcClientsBuilderBase<MutexType, CondVarType>::run() {
+    LOG_INFO(auth_logger, AUTH_DATASRC_CLIENTS_BUILDER_STARTED);
+
+    try {
+        bool keep_running = true;
+        while (keep_running) {
+            std::list<Command> current_commands;
+            {
+                // Move all new commands to local queue under the protection of
+                // queue_mutex_.  Note that list::splice() should never throw.
+                typename MutexType::Locker locker(*queue_mutex_);
+                while (command_queue_->empty()) {
+                    cond_->wait(*queue_mutex_);
+                }
+                current_commands.splice(current_commands.end(),
+                                        *command_queue_);
+            } // the lock is release here.
+
+            while (keep_running && !current_commands.empty()) {
+                keep_running = handleCommand(current_commands.front());
+                current_commands.pop_front();
+            }
+        }
+
+        LOG_INFO(auth_logger, AUTH_DATASRC_CLIENTS_BUILDER_STOPPED);
+    } catch (const std::exception& ex) {
+        // We explicitly catch exceptions so we can log it as soon as possible.
+        LOG_ERROR(auth_logger, AUTH_DATASRC_CLIENTS_BUILDER_FAILED).
+            arg(ex.what());
+        throw;
+    } catch (...) {
+        LOG_ERROR(auth_logger, AUTH_DATASRC_CLIENTS_BUILDER_FAILED_UNEXPECTED);
+        throw;
+    }
+}
+
+template <typename MutexType, typename CondVarType>
+bool
+DataSrcClientsBuilderBase<MutexType, CondVarType>::handleCommand(
+    const Command& command)
+{
+    const CommandID cid = command.first;
+    if (cid >= NUM_COMMANDS) {
+        // This shouldn't happen except for a bug within this file.
+        isc_throw(Unexpected, "internal bug: invalid command, ID: " << cid);
+    }
+
+    const boost::array<const char*, NUM_COMMANDS> command_desc = {
+        {"NOOP", "SHUTDOWN"}
+    };
+    LOG_DEBUG(auth_logger, DBGLVL_TRACE_BASIC,
+              AUTH_DATASRC_CLIENTS_BUILDER_COMMAND).arg(command_desc.at(cid));
+    switch (command.first) {
+    case SHUTDOWN:
+        return (false);
+    case NOOP:
+        doNoop();
+        break;
+    case NUM_COMMANDS:
+        assert(false);          // we rejected this case above
+    }
+    return (true);
+}
+} // namespace datasrc_clientmgr_internal
+
+/// \brief Shortcut type for normal data source clients manager.
+///
+/// In fact, for non test applications this is the only type of this kind
+/// to be considered.
+typedef DataSrcClientsMgrBase<
+    util::thread::Thread,
+    datasrc_clientmgr_internal::DataSrcClientsBuilder,
+    util::thread::Mutex, util::thread::CondVar> DataSrcClientsMgr;
+} // namespace auth
+} // namespace isc
+
+#endif  // DATASRC_CLIENTS_MGR_H
+
+// Local Variables:
+// mode: c++
+// End:

+ 6 - 0
src/bin/auth/main.cc

@@ -36,6 +36,8 @@
 #include <auth/auth_srv.h>
 #include <auth/auth_srv.h>
 #include <auth/auth_log.h>
 #include <auth/auth_log.h>
 #include <auth/datasrc_config.h>
 #include <auth/datasrc_config.h>
+#include <auth/datasrc_clients_mgr.h>
+
 #include <asiodns/asiodns.h>
 #include <asiodns/asiodns.h>
 #include <asiolink/asiolink.h>
 #include <asiolink/asiolink.h>
 #include <log/logger_support.h>
 #include <log/logger_support.h>
@@ -230,6 +232,10 @@ main(int argc, char* argv[]) {
         isc::server_common::initKeyring(*config_session);
         isc::server_common::initKeyring(*config_session);
         auth_server->setTSIGKeyRing(&isc::server_common::keyring);
         auth_server->setTSIGKeyRing(&isc::server_common::keyring);
 
 
+        // Instantiate the data source clients manager.  At the moment
+        // just so we actually create it in system tests.
+        DataSrcClientsMgr datasrc_clients_mgr;
+
         // Start the data source configuration.  We pass first_time and
         // Start the data source configuration.  We pass first_time and
         // config_session for the hack described in datasrcConfigHandler.
         // config_session for the hack described in datasrcConfigHandler.
         bool first_time = true;
         bool first_time = true;

+ 3 - 0
src/bin/auth/tests/Makefile.am

@@ -51,6 +51,9 @@ run_unittests_SOURCES += command_unittest.cc
 run_unittests_SOURCES += common_unittest.cc
 run_unittests_SOURCES += common_unittest.cc
 run_unittests_SOURCES += query_unittest.cc
 run_unittests_SOURCES += query_unittest.cc
 run_unittests_SOURCES += statistics_unittest.cc
 run_unittests_SOURCES += statistics_unittest.cc
+run_unittests_SOURCES += test_datasrc_clients_mgr.h test_datasrc_clients_mgr.cc
+run_unittests_SOURCES += datasrc_clients_builder_unittest.cc
+run_unittests_SOURCES += datasrc_clients_mgr_unittest.cc
 run_unittests_SOURCES += datasrc_config_unittest.cc
 run_unittests_SOURCES += datasrc_config_unittest.cc
 run_unittests_SOURCES += run_unittests.cc
 run_unittests_SOURCES += run_unittests.cc
 
 

+ 106 - 0
src/bin/auth/tests/datasrc_clients_builder_unittest.cc

@@ -0,0 +1,106 @@
+// Copyright (C) 2012  Internet Systems Consortium, Inc. ("ISC")
+//
+// Permission to use, copy, modify, and/or distribute this software for any
+// purpose with or without fee is hereby granted, provided that the above
+// copyright notice and this permission notice appear in all copies.
+//
+// THE SOFTWARE IS PROVIDED "AS IS" AND ISC DISCLAIMS ALL WARRANTIES WITH
+// REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY
+// AND FITNESS.  IN NO EVENT SHALL ISC BE LIABLE FOR ANY SPECIAL, DIRECT,
+// INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM
+// LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE
+// OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR
+// PERFORMANCE OF THIS SOFTWARE.
+
+#include <cc/data.h>
+
+#include <auth/datasrc_clients_mgr.h>
+#include "test_datasrc_clients_mgr.h"
+
+#include <gtest/gtest.h>
+
+#include <boost/function.hpp>
+
+using isc::data::ConstElementPtr;
+using namespace isc::auth::datasrc_clientmgr_internal;
+
+namespace {
+class DataSrcClientsBuilderTest : public ::testing::Test {
+protected:
+    DataSrcClientsBuilderTest() :
+        builder(&command_queue, &cond, &queue_mutex),
+        cond(command_queue, delayed_command_queue),
+        shutdown_cmd(SHUTDOWN, ConstElementPtr()),
+        noop_cmd(NOOP, ConstElementPtr())
+    {}
+
+    TestDataSrcClientsBuilder builder;
+    std::list<Command> command_queue; // test command queue
+    std::list<Command> delayed_command_queue; // commands available after wait
+    TestCondVar cond;
+    TestMutex queue_mutex;
+    const Command shutdown_cmd;
+    const Command noop_cmd;
+};
+
+TEST_F(DataSrcClientsBuilderTest, runSingleCommand) {
+    // A simplest case, just to check the basic behavior.
+    command_queue.push_back(shutdown_cmd);
+    builder.run();
+    EXPECT_TRUE(command_queue.empty());
+    EXPECT_EQ(0, cond.wait_count); // no wait because command queue is not empty
+    EXPECT_EQ(1, queue_mutex.lock_count);
+    EXPECT_EQ(1, queue_mutex.unlock_count);
+}
+
+TEST_F(DataSrcClientsBuilderTest, runMultiCommands) {
+    // Two NOOP commands followed by SHUTDOWN.  We should see two doNoop()
+    // calls.
+    command_queue.push_back(noop_cmd);
+    command_queue.push_back(noop_cmd);
+    command_queue.push_back(shutdown_cmd);
+    builder.run();
+    EXPECT_TRUE(command_queue.empty());
+    EXPECT_EQ(1, queue_mutex.lock_count);
+    EXPECT_EQ(1, queue_mutex.unlock_count);
+    EXPECT_EQ(2, queue_mutex.noop_count);
+}
+
+TEST_F(DataSrcClientsBuilderTest, exception) {
+    // Let the noop command handler throw exceptions and see if we can see
+    // them.
+    command_queue.push_back(noop_cmd);
+    queue_mutex.throw_from_noop = TestMutex::EXCLASS;
+    EXPECT_THROW(builder.run(), isc::Exception);
+
+    command_queue.push_back(noop_cmd);
+    queue_mutex.throw_from_noop = TestMutex::INTEGER;
+    EXPECT_THROW(builder.run(), int);
+}
+
+TEST_F(DataSrcClientsBuilderTest, condWait) {
+    // command_queue is originally empty, so it will require waiting on
+    // condvar.  specialized wait() will make the delayed command available.
+    delayed_command_queue.push_back(shutdown_cmd);
+    builder.run();
+
+    // There should be one call to wait()
+    EXPECT_EQ(1, cond.wait_count);
+    // wait() effectively involves one more set of lock/unlock, so we have
+    // two in total
+    EXPECT_EQ(2, queue_mutex.lock_count);
+    EXPECT_EQ(2, queue_mutex.unlock_count);
+}
+
+TEST_F(DataSrcClientsBuilderTest, shutdown) {
+    EXPECT_FALSE(builder.handleCommand(shutdown_cmd));
+}
+
+TEST_F(DataSrcClientsBuilderTest, badCommand) {
+    // out-of-range command ID
+    EXPECT_THROW(builder.handleCommand(Command(NUM_COMMANDS,
+                                               ConstElementPtr())),
+                 isc::Unexpected);
+}
+
+} // unnamed namespace

+ 86 - 0
src/bin/auth/tests/datasrc_clients_mgr_unittest.cc

@@ -0,0 +1,86 @@
+// Copyright (C) 2012  Internet Systems Consortium, Inc. ("ISC")
+//
+// Permission to use, copy, modify, and/or distribute this software for any
+// purpose with or without fee is hereby granted, provided that the above
+// copyright notice and this permission notice appear in all copies.
+//
+// THE SOFTWARE IS PROVIDED "AS IS" AND ISC DISCLAIMS ALL WARRANTIES WITH
+// REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY
+// AND FITNESS.  IN NO EVENT SHALL ISC BE LIABLE FOR ANY SPECIAL, DIRECT,
+// INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM
+// LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE
+// OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR
+// PERFORMANCE OF THIS SOFTWARE.
+
+#include <cc/data.h>
+
+#include <auth/datasrc_clients_mgr.h>
+#include "test_datasrc_clients_mgr.h"
+
+#include <gtest/gtest.h>
+
+#include <boost/function.hpp>
+
+using namespace isc::auth;
+using namespace isc::auth::datasrc_clientmgr_internal;
+
+namespace {
+void
+shutdownCheck() {
+    // Check for common points on shutdown.  The manager should have acquired
+    // the lock, put a SHUTDOWN command to the queue, and should have signaled
+    // the builder.
+    EXPECT_EQ(1, FakeDataSrcClientsBuilder::queue_mutex->lock_count);
+    EXPECT_EQ(1, FakeDataSrcClientsBuilder::cond->signal_count);
+    EXPECT_EQ(1, FakeDataSrcClientsBuilder::command_queue->size());
+    const Command& cmd = FakeDataSrcClientsBuilder::command_queue->front();
+    EXPECT_EQ(SHUTDOWN, cmd.first);
+    EXPECT_FALSE(cmd.second);   // no argument
+
+    // Finally, the manager should wait for the thread to terminate.
+    EXPECT_TRUE(FakeDataSrcClientsBuilder::thread_waited);
+}
+
+TEST(DataSrcClientsMgrTest, start) {
+    // When we create a manager, builder's run() method should be called.
+    FakeDataSrcClientsBuilder::started = false;
+    {
+        TestDataSrcClientsMgr mgr;
+        EXPECT_TRUE(FakeDataSrcClientsBuilder::started);
+        EXPECT_TRUE(FakeDataSrcClientsBuilder::command_queue->empty());
+
+        // Check pre-destroy conditions
+        EXPECT_EQ(0, FakeDataSrcClientsBuilder::cond->signal_count);
+        EXPECT_FALSE(FakeDataSrcClientsBuilder::thread_waited);
+    } // mgr and builder have been destroyed by this point.
+
+    // We stopped the manager implicitly (without shutdown()).  The manager
+    // will internally notify it
+    shutdownCheck();
+}
+
+TEST(DataSrcClientsMgrTest, shutdownWithUncaughtException) {
+    // Emulating the case when the builder exists on exception.  shutdown()
+    // will encounter UncaughtException exception and catch it.
+    EXPECT_NO_THROW({
+            TestDataSrcClientsMgr mgr;
+            FakeDataSrcClientsBuilder::thread_throw_on_wait =
+                FakeDataSrcClientsBuilder::THROW_UNCAUGHT_EX;
+        });
+}
+
+TEST(DataSrcClientsMgrTest, shutdownWithException) {
+    EXPECT_NO_THROW({
+            TestDataSrcClientsMgr mgr;
+            FakeDataSrcClientsBuilder::thread_throw_on_wait =
+                FakeDataSrcClientsBuilder::THROW_OTHER;
+        });
+}
+
+TEST(DataSrcClientsMgrTest, realThread) {
+    // Using the non-test definition with a real thread.  Just checking
+    // no disruption happens.
+    DataSrcClientsMgr mgr;
+}
+
+} // unnamed namespace

+ 75 - 0
src/bin/auth/tests/test_datasrc_clients_mgr.cc

@@ -0,0 +1,75 @@
+// Copyright (C) 2012  Internet Systems Consortium, Inc. ("ISC")
+//
+// Permission to use, copy, modify, and/or distribute this software for any
+// purpose with or without fee is hereby granted, provided that the above
+// copyright notice and this permission notice appear in all copies.
+//
+// THE SOFTWARE IS PROVIDED "AS IS" AND ISC DISCLAIMS ALL WARRANTIES WITH
+// REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY
+// AND FITNESS.  IN NO EVENT SHALL ISC BE LIABLE FOR ANY SPECIAL, DIRECT,
+// INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM
+// LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE
+// OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR
+// PERFORMANCE OF THIS SOFTWARE.
+
+#include <exceptions/exceptions.h>
+
+#include "test_datasrc_clients_mgr.h"
+
+namespace isc {
+namespace auth {
+namespace datasrc_clientmgr_internal {
+
+// Define static DataSrcClientsBuilder member variables.
+bool FakeDataSrcClientsBuilder::started = false;
+std::list<Command>* FakeDataSrcClientsBuilder::command_queue = NULL;
+std::list<Command> FakeDataSrcClientsBuilder::command_queue_copy;
+TestCondVar* FakeDataSrcClientsBuilder::cond = NULL;
+TestCondVar FakeDataSrcClientsBuilder::cond_copy;
+TestMutex* FakeDataSrcClientsBuilder::queue_mutex = NULL;
+TestMutex FakeDataSrcClientsBuilder::queue_mutex_copy;
+bool FakeDataSrcClientsBuilder::thread_waited = false;
+FakeDataSrcClientsBuilder::ExceptionFromWait
+FakeDataSrcClientsBuilder::thread_throw_on_wait =
+    FakeDataSrcClientsBuilder::NOTHROW;
+
+template<>
+void
+TestDataSrcClientsBuilder::doNoop() {
+    ++queue_mutex_->noop_count;
+    switch (queue_mutex_->throw_from_noop) {
+    case TestMutex::NONE:
+        break;                  // no throw
+    case TestMutex::EXCLASS:
+        isc_throw(Exception, "test exception");
+    case TestMutex::INTEGER:
+        throw 42;
+    }
+}
+} // namespace datasrc_clientmgr_internal
+
+template<>
+void
+TestDataSrcClientsMgr::cleanup() {
+    using namespace datasrc_clientmgr_internal;
+    // Make copy of some of the manager's member variables and reset the
+    // corresponding pointers.  The currently pointed objects are in the
+    // manager object, which are going to be invalidated.
+
+    FakeDataSrcClientsBuilder::command_queue_copy = command_queue_;
+    FakeDataSrcClientsBuilder::command_queue =
+        &FakeDataSrcClientsBuilder::command_queue_copy;
+    FakeDataSrcClientsBuilder::queue_mutex_copy = queue_mutex_;
+    FakeDataSrcClientsBuilder::queue_mutex =
+        &FakeDataSrcClientsBuilder::queue_mutex_copy;
+    FakeDataSrcClientsBuilder::cond_copy = cond_;
+    FakeDataSrcClientsBuilder::cond =
+        &FakeDataSrcClientsBuilder::cond_copy;
+}
+
+} // namespace auth
+} // namespace isc
+
+// Local Variables:
+// mode: c++
+// End:

+ 207 - 0
src/bin/auth/tests/test_datasrc_clients_mgr.h

@@ -0,0 +1,207 @@
+// Copyright (C) 2012  Internet Systems Consortium, Inc. ("ISC")
+//
+// Permission to use, copy, modify, and/or distribute this software for any
+// purpose with or without fee is hereby granted, provided that the above
+// copyright notice and this permission notice appear in all copies.
+//
+// THE SOFTWARE IS PROVIDED "AS IS" AND ISC DISCLAIMS ALL WARRANTIES WITH
+// REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY
+// AND FITNESS.  IN NO EVENT SHALL ISC BE LIABLE FOR ANY SPECIAL, DIRECT,
+// INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM
+// LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE
+// OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR
+// PERFORMANCE OF THIS SOFTWARE.
+
+#ifndef TEST_DATASRC_CLIENTS_MGR_H
+#define TEST_DATASRC_CLIENTS_MGR_H 1
+
+#include <exceptions/exceptions.h>
+
+#include <auth/datasrc_clients_mgr.h>
+
+#include <boost/function.hpp>
+
+#include <list>
+
+// In this file we provide specialization of thread, mutex, condition variable,
+// and DataSrcClientsBuilder for convenience of tests.  They don't use
+// actual threads or mutex, and allow tests to inspect some internal states
+// of the corresponding objects.
+//
+// In many cases, tests can use TestDataSrcClientsMgr (defined below) where
+// DataSrcClientsMgr is needed.
+
+// Below we extend the isc::auth::datasrc_clientmgr_internal namespace to
+// specialize the doNoop() method.
+namespace isc {
+namespace auth {
+namespace datasrc_clientmgr_internal {
+class TestMutex {
+public:
+    // for throw_from_noop.
+    // None: no throw from specialized doNoop()
+    // EXCLASS: throw some exception class object
+    // INTEGER: throw an integer
+    enum ExceptionFromNoop { NONE, EXCLASS, INTEGER };
+
+    TestMutex() : lock_count(0), unlock_count(0), noop_count(0),
+                  throw_from_noop(NONE)
+    {}
+    class Locker {
+    public:
+        Locker(TestMutex& mutex) : mutex_(mutex) {
+            ++mutex.lock_count;
+            if (mutex.lock_count > 100) { // 100 is an arbitrary choice
+                isc_throw(Unexpected,
+                          "too many test mutex count, likely a bug in test");
+            }
+        }
+        ~Locker() {
+            ++mutex_.unlock_count;
+        }
+    private:
+        TestMutex& mutex_;
+    };
+    size_t lock_count; // number of lock acquisitions; tests can check this
+    size_t unlock_count; // number of lock releases; tests can check this
+    size_t noop_count;          // allow doNoop() to modify this
+    ExceptionFromNoop throw_from_noop; // tests can set this to control doNoop
+};
+
+class TestCondVar {
+public:
+    TestCondVar() : wait_count(0), signal_count(0), command_queue_(NULL),
+                    delayed_command_queue_(NULL)
+    {}
+    TestCondVar(std::list<Command>& command_queue,
+                std::list<Command>& delayed_command_queue) :
+        wait_count(0),
+        signal_count(0),
+        command_queue_(&command_queue),
+        delayed_command_queue_(&delayed_command_queue)
+    {
+    }
+    void wait(TestMutex& mutex) {
+        // bookkeeping
+        ++mutex.unlock_count;
+        ++wait_count;
+        ++mutex.lock_count;
+
+        if (wait_count > 100) { // 100 is an arbitrary choice
+            isc_throw(Unexpected,
+                      "too many cond wait count, likely a bug in test");
+        }
+
+        // make the delayed commands available
+        command_queue_->splice(command_queue_->end(), *delayed_command_queue_);
+    }
+    void signal() {
+        ++signal_count;
+    }
+    size_t wait_count; // number of calls to wait(); tests can check this
+    size_t signal_count; // number of calls to signal(); tests can check this
+private:
+    std::list<Command>* command_queue_;
+    std::list<Command>* delayed_command_queue_;
+};
+
+// Convenient shortcut
+typedef DataSrcClientsBuilderBase<TestMutex, TestCondVar>
+TestDataSrcClientsBuilder;
+
+// We specialize this command handler for the convenience of tests.
+// It abuses our specialized Mutex to count the number of calls of this method.
+template<>
+void
+TestDataSrcClientsBuilder::doNoop();
+
+// A specialization of DataSrcClientsBuilder that allows tests to inspect
+// its internal states via static class variables.  Using static is suboptimal,
+// but DataSrcClientsMgr is highly encapsulated, this seems to be the best
+// possible compromise.
+class FakeDataSrcClientsBuilder {
+public:
+    // true iff a builder has started.
+    static bool started;
+
+    // These three correspond to the resource shared with the manager.
+    // xxx_copy will be set in the manager's destructor to record the
+    // final state of the manager.
+    static std::list<Command>* command_queue;
+    static TestCondVar* cond;
+    static TestMutex* queue_mutex;
+    static std::list<Command> command_queue_copy;
+    static TestCondVar cond_copy;
+    static TestMutex queue_mutex_copy;
+
+    // true iff the manager waited on the thread running the builder.
+    static bool thread_waited;
+
+    // If set to true by a test, TestThread::wait() throws an exception
+    // exception.
+    enum ExceptionFromWait { NOTHROW, THROW_UNCAUGHT_EX, THROW_OTHER };
+    static ExceptionFromWait thread_throw_on_wait;
+
+    FakeDataSrcClientsBuilder(
+        std::list<Command>* command_queue,
+        TestCondVar* cond,
+        TestMutex* queue_mutex)
+    {
+        FakeDataSrcClientsBuilder::started = false;
+        FakeDataSrcClientsBuilder::command_queue = command_queue;
+        FakeDataSrcClientsBuilder::cond = cond;
+        FakeDataSrcClientsBuilder::queue_mutex = queue_mutex;
+        FakeDataSrcClientsBuilder::thread_waited = false;
+        FakeDataSrcClientsBuilder::thread_throw_on_wait = NOTHROW;
+    }
+    void run() {
+        FakeDataSrcClientsBuilder::started = true;
+    }
+};
+
+// A fake thread class that doesn't really invoke thread but simply calls
+// the given main function (synchronously).  Tests can tweak the wait()
+// behavior via some static variables so it will throw some exceptions.
+class TestThread {
+public:
+    TestThread(const boost::function<void()>& main) {
+        main();
+    }
+    void wait() {
+        FakeDataSrcClientsBuilder::thread_waited = true;
+        switch (FakeDataSrcClientsBuilder::thread_throw_on_wait) {
+        case FakeDataSrcClientsBuilder::NOTHROW:
+            break;
+        case FakeDataSrcClientsBuilder::THROW_UNCAUGHT_EX:
+            isc_throw(util::thread::Thread::UncaughtException,
+                      "TestThread wait() saw an exception");
+        case FakeDataSrcClientsBuilder::THROW_OTHER:
+            isc_throw(Unexpected,
+                      "General emulated failure in TestThread wait()");
+        }
+    }
+};
+} // namespace datasrc_clientmgr_internal
+
+// Convenient shortcut
+typedef DataSrcClientsMgrBase<
+    datasrc_clientmgr_internal::TestThread,
+    datasrc_clientmgr_internal::FakeDataSrcClientsBuilder,
+    datasrc_clientmgr_internal::TestMutex,
+    datasrc_clientmgr_internal::TestCondVar> TestDataSrcClientsMgr;
+
+// A specialization of manager's "cleanup" called at the end of the
+// destructor.  We use this to record the final values of some of the class
+// member variables.
+template<>
+void
+TestDataSrcClientsMgr::cleanup();
+
+} // namespace auth
+} // namespace isc
+
+#endif  // TEST_DATASRC_CLIENTS_MGR_H
+
+// Local Variables:
+// mode: c++
+// End: