diff --git a/src/bgp/routing-instance/routepath_replicator.cc b/src/bgp/routing-instance/routepath_replicator.cc index e7793ed6a71..7412a849228 100644 --- a/src/bgp/routing-instance/routepath_replicator.cc +++ b/src/bgp/routing-instance/routepath_replicator.cc @@ -55,28 +55,67 @@ do { \ Rpr##obj::TraceMsg(trace_buf_, __FILE__, __LINE__, __VA_ARGS__); \ } while (false) +class TableState::DeleteActor : public LifetimeActor { +public: + DeleteActor(TableState *ts) + : LifetimeActor(ts->replicator()->server()->lifetime_manager()), + ts_(ts) { + } + + virtual bool MayDelete() const { + return ts_->MayDelete(); + } + + virtual void Shutdown() { + } + + virtual void Destroy() { + ts_->replicator()->DeleteTableState(ts_->table()); + } + +private: + TableState *ts_; +}; + TableState::TableState(RoutePathReplicator *replicator, BgpTable *table) : replicator_(replicator), table_(table), listener_id_(DBTableBase::kInvalidId), - deleted_(false), + deleter_(new DeleteActor(this)), table_delete_ref_(this, table->deleter()) { assert(table->deleter() != NULL); - route_count_ = 0; } TableState::~TableState() { } void TableState::ManagedDelete() { - if (!table_->IsVpnTable()) - return; - deleted_ = true; - replicator_->DeleteVpnTableState(); + deleter()->Delete(); +} + +bool TableState::deleted() const { + return deleter()->IsDeleted(); +} + +LifetimeActor *TableState::deleter() { + return deleter_.get(); +} + +const LifetimeActor *TableState::deleter() const { + return deleter_.get(); } bool TableState::MayDelete() const { - return (deleted_ && list_.empty() && route_count_ == 0); + if (list_.empty() && !route_count() && + !replicator()->BulkSyncExists(table())) + return true; + return false; +} + +void TableState::RetryDelete() { + if (!deleter()->IsDeleted()) + return; + deleter()->RetryDelete(); } void TableState::AddGroup(RtGroup *group) { @@ -114,24 +153,18 @@ RoutePathReplicator::RoutePathReplicator(BgpServer *server, : server_(server), family_(family), vpn_table_(NULL), - vpn_ts_(NULL), walk_trigger_(new TaskTrigger( boost::bind(&RoutePathReplicator::StartWalk, this), TaskScheduler::GetInstance()->GetTaskId("bgp::Config"), 0)), - unreg_trigger_(new TaskTrigger( - boost::bind(&RoutePathReplicator::UnregisterTables, this), - TaskScheduler::GetInstance()->GetTaskId("bgp::Config"), 0)), trace_buf_(SandeshTraceBufferCreate("RoutePathReplicator", 500)) { } RoutePathReplicator::~RoutePathReplicator() { - assert(!vpn_ts_); assert(table_state_list_.empty()); } void RoutePathReplicator::Initialize() { assert(!vpn_table_); - assert(!vpn_ts_); RoutingInstanceMgr *mgr = server_->routing_instance_mgr(); assert(mgr); @@ -140,15 +173,7 @@ void RoutePathReplicator::Initialize() { assert(master); vpn_table_ = master->GetTable(family_); assert(vpn_table_); - vpn_ts_ = AddTableState(vpn_table_); - assert(vpn_ts_); -} - -void RoutePathReplicator::DeleteVpnTableState() { - if (!vpn_ts_ || !vpn_ts_->MayDelete()) - return; - unreg_table_list_.insert(vpn_table_); - unreg_trigger_->Set(); + assert(AddTableState(vpn_table_)); } TableState *RoutePathReplicator::AddTableState(BgpTable *table, @@ -184,35 +209,12 @@ void RoutePathReplicator::RemoveTableState(BgpTable *table, RtGroup *group) { void RoutePathReplicator::DeleteTableState(BgpTable *table) { TableState *ts = FindTableState(table); assert(ts); - if (!ts->empty()) - return; - RPR_TRACE(UnregTable, table->name()); table->Unregister(ts->listener_id()); table_state_list_.erase(table); - if (ts == vpn_ts_) - vpn_ts_ = NULL; delete ts; } -// -// Enqueue BgpTable to unregister list if the TableState is empty. -// -// Skip if the BgpTable is still on the bulk sync list. This can -// happen if we're called from the bgp::Config Task and a previous -// walk is still in progress. -// -void RoutePathReplicator::UnregisterTableState(BgpTable *table) { - CHECK_CONCURRENCY("bgp::Config", "db::DBTable"); - const TableState *ts = FindTableState(table); - if (!ts->empty()) - return; - if (bulk_sync_.find(table) != bulk_sync_.end()) - return; - unreg_table_list_.insert(table); - unreg_trigger_->Set(); -} - TableState *RoutePathReplicator::FindTableState(BgpTable *table) { TableStateList::iterator loc = table_state_list_.find(table); return (loc != table_state_list_.end() ? loc->second : NULL); @@ -260,7 +262,7 @@ RoutePathReplicator::StartWalk() { } BgpTable *table = it->first; RPR_TRACE(Walk, table->name()); - const TableState *ts = FindTableState(table); + TableState *ts = FindTableState(table); assert(ts); DB *db = server()->database(); DBTableWalker::WalkId id = db->GetWalker()->WalkTable(table, NULL, @@ -272,16 +274,6 @@ RoutePathReplicator::StartWalk() { return true; } -bool RoutePathReplicator::UnregisterTables() { - CHECK_CONCURRENCY("bgp::Config"); - for (UnregTableList::iterator it = unreg_table_list_.begin(); - it != unreg_table_list_.end(); ++it) { - DeleteTableState(*it); - } - unreg_table_list_.clear(); - return true; -} - void RoutePathReplicator::BulkReplicationDone(DBTableBase *dbtable) { CHECK_CONCURRENCY("db::DBTable"); @@ -298,12 +290,14 @@ RoutePathReplicator::BulkReplicationDone(DBTableBase *dbtable) { } delete bulk_sync_state; bulk_sync_.erase(loc); - UnregisterTableState(table); + TableState *ts = FindTableState(table); + ts->RetryDelete(); } void RoutePathReplicator::JoinVpnTable(RtGroup *group) { CHECK_CONCURRENCY("bgp::Config"); - if (!vpn_ts_ || vpn_ts_->FindGroup(group)) + TableState *vpn_ts = FindTableState(vpn_table_); + if (!vpn_ts || vpn_ts->FindGroup(group)) return; RPR_TRACE(TableJoin, vpn_table_->name(), group->rt().ToString(), true); group->AddImportTable(family(), vpn_table_); @@ -314,14 +308,14 @@ void RoutePathReplicator::JoinVpnTable(RtGroup *group) { void RoutePathReplicator::LeaveVpnTable(RtGroup *group) { CHECK_CONCURRENCY("bgp::Config"); - if (!vpn_ts_) + TableState *vpn_ts = FindTableState(vpn_table_); + if (!vpn_ts) return; RPR_TRACE(TableLeave, vpn_table_->name(), group->rt().ToString(), true); group->RemoveImportTable(family(), vpn_table_); RPR_TRACE(TableLeave, vpn_table_->name(), group->rt().ToString(), false); group->RemoveExportTable(family(), vpn_table_); RemoveTableState(vpn_table_, group); - DeleteVpnTableState(); } // @@ -392,8 +386,6 @@ void RoutePathReplicator::Leave(BgpTable *table, const RouteTarget &rt, if (!table->empty()) { RequestWalk(table); walk_trigger_->Set(); - } else { - UnregisterTableState(table); } } @@ -404,7 +396,7 @@ void RoutePathReplicator::Leave(BgpTable *table, const RouteTarget &rt, } } -void RoutePathReplicator::DBStateSync(BgpTable *table, const TableState *ts, +void RoutePathReplicator::DBStateSync(BgpTable *table, TableState *ts, BgpRoute *rt, RtReplicated *dbstate, const RtReplicated::ReplicatedRtPathList *future) { set_synchronize(dbstate->GetMutableList(), future, @@ -414,9 +406,8 @@ void RoutePathReplicator::DBStateSync(BgpTable *table, const TableState *ts, if (dbstate->GetList().empty()) { rt->ClearState(table, ts->listener_id()); delete dbstate; - uint32_t prev_route_count = ts->decrement_route_count(); - if (prev_route_count == 1 && ts == vpn_ts_) - DeleteVpnTableState(); + if (table->GetDBStateCount(ts->listener_id()) == 0) + ts->RetryDelete(); } } @@ -480,7 +471,7 @@ static ExtCommunityPtr UpdateExtCommunity(BgpServer *server, // If primary table is a VRF table attach it's export targets to replicated // path in the VPN table. // -bool RoutePathReplicator::RouteListener(const TableState *ts, +bool RoutePathReplicator::RouteListener(TableState *ts, DBTablePartBase *root, DBEntryBase *entry) { CHECK_CONCURRENCY("db::DBTable"); @@ -511,7 +502,6 @@ bool RoutePathReplicator::RouteListener(const TableState *ts, if (dbstate == NULL) { dbstate = new RtReplicated(this); rt->SetState(table, id, dbstate); - ts->increment_route_count(); } // Get the export route target list from the routing instance. diff --git a/src/bgp/routing-instance/routepath_replicator.h b/src/bgp/routing-instance/routepath_replicator.h index 8438749f8f9..f53101f79d0 100644 --- a/src/bgp/routing-instance/routepath_replicator.h +++ b/src/bgp/routing-instance/routepath_replicator.h @@ -32,19 +32,17 @@ class TaskTrigger; // An instance is created for each VRF table and for the VPN table. // // An entry for VPN table is created when RoutePathReplicator is initialized. -// This entry gets deleted when the RoutePathReplicator is terminating. // An entry for a VRF table is created when processing a Join for the first -// export route target for the VRF. It is removed when a Leave for the last -// export route target for the VRF is processed. +// export route target for the VRF. +// TableState is removed when a Bgp table is deleted. +// TableState takes a delete reference to the BgpTable and DeleteActor of the +// TableState manages the unregister of this listener from the BgpTable and +// delete of TableState object. // // A TableState entry keeps track of all the export route targets for a VRF // by maintaining the GroupList. TableState cannot be deleted if GroupList -// is non-empty. -// -// The route_count_ keeps track of the number of routes exported to secondary -// tables. TableState can be deleted only if the count is 0. This mechanism -// is required for VPN table since we do not walk the entire VPN table when -// doing Leave processing for VRF import targets. +// is non-empty and table has replicated routes. Replicated routes are tracked +// using the DBStateCount of this listener // class TableState { public: @@ -56,31 +54,48 @@ class TableState { void ManagedDelete(); bool MayDelete() const; + LifetimeActor *deleter(); + const LifetimeActor *deleter() const; + bool deleted() const; + void AddGroup(RtGroup *group); void RemoveGroup(RtGroup *group); const RtGroup *FindGroup(RtGroup *group) const; bool empty() const { return list_.empty(); } - bool deleted() const { return deleted_; } - uint32_t route_count() const { return route_count_; } - uint32_t increment_route_count() const { - return route_count_.fetch_and_increment(); - } - uint32_t decrement_route_count() const { - return route_count_.fetch_and_decrement(); - } DBTableBase::ListenerId listener_id() const { return listener_id_; } void set_listener_id(DBTableBase::ListenerId listener_id) { assert(listener_id_ == DBTableBase::kInvalidId); listener_id_ = listener_id; } + uint32_t route_count() const { + return table_->GetDBStateCount(listener_id()); + } + + RoutePathReplicator *replicator() { + return replicator_; + } + + const RoutePathReplicator *replicator() const { + return replicator_; + } + + BgpTable *table() const { + return table_; + } + + BgpTable *table() { + return table_; + } + + void RetryDelete(); private: + class DeleteActor; RoutePathReplicator *replicator_; BgpTable *table_; DBTableBase::ListenerId listener_id_; - mutable tbb::atomic route_count_; - bool deleted_; + boost::scoped_ptr deleter_; LifetimeRef table_delete_ref_; GroupList list_; @@ -259,6 +274,7 @@ class RoutePathReplicator { private: friend class ReplicationTest; friend class RtReplicated; + friend class TableState; typedef std::map TableStateList; typedef std::map BulkSyncOrders; @@ -279,34 +295,30 @@ class RoutePathReplicator { void JoinVpnTable(RtGroup *group); void LeaveVpnTable(RtGroup *group); - bool RouteListener(const TableState *ts, DBTablePartBase *root, + bool RouteListener(TableState *ts, DBTablePartBase *root, DBEntryBase *entry); void DeleteSecondaryPath(BgpTable *table, BgpRoute *rt, const RtReplicated::SecondaryRouteInfo &rtinfo); - void DBStateSync(BgpTable *table, const TableState *ts, BgpRoute *rt, + void DBStateSync(BgpTable *table, TableState *ts, BgpRoute *rt, RtReplicated *dbstate, const RtReplicated::ReplicatedRtPathList *future); - bool VpnTableStateExists() const { return (vpn_ts_ != NULL); } - uint32_t VpnTableStateRouteCount() const { - return (vpn_ts_ ? vpn_ts_->route_count() : 0); + bool BulkSyncExists(BgpTable *table) const { + return (bulk_sync_.find(table) != bulk_sync_.end()); } BgpServer *server() { return server_; } Address::Family family() const { return family_; } + const BgpServer *server() const { return server_; }; - // Mutex to protect unreg_table_list_ and bulk_sync_ from multiple - // DBTable tasks. + // Mutex to protect bulk_sync_ from multiple DBTable tasks. tbb::mutex mutex_; BulkSyncOrders bulk_sync_; - UnregTableList unreg_table_list_; TableStateList table_state_list_; BgpServer *server_; Address::Family family_; BgpTable *vpn_table_; - TableState *vpn_ts_; boost::scoped_ptr walk_trigger_; - boost::scoped_ptr unreg_trigger_; SandeshTraceBufferPtr trace_buf_; }; diff --git a/src/bgp/test/routepath_replicator_test.cc b/src/bgp/test/routepath_replicator_test.cc index 6e305fd9d92..55ceff054a8 100644 --- a/src/bgp/test/routepath_replicator_test.cc +++ b/src/bgp/test/routepath_replicator_test.cc @@ -144,8 +144,29 @@ class ReplicationTest : public ::testing::Test { parser->Receive(&config_db_, netconf.data(), netconf.length(), 0); } + + void DeleteRoutingInstance(const string &instance_name, const string &rt_name) { + ifmap_test_util::IFMapMsgUnlink(&config_db_, "routing-instance", instance_name, + "virtual-network", instance_name, "virtual-network-routing-instance"); + ifmap_test_util::IFMapMsgUnlink(&config_db_, "routing-instance", instance_name, + "route-target", rt_name, "instance-target"); + ifmap_test_util::IFMapMsgNodeDelete( + &config_db_, "virtual-network", instance_name); + ifmap_test_util::IFMapMsgNodeDelete( + &config_db_, "routing-instance", instance_name); + ifmap_test_util::IFMapMsgNodeDelete( + &config_db_, "route-target", rt_name); + task_util::WaitForIdle(); + } + + void VerifyTableNoExists(const string &table_name) { + TASK_UTIL_EXPECT_TRUE( + bgp_server_->database()->FindTable(table_name) == NULL); + } + void AddInetRoute(IPeer *peer, const string &instance_name, - const string &prefix, int localpref, string rd = "") { + const string &prefix, int localpref, string rd = "", + const vector &rtarget_list = vector()) { boost::system::error_code error; Ip4Prefix nlri = Ip4Prefix::FromString(prefix, &error); EXPECT_FALSE(error); @@ -160,6 +181,15 @@ class ReplicationTest : public ::testing::Test { if (!rd.empty()) { attr_spec.push_back(&rd_spec); } + ExtCommunitySpec spec; + if (!rtarget_list.empty()) { + BOOST_FOREACH(string tgt, rtarget_list) { + RouteTarget rt(RouteTarget::FromString(tgt)); + spec.communities.push_back(get_value(rt.GetExtCommunity().begin(), 8)); + } + attr_spec.push_back(&spec); + } + BgpAttrPtr attr = bgp_server_->attr_db()->Locate(attr_spec); request.data.reset(new BgpTable::RequestData(attr, 0, 0)); BgpTable *table = static_cast( @@ -453,16 +483,46 @@ class ReplicationTest : public ::testing::Test { bgp_server_->rtarget_group_mgr()->EnableRouteTargetProcessing(); } - void VerifyVpnTableStateExists(bool exists) { + const TableState *LookupVpnTableState() { RoutePathReplicator *replicator = bgp_server_->replicator(Address::INETVPN); - TASK_UTIL_EXPECT_TRUE(replicator->VpnTableStateExists() == exists); + RoutingInstanceMgr *mgr = bgp_server_->routing_instance_mgr(); + RoutingInstance *master = + mgr->GetRoutingInstance(BgpConfigManager::kMasterInstance); + assert(master); + BgpTable *vpn_table = master->GetTable(Address::INETVPN); + const TableState *vpn_ts = replicator->FindTableState(vpn_table); + return vpn_ts; } - void VerifyVpnTableStateRouteCount(uint32_t count) { + void VerifyVpnTableStateExists(bool exists) { + if (exists) + TASK_UTIL_EXPECT_TRUE(LookupVpnTableState() != NULL); + else + TASK_UTIL_EXPECT_TRUE(LookupVpnTableState() == NULL); + } + + const TableState *LookupTableState(const string &instance_name) { RoutePathReplicator *replicator = bgp_server_->replicator(Address::INETVPN); - TASK_UTIL_EXPECT_EQ(count, replicator->VpnTableStateRouteCount()); + + BgpTable *table = static_cast( + bgp_server_->database()->FindTable(instance_name + ".inet.0")); + return replicator->FindTableState(table); + } + + const TableState *VerifyVRFTableStateExists(const string &instance_name, bool exists) { + const TableState *ts = NULL; + if (exists) + TASK_UTIL_EXPECT_TRUE((ts = LookupTableState(instance_name)) != NULL); + else + TASK_UTIL_EXPECT_TRUE((ts = LookupTableState(instance_name)) == NULL); + return ts; + } + + void VerifyVpnTableStateRouteCount(uint32_t count) { + const TableState *vpn_ts = LookupVpnTableState(); + TASK_UTIL_EXPECT_EQ(count, (vpn_ts ? vpn_ts->route_count() : 0)); } EventManager evm_; @@ -2228,6 +2288,161 @@ TEST_F(ReplicationTest, VpnTableStateDelete3) { TASK_UTIL_EXPECT_TRUE(bgp_server_->destroyed()); } +// Test a case where routing instance is deleted with replicated route +// In this case the route is replicated due to rtarget of the route not due +// to export_rt of the VRF. Simulate the static route scenario +TEST_F(ReplicationTest, DeleteInstanceWithReplicatedRoute) { + vector instance_names = list_of("blue")("red"); + multimap connections; + NetworkConfig(instance_names, connections); + task_util::WaitForIdle(); + + boost::system::error_code ec; + peers_.push_back( + new BgpPeerMock(Ip4Address::from_string("192.168.0.1", ec))); + + // Add route to blue table with the Rtarget that red imports. + AddInetRoute(peers_[0], "blue", "10.0.1.1/32", 100, "192.168.0.1:1", + list_of("target:64496:2")); + task_util::WaitForIdle(); + + DeleteRoutingInstance("blue", "target:64496:1"); + task_util::WaitForIdle(); + + DeleteInetRoute(peers_[0], "blue", "10.0.1.1/32"); + task_util::WaitForIdle(); + + DeleteRoutingInstance("red", "target:64496:2"); + task_util::WaitForIdle(); + + // Make sure that the blue inet table is gone. + VerifyTableNoExists("blue.inet.0"); + VerifyTableNoExists("red.inet.0"); +} + +// Test a case where routing instance is deleted with replicated route +// In this case the route is replicated due to rtarget of the route not due +// to export_rt of the VRF. Delete the routing instance that importing the route +TEST_F(ReplicationTest, DeleteInstanceWithReplicatedRoute_1) { + vector instance_names = list_of("blue")("red"); + multimap connections; + NetworkConfig(instance_names, connections); + task_util::WaitForIdle(); + + boost::system::error_code ec; + peers_.push_back( + new BgpPeerMock(Ip4Address::from_string("192.168.0.1", ec))); + + // Add route to blue table with the Rtarget that red imports. + AddInetRoute(peers_[0], "blue", "10.0.1.1/32", 100, "192.168.0.1:1", + list_of("target:64496:2")); + task_util::WaitForIdle(); + + DeleteRoutingInstance("red", "target:64496:2"); + task_util::WaitForIdle(); + + DeleteInetRoute(peers_[0], "blue", "10.0.1.1/32"); + task_util::WaitForIdle(); + + DeleteRoutingInstance("blue", "target:64496:1"); + task_util::WaitForIdle(); + + // Make sure that the blue inet table is gone. + VerifyTableNoExists("blue.inet.0"); + VerifyTableNoExists("red.inet.0"); +} + +// Test a case where routing instance is deleted with replicated routes +// Same as DeleteInstanceWithReplicatedRoute with multiple routes in +// different partition +TEST_F(ReplicationTest, DeleteInstanceWithReplicatedRoute_2) { + vector instance_names = list_of("blue")("red"); + multimap connections; + NetworkConfig(instance_names, connections); + task_util::WaitForIdle(); + + boost::system::error_code ec; + peers_.push_back( + new BgpPeerMock(Ip4Address::from_string("192.168.0.1", ec))); + + // Add multiple routes to blue table with the Rtarget that red imports. + AddInetRoute(peers_[0], "blue", "10.0.1.1/32", 100, "192.168.0.1:1", + list_of("target:64496:2")); + AddInetRoute(peers_[0], "blue", "10.0.1.2/32", 100, "192.168.0.1:1", + list_of("target:64496:2")); + AddInetRoute(peers_[0], "blue", "10.0.1.3/32", 100, "192.168.0.1:1", + list_of("target:64496:2")); + task_util::WaitForIdle(); + + DeleteRoutingInstance("blue", "target:64496:1"); + task_util::WaitForIdle(); + + DeleteInetRoute(peers_[0], "blue", "10.0.1.1/32"); + DeleteInetRoute(peers_[0], "blue", "10.0.1.2/32"); + DeleteInetRoute(peers_[0], "blue", "10.0.1.3/32"); + task_util::WaitForIdle(); + + DeleteRoutingInstance("red", "target:64496:2"); + task_util::WaitForIdle(); + + // Make sure that the blue inet table is gone. + VerifyTableNoExists("blue.inet.0"); + VerifyTableNoExists("red.inet.0"); +} + +TEST_F(ReplicationTest, TableStateOnVRFWithNoImportExportRT) { + vector instance_names = list_of("blue")("red"); + multimap connections = map_list_of("blue", "red"); + NetworkConfig(instance_names, connections); + task_util::WaitForIdle(); + + boost::system::error_code ec; + peers_.push_back( + new BgpPeerMock(Ip4Address::from_string("192.168.0.1", ec))); + + // Add route to blue table + AddInetRoute(peers_[0], "blue", "10.0.1.1/32", 100); + task_util::WaitForIdle(); + + TASK_UTIL_EXPECT_TRUE(InetRouteLookup("red", "10.0.1.1/32") != NULL); + + const TableState *ts_blue = VerifyVRFTableStateExists("blue", true); + const TableState *ts_red = VerifyVRFTableStateExists("red", true); + TASK_UTIL_EXPECT_EQ(1, ts_blue->route_count()); + TASK_UTIL_EXPECT_EQ(0, ts_red->route_count()); + + RemoveInstanceRouteTarget("blue", "target:64496:1"); + RemoveInstanceRouteTarget("red", "target:64496:2"); + + TASK_UTIL_EXPECT_EQ(0, GetInstanceImportRouteTargetList("red").size()); + TASK_UTIL_EXPECT_EQ(0, GetInstanceImportRouteTargetList("blue").size()); + TASK_UTIL_EXPECT_EQ(0, GetInstanceExportRouteTargetList("red").size()); + TASK_UTIL_EXPECT_EQ(0, GetInstanceExportRouteTargetList("blue").size()); + + ts_blue = VerifyVRFTableStateExists("blue", true); + ts_red = VerifyVRFTableStateExists("red", true); + + TASK_UTIL_EXPECT_TRUE(ts_red->empty()); + TASK_UTIL_EXPECT_TRUE(ts_blue->empty()); + TASK_UTIL_EXPECT_EQ(0, ts_blue->route_count()); + TASK_UTIL_EXPECT_EQ(0, ts_red->route_count()); + + DeleteInetRoute(peers_[0], "blue", "10.0.1.1/32"); + task_util::WaitForIdle(); + + DeleteRoutingInstance("blue", "target:64496:1"); + DeleteRoutingInstance("red", "target:64496:2"); + ifmap_test_util::IFMapMsgUnlink(&config_db_, + "routing-instance", "blue", + "routing-instance", "red", + "connection"); + task_util::WaitForIdle(); + + VerifyVRFTableStateExists("blue", false); + VerifyVRFTableStateExists("red", false); +} + + class TestEnvironment : public ::testing::Environment { virtual ~TestEnvironment() { } }; diff --git a/src/db/db_table.cc b/src/db/db_table.cc index ee05b18b9f8..dcaf9b7d07c 100644 --- a/src/db/db_table.cc +++ b/src/db/db_table.cc @@ -122,6 +122,11 @@ class DBTableBase::ListenerInfo { } } + uint64_t GetDBStateCount(ListenerId listener) { + assert(db_state_accounting_ && listener != DBTableBase::kInvalidId); + return state_count_[listener]; + } + void FillListeners(vector *listeners) const { tbb::spin_rw_mutex::scoped_lock read_lock(rw_mutex_, false); ListenerId id = 0; @@ -203,6 +208,10 @@ void DBTableBase::AddToDBStateCount(ListenerId listener, int count) { info_->AddToDBStateCount(listener, count); } +uint64_t DBTableBase::GetDBStateCount(ListenerId listener) { + return info_->GetDBStateCount(listener); +} + bool DBTableBase::MayDelete() const { if (HasListeners()) { return false; diff --git a/src/db/db_table.h b/src/db/db_table.h index bcf5637a495..8149bea8f7f 100644 --- a/src/db/db_table.h +++ b/src/db/db_table.h @@ -86,6 +86,8 @@ class DBTableBase { // Manage db state count for a listener. void AddToDBStateCount(ListenerId listener, int count); + uint64_t GetDBStateCount(ListenerId listener); + // Calculate the size across all partitions. // Must be called from Task which is mutually exclusive with db::DBTable. virtual size_t Size() const { return 0; }