Browse Source

[3221] Added dhcp-ddns IO support to dhcp::D2ClientMgr

Added creation and management of NameChangeSender to dhcpsrv::D2ClientMgr.
Added new file of unit tests, d2_udp_unittest.cc, to exercise D2ClientMgr
with a UDP sender.
Thomas Markwalder 11 years ago
parent
commit
b3493c2c68

+ 194 - 11
src/lib/dhcpsrv/d2_client.cc

@@ -12,6 +12,7 @@
 // OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR
 // PERFORMANCE OF THIS SOFTWARE.
 
+#include <dhcp_ddns/ncr_udp.h>
 #include <dhcpsrv/d2_client.h>
 #include <dhcpsrv/dhcpsrv_log.h>
 
@@ -22,6 +23,8 @@ using namespace std;
 namespace isc {
 namespace dhcp {
 
+//***************************** D2ClientConfig ********************************
+
 D2ClientConfig::D2ClientConfig(const  bool enable_updates,
                                const isc::asiolink::IOAddress& server_ip,
                                const size_t server_port,
@@ -135,7 +138,12 @@ operator<<(std::ostream& os, const D2ClientConfig& config) {
     return (os);
 }
 
-D2ClientMgr::D2ClientMgr() : d2_client_config_(new D2ClientConfig()) {
+
+//******************************** D2ClientMgr ********************************
+
+
+D2ClientMgr::D2ClientMgr() : d2_client_config_(new D2ClientConfig()),
+    name_change_sender_(), private_io_service_(), sender_io_service_(NULL) {
     // Default constructor initializes with a disabled configuration.
 }
 
@@ -149,16 +157,57 @@ D2ClientMgr::setD2ClientConfig(D2ClientConfigPtr& new_config) {
                   "D2ClientMgr cannot set DHCP-DDNS configuration to NULL.");
     }
 
-    // @todo When NameChangeSender is integrated, we will need to handle these
-    // scenarios:
-    // 1. D2 was enabled but now it is disabled
-    //     - destroy the sender, flush any queued
-    // 2. D2 is still enabled but server parameters have changed
-    //     - preserve any queued,  reconnect based on sender parameters
-    // 3. D2 was was disabled now it is enabled.
-    //     - create sender
-    //
-    // For now we just update the configuration.
+    // Don't do anything unless configuration values are actually different.
+    if (*d2_client_config_ != *new_config) {
+        if (!new_config->getEnableUpdates()) {
+            // Updating has been turned off, destroy current sender.
+            // Any queued requests are tossed.
+            name_change_sender_.reset();
+        } else {
+            dhcp_ddns::NameChangeSenderPtr new_sender;
+            switch (new_config->getNcrProtocol()) {
+            case dhcp_ddns::NCR_UDP: {
+                /// @todo Should we be able to configure a sender's client
+                /// side ip and port?  We should certainly be able to
+                /// configure a maximum queue size.  These were overlooked
+                /// but are covered in Trac# 3328.
+                isc::asiolink::IOAddress any_addr("0.0.0.0");
+                uint32_t any_port = 0;
+                uint32_t queueMax = 1024;
+
+                // Instantiate a new sender.
+                new_sender.reset(new dhcp_ddns::NameChangeUDPSender(
+                                                any_addr, any_port,
+                                                new_config->getServerIp(),
+                                                new_config->getServerPort(),
+                                                new_config->getNcrFormat(),
+                                                *this, queueMax));
+                break;
+                }
+            default:
+                // In theory you can't get here.
+                isc_throw(D2ClientError, "Invalid sender Protocol: "
+                          << new_config->getNcrProtocol());
+                break;
+            }
+
+            // Transfer queued requests from previous sender to the new one.
+            /// @todo - Should we consider anything queued to be wrong?
+            /// If only server values changed content might still be right but
+            /// if content values changed (e.g. suffix or an override flag)
+            /// then the queued contents might now be invalid.  There is
+            /// no way to regenerate them if they are wrong.
+            if (name_change_sender_) {
+                name_change_sender_->stopSending();
+                new_sender->assumeQueue(*name_change_sender_);
+            }
+
+            // Replace the old sender with the new one.
+            name_change_sender_ = new_sender;
+        }
+    }
+
+    // Update the configuration.
     d2_client_config_ = new_config;
     LOG_DEBUG(dhcpsrv_logger, DHCPSRV_DBG_TRACE, DHCPSRV_CFGMGR_CFG_DHCP_DDNS)
               .arg(!ddnsEnabled() ? "DHCP-DDNS updates disabled" :
@@ -257,7 +306,141 @@ D2ClientMgr::qualifyName(const std::string& partial_name) const {
     return (gen_name.str());
 }
 
+void
+D2ClientMgr::startSender(D2ClientErrorHandler error_handler) {
+    // Create a our own service instance when we are not being multiplexed
+    // into an external service..
+    private_io_service_.reset(new asiolink::IOService());
+    startSender(error_handler, *private_io_service_);
+}
+
+void
+D2ClientMgr::startSender(D2ClientErrorHandler error_handler,
+                         isc::asiolink::IOService& io_service) {
+    if (!name_change_sender_)  {
+        isc_throw(D2ClientError, "D2ClientMgr::startSender sender is null");
+    }
+
+    if (!error_handler) {
+        isc_throw(D2ClientError, "D2ClientMgr::startSender handler is null");
+    }
+
+    // Set the error handler.
+    client_error_handler_ = error_handler;
+
+    // Remember the io service being used.
+    sender_io_service_ = &io_service;
+
+    // Start the sender on the given service.
+    name_change_sender_->startSending(*sender_io_service_);
+
+    /// @todo need to register sender's select-fd with IfaceMgr once 3315 is
+    /// done.
+}
+
+bool
+D2ClientMgr::amSending() const {
+    return (name_change_sender_ && name_change_sender_->amSending());
+}
 
+void
+D2ClientMgr::stopSender() {
+    if (!name_change_sender_)  {
+        isc_throw(D2ClientError, "D2ClientMgr::stopSender sender is null");
+    }
+
+    /// @todo need to unregister sender's select-fd with IfaceMgr once 3315 is
+    /// done.
+
+    name_change_sender_->stopSending();
+}
+
+void
+D2ClientMgr::sendRequest(dhcp_ddns::NameChangeRequestPtr& ncr) {
+    if (!name_change_sender_) {
+        isc_throw(D2ClientError, "D2ClientMgr::sendRequest sender is null");
+    }
+
+    name_change_sender_->sendRequest(ncr);
+}
+
+size_t
+D2ClientMgr::getQueueSize() const {
+    if (!name_change_sender_) {
+        isc_throw(D2ClientError, "D2ClientMgr::getQueueSize sender is null");
+    }
+
+    return(name_change_sender_->getQueueSize());
+}
+
+
+const dhcp_ddns::NameChangeRequestPtr&
+D2ClientMgr::peekAt(const size_t index) const {
+    if (!name_change_sender_) {
+        isc_throw(D2ClientError, "D2ClientMgr::peekAt sender is null");
+    }
+
+    return (name_change_sender_->peekAt(index));
+}
+
+void
+D2ClientMgr::clearQueue() {
+    if (!name_change_sender_) {
+        isc_throw(D2ClientError, "D2ClientMgr::clearQueue sender is null");
+    }
+
+    name_change_sender_->clearSendQueue();
+}
+
+void
+D2ClientMgr::operator()(const dhcp_ddns::NameChangeSender::Result result,
+                        dhcp_ddns::NameChangeRequestPtr& ncr) {
+    if (result == dhcp_ddns::NameChangeSender::SUCCESS) {
+        LOG_DEBUG(dhcpsrv_logger, DHCPSRV_DBG_TRACE_DETAIL,
+                  DHCPSRV_DHCP_DDNS_NCR_SENT).arg(ncr->toText());
+    } else {
+        // Handler is mandatory but test it just to be safe.
+        /// @todo Until we have a better feel for how errors need to be
+        /// handled we farm it out to the application layer.
+        if (client_error_handler_) {
+            // Handler is not supposed to throw, but catch just in case.
+            try {
+                (client_error_handler_)(result, ncr);
+            } catch (const std::exception& ex) {
+                LOG_ERROR(dhcpsrv_logger, DHCPSRV_DHCP_DDNS_ERROR_EXCEPTION)
+                          .arg(ex.what());
+            }
+        } else {
+            LOG_ERROR(dhcpsrv_logger, DHCPSRV_DHCP_DDNS_HANDLER_NULL);
+        }
+   }
+}
+
+int
+D2ClientMgr::getSelectFd() {
+    if (!amSending()) {
+        isc_throw (D2ClientError, "D2ClientMgr::getSelectFd "
+                   " not in send mode");
+    }
+
+    return (name_change_sender_->getSelectFd());
+}
+
+void
+D2ClientMgr::runReadyIO() {
+    if (!sender_io_service_) {
+        // This should never happen.
+        isc_throw(D2ClientError, "D2ClientMgr::runReadyIO"
+                  " sender io service is null");
+    }
+
+    // We shouldn't be here if IO isn't ready to execute.
+    // By running poll we're gauranteed not to hang.
+    /// @todo Trac# 3325 requests that asiolink::IOService provide a
+    /// wrapper for poll().
+    sender_io_service_->get_io_service().poll();
+}
 
 };  // namespace dhcp
+
 };  // namespace isc

+ 178 - 5
src/lib/dhcpsrv/d2_client.h

@@ -211,14 +211,53 @@ operator<<(std::ostream& os, const D2ClientConfig& config);
 /// @brief Defines a pointer for D2ClientConfig instances.
 typedef boost::shared_ptr<D2ClientConfig> D2ClientConfigPtr;
 
+/// @brief Defines the type for D2 IO error handler.
+/// This callback is invoked when a send to b10-dhcp-ddns completes with a
+/// failed status.  This provides the application layer (Kea) with a means to
+/// handle the error appropriately.
+///
+/// @param result Result code of the send operation.
+/// @param ncr NameChangeRequest which failed to send.
+///
+/// @note Handlers are expected not to throw. In the event a hanlder does
+/// throw invoking code logs the exception and then swallows it.
+typedef
+boost::function<void(const dhcp_ddns::NameChangeSender::Result result,
+                     dhcp_ddns::NameChangeRequestPtr& ncr)> D2ClientErrorHandler;
+
 /// @brief D2ClientMgr isolates Kea from the details of being a D2 client.
 ///
-/// Provides services for managing the current D2ClientConfig and managing
-/// communications with D2. (@todo The latter will be added once communication
-/// with D2 is implemented through the integration of
-/// dhcp_ddns::NameChangeSender interface(s)).
+/// Provides services for managing the current dhcp-ddns configuration and
+/// as well as communications with b10-dhcp-ddns.  Regarding configuration it
+/// provides services to store, update, and access the current dhcp-ddns
+/// configuration.  As for b10-dhcp-ddns communications, D2ClientMgr creates
+/// maintains a NameChangeSender appropriate to the current configuration and
+/// provides services to start, stop, and post NCRs to the sender.  Additionally/// there are methods to examine the queue of requests currently waiting for
+/// transmission.
+///
+/// The manager also provides the mechanics to integrate the ASIO-based IO
+/// used by the NCR IPC with the select-driven IO used by Kea.  Senders expose
+/// a file descriptor, the "select-fd" that can monitored for read-readiness
+/// with the select() function (or variants).  D2ClientMgr provides a method,
+/// runReadyIO(), that will process all ready events on a sender's
+/// IOservice.  Track# 3315 is extending Kea's IfaceMgr to support the
+/// registration of multiple external sockets with callbacks that are then
+/// monitored with IO readiness via select().
+/// @todo D2ClientMgr will be modified to register the sender's select-fd and
+/// runReadyIO() with IfaceMgr when entering the send mode and will
+/// unregister when exiting send mode.
+///
+/// To place the manager in send mode, the calling layer must supply an error
+/// handler and optionally an IOService instance.  The error handler is invoked
+/// if a send completes with a failed status. This provides the calling layer
+/// an opportunity act upon the error.
+///
+/// If the caller supplies an IOService, that service will be used to process
+/// the sender's IO.  If not supplied, D2ClientMgr pass a private IOService
+/// into the sender.  Using a private service isolates the sender's IO from
+/// any other services.
 ///
-class D2ClientMgr {
+class D2ClientMgr : public dhcp_ddns::NameChangeSender::RequestSendHandler {
 public:
     /// @brief Constructor
     ///
@@ -331,9 +370,143 @@ public:
     template <class T>
     void adjustDomainName(const T& fqdn, T& fqdn_resp);
 
+    /// @brief Enables sending NameChangeRequests to b10-dhcp-ddns
+    ///
+    /// Places the NameChangeSender into send mode. This instructs the
+    /// sender to begin dequeuing and transmitting requests and to accept
+    /// additional requests via the sendRequest() method.
+    ///
+    /// @param error_handler application level error handler to cope with
+    /// sends that complete with a failed status.  A valid function must be
+    /// supplied as the manager cannot know how an application should deal
+    /// with send failures.
+    /// @param io_service IOService to be used for sender IO event processing
+    ///
+    /// @throw D2ClientError if sender instance is null. Underlying layer
+    /// may throw NCRSenderExceptions exceptions.
+    void startSender(D2ClientErrorHandler error_handler,
+                     isc::asiolink::IOService& io_service);
+
+    /// @brief Enables sending NameChangeRequests to b10-dhcp-ddns
+    ///
+    /// Places the NameChangeSender into send mode. This instructs the
+    /// sender to begin dequeuing and transmitting requests and to accept
+    /// additional requests via the sendRequest() method.  The manager
+    /// will create a new, private instance of an IOService for the sender
+    /// to use for IO event processing.
+    ///
+    /// @param error_handler application level error handler to cope with
+    /// sends that complete with a failed status.  A valid function must be
+    /// supplied as the manager cannot know how an application should deal
+    /// with send failures.
+    ///
+    /// @throw D2ClientError if sender instance is null. Underlying layer
+    /// may throw NCRSenderExceptions exceptions.
+    void startSender(D2ClientErrorHandler error_handler);
+
+    /// @brief Returns true if the sender is in send mode, false otherwise.
+    ///
+    /// A true value indicates that the sender is present and in accepting
+    /// messages for transmission, false otherwise.
+    bool amSending() const;
+
+    /// @brief Disables sending NameChangeRequests to b10-dhcp-ddns
+    ///
+    /// Takes the NameChangeSender out of send mode.  The sender will stop
+    /// transmitting requests, though any queued requests remain queued.
+    /// Attempts to queue additional requests via sendRequest will fail.
+    ///
+    /// @param io_service IOService to be used for sender IO event processing
+    ///
+    /// @throw D2ClientError if sender instance is null. Underlying layer
+    /// may throw NCRSenderExceptions exceptions.
+    void stopSender();
+
+    /// @brief Send the given NameChangeRequests to b10-dhcp-ddns
+    ///
+    /// Passes NameChangeRequests to the NCR sender for transmission to
+    /// b10-dhcp-ddns.
+    ///
+    /// @param ncr NameChangeRequest to send
+    ///
+    /// @throw D2ClientError if sender instance is null. Underlying layer
+    /// may throw NCRSenderExceptions exceptions.
+    void sendRequest(dhcp_ddns::NameChangeRequestPtr& ncr);
+
+    /// @brief Returns the number of NCRs queued for transmission.
+    size_t getQueueSize() const;
+
+    /// @brief Returns the nth NCR queued for transmission.
+    ///
+    /// Note that the entry is not removed from the queue.
+    /// @param index the index of the entry in the queue to fetch.
+    /// Valid values are 0 (front of the queue) to (queue size - 1).
+    /// @note This method is for test purposes only.
+    ///
+    /// @return Pointer reference to the queue entry.
+    ///
+    /// @throw D2ClientError if sender instance is null. Underlying layer
+    /// may throw NCRSenderExceptions exceptions.
+    const dhcp_ddns::NameChangeRequestPtr& peekAt(const size_t index) const;
+
+    /// @brief Removes all NCRs queued for transmission.
+    ///
+    /// @throw D2ClientError if sender instance is null. Underlying layer
+    /// may throw NCRSenderExceptions exceptions.
+    void clearQueue();
+
+    /// @brief Processes sender IO events
+    ///
+    /// Runs all handlers ready for execution on the sender's IO service.
+    void runReadyIO();
+
+protected:
+    /// @brief Function operator implementing the NCR sender callback.
+    ///
+    /// This method is invoked each time the NameChangeSender completes
+    /// an asychronous send.
+    ///
+    /// @param result contains that send outcome status.
+    /// @param ncr is a pointer to the NameChangeRequest that was
+    /// delivered (or attempted).
+    ///
+    /// @throw This method MUST NOT throw.
+    virtual void operator ()(const dhcp_ddns::NameChangeSender::Result result,
+                             dhcp_ddns::NameChangeRequestPtr& ncr);
+
+    /// @brief Fetches the sender's select-fd.
+    ///
+    /// The select-fd may be used with select() or poll().  If the sender has
+    /// IO waiting to process, the fd will evaluate as !EWOULDBLOCK.
+    /// @note This is only exposed for testing purposes.
+    ///
+    /// @return The sender's select-fd
+    ///
+    /// @throw D2ClientError if the sender does not exist or is not in send
+    /// mode.
+    int getSelectFd();
+
 private:
     /// @brief Container class for DHCP-DDNS configuration parameters.
     D2ClientConfigPtr d2_client_config_;
+
+    /// @brief Pointer to the current interface to DHCP-DDNS.
+    dhcp_ddns::NameChangeSenderPtr name_change_sender_;
+
+    /// @brief Private IOService to use if calling layer doesn't wish to
+    /// supply one.
+    boost::shared_ptr<asiolink::IOService> private_io_service_;
+
+    /// @brief Application supplied error handler invoked when a send
+    /// completes with a failed status.
+    D2ClientErrorHandler client_error_handler_;
+
+    /// @brief Pointer to the IOService currently being used by the sender.
+    /// @note We need to remember the io_service given to the sender however
+    /// we may have received only a referenece to it from the calling layer.
+    /// Use a raw pointer to store it.  This value should never be exposed
+    /// and is only valid while in send mode.
+    asiolink::IOService* sender_io_service_;
 };
 
 template <class T>

+ 15 - 0
src/lib/dhcpsrv/dhcpsrv_messages.mes

@@ -344,3 +344,18 @@ indicate an error in the source code, please submit a bug report.
 % DHCPSRV_UNKNOWN_DB unknown database type: %1
 The database access string specified a database type (given in the
 message) that is unknown to the software.  This is a configuration error.
+
+% DHCPSRV_DHCP_DDNS_HANDLER_NULL error handler for DHCP_DDNS IO is not set.
+This is an error message that occurs when an attempt to send a request to
+b10-dhcp-ddns fails and there is no registered error handler.  This is a
+programmatic error which should never occur and should be reported.
+
+% DHCPSRV_DHCP_DDNS_ERROR_EXCEPTION error handler for DHCP_DDNS IO generated an expected exception: %1
+This is an error message that occurs when an attempt to send a request to
+b10-dhcp-ddns fails there registered error handler threw an uncaught exception.
+This is a programmatic error which should not occur. By convention, the error
+handler should not propagate exceptions. Please report this error.
+
+% DHCPSRV_DHCP_DDNS_NCR_SENT NameChangeRequest sent to b10-dhcp-ddns: %1
+A debug message issued when a NameChangeRequest has been successfully sent to
+b10-dhcp-ddns.

+ 1 - 0
src/lib/dhcpsrv/tests/Makefile.am

@@ -53,6 +53,7 @@ libdhcpsrv_unittests_SOURCES += alloc_engine_unittest.cc
 libdhcpsrv_unittests_SOURCES += callout_handle_store_unittest.cc
 libdhcpsrv_unittests_SOURCES += cfgmgr_unittest.cc
 libdhcpsrv_unittests_SOURCES += d2_client_unittest.cc
+libdhcpsrv_unittests_SOURCES += d2_udp_unittest.cc
 libdhcpsrv_unittests_SOURCES += dbaccess_parser_unittest.cc
 libdhcpsrv_unittests_SOURCES += lease_unittest.cc
 libdhcpsrv_unittests_SOURCES += lease_mgr_factory_unittest.cc

+ 380 - 0
src/lib/dhcpsrv/tests/d2_udp_unittest.cc

@@ -0,0 +1,380 @@
+// Copyright (C) 2014 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.
+
+/// @file d2_upd_unittest.cc Unit tests for D2ClientMgr UDP communications.
+/// Note these tests are not intended to verify the actual send and receive
+/// across UDP sockets.  This level of testing is done in libdhcp-ddns.
+
+#include <asio.hpp>
+#include <asiolink/io_service.h>
+#include <config.h>
+#include <dhcpsrv/d2_client.h>
+#include <exceptions/exceptions.h>
+
+#include <boost/function.hpp>
+#include <boost/bind.hpp>
+
+#include <gtest/gtest.h>
+
+#include <sys/select.h>
+#include <sys/ioctl.h>
+
+using namespace std;
+using namespace isc::dhcp;
+using namespace isc;
+
+namespace {
+
+/// @brief Test fixture for excerising D2ClientMgr send management
+/// services.  It inherents from D2ClientMgr to allow overriding various
+/// methods and accessing otherwise restricted member.  In particular it
+/// overrides the NameChangeSender completion completion callback, allowing
+/// the injection of send errors.
+class D2ClientMgrTest : public D2ClientMgr, public ::testing::Test {
+public:
+    /// @brief If true simulates a send which completed with a failed status.
+    bool simulate_send_failure_;
+    /// @brief If true causes an exception throw in the client error handler.
+    bool error_handler_throw_;
+    /// @brief Tracks the number times the completion handler is called.
+    int callback_count_;
+    /// @brief Tracks the number of times the client error handler was called.
+    int error_handler_count_;
+
+    /// @brief Constructor
+    D2ClientMgrTest() : simulate_send_failure_(false),
+                       error_handler_throw_(false),
+                       callback_count_(0), error_handler_count_(0) {
+    }
+
+    /// @brief virtual Destructor
+    virtual ~D2ClientMgrTest(){
+    }
+
+    /// @brief Updates the D2ClientMgr's configuration to DDNS disabled.
+    void disableDdns() {
+        D2ClientConfigPtr new_cfg;
+        ASSERT_NO_THROW(new_cfg.reset(new D2ClientConfig()));
+        ASSERT_NO_THROW(setD2ClientConfig(new_cfg));
+        ASSERT_FALSE(ddnsEnabled());
+    }
+
+    /// @brief Updates the D2ClientMgr's configuration to DDNS enabled.
+    ///
+    /// @param server_address IP address of b10-dhcp-ddns.
+    /// @param server_port IP port number of b10-dhcp-ddns.
+    /// @param protocol NCR protocol to use. (Currently only UDP is
+    /// supported).
+    void enableDdns(const std::string& server_address,
+                    const size_t server_port,
+                    const dhcp_ddns::NameChangeProtocol protocol) {
+        // Update the configuration with one that is enabled.
+        D2ClientConfigPtr new_cfg;
+        ASSERT_NO_THROW(new_cfg.reset(new D2ClientConfig(true,
+                                  isc::asiolink::IOAddress(server_address),
+                                  server_port,
+                                  protocol, dhcp_ddns::FMT_JSON,
+                                  true, true, true, true,
+                                  "myhost", ".example.com.")));
+        ASSERT_NO_THROW(setD2ClientConfig(new_cfg));
+        ASSERT_TRUE(ddnsEnabled());
+    }
+
+    /// @brief Checks sender's select-fd against an expected state of readiness.
+    ///
+    /// Uses select() to determine if the sender's select_fd is marked as
+    /// ready to read, and compares this against the expected state.  The
+    /// select function is called with a timeout of 0.0 (non blocking).
+    ///
+    /// @param expect_ready Expected state of readiness (True if expecting
+    /// a ready to ready result,  false if expecting otherwise).
+    void selectCheck(bool expect_ready) {
+        fd_set read_fds;
+        int maxfd = 0;
+
+        FD_ZERO(&read_fds);
+
+        int select_fd = -1;
+        ASSERT_NO_THROW(select_fd = getSelectFd());
+
+        FD_SET(select_fd,  &read_fds);
+        maxfd = select_fd;
+
+        struct timeval select_timeout;
+        select_timeout.tv_sec = 0;
+        select_timeout.tv_usec = 0;
+
+        int result = (select(maxfd + 1, &read_fds, NULL, NULL,
+                      &select_timeout));
+
+        if (result < 0) {
+            const char *errstr = strerror(errno);
+            FAIL() << "select failed :" << errstr;
+        }
+
+        if (expect_ready) {
+            ASSERT_TRUE(result > 0);
+        } else {
+            ASSERT_TRUE(result == 0);
+        }
+    }
+
+    /// @brief Overrides base class completion callback.
+    ///
+    /// This method will be invoked each time a send completes. It allows
+    /// intervention prior to calling the production implemenation in the
+    /// base.  If simulate_send_failure_ is true, the base call impl will
+    /// be called with an error status, otherwise it will be called with
+    /// the result paramater given.
+    ///
+    /// @param result Result code of the send operation.
+    /// @param ncr NameChangeRequest which failed to send.
+    virtual void operator()(const dhcp_ddns::NameChangeSender::Result result,
+                            dhcp_ddns::NameChangeRequestPtr& ncr) {
+        ++callback_count_;
+        if (simulate_send_failure_) {
+            simulate_send_failure_ = false;
+            D2ClientMgr::operator()(dhcp_ddns::NameChangeSender::ERROR, ncr);
+        } else {
+            D2ClientMgr::operator()(result, ncr);
+        }
+    }
+
+    /// @brief Serves as the "application level" client error handler.
+    ///
+    /// This method is passed into calls to startSender as the client error
+    /// handler.  It should be invoked whenever the completion callback is
+    /// passed a result other than SUCCESS.  If error_handler_throw_
+    /// is true it will throw an exception.
+    ///
+    /// @param result unused - Result code of the send operation.
+    /// @param ncr unused -NameChangeRequest which failed to send.
+    void error_handler(const dhcp_ddns::NameChangeSender::Result /*result*/,
+                       dhcp_ddns::NameChangeRequestPtr& /*ncr*/) {
+        if (error_handler_throw_) {
+            error_handler_throw_ = false;
+            isc_throw(isc::InvalidOperation, "Simulated client handler throw");
+        }
+
+        ++error_handler_count_;
+    }
+
+    /// @brief Returns D2ClientErroHandler bound to this::error_handler_.
+    D2ClientErrorHandler getErrorHandler() {
+        return (boost::bind(&D2ClientMgrTest::error_handler, this, _1, _2));
+    }
+
+    /// @brief Contructs a NameChangeRequest message from a fixed JSON string.
+    dhcp_ddns::NameChangeRequestPtr buildTestNcr() {
+        // Build an NCR from json string.
+        const char* ncr_str =
+            "{"
+            " \"change_type\" : 0 , "
+            " \"forward_change\" : true , "
+            " \"reverse_change\" : false , "
+            " \"fqdn\" : \"myhost.example.com.\" , "
+            " \"ip_address\" : \"192.168.2.1\" , "
+            " \"dhcid\" : \"010203040A7F8E3D\" , "
+            " \"lease_expires_on\" : \"20140121132405\" , "
+            " \"lease_length\" : 1300 "
+            "}";
+
+        return (dhcp_ddns::NameChangeRequest::fromJSON(ncr_str));
+    }
+
+    /// Expose restricted members.
+    using D2ClientMgr::getSelectFd;
+};
+
+
+/// @brief Checks that D2ClientMgr disable and enable a UDP sender.
+TEST_F(D2ClientMgrTest, udpSenderEnableDisable) {
+    // Verify DDNS is disabled by default.
+    ASSERT_FALSE(ddnsEnabled());
+
+    // Verify we are not in send mode.
+    ASSERT_FALSE(amSending());
+
+    // Enable DDNS with server at 127.0.0.1/prot 53001 via UDP.
+    enableDdns("127.0.0.1", 530001, dhcp_ddns::NCR_UDP);
+    ASSERT_FALSE(amSending());
+
+    ASSERT_NO_THROW(startSender(getErrorHandler()));
+    ASSERT_TRUE(amSending());
+
+    // Verify that we take sender out of send mode.
+    ASSERT_NO_THROW(stopSender());
+    ASSERT_FALSE(amSending());
+}
+
+/// @brief Checks D2ClientMgr queuing methods with a UDP sender.
+TEST_F(D2ClientMgrTest, udpSenderQueing) {
+    // Enable DDNS with server at 127.0.0.1/prot 53001 via UDP.
+    enableDdns("127.0.0.1", 530001, dhcp_ddns::NCR_UDP);
+    ASSERT_FALSE(amSending());
+
+    // Queue should be empty.
+    EXPECT_EQ(0, getQueueSize());
+
+    // Trying to peek past the end of the queue should throw.
+    EXPECT_THROW(peekAt(1), dhcp_ddns::NcrSenderError);
+
+    // Trying to send a NCR when not in send mode should fail.
+    dhcp_ddns::NameChangeRequestPtr ncr = buildTestNcr();
+    EXPECT_THROW(sendRequest(ncr), dhcp_ddns::NcrSenderError);
+
+    // Place sender in send mode.
+    ASSERT_NO_THROW(startSender(getErrorHandler()));
+    ASSERT_TRUE(amSending());
+
+    // Send should succeed now.
+    ASSERT_NO_THROW(sendRequest(ncr));
+
+    // Queue should have 1 entry.
+    EXPECT_EQ(1, getQueueSize());
+
+    // Attempt to fetch the entry we just queued.
+    dhcp_ddns::NameChangeRequestPtr ncr2;
+    ASSERT_NO_THROW(ncr2 = peekAt(0));
+
+    // Verify what we queued matches what we fetched.
+    EXPECT_TRUE(*ncr == *ncr2);
+
+    // Clearing the queue while in send mode should fail.
+    ASSERT_THROW(clearQueue(), dhcp_ddns::NcrSenderError);
+
+    // We should still have 1 in the queue.
+    EXPECT_EQ(1, getQueueSize());
+
+    // Get out of send mode.
+    ASSERT_NO_THROW(stopSender());
+    ASSERT_FALSE(amSending());
+
+    // Clear queue should succeed now.
+    ASSERT_NO_THROW(clearQueue());
+    EXPECT_EQ(0, getQueueSize());
+}
+
+/// @brief Checks that D2ClientMgr can send with a UDP sender and
+/// a private IOService.
+TEST_F(D2ClientMgrTest, udpSend) {
+    // Enable DDNS with server at 127.0.0.1/prot 53001 via UDP.
+    enableDdns("127.0.0.1", 530001, dhcp_ddns::NCR_UDP);
+
+    // Trying to fetch the select-fd when not sending should fail.
+    ASSERT_THROW(getSelectFd(), D2ClientError);
+
+    // Place sender in send mode.
+    ASSERT_NO_THROW(startSender(getErrorHandler()));
+
+    // select_fd should evaluate to NOT ready to read.
+    selectCheck(false);
+
+    // Build a test request and send it.
+    dhcp_ddns::NameChangeRequestPtr ncr = buildTestNcr();
+    ASSERT_NO_THROW(sendRequest(ncr));
+
+    // select_fd should evaluate to ready to read.
+    selectCheck(true);
+
+    // Call service handler.
+    runReadyIO();
+
+    // select_fd should evaluate to not ready to read.
+    selectCheck(false);
+}
+
+/// @brief Checks that D2ClientMgr can send with a UDP sender and
+/// an external IOService.
+TEST_F(D2ClientMgrTest, udpSendExternalIOService) {
+    // Enable DDNS with server at 127.0.0.1/prot 53001 via UDP.
+    enableDdns("127.0.0.1", 530001, dhcp_ddns::NCR_UDP);
+
+    // Place sender in send mode using an external IO service.
+    asiolink::IOService io_service;
+    ASSERT_NO_THROW(startSender(getErrorHandler(), io_service));
+
+    // select_fd should evaluate to NOT ready to read.
+    selectCheck(false);
+
+    // Build a test request and send it.
+    dhcp_ddns::NameChangeRequestPtr ncr = buildTestNcr();
+    ASSERT_NO_THROW(sendRequest(ncr));
+
+    // select_fd should evaluate to ready to read.
+    selectCheck(true);
+
+    // Call service handler.
+    runReadyIO();
+
+    // select_fd should evaluate to not ready to read.
+    selectCheck(false);
+}
+
+/// @brief Checks that D2ClientMgr invokes the client error handler
+/// when send errors occur.
+TEST_F(D2ClientMgrTest, udpSendErrorHandler) {
+    // Enable DDNS with server at 127.0.0.1/prot 53001 via UDP.
+    enableDdns("127.0.0.1", 530001, dhcp_ddns::NCR_UDP);
+
+    // Trying to fetch the select-fd when not sending should fail.
+    ASSERT_THROW(getSelectFd(), D2ClientError);
+
+    // Place sender in send mode.
+    ASSERT_NO_THROW(startSender(getErrorHandler()));
+
+    // select_fd should evaluate to NOT ready to read.
+    selectCheck(false);
+
+    // Simulate a failed response in the send call back. This should
+    // cause the error handler to get invoked.
+    simulate_send_failure_ = true;
+
+    ASSERT_EQ(0, error_handler_count_);
+
+    // Send a test request.
+    dhcp_ddns::NameChangeRequestPtr ncr = buildTestNcr();
+    ASSERT_NO_THROW(sendRequest(ncr));
+
+    // select_fd should evaluate to ready to read.
+    selectCheck(true);
+
+    // Call service handler.
+    runReadyIO();
+
+    // select_fd should evaluate to not ready to read.
+    selectCheck(false);
+
+    ASSERT_EQ(1, error_handler_count_);
+
+    // Simulate a failed response in the send call back. This should
+    // cause the error handler to get invoked.
+    simulate_send_failure_ = true;
+    error_handler_throw_ = true;
+
+    // Send a test request.
+    ncr = buildTestNcr();
+    ASSERT_NO_THROW(sendRequest(ncr));
+
+    // Call the io service handler.
+    runReadyIO();
+
+    // Simulation flag should be false.
+    ASSERT_FALSE(error_handler_throw_);
+
+    // Count should still be 1.
+    ASSERT_EQ(1, error_handler_count_);
+}
+
+} // end of anonymous namespace