Skip to content

Commit

Permalink
Routepath Replicator should unregister from BgpTable only after all r…
Browse files Browse the repository at this point in the history
…eplicated routes are deleted

When the routing instance is deleted, route path replicator walks the route
table as part of Leave of rtgroup in import and export Route targets.
During the walk, it deletes all the replicated path/routes.
On walk complete, it unregisters from the DBTable. In the static route scenario,
the static route is added on the internal routing instance that has
"static-route-entries" property attached to it.
Such generated static route is replicated to destination VRF based on
"route-target-list" config in "static-route-entries.

Note: such internal routing instance doesn't have these route targets in its export_rt.

If this internal routing instance is deleted, route path replicator starts the
table walk as part of Leave of RtGroup in import and export. In case static
route module has not processed the config delete of static route entries,
it would not have deleted the static route added to inet route table.
On walk complete, it would unregister from the routing table.
Hence the replicated routes of Static route will never be deleted as
RouteReplicator module is no longer TableListener and doesn't process delete of
StaticRoute which happens when static route module process the delete request.

Proposed fix:
    1. Implement LifeTimeActor in TableState to manage the unregister of listener
and delete of tableState object. TableState object takes the delete reference
to the BgpTable and attempt to delete is only made if BgpTable is deleted.
TableState can be deleted if GroupList is empty & Replicated route count is zero
and table doesn't have pending tableWalks

    2. Implement GetDBStateCount() api in DBTableBase class to fetch DBState
count for a given listener

Added unit test code to Static route test and route replication test to
simulate the error condition and validate the fix

Change-Id: I9eb2b94aef9e112e29dea73fde5d38808e3b18b0
Closes-bug: #1482277
(cherry picked from commit c7a8736)
  • Loading branch information
bailkeri authored and Nischal Sheth committed Aug 14, 2015
1 parent b7bbed1 commit 04b0b74
Show file tree
Hide file tree
Showing 5 changed files with 330 additions and 102 deletions.
126 changes: 58 additions & 68 deletions src/bgp/routing-instance/routepath_replicator.cc
Expand Up @@ -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) {
Expand Down Expand Up @@ -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);
Expand All @@ -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,
Expand Down Expand Up @@ -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);
Expand Down Expand Up @@ -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,
Expand All @@ -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");
Expand All @@ -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_);
Expand All @@ -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();
}

//
Expand Down Expand Up @@ -392,8 +386,6 @@ void RoutePathReplicator::Leave(BgpTable *table, const RouteTarget &rt,
if (!table->empty()) {
RequestWalk(table);
walk_trigger_->Set();
} else {
UnregisterTableState(table);
}
}

Expand All @@ -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,
Expand All @@ -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();
}
}

Expand Down Expand Up @@ -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");

Expand Down Expand Up @@ -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.
Expand Down
70 changes: 41 additions & 29 deletions src/bgp/routing-instance/routepath_replicator.h
Expand Up @@ -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:
Expand All @@ -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<uint32_t> route_count_;
bool deleted_;
boost::scoped_ptr<DeleteActor> deleter_;
LifetimeRef<TableState> table_delete_ref_;
GroupList list_;

Expand Down Expand Up @@ -259,6 +274,7 @@ class RoutePathReplicator {
private:
friend class ReplicationTest;
friend class RtReplicated;
friend class TableState;

typedef std::map<BgpTable *, TableState *> TableStateList;
typedef std::map<BgpTable *, BulkSyncState *> BulkSyncOrders;
Expand All @@ -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<TaskTrigger> walk_trigger_;
boost::scoped_ptr<TaskTrigger> unreg_trigger_;
SandeshTraceBufferPtr trace_buf_;
};

Expand Down

0 comments on commit 04b0b74

Please sign in to comment.