summaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
-rw-r--r--gobgpd/main.go2
-rw-r--r--server/fsm.go13
-rw-r--r--server/fsm_test.go3
-rw-r--r--server/peer.go176
-rw-r--r--server/server.go447
-rw-r--r--table/adj.go16
-rw-r--r--table/destination.go98
-rw-r--r--table/destination_test.go43
-rw-r--r--table/message.go3
-rw-r--r--table/path.go40
-rw-r--r--table/policy.go152
-rw-r--r--table/policy_test.go86
-rw-r--r--table/table.go45
-rw-r--r--table/table_manager.go261
-rw-r--r--table/table_manager_test.go12
-rw-r--r--test/performance_test/main.go2
16 files changed, 677 insertions, 722 deletions
diff --git a/gobgpd/main.go b/gobgpd/main.go
index 9466e622..8d0dcf58 100644
--- a/gobgpd/main.go
+++ b/gobgpd/main.go
@@ -204,7 +204,7 @@ func main() {
// allocated before the above
// SetPolicy. But this should be
// handled more cleanly.
- if err := bgpServer.SetPolicy(newConfig.Policy); err != nil {
+ if err := bgpServer.SetRoutingPolicy(newConfig.Policy); err != nil {
log.Fatal(err)
}
} else {
diff --git a/server/fsm.go b/server/fsm.go
index 15d935bb..2d952065 100644
--- a/server/fsm.go
+++ b/server/fsm.go
@@ -86,7 +86,7 @@ type FSM struct {
rfMap map[bgp.RouteFamily]bool
confedCheck bool
peerInfo *table.PeerInfo
- peer *Peer
+ policy *table.RoutingPolicy
}
func (fsm *FSM) bgpMessageStateUpdate(MessageType uint8, isIn bool) {
@@ -138,7 +138,7 @@ func (fsm *FSM) bgpMessageStateUpdate(MessageType uint8, isIn bool) {
}
}
-func NewFSM(gConf *config.Global, pConf *config.Neighbor, peer *Peer) *FSM {
+func NewFSM(gConf *config.Global, pConf *config.Neighbor, policy *table.RoutingPolicy) *FSM {
adminState := ADMIN_STATE_UP
if pConf.NeighborState.AdminDown == true {
adminState = ADMIN_STATE_DOWN
@@ -155,7 +155,7 @@ func NewFSM(gConf *config.Global, pConf *config.Neighbor, peer *Peer) *FSM {
rfMap: make(map[bgp.RouteFamily]bool),
confedCheck: !config.IsConfederationMember(gConf, pConf) && config.IsEBGPPeer(gConf, pConf),
peerInfo: table.NewPeerInfo(gConf, pConf),
- peer: peer,
+ policy: policy,
}
fsm.t.Go(fsm.connectLoop)
return fsm
@@ -512,8 +512,13 @@ func (h *FSMHandler) recvMessageWithError() error {
// FIXME: we should use the original message for bmp/mrt
table.UpdatePathAttrs4ByteAs(body)
fmsg.PathList = table.ProcessMessage(m, h.fsm.peerInfo, fmsg.timestamp)
+ id := h.fsm.pConf.NeighborConfig.NeighborAddress.String()
policyMutex.RLock()
- h.fsm.peer.ApplyPolicy(table.POLICY_DIRECTION_IN, fmsg.PathList)
+ for _, path := range fmsg.PathList {
+ if h.fsm.policy.ApplyPolicy(id, table.POLICY_DIRECTION_IN, path) == nil {
+ path.Filter(id, table.POLICY_DIRECTION_IN)
+ }
+ }
policyMutex.RUnlock()
}
fmsg.payload = make([]byte, len(headerBuf)+len(bodyBuf))
diff --git a/server/fsm_test.go b/server/fsm_test.go
index aad9ff07..3b1bedd2 100644
--- a/server/fsm_test.go
+++ b/server/fsm_test.go
@@ -20,6 +20,7 @@ import (
log "github.com/Sirupsen/logrus"
"github.com/osrg/gobgp/config"
"github.com/osrg/gobgp/packet"
+ "github.com/osrg/gobgp/table"
"github.com/stretchr/testify/assert"
"net"
"strconv"
@@ -293,7 +294,7 @@ func makePeerAndHandler() (*Peer, *FSMHandler) {
capMap: make(map[bgp.BGPCapabilityCode][]bgp.ParameterCapabilityInterface),
}
- p.fsm = NewFSM(&gConf, &pConf, &Peer{})
+ p.fsm = NewFSM(&gConf, &pConf, table.NewRoutingPolicy())
incoming := make(chan *FsmMsg, 4096)
p.outgoing = make(chan *bgp.BGPMessage, 4096)
diff --git a/server/peer.go b/server/peer.go
index 8f43299e..1844236e 100644
--- a/server/peer.go
+++ b/server/peer.go
@@ -32,35 +32,41 @@ const (
)
type Peer struct {
- gConf config.Global
- conf config.Neighbor
- fsm *FSM
- rfMap map[bgp.RouteFamily]bool
- capMap map[bgp.BGPCapabilityCode][]bgp.ParameterCapabilityInterface
- adjRibIn *table.AdjRib
- adjRibOut *table.AdjRib
- outgoing chan *bgp.BGPMessage
- inPolicies []*table.Policy
- defaultInPolicy table.RouteType
- recvOpen *bgp.BGPMessage
- localRib *table.TableManager
+ tableId string
+ gConf config.Global
+ conf config.Neighbor
+ fsm *FSM
+ rfMap map[bgp.RouteFamily]bool
+ capMap map[bgp.BGPCapabilityCode][]bgp.ParameterCapabilityInterface
+ adjRibIn *table.AdjRib
+ adjRibOut *table.AdjRib
+ outgoing chan *bgp.BGPMessage
+ recvOpen *bgp.BGPMessage
+ policy *table.RoutingPolicy
+ localRib *table.TableManager
}
-func NewPeer(g config.Global, conf config.Neighbor, loc *table.TableManager) *Peer {
- rfs, _ := conf.AfiSafis.ToRfList()
+func NewPeer(g config.Global, conf config.Neighbor, loc *table.TableManager, policy *table.RoutingPolicy) *Peer {
peer := &Peer{
- gConf: g,
- conf: conf,
- rfMap: make(map[bgp.RouteFamily]bool),
- capMap: make(map[bgp.BGPCapabilityCode][]bgp.ParameterCapabilityInterface),
- outgoing: make(chan *bgp.BGPMessage, 128),
- adjRibIn: table.NewAdjRib(rfs),
- adjRibOut: table.NewAdjRib(rfs),
- localRib: loc,
+ gConf: g,
+ conf: conf,
+ rfMap: make(map[bgp.RouteFamily]bool),
+ capMap: make(map[bgp.BGPCapabilityCode][]bgp.ParameterCapabilityInterface),
+ outgoing: make(chan *bgp.BGPMessage, 128),
+ localRib: loc,
+ policy: policy,
+ }
+ tableId := table.GLOBAL_RIB_NAME
+ if peer.isRouteServerClient() {
+ tableId = conf.NeighborConfig.NeighborAddress.String()
}
+ peer.tableId = tableId
conf.NeighborState.SessionState = uint32(bgp.BGP_FSM_IDLE)
conf.Timers.TimersState.Downtime = time.Now().Unix()
- peer.fsm = NewFSM(&g, &conf, peer)
+ rfs, _ := conf.AfiSafis.ToRfList()
+ peer.adjRibIn = table.NewAdjRib(peer.ID(), rfs)
+ peer.adjRibOut = table.NewAdjRib(peer.ID(), rfs)
+ peer.fsm = NewFSM(&g, &conf, policy)
return peer
}
@@ -72,6 +78,14 @@ func (peer *Peer) Outgoing() chan *bgp.BGPMessage {
return peer.outgoing
}
+func (peer *Peer) ID() string {
+ return peer.conf.NeighborConfig.NeighborAddress.String()
+}
+
+func (peer *Peer) TableID() string {
+ return peer.tableId
+}
+
func (peer *Peer) isIBGPPeer() bool {
return peer.conf.NeighborConfig.PeerAs == peer.gConf.GlobalConfig.As
}
@@ -94,11 +108,18 @@ func (peer *Peer) getAccepted(rfList []bgp.RouteFamily) []*table.Path {
}
func (peer *Peer) getBestFromLocal(rfList []bgp.RouteFamily) ([]*table.Path, []*table.Path) {
- pathList, filtered := peer.ApplyPolicy(table.POLICY_DIRECTION_EXPORT, filterpath(peer, peer.localRib.GetBestPathList(rfList)))
- if peer.isRouteServerClient() == false {
- for _, path := range pathList {
- path.UpdatePathAttrs(&peer.gConf, &peer.conf)
+ pathList := []*table.Path{}
+ filtered := []*table.Path{}
+ for _, path := range peer.localRib.GetBestPathList(peer.TableID(), rfList) {
+ p := peer.policy.ApplyPolicy(peer.TableID(), table.POLICY_DIRECTION_EXPORT, filterpath(peer, path))
+ if p == nil {
+ filtered = append(filtered, path)
+ continue
+ }
+ if !peer.isRouteServerClient() {
+ p.UpdatePathAttrs(&peer.gConf, &peer.conf)
}
+ pathList = append(pathList, p)
}
return pathList, filtered
}
@@ -132,16 +153,13 @@ func open2Cap(open *bgp.BGPOpen, n *config.Neighbor) (map[bgp.BGPCapabilityCode]
return capMap, rfMap
}
-func (peer *Peer) handleBGPmessage(e *FsmMsg) ([]*table.Path, bool, []*bgp.BGPMessage) {
+func (peer *Peer) handleBGPmessage(e *FsmMsg) ([]*table.Path, []*bgp.BGPMessage) {
m := e.MsgData.(*bgp.BGPMessage)
- bgpMsgList := []*bgp.BGPMessage{}
- pathList := []*table.Path{}
log.WithFields(log.Fields{
"Topic": "Peer",
"Key": peer.conf.NeighborConfig.NeighborAddress,
"data": m,
}).Debug("received")
- update := false
switch m.Header.Type {
case bgp.BGP_MSG_OPEN:
@@ -178,11 +196,11 @@ func (peer *Peer) handleBGPmessage(e *FsmMsg) ([]*table.Path, bool, []*bgp.BGPMe
peer.adjRibOut.Drop(rfList)
accepted, filtered := peer.getBestFromLocal(rfList)
peer.adjRibOut.Update(accepted)
- pathList = append(pathList, accepted...)
for _, path := range filtered {
path.IsWithdraw = true
- pathList = append(pathList, path)
+ accepted = append(accepted, path)
}
+ return nil, table.CreateUpdateMsgFromPaths(accepted)
} else {
log.WithFields(log.Fields{
"Topic": "Peer",
@@ -191,11 +209,16 @@ func (peer *Peer) handleBGPmessage(e *FsmMsg) ([]*table.Path, bool, []*bgp.BGPMe
}
case bgp.BGP_MSG_UPDATE:
- update = true
peer.conf.Timers.TimersState.UpdateRecvTime = time.Now().Unix()
if len(e.PathList) > 0 {
peer.adjRibIn.Update(e.PathList)
- pathList = e.PathList
+ paths := make([]*table.Path, 0, len(e.PathList))
+ for _, path := range e.PathList {
+ if path.Filtered(peer.TableID()) != table.POLICY_DIRECTION_IN {
+ paths = append(paths, path)
+ }
+ }
+ return paths, nil
}
case bgp.BGP_MSG_NOTIFICATION:
body := m.Body.(*bgp.BGPNotification)
@@ -207,7 +230,7 @@ func (peer *Peer) handleBGPmessage(e *FsmMsg) ([]*table.Path, bool, []*bgp.BGPMe
"Data": body.Data,
}).Warn("received notification")
}
- return pathList, update, bgpMsgList
+ return nil, nil
}
func (peer *Peer) startFSMHandler(incoming chan *FsmMsg) {
@@ -351,87 +374,6 @@ func (peer *Peer) ToApiStruct() *api.Peer {
}
}
-func (peer *Peer) GetPolicy(d table.PolicyDirection) []*table.Policy {
- switch d {
- case table.POLICY_DIRECTION_IN:
- return peer.inPolicies
- default:
- return peer.localRib.GetPolicy(d)
- }
- return nil
-}
-
-func (peer *Peer) SetPolicy(d table.PolicyDirection, policies []*table.Policy) error {
- switch d {
- case table.POLICY_DIRECTION_IN:
- peer.inPolicies = policies
- default:
- return peer.localRib.SetPolicy(d, policies)
- }
- return nil
-}
-
-func (peer *Peer) GetDefaultPolicy(d table.PolicyDirection) table.RouteType {
- switch d {
- case table.POLICY_DIRECTION_IN:
- return peer.defaultInPolicy
- default:
- return peer.localRib.GetDefaultPolicy(d)
- }
- return table.ROUTE_TYPE_NONE
-}
-
-func (peer *Peer) SetDefaultPolicy(d table.PolicyDirection, typ table.RouteType) error {
- switch d {
- case table.POLICY_DIRECTION_IN:
- peer.defaultInPolicy = typ
- default:
- if peer.isRouteServerClient() {
- return peer.localRib.SetDefaultPolicy(d, typ)
- }
- }
- return nil
-}
-
-func (peer *Peer) ApplyPolicy(d table.PolicyDirection, paths []*table.Path) ([]*table.Path, []*table.Path) {
- newpaths := make([]*table.Path, 0, len(paths))
- filteredPaths := make([]*table.Path, 0)
- for _, path := range paths {
- result := table.ROUTE_TYPE_NONE
- newpath := path
- for _, p := range peer.GetPolicy(d) {
- result, newpath = p.Apply(path)
- if result != table.ROUTE_TYPE_NONE {
- break
- }
- }
-
- if result == table.ROUTE_TYPE_NONE {
- result = peer.GetDefaultPolicy(d)
- }
-
- switch result {
- case table.ROUTE_TYPE_ACCEPT:
- if d == table.POLICY_DIRECTION_IN {
- path.Filtered = false
- }
- newpaths = append(newpaths, newpath)
- case table.ROUTE_TYPE_REJECT:
- if d == table.POLICY_DIRECTION_IN {
- path.Filtered = true
- }
- filteredPaths = append(filteredPaths, path)
- log.WithFields(log.Fields{
- "Topic": "Peer",
- "Key": peer.conf.NeighborConfig.NeighborAddress,
- "Path": path,
- "Direction": d,
- }).Debug("reject")
- }
- }
- return newpaths, filteredPaths
-}
-
func (peer *Peer) DropAll(rfList []bgp.RouteFamily) {
peer.adjRibIn.Drop(rfList)
peer.adjRibOut.Drop(rfList)
diff --git a/server/server.go b/server/server.go
index 2bb89d8e..7b1617ee 100644
--- a/server/server.go
+++ b/server/server.go
@@ -34,10 +34,6 @@ import (
"time"
)
-const (
- GLOBAL_RIB_NAME = "global"
-)
-
var policyMutex sync.RWMutex
type SenderMsg struct {
@@ -120,6 +116,7 @@ func NewBgpServer(port int) *BgpServer {
b.listenPort = port
b.watchers = make(map[watcherType]watcher)
b.roaClient, _ = newROAClient(0, config.RpkiServers{})
+ b.policy = table.NewRoutingPolicy()
return &b
}
@@ -218,15 +215,8 @@ func (server *BgpServer) Serve() {
}
}(broadcastCh)
- toRFlist := func(l []config.AfiSafi) []bgp.RouteFamily {
- rfList := []bgp.RouteFamily{}
- for _, rf := range l {
- k, _ := bgp.GetRouteFamily(rf.AfiSafiName)
- rfList = append(rfList, k)
- }
- return rfList
- }
- server.globalRib = table.NewTableManager(GLOBAL_RIB_NAME, toRFlist(g.AfiSafis.AfiSafiList), g.MplsLabelRange.MinLabel, g.MplsLabelRange.MaxLabel)
+ rfs, _ := g.AfiSafis.ToRfList()
+ server.globalRib = table.NewTableManager(table.GLOBAL_RIB_NAME, rfs, g.MplsLabelRange.MinLabel, g.MplsLabelRange.MaxLabel)
server.listenerMap = make(map[string]*net.TCPListener)
acceptCh := make(chan *net.TCPConn)
l4, err1 := listenAndAccept("tcp4", server.listenPort, acceptCh)
@@ -362,26 +352,23 @@ func (server *BgpServer) Serve() {
}
SetTcpMD5SigSockopts(listener(config.NeighborConfig.NeighborAddress), addr, config.NeighborConfig.AuthPassword)
- var loc *table.TableManager
- if config.RouteServer.RouteServerConfig.RouteServerClient {
- loc = table.NewTableManager(config.NeighborConfig.NeighborAddress.String(), toRFlist(config.AfiSafis.AfiSafiList), g.MplsLabelRange.MinLabel, g.MplsLabelRange.MaxLabel)
- } else {
- loc = server.globalRib
- }
- peer := NewPeer(g, config, loc)
-
- server.setPolicyByConfig(peer, config.ApplyPolicy)
+ peer := NewPeer(g, config, server.globalRib, server.policy)
+ server.setPolicyByConfig(peer.ID(), config.ApplyPolicy)
if peer.isRouteServerClient() {
pathList := make([]*table.Path, 0)
rfList := peer.configuredRFlist()
for _, p := range server.neighborMap {
- if p.isRouteServerClient() == true {
- pathList = append(pathList, p.getAccepted(rfList)...)
+ if p.isRouteServerClient() {
+ for _, path := range p.getAccepted(rfList) {
+ path = server.policy.ApplyPolicy(peer.TableID(), table.POLICY_DIRECTION_IMPORT, path)
+ if path != nil {
+ pathList = append(pathList, path)
+ }
+ }
}
}
- pathList, _ = peer.ApplyPolicy(table.POLICY_DIRECTION_IMPORT, pathList)
if len(pathList) > 0 {
- peer.localRib.ProcessPaths(pathList)
+ server.globalRib.ProcessPaths(pathList)
}
}
server.neighborMap[addr] = peer
@@ -416,7 +403,7 @@ func (server *BgpServer) Serve() {
addr := config.NeighborConfig.NeighborAddress.String()
peer := server.neighborMap[addr]
peer.conf = config
- server.setPolicyByConfig(peer, config.ApplyPolicy)
+ server.setPolicyByConfig(peer.ID(), config.ApplyPolicy)
case e := <-server.fsmincomingCh:
peer, found := server.neighborMap[e.MsgSrc]
if !found {
@@ -452,121 +439,121 @@ func newSenderMsg(peer *Peer, messages []*bgp.BGPMessage) *SenderMsg {
}
}
-func filterpath(peer *Peer, pathList []*table.Path) []*table.Path {
- filtered := make([]*table.Path, 0)
-
- for _, path := range pathList {
- if _, ok := peer.rfMap[path.GetRouteFamily()]; !ok {
- continue
- }
+func filterpath(peer *Peer, path *table.Path) *table.Path {
+ if path == nil {
+ return nil
+ }
+ if _, ok := peer.rfMap[path.GetRouteFamily()]; !ok {
+ return nil
+ }
- remoteAddr := peer.conf.NeighborConfig.NeighborAddress
+ remoteAddr := peer.conf.NeighborConfig.NeighborAddress
- //iBGP handling
- if !path.IsLocal() && peer.isIBGPPeer() {
- ignore := true
- info := path.GetSource()
+ //iBGP handling
+ if !path.IsLocal() && peer.isIBGPPeer() {
+ ignore := true
+ info := path.GetSource()
- //if the path comes from eBGP peer
- if info.AS != peer.conf.NeighborConfig.PeerAs {
- ignore = false
- }
+ //if the path comes from eBGP peer
+ if info.AS != peer.conf.NeighborConfig.PeerAs {
+ ignore = false
+ }
+ // RFC4456 8. Avoiding Routing Information Loops
+ // A router that recognizes the ORIGINATOR_ID attribute SHOULD
+ // ignore a route received with its BGP Identifier as the ORIGINATOR_ID.
+ if id := path.GetOriginatorID(); peer.gConf.GlobalConfig.RouterId.Equal(id) {
+ log.WithFields(log.Fields{
+ "Topic": "Peer",
+ "Key": remoteAddr,
+ "OriginatorID": id,
+ "Data": path,
+ }).Debug("Originator ID is mine, ignore")
+ return nil
+ }
+ if info.RouteReflectorClient {
+ ignore = false
+ }
+ if peer.isRouteReflectorClient() {
// RFC4456 8. Avoiding Routing Information Loops
- // A router that recognizes the ORIGINATOR_ID attribute SHOULD
- // ignore a route received with its BGP Identifier as the ORIGINATOR_ID.
- if id := path.GetOriginatorID(); peer.gConf.GlobalConfig.RouterId.Equal(id) {
- log.WithFields(log.Fields{
- "Topic": "Peer",
- "Key": remoteAddr,
- "OriginatorID": id,
- "Data": path,
- }).Debug("Originator ID is mine, ignore")
- continue
- }
- if info.RouteReflectorClient {
- ignore = false
- }
- if peer.isRouteReflectorClient() {
- // RFC4456 8. Avoiding Routing Information Loops
- // If the local CLUSTER_ID is found in the CLUSTER_LIST,
- // the advertisement received SHOULD be ignored.
- for _, clusterId := range path.GetClusterList() {
- if clusterId.Equal(peer.fsm.peerInfo.RouteReflectorClusterID) {
- log.WithFields(log.Fields{
- "Topic": "Peer",
- "Key": remoteAddr,
- "ClusterID": clusterId,
- "Data": path,
- }).Debug("cluster list path attribute has local cluster id, ignore")
- continue
- }
+ // If the local CLUSTER_ID is found in the CLUSTER_LIST,
+ // the advertisement received SHOULD be ignored.
+ for _, clusterId := range path.GetClusterList() {
+ if clusterId.Equal(peer.fsm.peerInfo.RouteReflectorClusterID) {
+ log.WithFields(log.Fields{
+ "Topic": "Peer",
+ "Key": remoteAddr,
+ "ClusterID": clusterId,
+ "Data": path,
+ }).Debug("cluster list path attribute has local cluster id, ignore")
+ return nil
}
- ignore = false
- }
-
- if ignore {
- log.WithFields(log.Fields{
- "Topic": "Peer",
- "Key": remoteAddr,
- "Data": path,
- }).Debug("From same AS, ignore.")
- continue
}
+ ignore = false
}
- if remoteAddr.Equal(path.GetSource().Address) {
+ if ignore {
log.WithFields(log.Fields{
"Topic": "Peer",
"Key": remoteAddr,
"Data": path,
- }).Debug("From me, ignore.")
- continue
+ }).Debug("From same AS, ignore.")
+ return nil
}
+ }
- send := true
- for _, as := range path.GetAsList() {
- if as == peer.conf.NeighborConfig.PeerAs {
- send = false
- break
- }
- }
+ if remoteAddr.Equal(path.GetSource().Address) {
+ log.WithFields(log.Fields{
+ "Topic": "Peer",
+ "Key": remoteAddr,
+ "Data": path,
+ }).Debug("From me, ignore.")
+ return nil
+ }
- if !send {
+ for _, as := range path.GetAsList() {
+ if as == peer.conf.NeighborConfig.PeerAs {
log.WithFields(log.Fields{
"Topic": "Peer",
"Key": remoteAddr,
"Data": path,
}).Debug("AS PATH loop, ignore.")
- continue
+ return nil
}
- filtered = append(filtered, path.Clone(remoteAddr, path.IsWithdraw))
}
- return filtered
+ return path.Clone(remoteAddr, path.IsWithdraw)
}
func (server *BgpServer) dropPeerAllRoutes(peer *Peer) []*SenderMsg {
msgs := make([]*SenderMsg, 0)
for _, rf := range peer.configuredRFlist() {
+ dsts := server.globalRib.DeletePathsByPeer(peer.fsm.peerInfo, rf)
if peer.isRouteServerClient() {
+ pathList := make([]*table.Path, len(dsts))
for _, targetPeer := range server.neighborMap {
- rib := targetPeer.localRib
- if !targetPeer.isRouteServerClient() || rib.OwnerName() == peer.conf.NeighborConfig.NeighborAddress.String() {
+ if !targetPeer.isRouteServerClient() || targetPeer == peer || targetPeer.fsm.state != bgp.BGP_FSM_ESTABLISHED {
continue
}
- pathList, _ := rib.DeletePathsforPeer(peer.fsm.peerInfo, rf)
- if targetPeer.fsm.state != bgp.BGP_FSM_ESTABLISHED || len(pathList) == 0 {
- continue
+ i := 0
+ for _, dst := range dsts {
+ feed := dst.NewFeed(targetPeer.TableID())
+ pathList[i] = feed
+ i++
}
msgList := table.CreateUpdateMsgFromPaths(pathList)
msgs = append(msgs, newSenderMsg(targetPeer, msgList))
targetPeer.adjRibOut.Update(pathList)
}
} else {
- rib := server.globalRib
- pathList, _ := rib.DeletePathsforPeer(peer.fsm.peerInfo, rf)
+ pathList := make([]*table.Path, 0, len(dsts))
+ for _, dst := range dsts {
+ path := dst.NewFeed(table.GLOBAL_RIB_NAME)
+ if path != nil {
+ pathList = append(pathList, path)
+ }
+ }
if len(pathList) == 0 {
- continue
+ return msgs
}
server.broadcastBests(pathList)
@@ -606,7 +593,7 @@ func (server *BgpServer) broadcastBests(bests []*table.Path) {
result := &GrpcResponse{
Data: &api.Destination{
Prefix: path.GetNlri().String(),
- Paths: []*api.Path{path.ToApiStruct()},
+ Paths: []*api.Path{path.ToApiStruct(table.GLOBAL_RIB_NAME)},
},
}
remainReqs := make([]*GrpcRequest, 0, len(server.broadcastReqs))
@@ -662,29 +649,61 @@ func (server *BgpServer) broadcastPeerState(peer *Peer) {
func (server *BgpServer) propagateUpdate(peer *Peer, pathList []*table.Path) []*SenderMsg {
msgs := make([]*SenderMsg, 0)
+ rib := server.globalRib
if peer != nil && peer.isRouteServerClient() {
+ for _, path := range pathList {
+ path.Filter(peer.TableID(), table.POLICY_DIRECTION_IMPORT)
+ path.Filter(table.GLOBAL_RIB_NAME, table.POLICY_DIRECTION_IMPORT)
+ }
+ moded := []*table.Path{}
for _, targetPeer := range server.neighborMap {
- rib := targetPeer.localRib
- if !targetPeer.isRouteServerClient() || rib.OwnerName() == peer.conf.NeighborConfig.NeighborAddress.String() {
+ if !targetPeer.isRouteServerClient() || peer == targetPeer {
continue
}
- sendPathList, _ := targetPeer.ApplyPolicy(table.POLICY_DIRECTION_IMPORT, pathList)
- sendPathList, _ = rib.ProcessPaths(sendPathList)
- if targetPeer.fsm.state != bgp.BGP_FSM_ESTABLISHED || len(sendPathList) == 0 {
- continue
+ for _, before := range pathList {
+ after := server.policy.ApplyPolicy(targetPeer.TableID(), table.POLICY_DIRECTION_IMPORT, before)
+ if after == nil {
+ before.Filter(targetPeer.TableID(), table.POLICY_DIRECTION_IMPORT)
+ } else if after != before {
+ before.Filter(targetPeer.TableID(), table.POLICY_DIRECTION_IMPORT)
+ for _, n := range server.neighborMap {
+ if n == targetPeer {
+ continue
+ }
+ after.Filter(n.TableID(), table.POLICY_DIRECTION_IMPORT)
+ }
+ moded = append(moded, after)
+ }
}
- sendPathList, _ = targetPeer.ApplyPolicy(table.POLICY_DIRECTION_EXPORT, filterpath(targetPeer, sendPathList))
- if len(sendPathList) == 0 {
+ }
+ dsts := rib.ProcessPaths(append(pathList, moded...))
+ for _, targetPeer := range server.neighborMap {
+ if !targetPeer.isRouteServerClient() || targetPeer.fsm.state != bgp.BGP_FSM_ESTABLISHED {
continue
}
+ sendPathList := make([]*table.Path, 0, len(dsts))
+ for _, dst := range dsts {
+ path := server.policy.ApplyPolicy(targetPeer.TableID(), table.POLICY_DIRECTION_EXPORT, filterpath(targetPeer, dst.NewFeed(targetPeer.TableID())))
+ if path != nil {
+ sendPathList = append(sendPathList, path)
+ }
+ }
msgList := table.CreateUpdateMsgFromPaths(sendPathList)
targetPeer.adjRibOut.Update(sendPathList)
msgs = append(msgs, newSenderMsg(targetPeer, msgList))
}
} else {
- rib := server.globalRib
- pathList = rib.ApplyPolicy(table.POLICY_DIRECTION_IMPORT, pathList)
- sendPathList, _ := rib.ProcessPaths(pathList)
+ for idx, path := range pathList {
+ pathList[idx] = server.policy.ApplyPolicy(table.GLOBAL_RIB_NAME, table.POLICY_DIRECTION_IMPORT, path)
+ }
+ dsts := rib.ProcessPaths(pathList)
+ sendPathList := make([]*table.Path, 0, len(dsts))
+ for _, dst := range dsts {
+ path := dst.NewFeed(table.GLOBAL_RIB_NAME)
+ if path != nil {
+ sendPathList = append(sendPathList, path)
+ }
+ }
if len(sendPathList) == 0 {
return msgs
}
@@ -695,15 +714,18 @@ func (server *BgpServer) propagateUpdate(peer *Peer, pathList []*table.Path) []*
if targetPeer.isRouteServerClient() || targetPeer.fsm.state != bgp.BGP_FSM_ESTABLISHED {
continue
}
- f := rib.ApplyPolicy(table.POLICY_DIRECTION_EXPORT, filterpath(targetPeer, sendPathList))
- if len(f) == 0 {
- continue
- }
- for _, path := range f {
- path.UpdatePathAttrs(&server.bgpConfig.Global, &targetPeer.conf)
+ pathList := make([]*table.Path, len(sendPathList))
+ copy(pathList, sendPathList)
+ for idx, path := range pathList {
+ path = server.policy.ApplyPolicy(table.GLOBAL_RIB_NAME, table.POLICY_DIRECTION_EXPORT, filterpath(targetPeer, path))
+ if path != nil {
+ path.UpdatePathAttrs(&server.bgpConfig.Global, &targetPeer.conf)
+ }
+ pathList[idx] = path
}
- targetPeer.adjRibOut.Update(f)
- msgList := table.CreateUpdateMsgFromPaths(f)
+ targetPeer.adjRibOut.Update(pathList)
+ msgList := table.CreateUpdateMsgFromPaths(pathList)
+
msgs = append(msgs, newSenderMsg(targetPeer, msgList))
}
}
@@ -823,32 +845,12 @@ func (server *BgpServer) handleFSMMessage(peer *Peer, e *FsmMsg, incoming chan *
}
}
- pathList, update, msgList := peer.handleBGPmessage(e)
+ pathList, msgList := peer.handleBGPmessage(e)
if len(msgList) > 0 {
msgs = append(msgs, newSenderMsg(peer, msgList))
- break
}
- if update == false {
- if len(pathList) > 0 {
- msgList := table.CreateUpdateMsgFromPaths(pathList)
- msgs = append(msgs, newSenderMsg(peer, msgList))
- }
- break
- } else {
- if len(pathList) > 0 {
- server.roaClient.validate(pathList)
- }
- }
- // FIXME: refactor peer.handleBGPmessage and this func
- if peer.isRouteServerClient() {
- var accepted []*table.Path
- for _, p := range pathList {
- if p.Filtered == false {
- accepted = append(accepted, p)
- }
- }
- msgs = append(msgs, server.propagateUpdate(peer, accepted)...)
- } else {
+ if len(pathList) > 0 {
+ server.roaClient.validate(pathList)
msgs = append(msgs, server.propagateUpdate(peer, pathList)...)
}
default:
@@ -899,7 +901,7 @@ func (server *BgpServer) UpdatePolicy(policy config.RoutingPolicy) {
server.policyUpdateCh <- policy
}
-func (server *BgpServer) setPolicyByConfig(p policyPoint, c config.ApplyPolicy) {
+func (server *BgpServer) setPolicyByConfig(id string, c config.ApplyPolicy) {
for _, dir := range []table.PolicyDirection{table.POLICY_DIRECTION_IN, table.POLICY_DIRECTION_IMPORT, table.POLICY_DIRECTION_EXPORT} {
ps, def, err := server.policy.GetAssignmentFromConfig(dir, c)
if err != nil {
@@ -909,26 +911,24 @@ func (server *BgpServer) setPolicyByConfig(p policyPoint, c config.ApplyPolicy)
}).Errorf("failed to get policy info: %s", err)
continue
}
- p.SetDefaultPolicy(dir, def)
- p.SetPolicy(dir, ps)
+ server.policy.SetDefaultPolicy(id, dir, def)
+ server.policy.SetPolicy(id, dir, ps)
}
}
-func (server *BgpServer) SetPolicy(pl config.RoutingPolicy) error {
- p, err := table.NewRoutingPolicy(pl)
- if err != nil {
+func (server *BgpServer) SetRoutingPolicy(pl config.RoutingPolicy) error {
+ if err := server.policy.Reload(pl); err != nil {
log.WithFields(log.Fields{
"Topic": "Policy",
}).Errorf("failed to create routing policy: %s", err)
return err
}
- server.policy = p
- server.setPolicyByConfig(server.globalRib, server.bgpConfig.Global.ApplyPolicy)
+ server.setPolicyByConfig(table.GLOBAL_RIB_NAME, server.bgpConfig.Global.ApplyPolicy)
return nil
}
func (server *BgpServer) handlePolicy(pl config.RoutingPolicy) error {
- if err := server.SetPolicy(pl); err != nil {
+ if err := server.SetRoutingPolicy(pl); err != nil {
log.WithFields(log.Fields{
"Topic": "Policy",
}).Errorf("failed to set new policy: %s", err)
@@ -939,7 +939,7 @@ func (server *BgpServer) handlePolicy(pl config.RoutingPolicy) error {
"Topic": "Peer",
"Key": peer.conf.NeighborConfig.NeighborAddress,
}).Info("call set policy")
- server.setPolicyByConfig(peer, peer.conf.ApplyPolicy)
+ server.setPolicyByConfig(peer.ID(), peer.conf.ApplyPolicy)
}
return nil
}
@@ -1162,7 +1162,7 @@ func (server *BgpServer) handleModPathRequest(grpcReq *GrpcRequest) []*table.Pat
macIpAdv := evpnNlri.RouteTypeData.(*bgp.EVPNMacIPAdvertisementRoute)
etag := macIpAdv.ETag
mac := macIpAdv.MacAddress
- paths := server.globalRib.GetBestPathList([]bgp.RouteFamily{bgp.RF_EVPN})
+ paths := server.globalRib.GetBestPathList(table.GLOBAL_RIB_NAME, []bgp.RouteFamily{bgp.RF_EVPN})
if m := getMacMobilityExtendedCommunity(etag, mac, paths); m != nil {
extcomms = append(extcomms, m)
}
@@ -1256,7 +1256,7 @@ func (server *BgpServer) handleVrfRequest(req *GrpcRequest) []*table.Path {
result.ResponseErr = fmt.Errorf("unsupported route family: %s", bgp.RouteFamily(arg.Family))
break
}
- paths := rib.GetPathList(rf)
+ paths := rib.GetPathList(table.GLOBAL_RIB_NAME, rf)
dsts := make([]*api.Destination, 0, len(paths))
for _, path := range paths {
ok := table.CanImportToVrf(vrfs[name], path)
@@ -1265,7 +1265,7 @@ func (server *BgpServer) handleVrfRequest(req *GrpcRequest) []*table.Path {
}
dsts = append(dsts, &api.Destination{
Prefix: path.GetNlri().String(),
- Paths: []*api.Path{path.ToApiStruct()},
+ Paths: []*api.Path{path.ToApiStruct(table.GLOBAL_RIB_NAME)},
})
}
req.ResponseCh <- &GrpcResponse{
@@ -1357,12 +1357,14 @@ func (server *BgpServer) handleGrpc(grpcReq *GrpcRequest) []*SenderMsg {
return []*Peer{peer}, err
}
- sortedDsts := func(t *table.Table) []*api.Destination {
+ sortedDsts := func(id string, t *table.Table) []*api.Destination {
results := make([]*api.Destination, 0, len(t.GetDestinations()))
r := radix.New()
for _, dst := range t.GetDestinations() {
- r.Insert(dst.RadixKey, dst.ToApiStruct())
+ if d := dst.ToApiStruct(id); d != nil {
+ r.Insert(dst.RadixKey, d)
+ }
}
r.Walk(func(s string, v interface{}) bool {
results = append(results, v.(*api.Destination))
@@ -1405,6 +1407,7 @@ func (server *BgpServer) handleGrpc(grpcReq *GrpcRequest) []*SenderMsg {
Family: arg.Family,
}
rib := server.globalRib
+ id := table.GLOBAL_RIB_NAME
if grpcReq.RequestType == REQ_LOCAL_RIB {
peer, ok := server.neighborMap[arg.Name]
if !ok {
@@ -1415,7 +1418,7 @@ func (server *BgpServer) handleGrpc(grpcReq *GrpcRequest) []*SenderMsg {
err = fmt.Errorf("Neighbor %v doesn't have local rib", arg.Name)
goto ERROR
}
- rib = peer.localRib
+ id = peer.ID()
}
af := bgp.RouteFamily(arg.Family)
if _, ok := rib.Tables[af]; !ok {
@@ -1427,13 +1430,15 @@ func (server *BgpServer) handleGrpc(grpcReq *GrpcRequest) []*SenderMsg {
case bgp.RF_IPv4_UC, bgp.RF_IPv6_UC:
if len(arg.Destinations) > 0 {
dsts := []*api.Destination{}
- f := func(cidr string) (bool, error) {
+ f := func(id, cidr string) (bool, error) {
_, prefix, err := net.ParseCIDR(cidr)
if err != nil {
return false, err
}
if dst := rib.Tables[af].GetDestination(prefix.String()); dst != nil {
- dsts = append(dsts, dst.ToApiStruct())
+ if d := dst.ToApiStruct(id); d != nil {
+ dsts = append(dsts, d)
+ }
return true, nil
} else {
return false, nil
@@ -1441,14 +1446,14 @@ func (server *BgpServer) handleGrpc(grpcReq *GrpcRequest) []*SenderMsg {
}
for _, dst := range arg.Destinations {
key := dst.Prefix
- if _, err := f(key); err != nil {
+ if _, err := f(id, key); err != nil {
if host := net.ParseIP(key); host != nil {
masklen := 32
if af == bgp.RF_IPv6_UC {
masklen = 128
}
for i := masklen; i > 0; i-- {
- if y, _ := f(fmt.Sprintf("%s/%d", key, i)); y {
+ if y, _ := f(id, fmt.Sprintf("%s/%d", key, i)); y {
break
}
}
@@ -1458,18 +1463,20 @@ func (server *BgpServer) handleGrpc(grpcReq *GrpcRequest) []*SenderMsg {
ones, bits := prefix.Mask.Size()
for i := ones + 1; i <= bits; i++ {
prefix.Mask = net.CIDRMask(i, bits)
- f(prefix.String())
+ f(id, prefix.String())
}
}
}
d.Destinations = dsts
} else {
- d.Destinations = sortedDsts(rib.Tables[af])
+ d.Destinations = sortedDsts(id, rib.Tables[af])
}
default:
d.Destinations = make([]*api.Destination, 0, len(rib.Tables[af].GetDestinations()))
for _, dst := range rib.Tables[af].GetDestinations() {
- d.Destinations = append(d.Destinations, dst.ToApiStruct())
+ if s := dst.ToApiStruct(id); s != nil {
+ d.Destinations = append(d.Destinations, s)
+ }
}
}
grpcReq.ResponseCh <- &GrpcResponse{
@@ -1548,7 +1555,7 @@ func (server *BgpServer) handleGrpc(grpcReq *GrpcRequest) []*SenderMsg {
if found {
r.Insert(table.CidrToRadixkey(key), &api.Destination{
Prefix: key,
- Paths: []*api.Path{p.ToApiStruct()},
+ Paths: []*api.Path{p.ToApiStruct(peer.TableID())},
})
}
}
@@ -1560,7 +1567,7 @@ func (server *BgpServer) handleGrpc(grpcReq *GrpcRequest) []*SenderMsg {
for _, p := range paths {
results = append(results, &api.Destination{
Prefix: p.GetNlri().String(),
- Paths: []*api.Path{p.ToApiStruct()},
+ Paths: []*api.Path{p.ToApiStruct(peer.TableID())},
})
}
}
@@ -1608,9 +1615,11 @@ func (server *BgpServer) handleGrpc(grpcReq *GrpcRequest) []*SenderMsg {
}
for _, peer := range peers {
- pathList := peer.adjRibIn.PathList([]bgp.RouteFamily{grpcReq.RouteFamily}, false)
- if peer.isRouteServerClient() {
- pathList, _ = peer.ApplyPolicy(table.POLICY_DIRECTION_IN, pathList)
+ pathList := []*table.Path{}
+ for _, path := range peer.adjRibIn.PathList([]bgp.RouteFamily{grpcReq.RouteFamily}, false) {
+ if path = server.policy.ApplyPolicy(peer.ID(), table.POLICY_DIRECTION_IN, path); path != nil {
+ pathList = append(pathList, path.Clone(peer.conf.NeighborConfig.NeighborAddress, false))
+ }
}
msgs = append(msgs, server.propagateUpdate(peer, pathList)...)
}
@@ -1826,24 +1835,6 @@ func (server *BgpServer) handleGrpcModNeighbor(grpcReq *GrpcRequest) (sMsgs []*S
log.Infof("Peer %s is added", addr)
}
SetTcpMD5SigSockopts(listener(net.ParseIP(addr)), addr, arg.Peer.Conf.AuthPassword)
- var loc *table.TableManager
- if arg.Peer.RouteServer != nil {
- if arg.Peer.RouteServer.RouteServerClient {
- apitoRFlist := func(l []*api.AfiSafi) []bgp.RouteFamily {
- rfList := []bgp.RouteFamily{}
- for _, rf := range l {
- k, _ := bgp.GetRouteFamily(rf.Name)
- rfList = append(rfList, k)
- }
- return rfList
- }
- loc = table.NewTableManager(addr, apitoRFlist(arg.Peer.Afisafis.Afisafi), server.bgpConfig.Global.MplsLabelRange.MinLabel, server.bgpConfig.Global.MplsLabelRange.MaxLabel)
- } else {
- loc = server.globalRib
- }
- } else {
- loc = server.globalRib
- }
apitoConfig := func(a *api.Peer) config.Neighbor {
var pconf config.Neighbor
if a.Conf != nil {
@@ -1906,19 +1897,20 @@ func (server *BgpServer) handleGrpcModNeighbor(grpcReq *GrpcRequest) (sMsgs []*S
return pconf
}
configneigh := apitoConfig(arg.Peer)
- peer := NewPeer(server.bgpConfig.Global, configneigh, loc)
- server.setPolicyByConfig(peer, configneigh.ApplyPolicy)
+ peer := NewPeer(server.bgpConfig.Global, configneigh, server.globalRib, server.policy)
+ server.setPolicyByConfig(peer.ID(), configneigh.ApplyPolicy)
if peer.isRouteServerClient() {
pathList := make([]*table.Path, 0)
rfList := peer.configuredRFlist()
for _, p := range server.neighborMap {
- if p.isRouteServerClient() == true {
- pathList = append(pathList, p.getAccepted(rfList)...)
+ if p.isRouteServerClient() {
+ for _, path := range p.getAccepted(rfList) {
+ pathList = append(pathList, server.policy.ApplyPolicy(peer.TableID(), table.POLICY_DIRECTION_IMPORT, path))
+ }
}
}
- pathList, _ = peer.ApplyPolicy(table.POLICY_DIRECTION_IMPORT, pathList)
if len(pathList) > 0 {
- peer.localRib.ProcessPaths(pathList)
+ server.globalRib.ProcessPaths(pathList)
}
}
server.neighborMap[addr] = peer
@@ -2064,7 +2056,7 @@ func (server *BgpServer) handleGrpcGetPolicy(grpcReq *GrpcRequest) error {
func (server *BgpServer) policyInUse(x *table.Policy) bool {
for _, peer := range server.neighborMap {
for _, dir := range []table.PolicyDirection{table.POLICY_DIRECTION_IN, table.POLICY_DIRECTION_EXPORT, table.POLICY_DIRECTION_EXPORT} {
- for _, y := range peer.GetPolicy(dir) {
+ for _, y := range server.policy.GetPolicy(peer.ID(), dir) {
if x.Name() == y.Name() {
return true
}
@@ -2072,7 +2064,7 @@ func (server *BgpServer) policyInUse(x *table.Policy) bool {
}
}
for _, dir := range []table.PolicyDirection{table.POLICY_DIRECTION_EXPORT, table.POLICY_DIRECTION_EXPORT} {
- for _, y := range server.globalRib.GetPolicy(dir) {
+ for _, y := range server.policy.GetPolicy(table.GLOBAL_RIB_NAME, dir) {
if x.Name() == y.Name() {
return true
}
@@ -2144,53 +2136,46 @@ func (server *BgpServer) handleGrpcModPolicy(grpcReq *GrpcRequest) error {
return err
}
-type policyPoint interface {
- GetDefaultPolicy(table.PolicyDirection) table.RouteType
- GetPolicy(table.PolicyDirection) []*table.Policy
- SetDefaultPolicy(table.PolicyDirection, table.RouteType) error
- SetPolicy(table.PolicyDirection, []*table.Policy) error
-}
-
-func (server *BgpServer) getPolicyInfo(a *api.PolicyAssignment) (policyPoint, table.PolicyDirection, error) {
+func (server *BgpServer) getPolicyInfo(a *api.PolicyAssignment) (string, table.PolicyDirection, error) {
switch a.Resource {
case api.Resource_GLOBAL:
switch a.Type {
case api.PolicyType_IMPORT:
- return server.globalRib, table.POLICY_DIRECTION_IMPORT, nil
+ return table.GLOBAL_RIB_NAME, table.POLICY_DIRECTION_IMPORT, nil
case api.PolicyType_EXPORT:
- return server.globalRib, table.POLICY_DIRECTION_EXPORT, nil
+ return table.GLOBAL_RIB_NAME, table.POLICY_DIRECTION_EXPORT, nil
default:
- return nil, table.POLICY_DIRECTION_NONE, fmt.Errorf("invalid policy type")
+ return "", table.POLICY_DIRECTION_NONE, fmt.Errorf("invalid policy type")
}
case api.Resource_LOCAL:
peer, ok := server.neighborMap[a.Name]
if !ok {
- return nil, table.POLICY_DIRECTION_NONE, fmt.Errorf("not found peer %s", a.Name)
+ return "", table.POLICY_DIRECTION_NONE, fmt.Errorf("not found peer %s", a.Name)
}
switch a.Type {
case api.PolicyType_IN:
- return peer, table.POLICY_DIRECTION_IN, nil
+ return peer.ID(), table.POLICY_DIRECTION_IN, nil
case api.PolicyType_IMPORT:
- return peer, table.POLICY_DIRECTION_IMPORT, nil
+ return peer.ID(), table.POLICY_DIRECTION_IMPORT, nil
case api.PolicyType_EXPORT:
- return peer, table.POLICY_DIRECTION_EXPORT, nil
+ return peer.ID(), table.POLICY_DIRECTION_EXPORT, nil
default:
- return nil, table.POLICY_DIRECTION_NONE, fmt.Errorf("invalid policy type")
+ return "", table.POLICY_DIRECTION_NONE, fmt.Errorf("invalid policy type")
}
default:
- return nil, table.POLICY_DIRECTION_NONE, fmt.Errorf("invalid resource type")
+ return "", table.POLICY_DIRECTION_NONE, fmt.Errorf("invalid resource type")
}
}
func (server *BgpServer) handleGrpcGetPolicyAssignment(grpcReq *GrpcRequest) error {
arg := grpcReq.Data.(*api.PolicyAssignment)
- i, dir, err := server.getPolicyInfo(arg)
+ id, dir, err := server.getPolicyInfo(arg)
if err != nil {
return err
}
- arg.Default = i.GetDefaultPolicy(dir).ToApiStruct()
- ps := i.GetPolicy(dir)
+ arg.Default = server.policy.GetDefaultPolicy(id, dir).ToApiStruct()
+ ps := server.policy.GetPolicy(id, dir)
arg.Policies = make([]*api.Policy, 0, len(ps))
for _, x := range ps {
arg.Policies = append(arg.Policies, x.ToApiStruct())
@@ -2204,12 +2189,12 @@ func (server *BgpServer) handleGrpcGetPolicyAssignment(grpcReq *GrpcRequest) err
func (server *BgpServer) handleGrpcModPolicyAssignment(grpcReq *GrpcRequest) error {
var err error
var dir table.PolicyDirection
- var i policyPoint
+ var id string
policyMutex.Lock()
defer policyMutex.Unlock()
arg := grpcReq.Data.(*api.ModPolicyAssignmentArguments)
assignment := arg.Assignment
- i, dir, err = server.getPolicyInfo(assignment)
+ id, dir, err = server.getPolicyInfo(assignment)
if err != nil {
return err
}
@@ -2221,22 +2206,22 @@ func (server *BgpServer) handleGrpcModPolicyAssignment(grpcReq *GrpcRequest) err
}
ps = append(ps, p)
}
- cur := i.GetPolicy(dir)
+ cur := server.policy.GetPolicy(id, dir)
switch arg.Operation {
case api.Operation_ADD, api.Operation_REPLACE:
if arg.Operation == api.Operation_REPLACE || cur == nil {
- err = i.SetPolicy(dir, ps)
+ err = server.policy.SetPolicy(id, dir, ps)
} else {
- err = i.SetPolicy(dir, append(cur, ps...))
+ err = server.policy.SetPolicy(id, dir, append(cur, ps...))
}
if err != nil {
return err
}
switch assignment.Default {
case api.RouteAction_ACCEPT:
- err = i.SetDefaultPolicy(dir, table.ROUTE_TYPE_ACCEPT)
+ err = server.policy.SetDefaultPolicy(id, dir, table.ROUTE_TYPE_ACCEPT)
case api.RouteAction_REJECT:
- err = i.SetDefaultPolicy(dir, table.ROUTE_TYPE_REJECT)
+ err = server.policy.SetDefaultPolicy(id, dir, table.ROUTE_TYPE_REJECT)
}
case api.Operation_DEL:
n := make([]*table.Policy, 0, len(cur)-len(ps))
@@ -2252,13 +2237,13 @@ func (server *BgpServer) handleGrpcModPolicyAssignment(grpcReq *GrpcRequest) err
n = append(n, x)
}
}
- err = i.SetPolicy(dir, n)
+ err = server.policy.SetPolicy(id, dir, n)
case api.Operation_DEL_ALL:
- err = i.SetPolicy(dir, nil)
+ err = server.policy.SetPolicy(id, dir, nil)
if err != nil {
return err
}
- err = i.SetDefaultPolicy(dir, table.ROUTE_TYPE_NONE)
+ err = server.policy.SetDefaultPolicy(id, dir, table.ROUTE_TYPE_NONE)
}
return err
}
@@ -2473,7 +2458,7 @@ func (server *BgpServer) mkMrtRibMsgs(tbl *table.Table, t uint32) ([]*bgp.MRTMes
var seq uint32
msgs := make([]*bgp.MRTMessage, 0, len(tbl.GetDestinations()))
for _, dst := range tbl.GetDestinations() {
- l := dst.GetKnownPathList()
+ l := dst.GetKnownPathList(table.GLOBAL_RIB_NAME)
entries := make([]*bgp.RibEntry, 0, len(l))
for _, p := range l {
// mrt doesn't assume to dump locally generated routes
diff --git a/table/adj.go b/table/adj.go
index 019ad3c8..d7a0555d 100644
--- a/table/adj.go
+++ b/table/adj.go
@@ -21,16 +21,18 @@ import (
)
type AdjRib struct {
+ id string
accepted map[bgp.RouteFamily]int
table map[bgp.RouteFamily]map[string]*Path
}
-func NewAdjRib(rfList []bgp.RouteFamily) *AdjRib {
+func NewAdjRib(id string, rfList []bgp.RouteFamily) *AdjRib {
table := make(map[bgp.RouteFamily]map[string]*Path)
for _, rf := range rfList {
table[rf] = make(map[string]*Path)
}
return &AdjRib{
+ id: id,
table: table,
accepted: make(map[bgp.RouteFamily]int),
}
@@ -47,19 +49,21 @@ func (adj *AdjRib) Update(pathList []*Path) {
if path.IsWithdraw {
if found {
delete(adj.table[rf], key)
- if !old.Filtered {
+ if old.Filtered(adj.id) > POLICY_DIRECTION_NONE {
adj.accepted[rf]--
}
}
} else {
+ n := path.Filtered(adj.id)
if found {
- if old.Filtered && !path.Filtered {
+ o := old.Filtered(adj.id)
+ if o == POLICY_DIRECTION_IN && n == POLICY_DIRECTION_NONE {
adj.accepted[rf]++
- } else if !old.Filtered && path.Filtered {
+ } else if o == POLICY_DIRECTION_NONE && n == POLICY_DIRECTION_IN {
adj.accepted[rf]--
}
} else {
- if !path.Filtered {
+ if n == POLICY_DIRECTION_NONE {
adj.accepted[rf]++
}
}
@@ -75,7 +79,7 @@ func (adj *AdjRib) PathList(rfList []bgp.RouteFamily, accepted bool) []*Path {
pathList := make([]*Path, 0, adj.Count(rfList))
for _, rf := range rfList {
for _, rr := range adj.table[rf] {
- if accepted && rr.Filtered {
+ if accepted && rr.Filtered(adj.id) > POLICY_DIRECTION_NONE {
continue
}
pathList = append(pathList, rr)
diff --git a/table/destination.go b/table/destination.go
index 7485c987..f83e92e6 100644
--- a/table/destination.go
+++ b/table/destination.go
@@ -18,7 +18,6 @@ package table
import (
"bytes"
"encoding/binary"
- "encoding/json"
"fmt"
log "github.com/Sirupsen/logrus"
api "github.com/osrg/gobgp/api"
@@ -112,14 +111,13 @@ func NewPeerInfo(g *config.Global, p *config.Neighbor) *PeerInfo {
}
type Destination struct {
- routeFamily bgp.RouteFamily
- nlri bgp.AddrPrefixInterface
- knownPathList paths
- withdrawList paths
- newPathList paths
- bestPath *Path
- bestPathReason BestPathReason
- RadixKey string
+ routeFamily bgp.RouteFamily
+ nlri bgp.AddrPrefixInterface
+ oldKnownPathList paths
+ knownPathList paths
+ withdrawList paths
+ newPathList paths
+ RadixKey string
}
func NewDestination(nlri bgp.AddrPrefixInterface) *Destination {
@@ -137,24 +135,27 @@ func NewDestination(nlri bgp.AddrPrefixInterface) *Destination {
return d
}
-func (dd *Destination) MarshalJSON() ([]byte, error) {
- return json.Marshal(dd.ToApiStruct())
-}
-
-func (dd *Destination) ToApiStruct() *api.Destination {
+func (dd *Destination) ToApiStruct(id string) *api.Destination {
prefix := dd.GetNlri().String()
paths := func(arg []*Path) []*api.Path {
ret := make([]*api.Path, 0, len(arg))
+ first := true
for _, p := range arg {
- pp := p.ToApiStruct()
- if dd.GetBestPath() == p {
- pp.Best = true
+ if p.filtered[id] == POLICY_DIRECTION_NONE {
+ pp := p.ToApiStruct(id)
+ if first {
+ pp.Best = true
+ first = false
+ }
+ ret = append(ret, pp)
}
- ret = append(ret, pp)
}
return ret
}(dd.knownPathList)
+ if len(paths) == 0 {
+ return nil
+ }
return &api.Destination{
Prefix: prefix,
Paths: paths,
@@ -177,28 +178,32 @@ func (dd *Destination) setNlri(nlri bgp.AddrPrefixInterface) {
dd.nlri = nlri
}
-func (dd *Destination) getBestPathReason() BestPathReason {
- return dd.bestPathReason
-}
-
-func (dd *Destination) setBestPathReason(reason BestPathReason) {
- dd.bestPathReason = reason
-}
-
-func (dd *Destination) GetBestPath() *Path {
- return dd.bestPath
-}
-
-func (dd *Destination) setBestPath(path *Path) {
- dd.bestPath = path
+func (dd *Destination) GetKnownPathList(id string) []*Path {
+ list := make([]*Path, 0, len(dd.knownPathList))
+ for _, p := range dd.knownPathList {
+ if p.filtered[id] == POLICY_DIRECTION_NONE {
+ list = append(list, p)
+ }
+ }
+ return list
}
-func (dd *Destination) GetKnownPathList() []*Path {
- return dd.knownPathList
+func (dd *Destination) GetBestPath(id string) *Path {
+ for _, p := range dd.knownPathList {
+ if p.filtered[id] == POLICY_DIRECTION_NONE {
+ return p
+ }
+ }
+ return nil
}
-func (dd *Destination) setKnownPathList(List []*Path) {
- dd.knownPathList = List
+func (dd *Destination) oldBest(id string) *Path {
+ for _, p := range dd.oldKnownPathList {
+ if p.filtered[id] == POLICY_DIRECTION_NONE {
+ return p
+ }
+ }
+ return nil
}
func (dd *Destination) addWithdraw(withdraw *Path) {
@@ -229,8 +234,8 @@ func (dd *Destination) validatePath(path *Path) {
//
// Modifies destination's state related to stored paths. Removes withdrawn
// paths from known paths. Also, adds new paths to known paths.
-func (dest *Destination) Calculate() (*Path, BestPathReason, error) {
-
+func (dest *Destination) Calculate() {
+ dest.oldKnownPathList = dest.knownPathList
// First remove the withdrawn paths.
dest.explicitWithdraw()
// Do implicit withdrawal
@@ -240,7 +245,22 @@ func (dest *Destination) Calculate() (*Path, BestPathReason, error) {
// Clear new paths as we copied them.
dest.newPathList = make([]*Path, 0)
// Compute new best path
- return dest.computeKnownBestPath()
+ dest.computeKnownBestPath()
+}
+
+func (dest *Destination) NewFeed(id string) *Path {
+ old := dest.oldBest(id)
+ best := dest.GetBestPath(id)
+ if best != nil && best.Equal(old) {
+ return nil
+ }
+ if best == nil {
+ if old == nil {
+ return nil
+ }
+ return old.Clone(old.Owner, true)
+ }
+ return best
}
// Removes withdrawn paths.
diff --git a/table/destination_test.go b/table/destination_test.go
index dce14250..105d7104 100644
--- a/table/destination_test.go
+++ b/table/destination_test.go
@@ -63,49 +63,6 @@ func TestDestinationGetNlri(t *testing.T) {
r_nlri := dd.GetNlri()
assert.Equal(t, r_nlri, nlri)
}
-func TestDestinationSetBestPathReason(t *testing.T) {
- dd := &Destination{}
- reason := BestPathReason("reason1")
- dd.setBestPathReason(reason)
- r_reason := dd.getBestPathReason()
- assert.Equal(t, r_reason, reason)
-}
-func TestDestinationGetBestPathReason(t *testing.T) {
- dd := &Destination{}
- reason := BestPathReason("reason2")
- dd.setBestPathReason(reason)
- r_reason := dd.getBestPathReason()
- assert.Equal(t, r_reason, reason)
-}
-func TestDestinationSetBestPath(t *testing.T) {
- peerD := DestCreatePeer()
- pathD := DestCreatePath(peerD)
- ipv4d := NewDestination(pathD[0].GetNlri())
- ipv4d.setBestPath(pathD[0])
- r_pathD := ipv4d.GetBestPath()
- assert.Equal(t, r_pathD, pathD[0])
-}
-func TestDestinationGetBestPath(t *testing.T) {
- peerD := DestCreatePeer()
- pathD := DestCreatePath(peerD)
- ipv4d := NewDestination(pathD[0].GetNlri())
- ipv4d.setBestPath(pathD[0])
- r_pathD := ipv4d.GetBestPath()
- assert.Equal(t, r_pathD, pathD[0])
-}
-func TestDestinationCalculate(t *testing.T) {
- peerD := DestCreatePeer()
- pathD := DestCreatePath(peerD)
- ipv4d := NewDestination(pathD[0].GetNlri())
- //best path selection
- ipv4d.addNewPath(pathD[0])
- ipv4d.addNewPath(pathD[1])
- ipv4d.addNewPath(pathD[2])
- ipv4d.addWithdraw(pathD[2])
- _, _, e := ipv4d.Calculate()
- assert.Nil(t, e)
-}
-
func DestCreatePeer() []*PeerInfo {
peerD1 := &PeerInfo{AS: 65000}
peerD2 := &PeerInfo{AS: 65001}
diff --git a/table/message.go b/table/message.go
index 3ec0ef62..c6437d18 100644
--- a/table/message.go
+++ b/table/message.go
@@ -194,6 +194,9 @@ func CreateUpdateMsgFromPaths(pathList []*Path) []*bgp.BGPMessage {
pathByAttrs := make(map[uint32][]*bucket)
pathLen := len(pathList)
for _, path := range pathList {
+ if path == nil {
+ continue
+ }
y := func(p *Path) bool {
// the merging logic makes gobgpd slower so if
// paths are not many, let's avoid mering.
diff --git a/table/path.go b/table/path.go
index 7cfe37b0..73b02061 100644
--- a/table/path.go
+++ b/table/path.go
@@ -38,9 +38,10 @@ type Path struct {
NoImplicitWithdraw bool
Validation config.RpkiValidationResultType
IsFromZebra bool
- Filtered bool
Owner net.IP
reason BestPathReason
+ filtered map[string]PolicyDirection
+ key string
}
func NewPath(source *PeerInfo, nlri bgp.AddrPrefixInterface, isWithdraw bool, pattrs []bgp.PathAttributeInterface, medSetByTargetNeighbor bool, timestamp time.Time, noImplicitWithdraw bool) *Path {
@@ -67,6 +68,7 @@ func NewPath(source *PeerInfo, nlri bgp.AddrPrefixInterface, isWithdraw bool, pa
timestamp: timestamp,
NoImplicitWithdraw: noImplicitWithdraw,
Owner: owner,
+ filtered: make(map[string]PolicyDirection),
}
}
@@ -188,7 +190,7 @@ func (path *Path) IsIBGP() bool {
return path.source.AS == path.source.LocalAS
}
-func (path *Path) ToApiStruct() *api.Path {
+func (path *Path) ToApiStruct(id string) *api.Path {
nlri := path.GetNlri()
n, _ := nlri.Serialize()
rf := uint32(bgp.AfiSafiToRouteFamily(nlri.AFI(), nlri.SAFI()))
@@ -206,27 +208,11 @@ func (path *Path) ToApiStruct() *api.Path {
Age: int64(time.Now().Sub(path.timestamp).Seconds()),
IsWithdraw: path.IsWithdraw,
Validation: int32(path.Validation),
- Filtered: path.Filtered,
+ Filtered: path.Filtered(id) > POLICY_DIRECTION_NONE,
Rf: rf,
}
}
-func (path *Path) MarshalJSON() ([]byte, error) {
- return json.Marshal(struct {
- Source *PeerInfo `json:"source"`
- IsWithdraw bool `json:"is_withdraw"`
- Nlri bgp.AddrPrefixInterface `json:"nlri"`
- Pathattrs []bgp.PathAttributeInterface `json:"pattrs"`
- Filtered bool `json:"filtered"`
- }{
- Source: path.source,
- IsWithdraw: path.IsWithdraw,
- Nlri: path.nlri,
- Pathattrs: path.pathAttrs,
- Filtered: path.Filtered,
- })
-}
-
// create new PathAttributes
func (path *Path) Clone(owner net.IP, isWithdraw bool) *Path {
newPathAttrs := make([]bgp.PathAttributeInterface, len(path.pathAttrs))
@@ -237,9 +223,18 @@ func (path *Path) Clone(owner net.IP, isWithdraw bool) *Path {
p := NewPath(path.source, path.nlri, isWithdraw, newPathAttrs, false, path.timestamp, path.NoImplicitWithdraw)
p.Validation = path.Validation
p.Owner = owner
+ p.key = path.key
return p
}
+func (path *Path) Filter(id string, reason PolicyDirection) {
+ path.filtered[id] = reason
+}
+
+func (path *Path) Filtered(id string) PolicyDirection {
+ return path.filtered[id]
+}
+
func (path *Path) GetRouteFamily() bgp.RouteFamily {
return bgp.AfiSafiToRouteFamily(path.nlri.AFI(), path.nlri.SAFI())
}
@@ -332,7 +327,10 @@ func (path *Path) String() string {
}
func (path *Path) getPrefix() string {
- return path.nlri.String()
+ if path.key == "" {
+ return path.nlri.String()
+ }
+ return path.key
}
func (path *Path) GetAsPath() *bgp.PathAttributeAsPath {
@@ -655,7 +653,7 @@ func (lhs *Path) Equal(rhs *Path) bool {
return true
}
f := func(p *Path) []byte {
- s := p.ToApiStruct()
+ s := p.ToApiStruct(GLOBAL_RIB_NAME)
s.Age = 0
buf, _ := json.Marshal(s)
return buf
diff --git a/table/policy.go b/table/policy.go
index b3799e35..6e5a94b2 100644
--- a/table/policy.go
+++ b/table/policy.go
@@ -64,11 +64,23 @@ type PolicyDirection int
const (
POLICY_DIRECTION_NONE PolicyDirection = iota
+ POLICY_DIRECTION_IN
POLICY_DIRECTION_IMPORT
POLICY_DIRECTION_EXPORT
- POLICY_DIRECTION_IN
)
+func (d PolicyDirection) String() string {
+ switch d {
+ case POLICY_DIRECTION_IN:
+ return "in"
+ case POLICY_DIRECTION_IMPORT:
+ return "import"
+ case POLICY_DIRECTION_EXPORT:
+ return "export"
+ }
+ return fmt.Sprintf("unknown(%d)", d)
+}
+
type MatchOption int
const (
@@ -2597,10 +2609,115 @@ func NewPolicy(c config.PolicyDefinition, dmap DefinedSetMap) (*Policy, error) {
}, nil
}
+type Assignment struct {
+ inPolicies []*Policy
+ defaultInPolicy RouteType
+ importPolicies []*Policy
+ defaultImportPolicy RouteType
+ exportPolicies []*Policy
+ defaultExportPolicy RouteType
+}
+
type RoutingPolicy struct {
DefinedSetMap DefinedSetMap
PolicyMap map[string]*Policy
StatementMap map[string]*Statement
+ AssignmentMap map[string]*Assignment
+}
+
+func (r *RoutingPolicy) ApplyPolicy(id string, dir PolicyDirection, before *Path) *Path {
+ if before == nil {
+ return nil
+ }
+ if filtered := before.Filtered(id); filtered > POLICY_DIRECTION_NONE && filtered < dir {
+ return nil
+ }
+ result := ROUTE_TYPE_NONE
+ after := before
+ for _, p := range r.GetPolicy(id, dir) {
+ result, after = p.Apply(before)
+ if result != ROUTE_TYPE_NONE {
+ break
+ }
+ }
+ if result == ROUTE_TYPE_NONE {
+ result = r.GetDefaultPolicy(id, dir)
+ }
+ switch result {
+ case ROUTE_TYPE_ACCEPT:
+ return after
+ default:
+ return nil
+ }
+}
+
+func (r *RoutingPolicy) GetPolicy(id string, dir PolicyDirection) []*Policy {
+ a, ok := r.AssignmentMap[id]
+ if !ok {
+ return nil
+ }
+ switch dir {
+ case POLICY_DIRECTION_IN:
+ return a.inPolicies
+ case POLICY_DIRECTION_IMPORT:
+ return a.importPolicies
+ case POLICY_DIRECTION_EXPORT:
+ return a.exportPolicies
+ default:
+ return nil
+ }
+}
+
+func (r *RoutingPolicy) GetDefaultPolicy(id string, dir PolicyDirection) RouteType {
+ a, ok := r.AssignmentMap[id]
+ if !ok {
+ return ROUTE_TYPE_NONE
+ }
+ switch dir {
+ case POLICY_DIRECTION_IN:
+ return a.defaultInPolicy
+ case POLICY_DIRECTION_IMPORT:
+ return a.defaultImportPolicy
+ case POLICY_DIRECTION_EXPORT:
+ return a.defaultExportPolicy
+ default:
+ return ROUTE_TYPE_NONE
+ }
+
+}
+
+func (r *RoutingPolicy) SetPolicy(id string, dir PolicyDirection, policies []*Policy) error {
+ a, ok := r.AssignmentMap[id]
+ if !ok {
+ a = &Assignment{}
+ }
+ switch dir {
+ case POLICY_DIRECTION_IN:
+ a.inPolicies = policies
+ case POLICY_DIRECTION_IMPORT:
+ a.importPolicies = policies
+ case POLICY_DIRECTION_EXPORT:
+ a.exportPolicies = policies
+ }
+ r.AssignmentMap[id] = a
+ return nil
+}
+
+func (r *RoutingPolicy) SetDefaultPolicy(id string, dir PolicyDirection, typ RouteType) error {
+ a, ok := r.AssignmentMap[id]
+ if !ok {
+ a = &Assignment{}
+ }
+ switch dir {
+ case POLICY_DIRECTION_IN:
+ a.defaultInPolicy = typ
+ case POLICY_DIRECTION_IMPORT:
+ a.defaultImportPolicy = typ
+ case POLICY_DIRECTION_EXPORT:
+ a.defaultExportPolicy = typ
+ }
+ r.AssignmentMap[id] = a
+ return nil
}
func (r *RoutingPolicy) GetAssignmentFromConfig(dir PolicyDirection, a config.ApplyPolicy) ([]*Policy, RouteType, error) {
@@ -2660,14 +2777,14 @@ func (r *RoutingPolicy) StatementInUse(x *Statement) bool {
return false
}
-func NewRoutingPolicy(c config.RoutingPolicy) (*RoutingPolicy, error) {
+func (r *RoutingPolicy) Reload(c config.RoutingPolicy) error {
dmap := make(map[DefinedType]map[string]DefinedSet)
dmap[DEFINED_TYPE_PREFIX] = make(map[string]DefinedSet)
d := c.DefinedSets
for _, x := range d.PrefixSets.PrefixSetList {
y, err := NewPrefixSet(x)
if err != nil {
- return nil, err
+ return err
}
dmap[DEFINED_TYPE_PREFIX][y.Name()] = y
}
@@ -2675,7 +2792,7 @@ func NewRoutingPolicy(c config.RoutingPolicy) (*RoutingPolicy, error) {
for _, x := range d.NeighborSets.NeighborSetList {
y, err := NewNeighborSet(x)
if err != nil {
- return nil, err
+ return err
}
dmap[DEFINED_TYPE_NEIGHBOR][y.Name()] = y
}
@@ -2692,7 +2809,7 @@ func NewRoutingPolicy(c config.RoutingPolicy) (*RoutingPolicy, error) {
for _, x := range bd.AsPathSets.AsPathSetList {
y, err := NewAsPathSet(x)
if err != nil {
- return nil, err
+ return err
}
dmap[DEFINED_TYPE_AS_PATH][y.Name()] = y
}
@@ -2700,7 +2817,7 @@ func NewRoutingPolicy(c config.RoutingPolicy) (*RoutingPolicy, error) {
for _, x := range bd.CommunitySets.CommunitySetList {
y, err := NewCommunitySet(x)
if err != nil {
- return nil, err
+ return err
}
dmap[DEFINED_TYPE_COMMUNITY][y.Name()] = y
}
@@ -2708,7 +2825,7 @@ func NewRoutingPolicy(c config.RoutingPolicy) (*RoutingPolicy, error) {
for _, x := range bd.ExtCommunitySets.ExtCommunitySetList {
y, err := NewExtCommunitySet(x)
if err != nil {
- return nil, err
+ return err
}
dmap[DEFINED_TYPE_EXT_COMMUNITY][y.Name()] = y
}
@@ -2717,22 +2834,31 @@ func NewRoutingPolicy(c config.RoutingPolicy) (*RoutingPolicy, error) {
for _, x := range c.PolicyDefinitions.PolicyDefinitionList {
y, err := NewPolicy(x, dmap)
if err != nil {
- return nil, err
+ return err
}
pmap[y.Name()] = y
for _, s := range y.Statements {
_, ok := smap[s.Name]
if ok {
- return nil, fmt.Errorf("duplicated statement name. statement name must be unique.")
+ return fmt.Errorf("duplicated statement name. statement name must be unique.")
}
smap[s.Name] = s
}
}
+ r.DefinedSetMap = dmap
+ r.PolicyMap = pmap
+ r.StatementMap = smap
+ r.AssignmentMap = make(map[string]*Assignment)
+ return nil
+}
+
+func NewRoutingPolicy() *RoutingPolicy {
return &RoutingPolicy{
- DefinedSetMap: dmap,
- PolicyMap: pmap,
- StatementMap: smap,
- }, nil
+ DefinedSetMap: make(map[DefinedType]map[string]DefinedSet),
+ PolicyMap: make(map[string]*Policy),
+ StatementMap: make(map[string]*Statement),
+ AssignmentMap: make(map[string]*Assignment),
+ }
}
func CanImportToVrf(v *Vrf, path *Path) bool {
diff --git a/table/policy_test.go b/table/policy_test.go
index 92f6142b..c29b15c1 100644
--- a/table/policy_test.go
+++ b/table/policy_test.go
@@ -238,7 +238,8 @@ func TestPolicyNotMatch(t *testing.T) {
pd := createPolicyDefinition("pd1", s)
pl := createRoutingPolicy(ds, pd)
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
pType, newPath := r.PolicyMap["pd1"].Apply(path)
assert.Equal(t, ROUTE_TYPE_NONE, pType)
@@ -268,7 +269,8 @@ func TestPolicyMatchAndReject(t *testing.T) {
pd := createPolicyDefinition("pd1", s)
pl := createRoutingPolicy(ds, pd)
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
pType, newPath := r.PolicyMap["pd1"].Apply(path)
assert.Equal(t, ROUTE_TYPE_REJECT, pType)
@@ -299,7 +301,8 @@ func TestPolicyMatchAndAccept(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
pType, newPath := r.PolicyMap["pd1"].Apply(path)
assert.Equal(t, ROUTE_TYPE_ACCEPT, pType)
@@ -340,7 +343,8 @@ func TestPolicyRejectOnlyPrefixSet(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
pType, newPath := p.Apply(path1)
@@ -386,7 +390,8 @@ func TestPolicyRejectOnlyNeighborSet(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
pType, newPath := r.PolicyMap["pd1"].Apply(path1)
assert.Equal(t, ROUTE_TYPE_REJECT, pType)
@@ -438,7 +443,8 @@ func TestPolicyDifferentRoutefamilyOfPathAndPolicy(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
pType1, newPath1 := p.Apply(pathIPv4)
@@ -537,7 +543,8 @@ func TestAsPathLengthConditionWithOtherCondition(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
pType, newPath := p.Apply(path)
@@ -654,7 +661,8 @@ func TestAs4PathLengthConditionWithOtherCondition(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, _ := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ r.Reload(pl)
p, _ := NewPolicy(pl.PolicyDefinitions.PolicyDefinitionList[0], r.DefinedSetMap)
pType, newPath := p.Apply(path)
assert.Equal(t, ROUTE_TYPE_REJECT, pType)
@@ -997,7 +1005,8 @@ func TestAsPathConditionWithOtherCondition(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
pType, newPath := p.Apply(path)
@@ -1302,7 +1311,8 @@ func TestAs4PathConditionWithOtherCondition(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, _ := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ r.Reload(pl)
p, _ := NewPolicy(pl.PolicyDefinitions.PolicyDefinitionList[0], r.DefinedSetMap)
pType, newPath := p.Apply(path)
assert.Equal(t, ROUTE_TYPE_REJECT, pType)
@@ -1661,7 +1671,8 @@ func TestCommunityConditionEvaluateWithOtherCondition(t *testing.T) {
pl := createRoutingPolicy(ds, pd1, pd2)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
pType, newPath := p.Apply(path)
@@ -1705,7 +1716,8 @@ func TestPolicyMatchAndAddCommunities(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
@@ -1749,7 +1761,8 @@ func TestPolicyMatchAndReplaceCommunities(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
@@ -1793,7 +1806,8 @@ func TestPolicyMatchAndRemoveCommunities(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
pType, newPath := p.Apply(path)
@@ -1838,7 +1852,8 @@ func TestPolicyMatchAndRemoveCommunitiesRegexp(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
pType, newPath := p.Apply(path)
@@ -1883,7 +1898,8 @@ func TestPolicyMatchAndRemoveCommunitiesRegexp2(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
pType, newPath := p.Apply(path)
@@ -1928,7 +1944,8 @@ func TestPolicyMatchAndClearCommunities(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
@@ -2262,7 +2279,8 @@ func TestExtCommunityConditionEvaluateWithOtherCondition(t *testing.T) {
pd2 := createPolicyDefinition("pd2", s2)
pl := createRoutingPolicy(ds, pd1, pd2)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
pType, newPath := p.Apply(path)
@@ -2306,7 +2324,8 @@ func TestPolicyMatchAndReplaceMed(t *testing.T) {
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
@@ -2349,7 +2368,8 @@ func TestPolicyMatchAndAddingMed(t *testing.T) {
pd := createPolicyDefinition("pd1", s)
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
pType, newPath := p.Apply(path)
@@ -2393,7 +2413,8 @@ func TestPolicyMatchAndAddingMedOverFlow(t *testing.T) {
pd := createPolicyDefinition("pd1", s)
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
@@ -2438,7 +2459,8 @@ func TestPolicyMatchAndSubtractMed(t *testing.T) {
pd := createPolicyDefinition("pd1", s)
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
@@ -2483,7 +2505,8 @@ func TestPolicyMatchAndSubtractMedUnderFlow(t *testing.T) {
pd := createPolicyDefinition("pd1", s)
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
@@ -2525,7 +2548,8 @@ func TestPolicyMatchWhenPathHaveNotMed(t *testing.T) {
pd := createPolicyDefinition("pd1", s)
pl := createRoutingPolicy(ds, pd)
//test
- r, err := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ err := r.Reload(pl)
assert.Nil(t, err)
p := r.PolicyMap["pd1"]
@@ -2572,8 +2596,8 @@ func TestPolicyAsPathPrepend(t *testing.T) {
pd := createPolicyDefinition("pd1", s)
pl := createRoutingPolicy(ds, pd)
//test
- r, _ := NewRoutingPolicy(pl)
- // assert.Nil(t, err)
+ r := NewRoutingPolicy()
+ r.Reload(pl)
p := r.PolicyMap["pd1"]
pType, newPath := p.Apply(path)
@@ -2616,8 +2640,8 @@ func TestPolicyAsPathPrependLastAs(t *testing.T) {
pd := createPolicyDefinition("pd1", s)
pl := createRoutingPolicy(ds, pd)
//test
- r, _ := NewRoutingPolicy(pl)
- // assert.Nil(t, err)
+ r := NewRoutingPolicy()
+ r.Reload(pl)
p := r.PolicyMap["pd1"]
pType, newPath := p.Apply(path)
@@ -2666,7 +2690,8 @@ func TestPolicyAs4PathPrepend(t *testing.T) {
pd := createPolicyDefinition("pd1", s)
pl := createRoutingPolicy(ds, pd)
//test
- r, _ := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ r.Reload(pl)
p, _ := NewPolicy(pl.PolicyDefinitions.PolicyDefinitionList[0], r.DefinedSetMap)
pType, newPath := p.Apply(path)
@@ -2720,7 +2745,8 @@ func TestPolicyAs4PathPrependLastAs(t *testing.T) {
pd := createPolicyDefinition("pd1", s)
pl := createRoutingPolicy(ds, pd)
//test
- r, _ := NewRoutingPolicy(pl)
+ r := NewRoutingPolicy()
+ r.Reload(pl)
p, _ := NewPolicy(pl.PolicyDefinitions.PolicyDefinitionList[0], r.DefinedSetMap)
pType, newPath := p.Apply(path)
diff --git a/table/table.go b/table/table.go
index cf14d5c1..36a29ed4 100644
--- a/table/table.go
+++ b/table/table.go
@@ -51,28 +51,28 @@ func (t *Table) insert(path *Path) *Destination {
}
func (t *Table) DeleteDestByPeer(peerInfo *PeerInfo) []*Destination {
- changedDests := make([]*Destination, 0)
- for _, dest := range t.destinations {
- newKnownPathList := make([]*Path, 0)
- for _, p := range dest.GetKnownPathList() {
- if !p.GetSource().Equal(peerInfo) {
- newKnownPathList = append(newKnownPathList, p)
+ dsts := []*Destination{}
+ for _, dst := range t.destinations {
+ match := false
+ for _, p := range dst.knownPathList {
+ if p.GetSource().Equal(peerInfo) {
+ dst.addWithdraw(p)
+ match = true
}
}
- if len(newKnownPathList) != len(dest.GetKnownPathList()) {
- changedDests = append(changedDests, dest)
- dest.setKnownPathList(newKnownPathList)
+ if match {
+ dsts = append(dsts, dst)
}
}
- return changedDests
+ return dsts
}
func (t *Table) deletePathsByVrf(vrf *Vrf) []*Path {
pathList := make([]*Path, 0)
for _, dest := range t.destinations {
- for _, p := range dest.GetKnownPathList() {
+ for _, p := range dest.knownPathList {
var rd bgp.RouteDistinguisherInterface
- nlri := p.GetNlri()
+ nlri := p.nlri
switch nlri.(type) {
case *bgp.LabeledVPNIPAddrPrefix:
rd = nlri.(*bgp.LabeledVPNIPAddrPrefix).RD
@@ -104,7 +104,7 @@ func (t *Table) deleteRTCPathsByVrf(vrf *Vrf, vrfs map[string]*Vrf) []*Path {
nlri := dest.GetNlri().(*bgp.RouteTargetMembershipNLRI)
rhs := nlri.RouteTarget.String()
if lhs == rhs && isLastTargetUser(vrfs, target) {
- for _, p := range dest.GetKnownPathList() {
+ for _, p := range dest.knownPathList {
if p.IsLocal() {
p.IsWithdraw = true
pathList = append(pathList, p)
@@ -210,3 +210,22 @@ func (t *Table) setDestination(key string, dest *Destination) {
func (t *Table) tableKey(nlri bgp.AddrPrefixInterface) string {
return nlri.String()
}
+
+func (t *Table) Bests(id string) []*Path {
+ paths := make([]*Path, 0, len(t.destinations))
+ for _, dst := range t.destinations {
+ path := dst.GetBestPath(id)
+ if path != nil {
+ paths = append(paths, path)
+ }
+ }
+ return paths
+}
+
+func (t *Table) GetKnownPathList(id string) []*Path {
+ paths := make([]*Path, 0, len(t.destinations))
+ for _, dst := range t.destinations {
+ paths = append(paths, dst.GetKnownPathList(id)...)
+ }
+ return paths
+}
diff --git a/table/table_manager.go b/table/table_manager.go
index 637299b9..cc73c89b 100644
--- a/table/table_manager.go
+++ b/table/table_manager.go
@@ -24,6 +24,10 @@ import (
"time"
)
+const (
+ GLOBAL_RIB_NAME = "global"
+)
+
func nlri2Path(m *bgp.BGPMessage, p *PeerInfo, now time.Time) []*Path {
updateMsg := m.Body.(*bgp.BGPUpdate)
pathAttributes := updateMsg.PathAttributes
@@ -105,17 +109,13 @@ func ProcessMessage(m *bgp.BGPMessage, peerInfo *PeerInfo, timestamp time.Time)
}
type TableManager struct {
- Tables map[bgp.RouteFamily]*Table
- Vrfs map[string]*Vrf
- owner string
- minLabel uint32
- maxLabel uint32
- nextLabel uint32
- rfList []bgp.RouteFamily
- importPolicies []*Policy
- defaultImportPolicy RouteType
- exportPolicies []*Policy
- defaultExportPolicy RouteType
+ Tables map[bgp.RouteFamily]*Table
+ Vrfs map[string]*Vrf
+ owner string
+ minLabel uint32
+ maxLabel uint32
+ nextLabel uint32
+ rfList []bgp.RouteFamily
}
func NewTableManager(owner string, rfList []bgp.RouteFamily, minLabel, maxLabel uint32) *TableManager {
@@ -138,82 +138,6 @@ func (manager *TableManager) GetRFlist() []bgp.RouteFamily {
return manager.rfList
}
-func (manager *TableManager) GetPolicy(d PolicyDirection) []*Policy {
- switch d {
- case POLICY_DIRECTION_IMPORT:
- return manager.importPolicies
- case POLICY_DIRECTION_EXPORT:
- return manager.exportPolicies
- }
- return nil
-}
-
-func (manager *TableManager) SetPolicy(d PolicyDirection, policies []*Policy) error {
- switch d {
- case POLICY_DIRECTION_IMPORT:
- manager.importPolicies = policies
- case POLICY_DIRECTION_EXPORT:
- manager.exportPolicies = policies
- default:
- return fmt.Errorf("unsupported policy type: %d", d)
- }
- return nil
-}
-
-func (manager *TableManager) GetDefaultPolicy(d PolicyDirection) RouteType {
- switch d {
- case POLICY_DIRECTION_IMPORT:
- return manager.defaultImportPolicy
- case POLICY_DIRECTION_EXPORT:
- return manager.defaultExportPolicy
- }
- return ROUTE_TYPE_NONE
-}
-
-func (manager *TableManager) SetDefaultPolicy(d PolicyDirection, typ RouteType) error {
- switch d {
- case POLICY_DIRECTION_IMPORT:
- manager.defaultImportPolicy = typ
- case POLICY_DIRECTION_EXPORT:
- manager.defaultExportPolicy = typ
- default:
- return fmt.Errorf("unsupported policy type: %d", d)
- }
- return nil
-}
-
-func (manager *TableManager) ApplyPolicy(d PolicyDirection, paths []*Path) []*Path {
- newpaths := make([]*Path, 0, len(paths))
- for _, path := range paths {
- result := ROUTE_TYPE_NONE
- newpath := path
- for _, p := range manager.GetPolicy(d) {
- result, newpath = p.Apply(path)
- if result != ROUTE_TYPE_NONE {
- break
- }
- }
-
- if result == ROUTE_TYPE_NONE {
- result = manager.GetDefaultPolicy(d)
- }
-
- switch result {
- case ROUTE_TYPE_ACCEPT:
- newpaths = append(newpaths, newpath)
- case ROUTE_TYPE_REJECT:
- path.Filtered = true
- log.WithFields(log.Fields{
- "Topic": "Peer",
- "Key": path.GetSource().Address,
- "Path": path,
- "Direction": d,
- }).Debug("reject")
- }
- }
- return newpaths
-}
-
func (manager *TableManager) GetNextLabel(name, nexthop string, isWithdraw bool) (uint32, error) {
var label uint32
var err error
@@ -295,122 +219,67 @@ func (manager *TableManager) DeleteVrf(name string) ([]*Path, error) {
return msgs, nil
}
-func (manager *TableManager) calculate(destinationList []*Destination) ([]*Path, error) {
- newPaths := make([]*Path, 0)
-
- for _, destination := range destinationList {
- // compute best path
-
+func (manager *TableManager) calculate(destinations []*Destination) {
+ for _, destination := range destinations {
log.WithFields(log.Fields{
"Topic": "table",
"Owner": manager.owner,
"Key": destination.GetNlri().String(),
}).Debug("Processing destination")
+ destination.Calculate()
+ }
+}
- newBestPath, reason, err := destination.Calculate()
-
- if err != nil {
- log.Error(err)
- continue
- }
-
- destination.setBestPathReason(reason)
- currentBestPath := destination.GetBestPath()
-
- if newBestPath != nil && newBestPath.Equal(currentBestPath) {
- // best path is not changed
- log.WithFields(log.Fields{
- "Topic": "table",
- "Owner": manager.owner,
- "Key": destination.GetNlri().String(),
- "peer": newBestPath.GetSource().Address,
- "next_hop": newBestPath.GetNexthop().String(),
- "reason": reason,
- }).Debug("best path is not changed")
- continue
- }
-
- if newBestPath == nil {
- log.WithFields(log.Fields{
- "Topic": "table",
- "Owner": manager.owner,
- "Key": destination.GetNlri().String(),
- }).Debug("best path is nil")
-
- if len(destination.GetKnownPathList()) == 0 {
- // create withdraw path
- if currentBestPath != nil {
- log.WithFields(log.Fields{
- "Topic": "table",
- "Owner": manager.owner,
- "Key": destination.GetNlri().String(),
- "peer": currentBestPath.GetSource().Address,
- "next_hop": currentBestPath.GetNexthop().String(),
- }).Debug("best path is lost")
-
- p := destination.GetBestPath()
- newPaths = append(newPaths, p.Clone(p.Owner, true))
- }
- destination.setBestPath(nil)
- } else {
- log.WithFields(log.Fields{
- "Topic": "table",
- "Owner": manager.owner,
- "Key": destination.GetNlri().String(),
- }).Error("known path list is not empty")
- }
- } else {
- log.WithFields(log.Fields{
- "Topic": "table",
- "Owner": manager.owner,
- "Key": newBestPath.GetNlri().String(),
- "peer": newBestPath.GetSource().Address,
- "next_hop": newBestPath.GetNexthop(),
- "reason": reason,
- }).Debug("new best path")
-
- newPaths = append(newPaths, newBestPath)
- destination.setBestPath(newBestPath)
- }
-
- if len(destination.GetKnownPathList()) == 0 && destination.GetBestPath() == nil {
- rf := destination.getRouteFamily()
- t := manager.Tables[rf]
- t.deleteDest(destination)
- log.WithFields(log.Fields{
- "Topic": "table",
- "Owner": manager.owner,
- "Key": destination.GetNlri().String(),
- "route_family": rf,
- }).Debug("destination removed")
- }
+func (manager *TableManager) DeletePathsByPeer(info *PeerInfo, rf bgp.RouteFamily) []*Destination {
+ if t, ok := manager.Tables[rf]; ok {
+ dsts := t.DeleteDestByPeer(info)
+ manager.calculate(dsts)
+ return dsts
}
- return newPaths, nil
+ return nil
}
-func (manager *TableManager) DeletePathsforPeer(peerInfo *PeerInfo, rf bgp.RouteFamily) ([]*Path, error) {
+func (manager *TableManager) DeletePathsforPeer(id string, peerInfo *PeerInfo, rf bgp.RouteFamily) ([]*Path, error) {
if t, ok := manager.Tables[rf]; ok {
destinationList := t.DeleteDestByPeer(peerInfo)
- return manager.calculate(destinationList)
+ manager.calculate(destinationList)
+ paths := make([]*Path, 0, len(destinationList))
+ for _, dst := range destinationList {
+ paths = append(paths, dst.NewFeed(id))
+ }
+ return paths, nil
}
- return []*Path{}, nil
+ return nil, nil
}
-func (manager *TableManager) ProcessPaths(pathList []*Path) ([]*Path, error) {
- destinationList := make([]*Destination, 0, len(pathList))
+func (manager *TableManager) ProcessPaths(pathList []*Path) []*Destination {
+ m := make(map[string]bool, len(pathList))
+ dsts := make([]*Destination, 0, len(pathList))
for _, path := range pathList {
+ if path == nil {
+ continue
+ }
rf := path.GetRouteFamily()
if t, ok := manager.Tables[rf]; ok {
- destinationList = append(destinationList, t.insert(path))
+ dst := t.insert(path)
+ key := dst.GetNlri().String()
+ if !m[key] {
+ m[key] = true
+ dsts = append(dsts, dst)
+ }
if rf == bgp.RF_EVPN {
- dsts := manager.handleMacMobility(path)
- if len(dsts) > 0 {
- destinationList = append(destinationList, dsts...)
+ for _, dst := range manager.handleMacMobility(path) {
+ key := dst.GetNlri().String()
+ if !m[key] {
+ m[key] = true
+ dsts = append(dsts, dst)
+ }
}
}
}
}
- return manager.calculate(destinationList)
+ manager.calculate(dsts)
+ return dsts
}
// EVPN MAC MOBILITY HANDLING
@@ -427,7 +296,7 @@ func (manager *TableManager) handleMacMobility(path *Path) []*Destination {
if path.IsWithdraw || path.IsLocal() || nlri.RouteType != bgp.EVPN_ROUTE_TYPE_MAC_IP_ADVERTISEMENT {
return nil
}
- for _, path2 := range manager.GetPathList(bgp.RF_EVPN) {
+ for _, path2 := range manager.GetPathList(GLOBAL_RIB_NAME, bgp.RF_EVPN) {
if !path2.IsLocal() || path2.GetNlri().(*bgp.EVPNNLRI).RouteType != bgp.EVPN_ROUTE_TYPE_MAC_IP_ADVERTISEMENT {
continue
}
@@ -464,27 +333,19 @@ func (manager *TableManager) getDestinationCount(rfList []bgp.RouteFamily) int {
return count
}
-func (manager *TableManager) GetPathList(rf bgp.RouteFamily) []*Path {
- if _, ok := manager.Tables[rf]; !ok {
- return []*Path{}
- }
- destinations := manager.Tables[rf].GetDestinations()
- paths := make([]*Path, 0, len(destinations))
- for _, dest := range destinations {
- paths = append(paths, dest.knownPathList...)
- }
- return paths
-}
-
-func (manager *TableManager) GetBestPathList(rfList []bgp.RouteFamily) []*Path {
+func (manager *TableManager) GetBestPathList(id string, rfList []bgp.RouteFamily) []*Path {
paths := make([]*Path, 0, manager.getDestinationCount(rfList))
for _, rf := range rfList {
- if _, ok := manager.Tables[rf]; ok {
- destinations := manager.Tables[rf].GetDestinations()
- for _, dest := range destinations {
- paths = append(paths, dest.GetBestPath())
- }
+ if t, ok := manager.Tables[rf]; ok {
+ paths = append(paths, t.Bests(id)...)
}
}
return paths
}
+
+func (manager *TableManager) GetPathList(id string, rf bgp.RouteFamily) []*Path {
+ if t, ok := manager.Tables[rf]; ok {
+ return t.GetKnownPathList(id)
+ }
+ return nil
+}
diff --git a/table/table_manager_test.go b/table/table_manager_test.go
index f0e465b2..e441fb97 100644
--- a/table/table_manager_test.go
+++ b/table/table_manager_test.go
@@ -30,7 +30,15 @@ import (
// this function processes only BGPUpdate
func (manager *TableManager) ProcessUpdate(fromPeer *PeerInfo, message *bgp.BGPMessage) ([]*Path, error) {
paths := ProcessMessage(message, fromPeer, time.Now())
- return manager.ProcessPaths(paths)
+ dsts := manager.ProcessPaths(paths)
+ paths2 := make([]*Path, 0, len(paths))
+ for _, dst := range dsts {
+ p := dst.NewFeed(GLOBAL_RIB_NAME)
+ if p != nil {
+ paths2 = append(paths2, p)
+ }
+ }
+ return paths2, nil
}
func getLogger(lv log.Level) *log.Logger {
@@ -2148,7 +2156,7 @@ func TestProcessBGPUpdate_Timestamp(t *testing.T) {
nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")}
- adjRib := NewAdjRib([]bgp.RouteFamily{bgp.RF_IPv4_UC, bgp.RF_IPv6_UC})
+ adjRib := NewAdjRib("test", []bgp.RouteFamily{bgp.RF_IPv4_UC, bgp.RF_IPv6_UC})
m1 := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri)
peer := peerR1()
pList1 := ProcessMessage(m1, peer, time.Now())
diff --git a/test/performance_test/main.go b/test/performance_test/main.go
index 6929f50d..b7c92a94 100644
--- a/test/performance_test/main.go
+++ b/test/performance_test/main.go
@@ -31,7 +31,7 @@ import (
func newPeer(g config.Global, p config.Neighbor, incoming chan *server.FsmMsg) *server.Peer {
tbl := table.NewTableManager(g.GlobalConfig.RouterId.String(), []bgp.RouteFamily{bgp.RF_IPv4_UC, bgp.RF_IPv6_UC}, 0, 0)
- peer := server.NewPeer(g, p, tbl)
+ peer := server.NewPeer(g, p, tbl, table.NewRoutingPolicy())
server.NewFSMHandler(peer.Fsm(), incoming, peer.Outgoing())
return peer
}