Parcourir la source

[master] Merge branch 'trac2210'

Jelte Jansen il y a 12 ans
Parent
commit
5fd4177340

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

@@ -75,6 +75,43 @@ 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_RECONFIGURE_CONFIG_ERROR Error in data source configuration: %1
+The thread for maintaining data source clients has received a command to
+reconfigure, but the parameter data (the new configuration) contains an
+error. The most likely cause is that the datasource-specific configuration
+data is not what the data source expects. The system is still running with
+the data sources that were previously configured (i.e. as if the
+configuration has not changed), and the configuration data needs to be
+checked.
+The specific problem is printed in the log message.
+
+% AUTH_DATASRC_CLIENTS_BUILDER_RECONFIGURE_DATASRC_ERROR Error setting up data source: %1
+The thread for maintaining data source clients has received a command to
+reconfigure, but a data source failed to set up. This may be a problem with
+the data that is configured (e.g. unreadable files, inconsistent data,
+parser problems, database connection problems, etc.), but it could be a bug
+in the data source implementation as well. The system is still running with
+the data sources that were previously configured (i.e. as if the
+configuration has not changed).
+The specific problem is printed in the log message.
+
+% AUTH_DATASRC_CLIENTS_BUILDER_RECONFIGURE_ERROR Internal error setting up data source: %1
+The thread for maintaining data source clients has received a command to
+reconfigure, but raised an exception while setting up data sources. This is
+most likely an internal error in a data source, or a bug in the data source
+or the system itself, but it is probably a good idea to verify the
+configuration first. The system is still running with the data sources that
+were previously configured (i.e. as if the configuration has not changed).
+The specific problem is printed in the log message.
+
+% AUTH_DATASRC_CLIENTS_BUILDER_RECONFIGURE_STARTED data source reconfiguration started
+The thread for maintaining data source clients has received a command to
+reconfigure, and has now started this process.
+
+% AUTH_DATASRC_CLIENTS_BUILDER_RECONFIGURE_SUCCESS data source reconfiguration completed succesfully
+The thread for maintaining data source clients has finished reconfiguring
+the data source clients, and is now running with the new configuration.
+
 % AUTH_DATASRC_CLIENTS_BUILDER_STARTED data source builder thread started
 A separate thread for maintaining data source clients has been started.
 

+ 3 - 3
src/bin/auth/auth_srv.cc

@@ -269,7 +269,7 @@ public:
     const shared_ptr<TSIGKeyRing>* keyring_;
 
     /// The data source client list
-    AuthSrv::DataSrcClientListsPtr datasrc_client_lists_;
+    ClientListMapPtr datasrc_client_lists_;
 
     shared_ptr<ConfigurableClientList> getDataSrcClientList(
         const RRClass& rrclass)
@@ -933,8 +933,8 @@ AuthSrv::destroyDDNSForwarder() {
     }
 }
 
