const int Memfile_LeaseMgr::MINOR_VERSION_V6;
Memfile_LeaseMgr::Memfile_LeaseMgr(const DatabaseConnection::ParameterMap& parameters)
- : LeaseMgr(), lfc_setup_(), conn_(parameters), mutex_(new std::mutex) {
+ : TrackingLeaseMgr(), lfc_setup_(), conn_(parameters), mutex_(new std::mutex) {
bool conversion_needed = false;
// Check if the extended info tables are enabled.
// Increment class lease counters.
class_lease_counter_.addLease(lease);
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackAddLease(lease, true);
+ }
+
return (true);
}
static_cast<void>(addExtendedInfo6(lease));
}
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackAddLease(lease, true);
+ }
+
return (true);
}
// Adjust class lease counters.
class_lease_counter_.updateLease(lease, old_lease);
+
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackUpdateLease(lease, true);
+ }
}
void
break;
}
}
+
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackUpdateLease(lease, true);
+ }
}
void
// Decrement class lease counters.
class_lease_counter_.removeLease(lease);
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackDeleteLease(lease, true);
+ }
+
return (true);
}
}
deleteExtendedInfo6(lease->addr_);
}
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackDeleteLease(lease, true);
+ }
+
return (true);
}
}
#include <dhcpsrv/csv_lease_file6.h>
#include <dhcpsrv/memfile_lease_limits.h>
#include <dhcpsrv/memfile_lease_storage.h>
-#include <dhcpsrv/lease_mgr.h>
+#include <dhcpsrv/tracking_lease_mgr.h>
#include <boost/scoped_ptr.hpp>
#include <boost/shared_ptr.hpp>
/// is not specified, the default location in the installation
/// directory is used: <install-dir>/var/lib/kea/kea-leases4.csv and
/// <install-dir>/var/lib/kea/kea-leases6.csv.
-class Memfile_LeaseMgr : public LeaseMgr {
+class Memfile_LeaseMgr : public TrackingLeaseMgr {
public:
/// @defgroup v4 memfile backend versions
-// Copyright (C) 2012-2022 Internet Systems Consortium, Inc. ("ISC")
+// Copyright (C) 2012-2023 Internet Systems Consortium, Inc. ("ISC")
//
// This Source Code Form is subject to the terms of the Mozilla Public
// License, v. 2.0. If a copy of the MPL was not distributed with this
// If running in single-threaded mode, there's nothing to do here.
}
+// MySqlLeaseTrackingContextAlloc Constructor and Destructor
+
+MySqlLeaseMgr::MySqlLeaseTrackingContextAlloc::MySqlLeaseTrackingContextAlloc(
+ MySqlLeaseMgr& mgr, const LeasePtr& lease) : ctx_(), mgr_(mgr), lease_(lease) {
+
+ if (MultiThreadingMgr::instance().getMode()) {
+ // multi-threaded
+ {
+ // we need to protect the whole pool_ operation, hence extra scope {}
+ lock_guard<mutex> lock(mgr_.pool_->mutex_);
+ if (mgr_.hasCallbacks() && !mgr_.tryLock(lease)) {
+ isc_throw(DbOperationError, "unable to lock the lease " << lease->addr_);
+ }
+ if (!mgr_.pool_->pool_.empty()) {
+ ctx_ = mgr_.pool_->pool_.back();
+ mgr_.pool_->pool_.pop_back();
+ }
+ }
+ if (!ctx_) {
+ ctx_ = mgr_.createContext();
+ }
+ } else {
+ // single-threaded
+ if (mgr_.pool_->pool_.empty()) {
+ isc_throw(Unexpected, "No available MySQL lease context?!");
+ }
+ ctx_ = mgr_.pool_->pool_.back();
+ }
+}
+
+MySqlLeaseMgr::MySqlLeaseTrackingContextAlloc::~MySqlLeaseTrackingContextAlloc() {
+ if (MultiThreadingMgr::instance().getMode()) {
+ // multi-threaded
+ lock_guard<mutex> lock(mgr_.pool_->mutex_);
+ if (mgr_.hasCallbacks()) {
+ mgr_.unlock(lease_);
+ }
+ mgr_.pool_->pool_.push_back(ctx_);
+ }
+ // If running in single-threaded mode, there's nothing to do here.
+}
+
void
MySqlLeaseMgr::setExtendedInfoTablesEnabled(const db::DatabaseConnection::ParameterMap& /* parameters */) {
isc_throw(isc::NotImplemented, "extended info tables are not yet supported by mysql");
// MySqlLeaseMgr Constructor and Destructor
MySqlLeaseMgr::MySqlLeaseMgr(const DatabaseConnection::ParameterMap& parameters)
- : parameters_(parameters), timer_name_("") {
+ : TrackingLeaseMgr(), parameters_(parameters), timer_name_("") {
// Check if the extended info tables are enabled.
LeaseMgr::setExtendedInfoTablesEnabled(parameters);
MySqlLeaseMgr::addLeaseCommon(MySqlLeaseContextPtr& ctx,
StatementIndex stindex,
std::vector<MYSQL_BIND>& bind) {
-
// Bind the parameters to the statement
int status = mysql_stmt_bind_param(ctx->conn_.statements_[stindex], &bind[0]);
checkError(ctx, status, stindex, "unable to bind parameters");
.arg(lease->addr_.toText());
// Get a context
- MySqlLeaseContextAlloc get_context(*this);
+ MySqlLeaseTrackingContextAlloc get_context(*this, lease);
MySqlLeaseContextPtr ctx = get_context.ctx_;
// Create the MYSQL_BIND array for the lease
// of the Lease up to the point of insertion in the database).
lease->updateCurrentExpirationTime();
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackAddLease(lease, false);
+ }
+
return (result);
}
.arg(lease->type_);
// Get a context
- MySqlLeaseContextAlloc get_context(*this);
+ MySqlLeaseTrackingContextAlloc get_context(*this, lease);
MySqlLeaseContextPtr ctx = get_context.ctx_;
// Create the MYSQL_BIND array for the lease
// of the Lease up to the point of insertion in the database).
lease->updateCurrentExpirationTime();
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackAddLease(lease, false);
+ }
+
return (result);
}
.arg(lease->addr_.toText());
// Get a context
- MySqlLeaseContextAlloc get_context(*this);
+ MySqlLeaseTrackingContextAlloc get_context(*this, lease);
MySqlLeaseContextPtr ctx = get_context.ctx_;
// Create the MYSQL_BIND array for the data being updated
// Update lease current expiration time.
lease->updateCurrentExpirationTime();
+
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackUpdateLease(lease, false);
+ }
}
void
.arg(lease->type_);
// Get a context
- MySqlLeaseContextAlloc get_context(*this);
+ MySqlLeaseTrackingContextAlloc get_context(*this, lease);
MySqlLeaseContextPtr ctx = get_context.ctx_;
// Create the MYSQL_BIND array for the data being updated
// Update lease current expiration time.
lease->updateCurrentExpirationTime();
+
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackUpdateLease(lease, false);
+ }
}
// Delete lease methods. Similar to other groups of methods, these comprise
// handles the common processing.
uint64_t
-MySqlLeaseMgr::deleteLeaseCommon(StatementIndex stindex,
+MySqlLeaseMgr::deleteLeaseCommon(MySqlLeaseContextPtr& ctx,
+ StatementIndex stindex,
MYSQL_BIND* bind) {
-
- // Get a context
- MySqlLeaseContextAlloc get_context(*this);
- MySqlLeaseContextPtr ctx = get_context.ctx_;
-
// Bind the input parameters to the statement
int status = mysql_stmt_bind_param(ctx->conn_.statements_[stindex], bind);
checkError(ctx, status, stindex, "unable to bind WHERE clause parameter");
inbind[1].buffer = reinterpret_cast<char*>(&expire);
inbind[1].buffer_length = sizeof(expire);
- auto affected_rows = deleteLeaseCommon(DELETE_LEASE4, inbind);
+ // Get a context
+ MySqlLeaseTrackingContextAlloc get_context(*this, lease);
+ MySqlLeaseContextPtr ctx = get_context.ctx_;
+
+ auto affected_rows = deleteLeaseCommon(ctx, DELETE_LEASE4, inbind);
// Check success case first as it is the most likely outcome.
if (affected_rows == 1) {
+ if (hasCallbacks()) {
+ trackDeleteLease(lease, false);
+ }
return (true);
}
inbind[1].buffer = reinterpret_cast<char*>(&expire);
inbind[1].buffer_length = sizeof(expire);
- auto affected_rows = deleteLeaseCommon(DELETE_LEASE6, inbind);
+ // Get a context
+ MySqlLeaseTrackingContextAlloc get_context(*this, lease);
+ MySqlLeaseContextPtr ctx = get_context.ctx_;
+
+ auto affected_rows = deleteLeaseCommon(ctx, DELETE_LEASE6, inbind);
// Check success case first as it is the most likely outcome.
if (affected_rows == 1) {
+ if (hasCallbacks()) {
+ trackDeleteLease(lease, false);
+ }
return (true);
}
inbind[1].buffer = reinterpret_cast<char*>(&expire_time);
inbind[1].buffer_length = sizeof(expire_time);
+ // Get a context
+ MySqlLeaseContextAlloc get_context(*this);
+ MySqlLeaseContextPtr ctx = get_context.ctx_;
+
// Get the number of deleted leases and log it.
- uint64_t deleted_leases = deleteLeaseCommon(statement_index, inbind);
+ uint64_t deleted_leases = deleteLeaseCommon(ctx, statement_index, inbind);
LOG_DEBUG(dhcpsrv_logger, DHCPSRV_DBG_TRACE_DETAIL, DHCPSRV_MYSQL_DELETED_EXPIRED_RECLAIMED)
.arg(deleted_leases);
-// Copyright (C) 2012-2022 Internet Systems Consortium, Inc. ("ISC")
+// Copyright (C) 2012-2023 Internet Systems Consortium, Inc. ("ISC")
//
// This Source Code Form is subject to the terms of the Mozilla Public
// License, v. 2.0. If a copy of the MPL was not distributed with this
#include <asiolink/io_service.h>
#include <dhcp/hwaddr.h>
#include <dhcpsrv/dhcpsrv_exceptions.h>
-#include <dhcpsrv/lease_mgr.h>
+#include <dhcpsrv/tracking_lease_mgr.h>
#include <mysql/mysql_connection.h>
#include <boost/scoped_ptr.hpp>
/// database. Use of this backend presupposes that a MySQL database is
/// available and that the Kea schema has been created within it.
-class MySqlLeaseMgr : public LeaseMgr {
+class MySqlLeaseMgr : public TrackingLeaseMgr {
public:
/// @brief Constructor
/// to the prepared statement, executes the statement and checks to
/// see how many rows were deleted.
///
+ /// @param ctx Context
/// @param stindex Index of prepared statement to be executed
/// @param bind Array of MYSQL_BIND objects representing the parameters.
/// (Note that the number is determined by the number of parameters
///
/// @throw isc::db::DbOperationError An operation on the open database has
/// failed.
- uint64_t deleteLeaseCommon(StatementIndex stindex,
+ uint64_t deleteLeaseCommon(MySqlLeaseContextPtr& ctx,
+ StatementIndex stindex,
MYSQL_BIND* bind);
/// @brief Delete expired-reclaimed leases.
/// @return The number of updates in the database or 0.
virtual size_t buildExtendedInfoTables6(bool update, bool current) override;
- /// @brief Context RAII Allocator.
+ /// @brief Context RAII allocator.
class MySqlLeaseContextAlloc {
public:
const MySqlLeaseMgr& mgr_;
};
+ /// @brief Context RAII allocator for lease tracking.
+ ///
+ /// This context should be used in the non-const calls that
+ /// may trigger callbacks for lease tracking.
+ class MySqlLeaseTrackingContextAlloc {
+ public:
+
+ /// @brief Constructor
+ ///
+ /// This constructor takes a context of the pool if one is available
+ /// or creates a new one.
+ ///
+ /// @param mgr A parent instance
+ /// @param lease allocated or deallocated lease instance.
+ MySqlLeaseTrackingContextAlloc(MySqlLeaseMgr& mgr, const LeasePtr& lease);
+
+ /// @brief Destructor
+ ///
+ /// This destructor puts back the context in the pool.
+ ~MySqlLeaseTrackingContextAlloc();
+
+ /// @brief The context
+ MySqlLeaseContextPtr ctx_;
+
+ private:
+
+ /// @brief The manager
+ MySqlLeaseMgr& mgr_;
+
+ /// @brief Tracked lease instance.
+ LeasePtr lease_;
+ };
+
protected:
/// Extended information / Bulk Lease Query shared interface.
// If running in single-threaded mode, there's nothing to do here.
}
+// PgSqlLeaseTrackingContextAlloc Constructor and Destructor
+
+PgSqlLeaseMgr::PgSqlLeaseTrackingContextAlloc::PgSqlLeaseTrackingContextAlloc(
+ PgSqlLeaseMgr& mgr, const LeasePtr& lease) : ctx_(), mgr_(mgr), lease_(lease) {
+
+ if (MultiThreadingMgr::instance().getMode()) {
+ // multi-threaded
+ {
+ // we need to protect the whole pool_ operation, hence extra scope {}
+ lock_guard<mutex> lock(mgr_.pool_->mutex_);
+ if (mgr_.hasCallbacks() && !mgr_.tryLock(lease)) {
+ isc_throw(DbOperationError, "unable to lock the lease " << lease->addr_);
+ }
+ if (!mgr_.pool_->pool_.empty()) {
+ ctx_ = mgr_.pool_->pool_.back();
+ mgr_.pool_->pool_.pop_back();
+ }
+ }
+ if (!ctx_) {
+ ctx_ = mgr_.createContext();
+ }
+ } else {
+ // single-threaded
+ if (mgr_.pool_->pool_.empty()) {
+ isc_throw(Unexpected, "No available PostgreSQL lease context?!");
+ }
+ ctx_ = mgr_.pool_->pool_.back();
+ }
+}
+
+PgSqlLeaseMgr::PgSqlLeaseTrackingContextAlloc::~PgSqlLeaseTrackingContextAlloc() {
+ if (MultiThreadingMgr::instance().getMode()) {
+ // multi-threaded
+ lock_guard<mutex> lock(mgr_.pool_->mutex_);
+ if (mgr_.hasCallbacks()) {
+ mgr_.unlock(lease_);
+ }
+ mgr_.pool_->pool_.push_back(ctx_);
+ }
+ // If running in single-threaded mode, there's nothing to do here.
+}
+
void
PgSqlLeaseMgr::setExtendedInfoTablesEnabled(const db::DatabaseConnection::ParameterMap& /* parameters */) {
isc_throw(isc::NotImplemented, "extended info tables are not yet supported by mysql");
// PgSqlLeaseMgr Constructor and Destructor
PgSqlLeaseMgr::PgSqlLeaseMgr(const DatabaseConnection::ParameterMap& parameters)
- : parameters_(parameters), timer_name_("") {
+ : TrackingLeaseMgr(), parameters_(parameters), timer_name_("") {
// Check if the extended info tables are enabled.
LeaseMgr::setExtendedInfoTablesEnabled(parameters);
.arg(lease->addr_.toText());
// Get a context
- PgSqlLeaseContextAlloc get_context(*this);
+ PgSqlLeaseTrackingContextAlloc get_context(*this, lease);
PgSqlLeaseContextPtr ctx = get_context.ctx_;
PsqlBindArray bind_array;
// of the Lease up to the point of insertion in the database).
lease->updateCurrentExpirationTime();
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackAddLease(lease, false);
+ }
+
return (result);
}
.arg(lease->type_);
// Get a context
- PgSqlLeaseContextAlloc get_context(*this);
+ PgSqlLeaseTrackingContextAlloc get_context(*this, lease);
PgSqlLeaseContextPtr ctx = get_context.ctx_;
PsqlBindArray bind_array;
// of the Lease up to the point of insertion in the database).
lease->updateCurrentExpirationTime();
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackAddLease(lease, false);
+ }
+
return (result);
}
.arg(lease->addr_.toText());
// Get a context
- PgSqlLeaseContextAlloc get_context(*this);
+ PgSqlLeaseTrackingContextAlloc get_context(*this, lease);
PgSqlLeaseContextPtr ctx = get_context.ctx_;
// Create the BIND array for the data being updated
// Update lease current expiration time.
lease->updateCurrentExpirationTime();
+
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackUpdateLease(lease, false);
+ }
}
void
.arg(lease->type_);
// Get a context
- PgSqlLeaseContextAlloc get_context(*this);
+ PgSqlLeaseTrackingContextAlloc get_context(*this, lease);
PgSqlLeaseContextPtr ctx = get_context.ctx_;
// Create the BIND array for the data being updated
// Update lease current expiration time.
lease->updateCurrentExpirationTime();
+
+ // Run installed callbacks.
+ if (hasCallbacks()) {
+ trackUpdateLease(lease, false);
+ }
}
uint64_t
-PgSqlLeaseMgr::deleteLeaseCommon(StatementIndex stindex,
+PgSqlLeaseMgr::deleteLeaseCommon(PgSqlLeaseContextPtr& ctx,
+ StatementIndex stindex,
PsqlBindArray& bind_array) {
- // Get a context
- PgSqlLeaseContextAlloc get_context(*this);
- PgSqlLeaseContextPtr ctx = get_context.ctx_;
-
PgSqlResult r(PQexecPrepared(ctx->conn_, tagged_statements[stindex].name,
tagged_statements[stindex].nbparams,
&bind_array.values_[0],
}
bind_array.add(expire_str);
- auto affected_rows = deleteLeaseCommon(DELETE_LEASE4, bind_array);
+ // Get a context
+ PgSqlLeaseTrackingContextAlloc get_context(*this, lease);
+ PgSqlLeaseContextPtr ctx = get_context.ctx_;
+
+ auto affected_rows = deleteLeaseCommon(ctx, DELETE_LEASE4, bind_array);
// Check success case first as it is the most likely outcome.
if (affected_rows == 1) {
+ if (hasCallbacks()) {
+ trackDeleteLease(lease, false);
+ }
return (true);
}
}
bind_array.add(expire_str);
- auto affected_rows = deleteLeaseCommon(DELETE_LEASE6, bind_array);
+ // Get a context
+ PgSqlLeaseTrackingContextAlloc get_context(*this, lease);
+ PgSqlLeaseContextPtr ctx = get_context.ctx_;
+
+ auto affected_rows = deleteLeaseCommon(ctx, DELETE_LEASE6, bind_array);
// Check success case first as it is the most likely outcome.
if (affected_rows == 1) {
+ if (hasCallbacks()) {
+ trackDeleteLease(lease, false);
+ }
return (true);
}
static_cast<time_t>(secs));
bind_array.add(expiration_str);
+ // Get a context
+ PgSqlLeaseContextAlloc get_context(*this);
+ PgSqlLeaseContextPtr ctx = get_context.ctx_;
+
// Delete leases.
- return (deleteLeaseCommon(statement_index, bind_array));
+ return (deleteLeaseCommon(ctx, statement_index, bind_array));
}
string
-// Copyright (C) 2013-2022 Internet Systems Consortium, Inc. ("ISC")
+// Copyright (C) 2013-2023 Internet Systems Consortium, Inc. ("ISC")
//
// This Source Code Form is subject to the terms of the Mozilla Public
// License, v. 2.0. If a copy of the MPL was not distributed with this
#include <dhcp/hwaddr.h>
#include <dhcpsrv/dhcpsrv_exceptions.h>
#include <dhcpsrv/lease_mgr.h>
+#include <dhcpsrv/tracking_lease_mgr.h>
#include <pgsql/pgsql_connection.h>
#include <pgsql/pgsql_exchange.h>
/// database. Use of this backend presupposes that a PostgreSQL database is
/// available and that the Kea schema has been created within it.
-class PgSqlLeaseMgr : public LeaseMgr {
+class PgSqlLeaseMgr : public TrackingLeaseMgr {
public:
/// @brief Constructor
/// to the prepared statement, executes the statement and checks to
/// see how many rows were deleted.
///
+ /// @param ctx Context
/// @param stindex Index of prepared statement to be executed
/// @param bind_array Array containing lease values and where clause
/// parameters for the delete
///
/// @throw isc::db::DbOperationError An operation on the open database has
/// failed.
- uint64_t deleteLeaseCommon(StatementIndex stindex,
+ uint64_t deleteLeaseCommon(PgSqlLeaseContextPtr& ctx,
+ StatementIndex stindex,
db::PsqlBindArray& bind_array);
/// @brief Delete expired-reclaimed leases.
/// @brief Write V6 leases to a file.
virtual void writeLeases6(const std::string& /*filename*/) override;
- /// @brief Context RAII Allocator.
+ /// @brief Context RAII allocator.
class PgSqlLeaseContextAlloc {
public:
const PgSqlLeaseMgr& mgr_;
};
+ /// @brief Context RAII allocator for lease tracking.
+ ///
+ /// This context should be used in the non-const calls that
+ /// may trigger callbacks for lease tracking.
+ class PgSqlLeaseTrackingContextAlloc {
+ public:
+
+ /// @brief Constructor
+ ///
+ /// This constructor takes a context of the pool if one is available
+ /// or creates a new one.
+ ///
+ /// @param mgr A parent instance
+ /// @param lease allocated or deallocated lease instance.
+ PgSqlLeaseTrackingContextAlloc(PgSqlLeaseMgr& mgr, const LeasePtr& lease);
+
+ /// @brief Destructor
+ ///
+ /// This destructor puts back the context in the pool.
+ ~PgSqlLeaseTrackingContextAlloc();
+
+ /// @brief The context
+ PgSqlLeaseContextPtr ctx_;
+
+ private:
+
+ /// @brief The manager
+ PgSqlLeaseMgr& mgr_;
+
+ /// @brief Tracked lease instance.
+ LeasePtr lease_;
+ };
+
protected:
/// Extended information / Bulk Lease Query shared interface.
#include <gtest/gtest.h>
+#include <functional>
#include <limits>
#include <sstream>
}
void
-GenericLeaseMgrTest::logCallback(const TrackingLeaseMgr::CallbackType type, SubnetID subnet_id,
- const LeasePtr& lease) {
- logs_.push_back(Log{type, subnet_id, lease});
+GenericLeaseMgrTest::logCallback(TrackingLeaseMgr::CallbackType type, SubnetID subnet_id,
+ LeasePtr lease, bool mt_safe) {
+ auto locked = (lmptr_ ? lmptr_->isLocked(lease) : false);
+ logs_.push_back(Log{type, subnet_id, lease, mt_safe, locked});
}
int
ASSERT_FALSE(lease);
}
+void
+GenericLeaseMgrTest::testTrackAddLease4(bool expect_locked, bool expect_mt_safe) {
+ // Register a callback for all subnets.
+ lmptr_->registerCallback(TrackingLeaseMgr::TRACK_ADD_LEASE, 0, "flq",
+ std::bind(&GenericLeaseMgrTest::logCallback,
+ this,
+ TrackingLeaseMgr::TRACK_ADD_LEASE,
+ 0,
+ ph::_1,
+ ph::_2));
+ // Add a lease. It should trigger the callback.
+ Lease4Ptr lease = initializeLease4(straddress4_[1]);
+ EXPECT_TRUE(lmptr_->addLease(lease));
+
+ // Make sure that the callback has been invoked.
+ ASSERT_EQ(1, logs_.size());
+
+ // This flag should be false for the Memfile backend and true
+ // for the SQL backends.
+ if (expect_locked) {
+ EXPECT_TRUE(logs_[0].locked);
+ } else {
+ EXPECT_FALSE(logs_[0].locked);
+ }
+ // This flag should be set to true for the Memfile backends.
+ // It should be false for other backends. If the backends do
+ // not provide the MT-safe context, the callbacks must protect
+ // against the concurrent access on their own.
+ if (expect_mt_safe) {
+ EXPECT_TRUE(logs_[0].mt_safe);
+ } else {
+ EXPECT_FALSE(logs_[0].mt_safe);
+ }
+
+ // The lease locks should have been released.
+ EXPECT_FALSE(lmptr_->isLocked(lease));
+}
+
+void
+GenericLeaseMgrTest::testTrackAddLease6(bool expect_locked, bool expect_mt_safe) {
+ // Register a callback for all subnets.
+ lmptr_->registerCallback(TrackingLeaseMgr::TRACK_ADD_LEASE, 0, "flq",
+ std::bind(&GenericLeaseMgrTest::logCallback,
+ this,
+ TrackingLeaseMgr::TRACK_ADD_LEASE,
+ 0,
+ ph::_1,
+ ph::_2));
+ // Add a lease. It should trigger the callback.
+ Lease6Ptr lease = initializeLease6(straddress6_[1]);
+ EXPECT_TRUE(lmptr_->addLease(lease));
+
+ // Make sure that the callback has been invoked.
+ ASSERT_EQ(1, logs_.size());
+
+ // This flag should be false for the Memfile backend and true
+ // for the SQL backends.
+ if (expect_locked) {
+ EXPECT_TRUE(logs_[0].locked);
+ } else {
+ EXPECT_FALSE(logs_[0].locked);
+ }
+ // This flag should be set to true for the Memfile backends.
+ // It should be false for other backends. If the backends do
+ // not provide the MT-safe context, the callbacks must protect
+ // against the concurrent access on their own.
+ if (expect_mt_safe) {
+ EXPECT_TRUE(logs_[0].mt_safe);
+ } else {
+ EXPECT_FALSE(logs_[0].mt_safe);
+ }
+
+ // The lease locks should have been released.
+ EXPECT_FALSE(lmptr_->isLocked(lease));
+}
+
+void
+GenericLeaseMgrTest::testTrackUpdateLease4(bool expect_locked, bool expect_mt_safe) {
+ // Register a callback for all subnets.
+ lmptr_->registerCallback(TrackingLeaseMgr::TRACK_UPDATE_LEASE, 0, "flq",
+ std::bind(&GenericLeaseMgrTest::logCallback,
+ this,
+ TrackingLeaseMgr::TRACK_UPDATE_LEASE,
+ 0,
+ ph::_1,
+ ph::_2));
+ // Add a lease.
+ Lease4Ptr lease = initializeLease4(straddress4_[1]);
+ EXPECT_TRUE(lmptr_->addLease(lease));
+ EXPECT_TRUE(logs_.empty());
+
+ lmptr_->updateLease4(lease);
+
+ // Make sure that the callback has been invoked.
+ ASSERT_EQ(1, logs_.size());
+
+ // This flag should be false for the Memfile backend and true
+ // for the SQL backends.
+ if (expect_locked) {
+ EXPECT_TRUE(logs_[0].locked);
+ } else {
+ EXPECT_FALSE(logs_[0].locked);
+ }
+ // This flag should be set to true for the Memfile backends.
+ // It should be false for other backends. If the backends do
+ // not provide the MT-safe context, the callbacks must protect
+ // against the concurrent access on their own.
+ if (expect_mt_safe) {
+ EXPECT_TRUE(logs_[0].mt_safe);
+ } else {
+ EXPECT_FALSE(logs_[0].mt_safe);
+ }
+
+ // The lease locks should have been released.
+ EXPECT_FALSE(lmptr_->isLocked(lease));
+}
+
+void
+GenericLeaseMgrTest::testTrackUpdateLease6(bool expect_locked, bool expect_mt_safe) {
+ // Register a callback for all subnets.
+ lmptr_->registerCallback(TrackingLeaseMgr::TRACK_UPDATE_LEASE, 0, "flq",
+ std::bind(&GenericLeaseMgrTest::logCallback,
+ this,
+ TrackingLeaseMgr::TRACK_UPDATE_LEASE,
+ 0,
+ ph::_1,
+ ph::_2));
+ // Add a lease.
+ Lease6Ptr lease = initializeLease6(straddress6_[1]);
+ EXPECT_TRUE(lmptr_->addLease(lease));
+ EXPECT_TRUE(logs_.empty());
+
+ lmptr_->updateLease6(lease);
+
+ // Make sure that the callback has been invoked.
+ ASSERT_EQ(1, logs_.size());
+
+ // This flag should be false for the Memfile backend and true
+ // for the SQL backends.
+ if (expect_locked) {
+ EXPECT_TRUE(logs_[0].locked);
+ } else {
+ EXPECT_FALSE(logs_[0].locked);
+ }
+ // This flag should be set to true for the Memfile backends.
+ // It should be false for other backends. If the backends do
+ // not provide the MT-safe context, the callbacks must protect
+ // against the concurrent access on their own.
+ if (expect_mt_safe) {
+ EXPECT_TRUE(logs_[0].mt_safe);
+ } else {
+ EXPECT_FALSE(logs_[0].mt_safe);
+ }
+
+ // The lease locks should have been released.
+ EXPECT_FALSE(lmptr_->isLocked(lease));
+}
+
+void
+GenericLeaseMgrTest::testTrackDeleteLease4(bool expect_locked, bool expect_mt_safe) {
+ // Register a callback for all subnets.
+ lmptr_->registerCallback(TrackingLeaseMgr::TRACK_DELETE_LEASE, 0, "flq",
+ std::bind(&GenericLeaseMgrTest::logCallback,
+ this,
+ TrackingLeaseMgr::TRACK_DELETE_LEASE,
+ 0,
+ ph::_1,
+ ph::_2));
+ // Add a lease.
+ Lease4Ptr lease = initializeLease4(straddress4_[1]);
+ EXPECT_TRUE(lmptr_->addLease(lease));
+ EXPECT_TRUE(logs_.empty());
+
+ lmptr_->deleteLease(lease);
+
+ // Make sure that the callback has been invoked.
+ ASSERT_EQ(1, logs_.size());
+
+ // This flag should be false for the Memfile backend and true
+ // for the SQL backends.
+ if (expect_locked) {
+ EXPECT_TRUE(logs_[0].locked);
+ } else {
+ EXPECT_FALSE(logs_[0].locked);
+ }
+ // This flag should be set to true for the Memfile backends.
+ // It should be false for other backends. If the backends do
+ // not provide the MT-safe context, the callbacks must protect
+ // against the concurrent access on their own.
+ if (expect_mt_safe) {
+ EXPECT_TRUE(logs_[0].mt_safe);
+ } else {
+ EXPECT_FALSE(logs_[0].mt_safe);
+ }
+
+ // The lease locks should have been released.
+ EXPECT_FALSE(lmptr_->isLocked(lease));
+}
+
+void
+GenericLeaseMgrTest::testTrackDeleteLease6(bool expect_locked, bool expect_mt_safe) {
+ // Register a callback for all subnets.
+ lmptr_->registerCallback(TrackingLeaseMgr::TRACK_DELETE_LEASE, 0, "flq",
+ std::bind(&GenericLeaseMgrTest::logCallback,
+ this,
+ TrackingLeaseMgr::TRACK_DELETE_LEASE,
+ 0,
+ ph::_1,
+ ph::_2));
+ // Add a lease.
+ Lease6Ptr lease = initializeLease6(straddress6_[1]);
+ EXPECT_TRUE(lmptr_->addLease(lease));
+ EXPECT_TRUE(logs_.empty());
+
+ lmptr_->deleteLease(lease);
+
+ // Make sure that the callback has been invoked.
+ ASSERT_EQ(1, logs_.size());
+
+ // This flag should be false for the Memfile backend and true
+ // for the SQL backends.
+ if (expect_locked) {
+ EXPECT_TRUE(logs_[0].locked);
+ } else {
+ EXPECT_FALSE(logs_[0].locked);
+ }
+ // This flag should be set to true for the Memfile backends.
+ // It should be false for other backends. If the backends do
+ // not provide the MT-safe context, the callbacks must protect
+ // against the concurrent access on their own.
+ if (expect_mt_safe) {
+ EXPECT_TRUE(logs_[0].mt_safe);
+ } else {
+ EXPECT_FALSE(logs_[0].mt_safe);
+ }
+
+ // The lease locks should have been released.
+ EXPECT_FALSE(lmptr_->isLocked(lease));
+}
+
} // namespace test
} // namespace dhcp
} // namespace isc
TrackingLeaseMgr::CallbackType type;
SubnetID subnet_id;
LeasePtr lease;
+ bool mt_safe;
+ bool locked;
} Log;
/// @brief Default constructor.
/// @param type callback type.
/// @param subnet_id subnet identifier.
/// @param lease lease instance.
- void logCallback(const TrackingLeaseMgr::CallbackType type, SubnetID subnet_id,
- const LeasePtr& lease);
+ /// @param mt a boolean flag indicating if the function has been called
+ /// in the thread-safe context.
+ void logCallback(TrackingLeaseMgr::CallbackType type, SubnetID subnet_id,
+ LeasePtr lease, bool mt_safe);
/// @brief Counts log entries.
///
/// @brief Checks a few v6 lease limit checking scenarios.
void testLeaseLimits6();
+ /// @brief Checks if the backends call the callbacks when an
+ /// IPv4 lease is added.
+ ///
+ /// @param expect_locked a boolean flag indicating if the test should
+ /// expect that the lease is locked before the callback.
+ /// @param expect_mt_safe a boolean flag indicating if the test should
+ /// expect that the callbacks are called in the MT-safe context.
+ void testTrackAddLease4(bool expect_locked, bool expect_mt_safe);
+
+ /// @brief Checks if the backends call the callbacks when an
+ /// IPv6 lease is added.
+ ///
+ /// @param expect_locked a boolean flag indicating if the test should
+ /// expect that the lease is locked before the callback.
+ /// @param expect_mt_safe a boolean flag indicating if the test should
+ /// expect that the callbacks are called in the MT-safe context.
+ void testTrackAddLease6(bool expect_locked, bool expect_mt_safe);
+
+ /// @brief Checks if the backends call the callbacks when an
+ /// IPv4 lease is updated.
+ ///
+ /// @param expect_locked a boolean flag indicating if the test should
+ /// expect that the lease is locked before the callback.
+ /// @param expect_mt_safe a boolean flag indicating if the test should
+ /// expect that the callbacks are called in the MT-safe context.
+ void testTrackUpdateLease4(bool expect_locked, bool expect_mt_safe);
+
+ /// @brief Checks if the backends call the callbacks when an
+ /// IPv6 lease is updated.
+ ///
+ /// @param expect_locked a boolean flag indicating if the test should
+ /// expect that the lease is locked before the callback.
+ /// @param expect_mt_safe a boolean flag indicating if the test should
+ /// expect that the callbacks are called in the MT-safe context.
+ void testTrackUpdateLease6(bool expect_locked, bool expect_mt_safe);
+
+ /// @brief Checks if the backends call the callbacks when an
+ /// IPv4 lease is deleted.
+ ///
+ /// @param expect_locked a boolean flag indicating if the test should
+ /// expect that the lease is locked before the callback.
+ /// @param expect_mt_safe a boolean flag indicating if the test should
+ /// expect that the callbacks are called in the MT-safe context.
+ void testTrackDeleteLease4(bool expect_locked, bool expect_mt_safe);
+
+ /// @brief Checks if the backends call the callbacks when an
+ /// IPv6 lease is deleted.
+ ///
+ /// @param expect_locked a boolean flag indicating if the test should
+ /// expect that the lease is locked before the callback.
+ /// @param expect_mt_safe a boolean flag indicating if the test should
+ /// expect that the callbacks are called in the MT-safe context.
+ void testTrackDeleteLease6(bool expect_locked, bool expect_mt_safe);
+
/// @brief String forms of IPv4 addresses
std::vector<std::string> straddress4_;
std::vector<Log> logs_;
/// @brief Pointer to the lease manager
- LeaseMgr* lmptr_;
+ TrackingLeaseMgr* lmptr_;
};
class LeaseMgrDbLostCallbackTest : public ::testing::Test {
" lease database backend.\n";
throw;
}
- lmptr_ = &(LeaseMgrFactory::instance());
+ lmptr_ = static_cast<TrackingLeaseMgr*>(&(LeaseMgrFactory::instance()));
}
/// @brief Runs IOService and stops after a specified time.
// Recreate Memfile_LeaseMgr.
LeaseMgrFactory::destroy();
ASSERT_NO_THROW(LeaseMgrFactory::create(dbaccess));
- lmptr_ = &(LeaseMgrFactory::instance());
+ lmptr_ = static_cast<TrackingLeaseMgr*>(&(LeaseMgrFactory::instance()));
// We will store addresses here, so it will be easier to randomly
// pick a lease.
// Recreate Memfile_LeaseMgr.
LeaseMgrFactory::destroy();
LeaseMgrFactory::create(dbaccess);
- lmptr_ = &(LeaseMgrFactory::instance());
+ lmptr_ = static_cast<TrackingLeaseMgr*>(&(LeaseMgrFactory::instance()));
});
// Ok, let's check if the leases are really accessible.
// Recreate Memfile_LeaseMgr.
LeaseMgrFactory::destroy();
ASSERT_NO_THROW(LeaseMgrFactory::create(dbaccess));
- lmptr_ = &(LeaseMgrFactory::instance());
+ lmptr_ = static_cast<TrackingLeaseMgr*>(&(LeaseMgrFactory::instance()));
// We will store addresses here, so it will be easier to randomly
// pick a lease.
// Recreate Memfile_LeaseMgr.
LeaseMgrFactory::destroy();
LeaseMgrFactory::create(dbaccess);
- lmptr_ = &(LeaseMgrFactory::instance());
+ lmptr_ = static_cast<TrackingLeaseMgr*>(&(LeaseMgrFactory::instance()));
});
// Ok, let's check if the leases are really accessible.
EXPECT_EQ(exp_remote_id, ex_info->id_);
}
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is added.
+TEST_F(MemfileLeaseMgrTest, trackAddLease4) {
+ startBackend(V4);
+ // Expect that lease is not locked and the MT-safe context.
+ testTrackAddLease4(false, true);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is added.
+TEST_F(MemfileLeaseMgrTest, trackAddLease4MultiThreading) {
+ startBackend(V4);
+ MultiThreadingMgr::instance().setMode(true);
+ // Expect that lease is not locked and the MT-safe context.
+ testTrackAddLease4(false, true);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is added.
+TEST_F(MemfileLeaseMgrTest, trackAddLease6) {
+ startBackend(V6);
+ // Expect that lease is not locked and the MT-safe context.
+ testTrackAddLease6(false, true);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is added.
+TEST_F(MemfileLeaseMgrTest, trackAddLease6MultiThreading) {
+ startBackend(V6);
+ MultiThreadingMgr::instance().setMode(true);
+ // Expect that lease is not locked and the MT-safe context.
+ testTrackAddLease6(false, true);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is added.
+TEST_F(MemfileLeaseMgrTest, trackUpdateLease4) {
+ startBackend(V4);
+ // Expect that lease is not locked and the MT-safe context.
+ testTrackUpdateLease4(false, true);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is added.
+TEST_F(MemfileLeaseMgrTest, trackUpdateLease4MultiThreading) {
+ startBackend(V4);
+ MultiThreadingMgr::instance().setMode(true);
+ // Expect that lease is not locked and the MT-safe context.
+ testTrackUpdateLease4(false, true);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is added.
+TEST_F(MemfileLeaseMgrTest, trackUpdateLease6) {
+ startBackend(V6);
+ // Expect that lease is not locked and the MT-safe context.
+ testTrackUpdateLease6(false, true);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is added.
+TEST_F(MemfileLeaseMgrTest, trackUpdateLease6MultiThreading) {
+ startBackend(V6);
+ MultiThreadingMgr::instance().setMode(true);
+ // Expect that lease is not locked and the MT-safe context.
+ testTrackUpdateLease6(false, true);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is added.
+TEST_F(MemfileLeaseMgrTest, trackDeleteLease4) {
+ startBackend(V4);
+ // Expect that lease is not locked and the MT-safe context.
+ testTrackDeleteLease4(false, true);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is added.
+TEST_F(MemfileLeaseMgrTest, trackDeleteLease4MultiThreading) {
+ startBackend(V4);
+ MultiThreadingMgr::instance().setMode(true);
+ // Expect that lease is not locked and the MT-safe context.
+ testTrackDeleteLease4(false, true);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is added.
+TEST_F(MemfileLeaseMgrTest, trackDeleteLease6) {
+ startBackend(V6);
+ // Expect that lease is not locked and the MT-safe context.
+ testTrackDeleteLease6(false, true);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is added.
+TEST_F(MemfileLeaseMgrTest, trackDeleteLease6MultiThreading) {
+ startBackend(V6);
+ MultiThreadingMgr::instance().setMode(true);
+ // Expect that lease is not locked and the MT-safe context.
+ testTrackDeleteLease6(false, true);
+}
+
+
} // namespace
throw;
}
- lmptr_ = &(LeaseMgrFactory::instance());
+ lmptr_ = static_cast<TrackingLeaseMgr*>(&(LeaseMgrFactory::instance()));
MultiThreadingMgr::instance().setMode(false);
}
void reopen(Universe) {
LeaseMgrFactory::destroy();
LeaseMgrFactory::create(validMySQLConnectionString());
- lmptr_ = &(LeaseMgrFactory::instance());
+ lmptr_ = static_cast<TrackingLeaseMgr*>(&(LeaseMgrFactory::instance()));
}
};
testLeaseLimits6();
}
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is added.
+TEST_F(MySqlLeaseMgrTest, trackAddLease4) {
+ // It is unnecessary to lock the lease in ST. The backend does not
+ // provide the MT-safe context for the callbacks.
+ testTrackAddLease4(false, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is added.
+TEST_F(MySqlLeaseMgrTest, trackAddLease4MultiThreading) {
+ MultiThreadingMgr::instance().setMode(true);
+ // The lease should be locked in the MT mode. The backend does not
+ // provide an MT-safe context.
+ testTrackAddLease4(true, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is added.
+TEST_F(MySqlLeaseMgrTest, trackAddLease6) {
+ // It is unnecessary to lock the lease in ST. The backend does not
+ // provide the MT-safe context for the callbacks.
+ testTrackAddLease6(false, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is added.
+TEST_F(MySqlLeaseMgrTest, trackAddLease6MultiThreading) {
+ MultiThreadingMgr::instance().setMode(true);
+ // The lease should be locked in the MT mode. The backend does not
+ // provide an MT-safe context.
+ testTrackAddLease6(true, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is updated.
+TEST_F(MySqlLeaseMgrTest, trackUpdateLease4) {
+ // It is unnecessary to lock the lease in ST. The backend does not
+ // provide the MT-safe context for the callbacks.
+ testTrackUpdateLease4(false, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is updated.
+TEST_F(MySqlLeaseMgrTest, trackUpdateLease4MultiThreading) {
+ MultiThreadingMgr::instance().setMode(true);
+ // The lease should be locked in the MT mode. The backend does not
+ // provide an MT-safe context.
+ testTrackUpdateLease4(true, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is updated.
+TEST_F(MySqlLeaseMgrTest, trackUpdateLease6) {
+ // It is unnecessary to lock the lease in ST. The backend does not
+ // provide the MT-safe context for the callbacks.
+ testTrackUpdateLease6(false, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is updated.
+TEST_F(MySqlLeaseMgrTest, trackUpdateLease6MultiThreading) {
+ MultiThreadingMgr::instance().setMode(true);
+ // The lease should be locked in the MT mode. The backend does not
+ // provide an MT-safe context.
+ testTrackUpdateLease6(true, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is deleted.
+TEST_F(MySqlLeaseMgrTest, trackDeleteLease4) {
+ // It is unnecessary to lock the lease in ST. The backend does not
+ // provide the MT-safe context for the callbacks.
+ testTrackDeleteLease4(false, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is deleted.
+TEST_F(MySqlLeaseMgrTest, trackDeleteLease4MultiThreading) {
+ MultiThreadingMgr::instance().setMode(true);
+ // The lease should be locked in the MT mode. The backend does not
+ // provide an MT-safe context.
+ testTrackDeleteLease4(true, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is deleted.
+TEST_F(MySqlLeaseMgrTest, trackDeleteLease6) {
+ // It is unnecessary to lock the lease in ST. The backend does not
+ // provide the MT-safe context for the callbacks.
+ testTrackDeleteLease6(false, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is deleted.
+TEST_F(MySqlLeaseMgrTest, trackDeleteLease6MultiThreading) {
+ MultiThreadingMgr::instance().setMode(true);
+ // The lease should be locked in the MT mode. The backend does not
+ // provide an MT-safe context.
+ testTrackDeleteLease6(true, false);
+}
+
} // namespace
throw;
}
- lmptr_ = &(LeaseMgrFactory::instance());
+ lmptr_ = static_cast<TrackingLeaseMgr*>(&(LeaseMgrFactory::instance()));
MultiThreadingMgr::instance().setMode(false);
}
void reopen(Universe) {
LeaseMgrFactory::destroy();
LeaseMgrFactory::create(validPgSQLConnectionString());
- lmptr_ = &(LeaseMgrFactory::instance());
+ lmptr_ = static_cast<TrackingLeaseMgr*>(&(LeaseMgrFactory::instance()));
}
};
testLeaseLimits6();
}
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is added.
+TEST_F(PgSqlLeaseMgrTest, trackAddLease4) {
+ // It is unnecessary to lock the lease in ST. The backend does not
+ // provide the MT-safe context for the callbacks.
+ testTrackAddLease4(false, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is added.
+TEST_F(PgSqlLeaseMgrTest, trackAddLease4MultiThreading) {
+ MultiThreadingMgr::instance().setMode(true);
+ // The lease should be locked in the MT mode. The backend does not
+ // provide an MT-safe context.
+ testTrackAddLease4(true, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is added.
+TEST_F(PgSqlLeaseMgrTest, trackAddLease6) {
+ // It is unnecessary to lock the lease in ST. The backend does not
+ // provide the MT-safe context for the callbacks.
+ testTrackAddLease6(false, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is added.
+TEST_F(PgSqlLeaseMgrTest, trackAddLease6MultiThreading) {
+ MultiThreadingMgr::instance().setMode(true);
+ // The lease should be locked in the MT mode. The backend does not
+ // provide an MT-safe context.
+ testTrackAddLease6(true, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is updated.
+TEST_F(PgSqlLeaseMgrTest, trackUpdateLease4) {
+ // It is unnecessary to lock the lease in ST. The backend does not
+ // provide the MT-safe context for the callbacks.
+ testTrackUpdateLease4(false, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is updated.
+TEST_F(PgSqlLeaseMgrTest, trackUpdateLease4MultiThreading) {
+ MultiThreadingMgr::instance().setMode(true);
+ // The lease should be locked in the MT mode. The backend does not
+ // provide an MT-safe context.
+ testTrackUpdateLease4(true, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is updated.
+TEST_F(PgSqlLeaseMgrTest, trackUpdateLease6) {
+ // It is unnecessary to lock the lease in ST. The backend does not
+ // provide the MT-safe context for the callbacks.
+ testTrackUpdateLease6(false, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is updated.
+TEST_F(PgSqlLeaseMgrTest, trackUpdateLease6MultiThreading) {
+ MultiThreadingMgr::instance().setMode(true);
+ // The lease should be locked in the MT mode. The backend does not
+ // provide an MT-safe context.
+ testTrackUpdateLease6(true, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is deleted.
+TEST_F(PgSqlLeaseMgrTest, trackDeleteLease4) {
+ // It is unnecessary to lock the lease in ST. The backend does not
+ // provide the MT-safe context for the callbacks.
+ testTrackDeleteLease4(false, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv4 lease is deleted.
+TEST_F(PgSqlLeaseMgrTest, trackDeleteLease4MultiThreading) {
+ MultiThreadingMgr::instance().setMode(true);
+ // The lease should be locked in the MT mode. The backend does not
+ // provide an MT-safe context.
+ testTrackDeleteLease4(true, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is deleted.
+TEST_F(PgSqlLeaseMgrTest, trackDeleteLease6) {
+ // It is unnecessary to lock the lease in ST. The backend does not
+ // provide the MT-safe context for the callbacks.
+ testTrackDeleteLease6(false, false);
+}
+
+/// @brief Checks if the backends call the callbacks when an
+/// IPv6 lease is deleted.
+TEST_F(PgSqlLeaseMgrTest, trackDeleteLease6MultiThreading) {
+ MultiThreadingMgr::instance().setMode(true);
+ // The lease should be locked in the MT mode. The backend does not
+ // provide an MT-safe context.
+ testTrackDeleteLease6(true, false);
+}
+
} // namespace
// An attempt to lock an already locked lease should fail.
EXPECT_TRUE(mgr.tryLock(initializeLease<Lease4>(1, "192.0.2.1")));
EXPECT_FALSE(mgr.tryLock(initializeLease<Lease4>(1, "192.0.2.1")));
+ EXPECT_TRUE(mgr.isLocked(initializeLease<Lease4>(1, "192.0.2.1")));
// We can lock another lease but we cannot lock an already locked one.
EXPECT_TRUE(mgr.tryLock(initializeLease<Lease4>(1, "192.0.2.2")));
EXPECT_FALSE(mgr.tryLock(initializeLease<Lease4>(1, "192.0.2.1")));
EXPECT_FALSE(mgr.tryLock(initializeLease<Lease4>(2, "192.0.2.2")));
+ EXPECT_TRUE(mgr.isLocked(initializeLease<Lease4>(1, "192.0.1.2")));
+ EXPECT_TRUE(mgr.isLocked(initializeLease<Lease4>(2, "192.0.2.2")));
// If we unlock the lease, it can be locked again. However, unlocking
// the lease should not affect other locks.
mgr.unlock(initializeLease<Lease4>(1, "192.0.2.1"));
+ EXPECT_FALSE(mgr.isLocked(initializeLease<Lease4>(1, "192.0.2.1")));
+ EXPECT_TRUE(mgr.isLocked(initializeLease<Lease4>(2, "192.0.2.2")));
EXPECT_FALSE(mgr.tryLock(initializeLease<Lease4>(2, "192.0.2.2")));
EXPECT_TRUE(mgr.tryLock(initializeLease<Lease4>(1, "192.0.2.1")));
}
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
0,
- _1)));
+ _1, _2)));
// Callback for lease add and subnet id 1.
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_ADD_LEASE, 1, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
1,
- _1)));
+ _1, _2)));
// Callback for lease add and subnet id 2.
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_ADD_LEASE, 2, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
1,
- _1)));
+ _1, _2)));
// Callback for lease update and subnet id 0.
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_UPDATE_LEASE, 0, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_UPDATE_LEASE,
0,
- _1)));
+ _1, _2)));
// Callback for lease delete and subnet id 0.
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_DELETE_LEASE, 0, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_DELETE_LEASE,
1,
- _1)));
+ _1, _2)));
// This call should trigger the lease add callbacks for subnet id 0 and 1.
EXPECT_NO_THROW(mgr.trackAddLease(initializeLease<Lease4>(1, "192.0.2.1"), false));
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
0,
- _1)));
+ _1, _2)));
// Another attempt should fail.
EXPECT_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_ADD_LEASE, 0, "flq",
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
0,
- _1)),
+ _1, _2)),
InvalidOperation);
// It should succeed for a different owner.
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
0,
- _1)));
+ _1, _2)));
// It should also succeed for a different subnet id.
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_ADD_LEASE, 5, "qlf",
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
5,
- _1)));
+ _1, _2)));
// But, another attempt for the subnet id should fail.
EXPECT_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_ADD_LEASE, 5, "qlf",
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
5,
- _1)),
+ _1, _2)),
InvalidOperation);
}
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
0,
- _1)));
+ _1, _2)));
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_UPDATE_LEASE, 0, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_UPDATE_LEASE,
0,
- _1)));
+ _1, _2)));
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_DELETE_LEASE, 0, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_DELETE_LEASE,
0,
- _1)));
+ _1, _2)));
EXPECT_NO_THROW(mgr.trackUpdateLease(initializeLease<Lease4>(1, "192.0.2.1"), false));
EXPECT_EQ(1, logs_.size());
EXPECT_EQ(1, countLogs(TrackingLeaseMgr::TRACK_UPDATE_LEASE, 0));
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
0,
- _1)));
+ _1, _2)));
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_UPDATE_LEASE, 0, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_UPDATE_LEASE,
0,
- _1)));
+ _1, _2)));
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_DELETE_LEASE, 0, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_DELETE_LEASE,
0,
- _1)));
+ _1, _2)));
EXPECT_NO_THROW(mgr.trackDeleteLease(initializeLease<Lease4>(1, "192.0.2.1"), false));
EXPECT_EQ(1, logs_.size());
EXPECT_EQ(1, countLogs(TrackingLeaseMgr::TRACK_DELETE_LEASE, 0));
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
0,
- _1)));
+ _1, _2)));
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_ADD_LEASE, 1, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
1,
- _1)));
+ _1, _2)));
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_ADD_LEASE, 2, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
2,
- _1)));
+ _1, _2)));
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_UPDATE_LEASE, 1, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_UPDATE_LEASE,
1,
- _1)));
+ _1, _2)));
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_UPDATE_LEASE, 2, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_UPDATE_LEASE,
2,
- _1)));
+ _1, _2)));
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_DELETE_LEASE, 1, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_DELETE_LEASE,
1,
- _1)));
+ _1, _2)));
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_DELETE_LEASE, 2, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_DELETE_LEASE,
2,
- _1)));
+ _1, _2)));
// Unregister the callbacks for subnet id 1.
EXPECT_NO_THROW(mgr.unregisterCallbacks(SubnetID(1)));
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
0,
- _1)));
+ _1, _2)));
EXPECT_NO_THROW(mgr.registerCallback(TrackingLeaseMgr::TRACK_UPDATE_LEASE, 0, "flq",
std::bind(&TrackingLeaseMgrTest::logCallback,
this,
TrackingLeaseMgr::TRACK_UPDATE_LEASE,
0,
- _1)));
+ _1, _2)));
// Make sure they have been registered.
EXPECT_TRUE(mgr.hasCallbacks());
this,
TrackingLeaseMgr::TRACK_ADD_LEASE,
0,
- _1)));
+ _1, _2)));
EXPECT_TRUE(mgr.hasCallbacks());
}
locked_leases_.erase(lease->addr_);
}
+bool
+TrackingLeaseMgr::isLocked(const LeasePtr& lease) {
+ return (locked_leases_.find(lease->addr_) != locked_leases_.end());
+}
+
void
TrackingLeaseMgr::trackAddLease(const LeasePtr& lease, bool mt_safe) {
runCallbacks(TRACK_ADD_LEASE, lease, mt_safe);
/// @param lease a lease instance for which unlocking should be attempted.
void unlock(const LeasePtr& lease);
+public:
+
+ /// @brief Checks if the lease is locked.
+ ///
+ /// This function is useful in the unit tests.
+ ///
+ /// @return true if the lease is locked, false otherwise.
+ bool isLocked(const LeasePtr& lease);
+
+protected:
+
/// @brief Invokes the callbacks when a new lease is added.
///
/// It executes all callbacks of the @c TRACK_ADD_LEASE type for a subnet id of 0