-AuthSrv::DataSrcClientListsPtr
-AuthSrv::swapDataSrcClientLists(DataSrcClientListsPtr new_lists) {
+ClientListMapPtr
+AuthSrv::swapDataSrcClientLists(ClientListMapPtr new_lists) {
     // TODO: Debug-build only check
     if (!impl_->mutex_.locked()) {
         isc_throw(isc::Unexpected, "Not locked!");

+ 3 - 8
src/bin/auth/auth_srv.h

@@ -19,6 +19,7 @@
 
 #include <datasrc/factory.h>
 #include <datasrc/client_list.h>
+#include <datasrc/datasrc_config.h>
 
 #include <dns/message.h>
 #include <dns/opcode.h>
@@ -302,12 +303,6 @@ public:
     /// If there was no forwarder yet, this method does nothing.
     void destroyDDNSForwarder();
 
-    /// \brief Shortcut typedef used for swapDataSrcClientLists().
-    typedef boost::shared_ptr<std::map<
-        isc::dns::RRClass, boost::shared_ptr<
-                               isc::datasrc::ConfigurableClientList> > >
-    DataSrcClientListsPtr;
-
     /// \brief Swap the currently used set of data source client lists with
     /// given one.
     ///
@@ -333,8 +328,8 @@ public:
     /// \param new_lists Shared pointer to a new set of data source client
     /// lists.
     /// \return The previous set of lists.  It can be NULL.
-    DataSrcClientListsPtr swapDataSrcClientLists(DataSrcClientListsPtr
-                                                 new_lists);
+    isc::datasrc::ClientListMapPtr
+        swapDataSrcClientLists(isc::datasrc::ClientListMapPtr new_lists);
 
     /// \brief Returns the currently used client list for the class.
     ///

+ 63 - 12
src/bin/auth/datasrc_clients_mgr.h

@@ -21,12 +21,16 @@
 #include <log/logger_support.h>
 #include <log/log_dbglevels.h>
 
+#include <auth/datasrc_config.h>
 #include <cc/data.h>
+#include <datasrc/client_list.h>
+#include <dns/rrclass.h>
 
 #include <auth/auth_log.h>
 
 #include <boost/array.hpp>
 #include <boost/bind.hpp>
+#include <boost/shared_ptr.hpp>
 
 #include <list>
 #include <utility>
@@ -43,6 +47,9 @@ namespace datasrc_clientmgr_internal {
 /// \brief ID of commands from the DataSrcClientsMgr to DataSrcClientsBuilder.
 enum CommandID {
     NOOP,         ///< Do nothing.  Only useful for tests; no argument
+    RECONFIGURE,  ///< Reconfigure the datasource client lists,
+                  ///  the argument to the command is the full new
+                  ///  datasources configuration.
     SHUTDOWN,     ///< Shutdown the builder; no argument
     NUM_COMMANDS
 };
@@ -92,7 +99,8 @@ public:
     /// \throw std::bad_alloc internal memory allocation failure.
     /// \throw isc::Unexpected general unexpected system errors.
     DataSrcClientsMgrBase() :
-        builder_(&command_queue_, &cond_, &queue_mutex_),
+        builder_(&command_queue_, &cond_, &queue_mutex_, &clients_map_,
+                 &map_mutex_),
         builder_thread_(boost::bind(&BuilderType::run, &builder_))
     {}
 
@@ -161,10 +169,9 @@ private:
     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
+    datasrc::ClientListMapPtr clients_map_;
+                                // map of actual data source client objects
+    MutexType map_mutex_;       // mutex to protect the clients map
 
     BuilderType builder_;
     ThreadType builder_thread_; // for safety this should be placed last
@@ -199,12 +206,12 @@ public:
     ///
     /// \throw None
     DataSrcClientsBuilderBase(std::list<Command>* command_queue,
-                              CondVarType* cond, MutexType* queue_mutex
-#ifdef notyet
-                              // In #2210 or #2212 we pass other data
-#endif
+                              CondVarType* cond, MutexType* queue_mutex,
+                              datasrc::ClientListMapPtr* clients_map,
+                              MutexType* map_mutex
         ) :
-        command_queue_(command_queue), cond_(cond), queue_mutex_(queue_mutex)
+        command_queue_(command_queue), cond_(cond), queue_mutex_(queue_mutex),
+        clients_map_(clients_map), map_mutex_(map_mutex)
     {}
 
     /// \brief The main loop.
@@ -225,10 +232,51 @@ private:
     // implementation really does nothing.
     void doNoop() {}
 
+    void doReconfigure(const data::ConstElementPtr& config) {
+        if (config) {
+            LOG_INFO(auth_logger,
+                     AUTH_DATASRC_CLIENTS_BUILDER_RECONFIGURE_STARTED);
+            try {
+                // Define new_clients_map outside of the block that
+                // has the lock scope; this way, after the swap,
+                // the lock is guaranteed to be released before
+                // the old data is destroyed, minimizing the lock
+                // duration.
+                datasrc::ClientListMapPtr new_clients_map =
+                    configureDataSource(config);
+                {
+                    typename MutexType::Locker locker(*map_mutex_);
+                    new_clients_map.swap(*clients_map_);
+                } // lock is released by leaving scope
+                LOG_INFO(auth_logger,
+                         AUTH_DATASRC_CLIENTS_BUILDER_RECONFIGURE_SUCCESS);
+            } catch (const datasrc::ConfigurableClientList::ConfigurationError&
+                     config_error) {
+                LOG_ERROR(auth_logger,
+                    AUTH_DATASRC_CLIENTS_BUILDER_RECONFIGURE_CONFIG_ERROR).
+                    arg(config_error.what());
+            } catch (const datasrc::DataSourceError& ds_error) {
+                LOG_ERROR(auth_logger,
+                    AUTH_DATASRC_CLIENTS_BUILDER_RECONFIGURE_DATASRC_ERROR).
+                    arg(ds_error.what());
+            } catch (const isc::Exception& isc_error) {
+                LOG_ERROR(auth_logger,
+                    AUTH_DATASRC_CLIENTS_BUILDER_RECONFIGURE_ERROR).
+                    arg(isc_error.what());
+            }
+            // other exceptions are propagated, see
+            // http://bind10.isc.org/ticket/2210#comment:13
+
+            // old clients_map_ data is released by leaving scope
+        }
+    }
+
     // The following are shared with the manager
     std::list<Command>* command_queue_;
     CondVarType* cond_;
     MutexType* queue_mutex_;
+    datasrc::ClientListMapPtr* clients_map_;
+    MutexType* map_mutex_;
 };
 
 // Shortcut typedef for normal use
@@ -253,7 +301,7 @@ DataSrcClientsBuilderBase<MutexType, CondVarType>::run() {
                 }
                 current_commands.splice(current_commands.end(),
                                         *command_queue_);
-            } // the lock is release here.
+            } // the lock is released here.
 
             while (keep_running && !current_commands.empty()) {
                 keep_running = handleCommand(current_commands.front());
@@ -285,11 +333,14 @@ DataSrcClientsBuilderBase<MutexType, CondVarType>::handleCommand(
     }
 
     const boost::array<const char*, NUM_COMMANDS> command_desc = {
-        {"NOOP", "SHUTDOWN"}
+        {"NOOP", "RECONFIGURE", "SHUTDOWN"}
     };
     LOG_DEBUG(auth_logger, DBGLVL_TRACE_BASIC,
               AUTH_DATASRC_CLIENTS_BUILDER_COMMAND).arg(command_desc.at(cid));
     switch (command.first) {
+    case RECONFIGURE:
+        doReconfigure(command.second);
+        break;
     case SHUTDOWN:
         return (false);
     case NOOP:

+ 1 - 2
src/bin/auth/datasrc_config.cc

@@ -13,12 +13,11 @@
 // PERFORMANCE OF THIS SOFTWARE.
 
 #include <cc/data.h>
-#include "auth_srv.h"
 #include "datasrc_config.h"
 
 // This is a trivial specialization for the commonly used version.
 // Defined in .cc to avoid accidental creation of multiple copies.
-AuthSrv::DataSrcClientListsPtr
+isc::datasrc::ClientListMapPtr
 configureDataSource(const isc::data::ConstElementPtr& config) {
     return (configureDataSourceGeneric<
             isc::datasrc::ConfigurableClientList>(config));

+ 4 - 3
src/bin/auth/datasrc_config.h

@@ -23,7 +23,7 @@
 #include <boost/shared_ptr.hpp>
 
 #include <utility>
-#include <set>
+#include <map>
 
 /// \brief Configure data source client lists
 ///
@@ -48,6 +48,8 @@
 /// \param config The configuration value to parse. It is in the form
 ///     as an update from the config manager.
 /// \return A map from RR classes to configured lists.
+/// \throw ConfigurationError if the config element is not in the expected
+///        format (A map of lists)
 template<class List>
 boost::shared_ptr<std::map<isc::dns::RRClass,
                            boost::shared_ptr<List> > > // = ListMap below
@@ -58,7 +60,6 @@ configureDataSourceGeneric(const isc::data::ConstElementPtr& config) {
 
     boost::shared_ptr<ListMap> new_lists(new ListMap);
 
-    // Go through the configuration and create corresponding list.
     const Map& map(config->mapValue());
     for (Map::const_iterator it(map.begin()); it != map.end(); ++it) {
         const isc::dns::RRClass rrclass(it->first);
@@ -73,7 +74,7 @@ configureDataSourceGeneric(const isc::data::ConstElementPtr& config) {
 
 /// \brief Concrete version of configureDataSource() for the
 ///     use with authoritative server implementation.
-AuthSrv::DataSrcClientListsPtr
+isc::datasrc::ClientListMapPtr
 configureDataSource(const isc::data::ConstElementPtr& config);
 
 #endif  // DATASRC_CONFIG_H

+ 1 - 1
src/bin/auth/main.cc

@@ -96,7 +96,7 @@ datasrcConfigHandler(AuthSrv* server, bool* first_time,
 {
     assert(server != NULL);
     if (config->contains("classes")) {
-        AuthSrv::DataSrcClientListsPtr lists;
+        isc::datasrc::ClientListMapPtr lists;
 
         if (*first_time) {
             // HACK: The default is not passed to the handler in the first

+ 4 - 4
src/bin/auth/tests/auth_srv_unittest.cc

@@ -727,7 +727,7 @@ TEST_F(AuthSrvTest, notifyWithSessionMessageError) {
 
 void
 installDataSrcClientLists(AuthSrv& server,
-                          AuthSrv::DataSrcClientListsPtr lists)
+                          ClientListMapPtr lists)
 {
     thread::Mutex::Locker locker(server.getDataSrcClientListMutex());
     server.swapDataSrcClientLists(lists);
@@ -1444,7 +1444,7 @@ TEST_F(AuthSrvTest,
         boost::shared_ptr<isc::datasrc::ConfigurableClientList>
             list(new FakeList(server.getDataSrcClientList(RRClass::IN()),
                               THROW_NEVER, false));
-        AuthSrv::DataSrcClientListsPtr lists(new std::map<RRClass, ListPtr>);
+        ClientListMapPtr lists(new std::map<RRClass, ListPtr>);
         lists->insert(pair<RRClass, ListPtr>(RRClass::IN(), list));
         server.swapDataSrcClientLists(lists);
     }
@@ -1475,7 +1475,7 @@ setupThrow(AuthSrv& server, ThrowWhen throw_when, bool isc_exception,
     boost::shared_ptr<isc::datasrc::ConfigurableClientList>
         list(new FakeList(server.getDataSrcClientList(RRClass::IN()),
                           throw_when, isc_exception, rrset));
-    AuthSrv::DataSrcClientListsPtr lists(new std::map<RRClass, ListPtr>);
+    ClientListMapPtr lists(new std::map<RRClass, ListPtr>);
     lists->insert(pair<RRClass, ListPtr>(RRClass::IN(), list));
     server.swapDataSrcClientLists(lists);
 }
@@ -1792,7 +1792,7 @@ TEST_F(AuthSrvTest, clientList) {
     isc::util::thread::Mutex::Locker locker(
         server.getDataSrcClientListMutex());
 
-    AuthSrv::DataSrcClientListsPtr lists; // initially empty
+    ClientListMapPtr lists; // initially empty
 
     // The lists don't exist. Therefore, the list of RRClasses is empty.
     EXPECT_TRUE(server.swapDataSrcClientLists(lists)->empty());

+ 1 - 5
src/bin/auth/tests/command_unittest.cc

@@ -18,8 +18,6 @@
 
 #include <util/threads/sync.h>
 
-#include <auth/auth_srv.h>
-#include <auth/auth_config.h>
 #include <auth/command.h>
 #include <auth/datasrc_config.h>
 
@@ -193,9 +191,7 @@ zoneChecks(AuthSrv& server) {
 }
 
 void
-installDataSrcClientLists(AuthSrv& server,
-                          AuthSrv::DataSrcClientListsPtr lists)
-{
+installDataSrcClientLists(AuthSrv& server, ClientListMapPtr lists) {
     isc::util::thread::Mutex::Locker locker(
         server.getDataSrcClientListMutex());
     server.swapDataSrcClientLists(lists);

+ 89 - 1
src/bin/auth/tests/datasrc_clients_builder_unittest.cc

@@ -22,13 +22,14 @@
 #include <boost/function.hpp>
 
 using isc::data::ConstElementPtr;
+using namespace isc::datasrc;
 using namespace isc::auth::datasrc_clientmgr_internal;
 
 namespace {
 class DataSrcClientsBuilderTest : public ::testing::Test {
 protected:
     DataSrcClientsBuilderTest() :
-        builder(&command_queue, &cond, &queue_mutex),
+        builder(&command_queue, &cond, &queue_mutex, &clients_map, &map_mutex),
         cond(command_queue, delayed_command_queue),
         shutdown_cmd(SHUTDOWN, ConstElementPtr()),
         noop_cmd(NOOP, ConstElementPtr())
@@ -37,8 +38,10 @@ protected:
     TestDataSrcClientsBuilder builder;
     std::list<Command> command_queue; // test command queue
     std::list<Command> delayed_command_queue; // commands available after wait
+    ClientListMapPtr clients_map; // configured clients
     TestCondVar cond;
     TestMutex queue_mutex;
+    TestMutex map_mutex;
     const Command shutdown_cmd;
     const Command noop_cmd;
 };
@@ -92,6 +95,91 @@ TEST_F(DataSrcClientsBuilderTest, condWait) {
     EXPECT_EQ(2, queue_mutex.unlock_count);
 }
 
+TEST_F(DataSrcClientsBuilderTest, reconfigure) {
+    // Full testing of different configurations is not here, but we
+    // do check a few cases of correct and erroneous input, to verify
+    // the error handling
+
+    // A command structure we'll modify to send different commands
+    Command reconfig_cmd(RECONFIGURE, ConstElementPtr());
+
+    // Initially, no clients should be there
+    EXPECT_EQ(ClientListMapPtr(), clients_map);
+
+    // A config that doesn't do much except be accepted
+    ConstElementPtr good_config = isc::data::Element::fromJSON(
+        "{"
+        "\"IN\": [{"
+        "   \"type\": \"MasterFiles\","
+        "   \"params\": {},"
+        "   \"cache-enable\": true"
+        "}]"
+        "}"
+    );
+
+    // A configuration that is 'correct' in the top-level, but contains
+    // bad data for the type it specifies
+    ConstElementPtr bad_config = isc::data::Element::fromJSON(
+        "{"
+        "\"IN\": [{"
+        "   \"type\": \"MasterFiles\","
+        "   \"params\": { \"foo\": [ 1, 2, 3, 4  ]},"
+        "   \"cache-enable\": true"
+        "}]"
+        "}"
+    );
+
+    reconfig_cmd.second = good_config;
+    EXPECT_TRUE(builder.handleCommand(reconfig_cmd));
+    EXPECT_EQ(1, clients_map->size());
+    EXPECT_EQ(1, map_mutex.lock_count);
+
+    // Store the nonempty clients map we now have
+    ClientListMapPtr working_config_clients(clients_map);
+
+    // If a 'bad' command argument got here, the config validation should
+    // have failed already, but still, the handler should return true,
+    // and the clients_map should not be updated.
+    reconfig_cmd.second = isc::data::Element::create("{ \"foo\": \"bar\" }");
+    EXPECT_TRUE(builder.handleCommand(reconfig_cmd));
+    EXPECT_EQ(working_config_clients, clients_map);
+    // Building failed, so map mutex should not have been locked again
+    EXPECT_EQ(1, map_mutex.lock_count);
+
+    // The same for a configuration that has bad data for the type it
+    // specifies
+    reconfig_cmd.second = bad_config;
+    builder.handleCommand(reconfig_cmd);
+    EXPECT_TRUE(builder.handleCommand(reconfig_cmd));
+    EXPECT_EQ(working_config_clients, clients_map);
+    // Building failed, so map mutex should not have been locked again
+    EXPECT_EQ(1, map_mutex.lock_count);
+
+    // The same goes for an empty parameter (it should at least be
+    // an empty map)
+    reconfig_cmd.second = ConstElementPtr();
+    EXPECT_TRUE(builder.handleCommand(reconfig_cmd));
+    EXPECT_EQ(working_config_clients, clients_map);
+    EXPECT_EQ(1, map_mutex.lock_count);
+
+    // Reconfigure again with the same good clients, the result should
+    // be a different map than the original, but not an empty one.
+    reconfig_cmd.second = good_config;
+    EXPECT_TRUE(builder.handleCommand(reconfig_cmd));
+    EXPECT_NE(working_config_clients, clients_map);
+    EXPECT_EQ(1, clients_map->size());
+    EXPECT_EQ(2, map_mutex.lock_count);
+
+    // And finally, try an empty config to disable all datasource clients
+    reconfig_cmd.second = isc::data::Element::createMap();
+    EXPECT_TRUE(builder.handleCommand(reconfig_cmd));
+    EXPECT_EQ(0, clients_map->size());
+    EXPECT_EQ(3, map_mutex.lock_count);
+
+    // Also check if it has been cleanly unlocked every time
+    EXPECT_EQ(3, map_mutex.unlock_count);
+}
+
 TEST_F(DataSrcClientsBuilderTest, shutdown) {
     EXPECT_FALSE(builder.handleCommand(shutdown_cmd));
 }

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

@@ -13,6 +13,7 @@
 // PERFORMANCE OF THIS SOFTWARE.
 
 #include <exceptions/exceptions.h>
+#include <auth/datasrc_config.h>
 
 #include "test_datasrc_clients_mgr.h"
 
@@ -27,6 +28,9 @@ std::list<Command> FakeDataSrcClientsBuilder::command_queue_copy;
 TestCondVar* FakeDataSrcClientsBuilder::cond = NULL;
 TestCondVar FakeDataSrcClientsBuilder::cond_copy;
 TestMutex* FakeDataSrcClientsBuilder::queue_mutex = NULL;
+isc::datasrc::ClientListMapPtr*
+    FakeDataSrcClientsBuilder::clients_map = NULL;
+TestMutex* FakeDataSrcClientsBuilder::map_mutex = NULL;
 TestMutex FakeDataSrcClientsBuilder::queue_mutex_copy;
 bool FakeDataSrcClientsBuilder::thread_waited = false;
 FakeDataSrcClientsBuilder::ExceptionFromWait

+ 8 - 1
src/bin/auth/tests/test_datasrc_clients_mgr.h

@@ -18,6 +18,7 @@
 #include <exceptions/exceptions.h>
 
 #include <auth/datasrc_clients_mgr.h>
+#include <datasrc/datasrc_config.h>
 
 #include <boost/function.hpp>
 
@@ -130,6 +131,8 @@ public:
     static std::list<Command>* command_queue;
     static TestCondVar* cond;
     static TestMutex* queue_mutex;
+    static isc::datasrc::ClientListMapPtr* clients_map;
+    static TestMutex* map_mutex;
     static std::list<Command> command_queue_copy;
     static TestCondVar cond_copy;
     static TestMutex queue_mutex_copy;
@@ -145,12 +148,16 @@ public:
     FakeDataSrcClientsBuilder(
         std::list<Command>* command_queue,
         TestCondVar* cond,
-        TestMutex* queue_mutex)
+        TestMutex* queue_mutex,
+        isc::datasrc::ClientListMapPtr* clients_map,
+        TestMutex* map_mutex)
     {
         FakeDataSrcClientsBuilder::started = false;
         FakeDataSrcClientsBuilder::command_queue = command_queue;
         FakeDataSrcClientsBuilder::cond = cond;
         FakeDataSrcClientsBuilder::queue_mutex = queue_mutex;
+        FakeDataSrcClientsBuilder::clients_map = clients_map;
+        FakeDataSrcClientsBuilder::map_mutex = map_mutex;
         FakeDataSrcClientsBuilder::thread_waited = false;
         FakeDataSrcClientsBuilder::thread_throw_on_wait = NOTHROW;
     }

+ 5 - 1
src/lib/datasrc/client_list.h

@@ -37,7 +37,6 @@ typedef boost::shared_ptr<DataSourceClient> DataSourceClientPtr;
 class DataSourceClientContainer;
 typedef boost::shared_ptr<DataSourceClientContainer>
     DataSourceClientContainerPtr;
-
 // XXX: it's better to even hide the existence of the "memory" namespace.
 // We should probably consider pimpl for details of ConfigurableClientList
 // and hide real definitions except for itself and tests.
@@ -391,6 +390,11 @@ protected:
     DataSources data_sources_;
 };
 
+/// \brief Shortcut typedef for maps of client_lists.
+typedef boost::shared_ptr<std::map<
+    isc::dns::RRClass, boost::shared_ptr<ConfigurableClientList> > >
+        ClientListMapPtr;
+
 } // namespace datasrc
 } // namespace isc