summaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
-rw-r--r--api/grpc_server.go47
-rw-r--r--client/client.go4
-rw-r--r--gobgp/lib/path.go2
-rw-r--r--server/peer.go231
-rw-r--r--server/rpki.go24
-rw-r--r--server/server.go333
-rw-r--r--server/server_test.go5
-rw-r--r--server/zclient.go2
-rw-r--r--table/destination.go14
-rw-r--r--table/policy.go10
10 files changed, 341 insertions, 331 deletions
diff --git a/api/grpc_server.go b/api/grpc_server.go
index cec1c272..4d366482 100644
--- a/api/grpc_server.go
+++ b/api/grpc_server.go
@@ -369,7 +369,7 @@ func NewValidationFromTableStruct(v *table.Validation) *RPKIValidation {
}
}
-func ToPathApi(path *table.Path) *Path {
+func ToPathApi(path *table.Path, v *table.Validation) *Path {
nlri := path.GetNlri()
n, _ := nlri.Serialize()
family := uint32(bgp.AfiSafiToRouteFamily(nlri.AFI(), nlri.SAFI()))
@@ -381,13 +381,18 @@ func ToPathApi(path *table.Path) *Path {
}
return ret
}(path.GetPathAttrs())
+ vv := config.RPKI_VALIDATION_RESULT_TYPE_NONE.ToInt()
+ if v != nil {
+ vv = v.Status.ToInt()
+ }
+
p := &Path{
Nlri: n,
Pattrs: pattrs,
Age: path.GetTimestamp().Unix(),
IsWithdraw: path.IsWithdraw,
- Validation: int32(path.ValidationStatus().ToInt()),
- ValidationDetail: NewValidationFromTableStruct(path.Validation()),
+ Validation: int32(vv),
+ ValidationDetail: NewValidationFromTableStruct(v),
Family: family,
Stale: path.IsStale(),
IsFromExternal: path.IsFromExternal(),
@@ -404,6 +409,14 @@ func ToPathApi(path *table.Path) *Path {
return p
}
+func getValidation(v []*table.Validation, i int) *table.Validation {
+ if v == nil {
+ return nil
+ } else {
+ return v[i]
+ }
+}
+
func (s *Server) GetRib(ctx context.Context, arg *GetRibRequest) (*GetRibResponse, error) {
if arg == nil || arg.Table == nil {
return nil, fmt.Errorf("invalid request")
@@ -429,16 +442,17 @@ func (s *Server) GetRib(ctx context.Context, arg *GetRibRequest) (*GetRibRespons
var in bool
var err error
var tbl *table.Table
+ var v []*table.Validation
family := bgp.RouteFamily(arg.Table.Family)
switch arg.Table.Type {
case Resource_LOCAL, Resource_GLOBAL:
- tbl, err = s.bgpServer.GetRib(arg.Table.Name, family, f())
+ tbl, v, err = s.bgpServer.GetRib(arg.Table.Name, family, f())
case Resource_ADJ_IN:
in = true
fallthrough
case Resource_ADJ_OUT:
- tbl, err = s.bgpServer.GetAdjRib(arg.Table.Name, family, in, f())
+ tbl, v, err = s.bgpServer.GetAdjRib(arg.Table.Name, family, in, f())
case Resource_VRF:
tbl, err = s.bgpServer.GetVrfRib(arg.Table.Name, family, []*table.LookupPrefix{})
default:
@@ -451,13 +465,15 @@ func (s *Server) GetRib(ctx context.Context, arg *GetRibRequest) (*GetRibRespons
tblDsts := tbl.GetDestinations()
dsts := make([]*Destination, 0, len(tblDsts))
+ idx := 0
for _, dst := range tblDsts {
dsts = append(dsts, &Destination{
Prefix: dst.GetNlri().String(),
Paths: func(paths []*table.Path) []*Path {
l := make([]*Path, 0, len(paths))
for i, p := range paths {
- pp := ToPathApi(p)
+ pp := ToPathApi(p, getValidation(v, idx))
+ idx++
switch arg.Table.Type {
case Resource_LOCAL, Resource_GLOBAL:
if i == 0 && !table.SelectionOptions.DisableBestPathSelection {
@@ -494,14 +510,15 @@ func (s *Server) GetPath(arg *GetPathRequest, stream GobgpApi_GetPathServer) err
family := bgp.RouteFamily(arg.Family)
var tbl *table.Table
var err error
+ var v []*table.Validation
switch arg.Type {
case Resource_LOCAL, Resource_GLOBAL:
- tbl, err = s.bgpServer.GetRib(arg.Name, family, f())
+ tbl, v, err = s.bgpServer.GetRib(arg.Name, family, f())
case Resource_ADJ_IN:
in = true
fallthrough
case Resource_ADJ_OUT:
- tbl, err = s.bgpServer.GetAdjRib(arg.Name, family, in, f())
+ tbl, v, err = s.bgpServer.GetAdjRib(arg.Name, family, in, f())
case Resource_VRF:
tbl, err = s.bgpServer.GetVrfRib(arg.Name, family, []*table.LookupPrefix{})
default:
@@ -511,11 +528,13 @@ func (s *Server) GetPath(arg *GetPathRequest, stream GobgpApi_GetPathServer) err
return err
}
+ idx := 0
return func() error {
for _, dst := range tbl.GetDestinations() {
- for idx, path := range dst.GetAllKnownPathList() {
- p := ToPathApi(path)
- if idx == 0 && !table.SelectionOptions.DisableBestPathSelection {
+ for i, path := range dst.GetAllKnownPathList() {
+ p := ToPathApi(path, getValidation(v, idx))
+ idx++
+ if i == 0 && !table.SelectionOptions.DisableBestPathSelection {
switch arg.Type {
case Resource_LOCAL, Resource_GLOBAL:
p.Best = true
@@ -562,11 +581,11 @@ func (s *Server) MonitorRib(arg *MonitorRibRequest, stream GobgpApi_MonitorRibSe
continue
}
if dst, y := dsts[path.GetNlri().String()]; y {
- dst.Paths = append(dst.Paths, ToPathApi(path))
+ dst.Paths = append(dst.Paths, ToPathApi(path, nil))
} else {
dsts[path.GetNlri().String()] = &Destination{
Prefix: path.GetNlri().String(),
- Paths: []*Path{ToPathApi(path)},
+ Paths: []*Path{ToPathApi(path, nil)},
}
}
}
@@ -863,7 +882,7 @@ func (s *Server) DeleteBmp(ctx context.Context, arg *DeleteBmpRequest) (*DeleteB
}
func (s *Server) ValidateRib(ctx context.Context, arg *ValidateRibRequest) (*ValidateRibResponse, error) {
- return &ValidateRibResponse{}, s.bgpServer.ValidateRib(arg.Prefix)
+ return &ValidateRibResponse{}, nil
}
func (s *Server) AddRpki(ctx context.Context, arg *AddRpkiRequest) (*AddRpkiResponse, error) {
diff --git a/client/client.go b/client/client.go
index c31ac2b3..d2f56cd8 100644
--- a/client/client.go
+++ b/client/client.go
@@ -354,7 +354,7 @@ type AddPathByStreamClient struct {
func (c *AddPathByStreamClient) Send(paths ...*table.Path) error {
ps := make([]*api.Path, 0, len(paths))
for _, p := range paths {
- ps = append(ps, api.ToPathApi(p))
+ ps = append(ps, api.ToPathApi(p, nil))
}
return c.stream.Send(&api.InjectMrtRequest{
Resource: api.Resource_GLOBAL,
@@ -385,7 +385,7 @@ func (cli *Client) addPath(vrfID string, pathList []*table.Path) ([]byte, error)
r, err := cli.cli.AddPath(context.Background(), &api.AddPathRequest{
Resource: resource,
VrfId: vrfID,
- Path: api.ToPathApi(path),
+ Path: api.ToPathApi(path, nil),
})
if err != nil {
return nil, err
diff --git a/gobgp/lib/path.go b/gobgp/lib/path.go
index 67cf8b06..c74a7bc0 100644
--- a/gobgp/lib/path.go
+++ b/gobgp/lib/path.go
@@ -58,7 +58,7 @@ func serialize_path(rf C.int, input *C.char) *C.path {
return nil
}
path := C.new_path()
- p := api.ToPathApi(pp)
+ p := api.ToPathApi(pp, nil)
if len(p.Nlri) > 0 {
path.nlri.len = C.int(len(p.Nlri))
path.nlri.value = C.CString(string(p.Nlri))
diff --git a/server/peer.go b/server/peer.go
index 3bdd9965..80fc6a1b 100644
--- a/server/peer.go
+++ b/server/peer.go
@@ -313,105 +313,6 @@ func (peer *Peer) getAccepted(rfList []bgp.RouteFamily) []*table.Path {
return peer.adjRibIn.PathList(rfList, true)
}
-func (peer *Peer) filterpath(path, old *table.Path) *table.Path {
- // Special handling for RTM NLRI.
- if path != nil && path.GetRouteFamily() == bgp.RF_RTC_UC && !path.IsWithdraw {
- // If the given "path" is locally generated and the same with "old", we
- // assumes "path" was already sent before. This assumption avoids the
- // infinite UPDATE loop between Route Reflector and its clients.
- if path.IsLocal() && path == old {
- log.WithFields(log.Fields{
- "Topic": "Peer",
- "Key": peer.fsm.pConf.State.NeighborAddress,
- "Path": path,
- }).Debug("given rtm nlri is already sent, skipping to advertise")
- return nil
- }
-
- if old != nil && old.IsLocal() {
- // We assumes VRF with the specific RT is deleted.
- path = old.Clone(true)
- } else if peer.isRouteReflectorClient() {
- // We need to send the path even if the peer is originator of the
- // path in order to signal that the client should distribute route
- // with the given RT.
- } else {
- // We send a path even if it is not the best path. See comments in
- // (*Destination) GetChanges().
- dst := peer.localRib.GetDestination(path)
- path = nil
- for _, p := range dst.GetKnownPathList(peer.TableID(), peer.AS()) {
- // Just take care not to send back.
- if peer.ID() != p.GetSource().Address.String() {
- path = p
- break
- }
- }
- }
- }
-
- // only allow vpnv4 and vpnv6 paths to be advertised to VRFed neighbors.
- // also check we can import this path using table.CanImportToVrf()
- // if we can, make it local path by calling (*Path).ToLocal()
- if path != nil && peer.fsm.pConf.Config.Vrf != "" {
- if f := path.GetRouteFamily(); f != bgp.RF_IPv4_VPN && f != bgp.RF_IPv6_VPN {
- return nil
- }
- vrf := peer.localRib.Vrfs[peer.fsm.pConf.Config.Vrf]
- if table.CanImportToVrf(vrf, path) {
- path = path.ToLocal()
- } else {
- return nil
- }
- }
-
- // replace-peer-as handling
- if path != nil && !path.IsWithdraw && peer.fsm.pConf.AsPathOptions.State.ReplacePeerAs {
- path = path.ReplaceAS(peer.fsm.pConf.Config.LocalAs, peer.fsm.pConf.Config.PeerAs)
- }
-
- if path = filterpath(peer, path, old); path == nil {
- return nil
- }
-
- path = table.UpdatePathAttrs(peer.fsm.gConf, peer.fsm.pConf, peer.fsm.peerInfo, path)
-
- options := &table.PolicyOptions{
- Info: peer.fsm.peerInfo,
- }
- path = peer.policy.ApplyPolicy(peer.TableID(), table.POLICY_DIRECTION_EXPORT, path, options)
- // When 'path' is filtered (path == nil), check 'old' has been sent to this peer.
- // If it has, send withdrawal to the peer.
- if path == nil && old != nil {
- o := peer.policy.ApplyPolicy(peer.TableID(), table.POLICY_DIRECTION_EXPORT, old, options)
- if o != nil {
- path = old.Clone(true)
- }
- }
-
- // draft-uttaro-idr-bgp-persistence-02
- // 4.3. Processing LLGR_STALE Routes
- //
- // The route SHOULD NOT be advertised to any neighbor from which the
- // Long-lived Graceful Restart Capability has not been received. The
- // exception is described in the Optional Partial Deployment
- // Procedure section (Section 4.7). Note that this requirement
- // implies that such routes should be withdrawn from any such neighbor.
- if path != nil && !path.IsWithdraw && !peer.isLLGREnabledFamily(path.GetRouteFamily()) && path.IsLLGRStale() {
- // we send unnecessary withdrawn even if we didn't
- // sent the route.
- path = path.Clone(true)
- }
-
- // remove local-pref attribute
- // we should do this after applying export policy since policy may
- // set local-preference
- if path != nil && !peer.isIBGPPeer() && !peer.isRouteServerClient() {
- path.RemoveLocalPref()
- }
- return path
-}
-
func (peer *Peer) filterPathFromSourcePeer(path, old *table.Path) *table.Path {
if peer.ID() != path.GetSource().Address.String() {
return path
@@ -447,86 +348,6 @@ func (peer *Peer) filterPathFromSourcePeer(path, old *table.Path) *table.Path {
return nil
}
-func (peer *Peer) getBestFromLocal(rfList []bgp.RouteFamily) ([]*table.Path, []*table.Path) {
- pathList := []*table.Path{}
- filtered := []*table.Path{}
- for _, family := range peer.toGlobalFamilies(rfList) {
- pl := func() []*table.Path {
- if peer.isAddPathSendEnabled(family) {
- return peer.localRib.GetPathList(peer.TableID(), peer.AS(), []bgp.RouteFamily{family})
- }
- return peer.localRib.GetBestPathList(peer.TableID(), peer.AS(), []bgp.RouteFamily{family})
- }()
- for _, path := range pl {
- if p := peer.filterpath(path, nil); p != nil {
- pathList = append(pathList, p)
- } else {
- filtered = append(filtered, path)
- }
- }
- }
- if peer.isGracefulRestartEnabled() {
- for _, family := range rfList {
- pathList = append(pathList, table.NewEOR(family))
- }
- }
- return pathList, filtered
-}
-
-func (peer *Peer) processOutgoingPaths(paths, olds []*table.Path) []*table.Path {
- if peer.fsm.state != bgp.BGP_FSM_ESTABLISHED {
- return nil
- }
- if peer.fsm.pConf.GracefulRestart.State.LocalRestarting {
- log.WithFields(log.Fields{
- "Topic": "Peer",
- "Key": peer.fsm.pConf.State.NeighborAddress,
- }).Debug("now syncing, suppress sending updates")
- return nil
- }
-
- outgoing := make([]*table.Path, 0, len(paths))
-
- for idx, path := range paths {
- var old *table.Path
- if olds != nil {
- old = olds[idx]
- }
- if p := peer.filterpath(path, old); p != nil {
- outgoing = append(outgoing, p)
- }
- }
- return outgoing
-}
-
-func (peer *Peer) handleRouteRefresh(e *FsmMsg) []*table.Path {
- m := e.MsgData.(*bgp.BGPMessage)
- rr := m.Body.(*bgp.BGPRouteRefresh)
- rf := bgp.AfiSafiToRouteFamily(rr.AFI, rr.SAFI)
- if _, ok := peer.fsm.rfMap[rf]; !ok {
- log.WithFields(log.Fields{
- "Topic": "Peer",
- "Key": peer.ID(),
- "Data": rf,
- }).Warn("Route family isn't supported")
- return nil
- }
- if _, ok := peer.fsm.capMap[bgp.BGP_CAP_ROUTE_REFRESH]; !ok {
- log.WithFields(log.Fields{
- "Topic": "Peer",
- "Key": peer.ID(),
- }).Warn("ROUTE_REFRESH received but the capability wasn't advertised")
- return nil
- }
- rfList := []bgp.RouteFamily{rf}
- accepted, filtered := peer.getBestFromLocal(rfList)
- for _, path := range filtered {
- path.IsWithdraw = true
- accepted = append(accepted, path)
- }
- return accepted
-}
-
func (peer *Peer) doPrefixLimit(k bgp.RouteFamily, c *config.PrefixLimitConfig) *bgp.BGPMessage {
if maxPrefixes := int(c.MaxPrefixes); maxPrefixes > 0 {
count := peer.adjRibIn.Count([]bgp.RouteFamily{k})
@@ -652,58 +473,6 @@ func (peer *Peer) PassConn(conn *net.TCPConn) {
}
}
-func (peer *Peer) ToConfig(getAdvertised bool) *config.Neighbor {
- // create copy which can be access to without mutex
- conf := *peer.fsm.pConf
-
- conf.AfiSafis = make([]config.AfiSafi, len(peer.fsm.pConf.AfiSafis))
- for i, af := range peer.fsm.pConf.AfiSafis {
- conf.AfiSafis[i] = af
- conf.AfiSafis[i].AddPaths.State.Receive = peer.isAddPathReceiveEnabled(af.State.Family)
- if peer.isAddPathSendEnabled(af.State.Family) {
- conf.AfiSafis[i].AddPaths.State.SendMax = af.AddPaths.State.SendMax
- } else {
- conf.AfiSafis[i].AddPaths.State.SendMax = 0
- }
- }
-
- remoteCap := make([]bgp.ParameterCapabilityInterface, 0, len(peer.fsm.capMap))
- for _, caps := range peer.fsm.capMap {
- for _, m := range caps {
- // need to copy all values here
- buf, _ := m.Serialize()
- c, _ := bgp.DecodeCapability(buf)
- remoteCap = append(remoteCap, c)
- }
- }
- conf.State.RemoteCapabilityList = remoteCap
- conf.State.LocalCapabilityList = capabilitiesFromConfig(peer.fsm.pConf)
-
- conf.State.SessionState = config.IntToSessionStateMap[int(peer.fsm.state)]
- conf.State.AdminState = config.IntToAdminStateMap[int(peer.fsm.adminState)]
-
- if peer.fsm.state == bgp.BGP_FSM_ESTABLISHED {
- rfList := peer.configuredRFlist()
- if getAdvertised {
- pathList, filtered := peer.getBestFromLocal(rfList)
- conf.State.AdjTable.Advertised = uint32(len(pathList))
- conf.State.AdjTable.Filtered = uint32(len(filtered))
- } else {
- conf.State.AdjTable.Advertised = 0
- }
- conf.State.AdjTable.Received = uint32(peer.adjRibIn.Count(rfList))
- conf.State.AdjTable.Accepted = uint32(peer.adjRibIn.Accepted(rfList))
-
- conf.Transport.State.LocalAddress, conf.Transport.State.LocalPort = peer.fsm.LocalHostPort()
- _, conf.Transport.State.RemotePort = peer.fsm.RemoteHostPort()
- buf, _ := peer.fsm.recvOpen.Serialize()
- // need to copy all values here
- conf.State.ReceivedOpenMessage, _ = bgp.ParseBGPMessage(buf)
- conf.State.RemoteRouterId = peer.fsm.peerInfo.ID.To4().String()
- }
- return &conf
-}
-
func (peer *Peer) DropAll(rfList []bgp.RouteFamily) {
peer.adjRibIn.Drop(rfList)
}
diff --git a/server/rpki.go b/server/rpki.go
index 371fda4b..6356888a 100644
--- a/server/rpki.go
+++ b/server/rpki.go
@@ -112,6 +112,10 @@ func NewROAManager(as uint32) (*roaManager, error) {
return m, nil
}
+func (c *roaManager) enabled() bool {
+ return len(c.clientMap) != 0
+}
+
func (m *roaManager) SetAS(as uint32) error {
if m.AS != 0 {
return fmt.Errorf("AS was already configured")
@@ -570,21 +574,15 @@ func ValidatePath(ownAs uint32, tree *radix.Tree, cidr string, asPath *bgp.PathA
return validation
}
-func (c *roaManager) validate(pathList []*table.Path) {
- if len(c.clientMap) == 0 {
- // RPKI isn't enabled
- return
+func (c *roaManager) validate(path *table.Path) *table.Validation {
+ if len(c.clientMap) == 0 || path.IsWithdraw || path.IsEOR() {
+ // RPKI isn't enabled or invalid path
+ return nil
}
-
- for _, path := range pathList {
- if path.IsWithdraw || path.IsEOR() {
- continue
- }
- if tree, ok := c.Roas[path.GetRouteFamily()]; ok {
- v := ValidatePath(c.AS, tree, path.GetNlri().String(), path.GetAsPath())
- path.SetValidation(v)
- }
+ if tree, ok := c.Roas[path.GetRouteFamily()]; ok {
+ return ValidatePath(c.AS, tree, path.GetNlri().String(), path.GetAsPath())
}
+ return nil
}
type roaClient struct {
diff --git a/server/server.go b/server/server.go
index 75d71a6b..3c7bd336 100644
--- a/server/server.go
+++ b/server/server.go
@@ -463,6 +463,109 @@ func filterpath(peer *Peer, path, old *table.Path) *table.Path {
return path
}
+func (s *BgpServer) filterpath(peer *Peer, path, old *table.Path) *table.Path {
+ // Special handling for RTM NLRI.
+ if path != nil && path.GetRouteFamily() == bgp.RF_RTC_UC && !path.IsWithdraw {
+ // If the given "path" is locally generated and the same with "old", we
+ // assumes "path" was already sent before. This assumption avoids the
+ // infinite UPDATE loop between Route Reflector and its clients.
+ if path.IsLocal() && path == old {
+ log.WithFields(log.Fields{
+ "Topic": "Peer",
+ "Key": peer.fsm.pConf.State.NeighborAddress,
+ "Path": path,
+ }).Debug("given rtm nlri is already sent, skipping to advertise")
+ return nil
+ }
+
+ if old != nil && old.IsLocal() {
+ // We assumes VRF with the specific RT is deleted.
+ path = old.Clone(true)
+ } else if peer.isRouteReflectorClient() {
+ // We need to send the path even if the peer is originator of the
+ // path in order to signal that the client should distribute route
+ // with the given RT.
+ } else {
+ // We send a path even if it is not the best path. See comments in
+ // (*Destination) GetChanges().
+ dst := peer.localRib.GetDestination(path)
+ path = nil
+ for _, p := range dst.GetKnownPathList(peer.TableID(), peer.AS()) {
+ // Just take care not to send back.
+ if peer.ID() != p.GetSource().Address.String() {
+ path = p
+ break
+ }
+ }
+ }
+ }
+
+ // only allow vpnv4 and vpnv6 paths to be advertised to VRFed neighbors.
+ // also check we can import this path using table.CanImportToVrf()
+ // if we can, make it local path by calling (*Path).ToLocal()
+ if path != nil && peer.fsm.pConf.Config.Vrf != "" {
+ if f := path.GetRouteFamily(); f != bgp.RF_IPv4_VPN && f != bgp.RF_IPv6_VPN {
+ return nil
+ }
+ vrf := peer.localRib.Vrfs[peer.fsm.pConf.Config.Vrf]
+ if table.CanImportToVrf(vrf, path) {
+ path = path.ToLocal()
+ } else {
+ return nil
+ }
+ }
+
+ // replace-peer-as handling
+ if path != nil && !path.IsWithdraw && peer.fsm.pConf.AsPathOptions.State.ReplacePeerAs {
+ path = path.ReplaceAS(peer.fsm.pConf.Config.LocalAs, peer.fsm.pConf.Config.PeerAs)
+ }
+
+ if path = filterpath(peer, path, old); path == nil {
+ return nil
+ }
+
+ path = table.UpdatePathAttrs(peer.fsm.gConf, peer.fsm.pConf, peer.fsm.peerInfo, path)
+
+ options := &table.PolicyOptions{
+ Info: peer.fsm.peerInfo,
+ }
+ if v := s.roaManager.validate(path); v != nil {
+ options.ValidationResult = v
+ }
+
+ path = peer.policy.ApplyPolicy(peer.TableID(), table.POLICY_DIRECTION_EXPORT, path, options)
+ // When 'path' is filtered (path == nil), check 'old' has been sent to this peer.
+ // If it has, send withdrawal to the peer.
+ if path == nil && old != nil {
+ o := peer.policy.ApplyPolicy(peer.TableID(), table.POLICY_DIRECTION_EXPORT, old, options)
+ if o != nil {
+ path = old.Clone(true)
+ }
+ }
+
+ // draft-uttaro-idr-bgp-persistence-02
+ // 4.3. Processing LLGR_STALE Routes
+ //
+ // The route SHOULD NOT be advertised to any neighbor from which the
+ // Long-lived Graceful Restart Capability has not been received. The
+ // exception is described in the Optional Partial Deployment
+ // Procedure section (Section 4.7). Note that this requirement
+ // implies that such routes should be withdrawn from any such neighbor.
+ if path != nil && !path.IsWithdraw && !peer.isLLGREnabledFamily(path.GetRouteFamily()) && path.IsLLGRStale() {
+ // we send unnecessary withdrawn even if we didn't
+ // sent the route.
+ path = path.Clone(true)
+ }
+
+ // remove local-pref attribute
+ // we should do this after applying export policy since policy may
+ // set local-preference
+ if path != nil && !peer.isIBGPPeer() && !peer.isRouteServerClient() {
+ path.RemoveLocalPref()
+ }
+ return path
+}
+
func clonePathList(pathList []*table.Path) []*table.Path {
l := make([]*table.Path, 0, len(pathList))
for _, p := range pathList {
@@ -501,6 +604,58 @@ func (server *BgpServer) notifyBestWatcher(best []*table.Path, multipath [][]*ta
server.notifyWatcher(WATCH_EVENT_TYPE_BEST_PATH, w)
}
+func (s *BgpServer) ToConfig(peer *Peer, getAdvertised bool) *config.Neighbor {
+ // create copy which can be access to without mutex
+ conf := *peer.fsm.pConf
+
+ conf.AfiSafis = make([]config.AfiSafi, len(peer.fsm.pConf.AfiSafis))
+ for i, af := range peer.fsm.pConf.AfiSafis {
+ conf.AfiSafis[i] = af
+ conf.AfiSafis[i].AddPaths.State.Receive = peer.isAddPathReceiveEnabled(af.State.Family)
+ if peer.isAddPathSendEnabled(af.State.Family) {
+ conf.AfiSafis[i].AddPaths.State.SendMax = af.AddPaths.State.SendMax
+ } else {
+ conf.AfiSafis[i].AddPaths.State.SendMax = 0
+ }
+ }
+
+ remoteCap := make([]bgp.ParameterCapabilityInterface, 0, len(peer.fsm.capMap))
+ for _, caps := range peer.fsm.capMap {
+ for _, m := range caps {
+ // need to copy all values here
+ buf, _ := m.Serialize()
+ c, _ := bgp.DecodeCapability(buf)
+ remoteCap = append(remoteCap, c)
+ }
+ }
+ conf.State.RemoteCapabilityList = remoteCap
+ conf.State.LocalCapabilityList = capabilitiesFromConfig(peer.fsm.pConf)
+
+ conf.State.SessionState = config.IntToSessionStateMap[int(peer.fsm.state)]
+ conf.State.AdminState = config.IntToAdminStateMap[int(peer.fsm.adminState)]
+
+ if peer.fsm.state == bgp.BGP_FSM_ESTABLISHED {
+ rfList := peer.configuredRFlist()
+ if getAdvertised {
+ pathList, filtered := s.getBestFromLocal(peer, rfList)
+ conf.State.AdjTable.Advertised = uint32(len(pathList))
+ conf.State.AdjTable.Filtered = uint32(len(filtered))
+ } else {
+ conf.State.AdjTable.Advertised = 0
+ }
+ conf.State.AdjTable.Received = uint32(peer.adjRibIn.Count(rfList))
+ conf.State.AdjTable.Accepted = uint32(peer.adjRibIn.Accepted(rfList))
+
+ conf.Transport.State.LocalAddress, conf.Transport.State.LocalPort = peer.fsm.LocalHostPort()
+ _, conf.Transport.State.RemotePort = peer.fsm.RemoteHostPort()
+ buf, _ := peer.fsm.recvOpen.Serialize()
+ // need to copy all values here
+ conf.State.ReceivedOpenMessage, _ = bgp.ParseBGPMessage(buf)
+ conf.State.RemoteRouterId = peer.fsm.peerInfo.ID.To4().String()
+ }
+ return &conf
+}
+
func (server *BgpServer) notifyPrePolicyUpdateWatcher(peer *Peer, pathList []*table.Path, msg *bgp.BGPMessage, timestamp time.Time, payload []byte) {
if !server.isWatched(WATCH_EVENT_TYPE_PRE_UPDATE) || peer == nil {
return
@@ -523,7 +678,7 @@ func (server *BgpServer) notifyPrePolicyUpdateWatcher(peer *Peer, pathList []*ta
Payload: payload,
PostPolicy: false,
PathList: cloned,
- Neighbor: peer.ToConfig(false),
+ Neighbor: server.ToConfig(peer, false),
}
server.notifyWatcher(WATCH_EVENT_TYPE_PRE_UPDATE, ev)
}
@@ -548,7 +703,7 @@ func (server *BgpServer) notifyPostPolicyUpdateWatcher(peer *Peer, pathList []*t
Timestamp: cloned[0].GetTimestamp(),
PostPolicy: true,
PathList: cloned,
- Neighbor: peer.ToConfig(false),
+ Neighbor: server.ToConfig(peer, false),
}
server.notifyWatcher(WATCH_EVENT_TYPE_POST_UPDATE, ev)
}
@@ -609,17 +764,89 @@ func (server *BgpServer) notifyRecvMessageWatcher(peer *Peer, timestamp time.Tim
server.notifyMessageWatcher(peer, timestamp, msg, false)
}
+func (s *BgpServer) getBestFromLocal(peer *Peer, rfList []bgp.RouteFamily) ([]*table.Path, []*table.Path) {
+ pathList := []*table.Path{}
+ filtered := []*table.Path{}
+ for _, family := range peer.toGlobalFamilies(rfList) {
+ pl := func() []*table.Path {
+ if peer.isAddPathSendEnabled(family) {
+ return peer.localRib.GetPathList(peer.TableID(), peer.AS(), []bgp.RouteFamily{family})
+ }
+ return peer.localRib.GetBestPathList(peer.TableID(), peer.AS(), []bgp.RouteFamily{family})
+ }()
+ for _, path := range pl {
+ if p := s.filterpath(peer, path, nil); p != nil {
+ pathList = append(pathList, p)
+ } else {
+ filtered = append(filtered, path)
+ }
+ }
+ }
+ if peer.isGracefulRestartEnabled() {
+ for _, family := range rfList {
+ pathList = append(pathList, table.NewEOR(family))
+ }
+ }
+ return pathList, filtered
+}
+
+func (s *BgpServer) processOutgoingPaths(peer *Peer, paths, olds []*table.Path) []*table.Path {
+ if peer.fsm.state != bgp.BGP_FSM_ESTABLISHED {
+ return nil
+ }
+ if peer.fsm.pConf.GracefulRestart.State.LocalRestarting {
+ log.WithFields(log.Fields{
+ "Topic": "Peer",
+ "Key": peer.fsm.pConf.State.NeighborAddress,
+ }).Debug("now syncing, suppress sending updates")
+ return nil
+ }
+
+ outgoing := make([]*table.Path, 0, len(paths))
+
+ for idx, path := range paths {
+ var old *table.Path
+ if olds != nil {
+ old = olds[idx]
+ }
+ if p := s.filterpath(peer, path, old); p != nil {
+ outgoing = append(outgoing, p)
+ }
+ }
+ return outgoing
+}
+
+func (s *BgpServer) handleRouteRefresh(peer *Peer, e *FsmMsg) []*table.Path {
+ m := e.MsgData.(*bgp.BGPMessage)
+ rr := m.Body.(*bgp.BGPRouteRefresh)
+ rf := bgp.AfiSafiToRouteFamily(rr.AFI, rr.SAFI)
+ if _, ok := peer.fsm.rfMap[rf]; !ok {
+ log.WithFields(log.Fields{
+ "Topic": "Peer",
+ "Key": peer.ID(),
+ "Data": rf,
+ }).Warn("Route family isn't supported")
+ return nil
+ }
+ if _, ok := peer.fsm.capMap[bgp.BGP_CAP_ROUTE_REFRESH]; !ok {
+ log.WithFields(log.Fields{
+ "Topic": "Peer",
+ "Key": peer.ID(),
+ }).Warn("ROUTE_REFRESH received but the capability wasn't advertised")
+ return nil
+ }
+ rfList := []bgp.RouteFamily{rf}
+ accepted, filtered := s.getBestFromLocal(peer, rfList)
+ for _, path := range filtered {
+ path.IsWithdraw = true
+ accepted = append(accepted, path)
+ }
+ return accepted
+}
+
func (server *BgpServer) propagateUpdate(peer *Peer, pathList []*table.Path) {
rs := peer != nil && peer.isRouteServerClient()
vrf := !rs && peer != nil && peer.fsm.pConf.Config.Vrf != ""
- var policyOptions *table.PolicyOptions
- if !rs && peer != nil {
- policyOptions = &table.PolicyOptions{
- Info: peer.fsm.peerInfo,
- }
- } else {
- policyOptions = nil
- }
tableId := table.GLOBAL_RIB_NAME
rib := server.globalRib
@@ -633,6 +860,15 @@ func (server *BgpServer) propagateUpdate(peer *Peer, pathList []*table.Path) {
path = path.ToGlobal(rib.Vrfs[peer.fsm.pConf.Config.Vrf])
}
+ policyOptions := &table.PolicyOptions{}
+
+ if !rs && peer != nil {
+ policyOptions.Info = peer.fsm.peerInfo
+ }
+ if v := server.roaManager.validate(path); v != nil {
+ policyOptions.ValidationResult = v
+ }
+
if p := server.policy.ApplyPolicy(tableId, table.POLICY_DIRECTION_IMPORT, path, policyOptions); p != nil {
path = p
} else {
@@ -663,7 +899,7 @@ func (server *BgpServer) propagateUpdate(peer *Peer, pathList []*table.Path) {
}
var candidates []*table.Path
if path.IsWithdraw {
- candidates, _ = peer.getBestFromLocal(peer.configuredRFlist())
+ candidates, _ = server.getBestFromLocal(peer, peer.configuredRFlist())
} else {
candidates = server.globalRib.GetBestPathList(peer.TableID(), 0, fs)
}
@@ -680,9 +916,9 @@ func (server *BgpServer) propagateUpdate(peer *Peer, pathList []*table.Path) {
}
}
if path.IsWithdraw {
- paths = peer.processOutgoingPaths(nil, paths)
+ paths = server.processOutgoingPaths(peer, nil, paths)
} else {
- paths = peer.processOutgoingPaths(paths, nil)
+ paths = server.processOutgoingPaths(peer, paths, nil)
}
sendFsmOutgoingMsg(peer, paths, nil, false)
}
@@ -752,7 +988,7 @@ func (server *BgpServer) propagateUpdateToNeighbors(source *Peer, newPath *table
if !needOld {
oldList = nil
}
- if paths := targetPeer.processOutgoingPaths(bestList, oldList); len(paths) > 0 {
+ if paths := server.processOutgoingPaths(targetPeer, bestList, oldList); len(paths) > 0 {
sendFsmOutgoingMsg(targetPeer, paths, nil, false)
}
}
@@ -892,7 +1128,7 @@ func (server *BgpServer) handleFSMMessage(peer *Peer, e *FsmMsg) {
var pathList []*table.Path
_, y := peer.fsm.rfMap[bgp.RF_RTC_UC]
if c := peer.fsm.pConf.GetAfiSafi(bgp.RF_RTC_UC); y && !peer.fsm.pConf.GracefulRestart.State.PeerRestarting && c.RouteTargetMembership.Config.DeferralTime > 0 {
- pathList, _ = peer.getBestFromLocal([]bgp.RouteFamily{bgp.RF_RTC_UC})
+ pathList, _ = server.getBestFromLocal(peer, []bgp.RouteFamily{bgp.RF_RTC_UC})
t := c.RouteTargetMembership.Config.DeferralTime
for _, f := range peer.configuredRFlist() {
if f != bgp.RF_RTC_UC {
@@ -900,7 +1136,7 @@ func (server *BgpServer) handleFSMMessage(peer *Peer, e *FsmMsg) {
}
}
} else {
- pathList, _ = peer.getBestFromLocal(peer.configuredRFlist())
+ pathList, _ = server.getBestFromLocal(peer, peer.configuredRFlist())
}
if len(pathList) > 0 {
@@ -948,7 +1184,7 @@ func (server *BgpServer) handleFSMMessage(peer *Peer, e *FsmMsg) {
if peer.fsm.state != bgp.BGP_FSM_ESTABLISHED || e.timestamp.Unix() < peer.fsm.pConf.Timers.State.Uptime {
return
}
- if paths := peer.handleRouteRefresh(e); len(paths) > 0 {
+ if paths := server.handleRouteRefresh(peer, e); len(paths) > 0 {
sendFsmOutgoingMsg(peer, paths, nil, false)
return
}
@@ -962,7 +1198,6 @@ func (server *BgpServer) handleFSMMessage(peer *Peer, e *FsmMsg) {
if peer.fsm.state != bgp.BGP_FSM_ESTABLISHED || e.timestamp.Unix() < peer.fsm.pConf.Timers.State.Uptime {
return
}
- server.roaManager.validate(e.PathList)
pathList, eor, notification := peer.handleUpdate(e)
if notification != nil {
sendFsmOutgoingMsg(peer, nil, notification, true)
@@ -1011,7 +1246,7 @@ func (server *BgpServer) handleFSMMessage(peer *Peer, e *FsmMsg) {
if !p.isGracefulRestartEnabled() {
continue
}
- paths, _ := p.getBestFromLocal(p.configuredRFlist())
+ paths, _ := server.getBestFromLocal(p, p.configuredRFlist())
if len(paths) > 0 {
sendFsmOutgoingMsg(p, paths, nil, false)
}
@@ -1053,7 +1288,7 @@ func (server *BgpServer) handleFSMMessage(peer *Peer, e *FsmMsg) {
families = append(families, f)
}
}
- if paths, _ := peer.getBestFromLocal(families); len(paths) > 0 {
+ if paths, _ := server.getBestFromLocal(peer, families); len(paths) > 0 {
sendFsmOutgoingMsg(peer, paths, nil, false)
}
}
@@ -1501,7 +1736,7 @@ func (s *BgpServer) softResetOut(addr string, family bgp.RouteFamily, deferral b
}
}
- pathList, filtered := peer.getBestFromLocal(families)
+ pathList, filtered := s.getBestFromLocal(peer, families)
if len(pathList) > 0 {
sendFsmOutgoingMsg(peer, pathList, nil, false)
}
@@ -1550,7 +1785,19 @@ func (s *BgpServer) SoftReset(addr string, family bgp.RouteFamily) error {
}, true)
}
-func (s *BgpServer) GetRib(addr string, family bgp.RouteFamily, prefixes []*table.LookupPrefix) (rib *table.Table, err error) {
+func (s *BgpServer) validateTable(r *table.Table) (v []*table.Validation) {
+ if s.roaManager.enabled() {
+ v := make([]*table.Validation, 0, len(r.GetDestinations()))
+ for _, d := range r.GetDestinations() {
+ for _, p := range d.GetAllKnownPathList() {
+ v = append(v, s.roaManager.validate(p))
+ }
+ }
+ }
+ return
+}
+
+func (s *BgpServer) GetRib(addr string, family bgp.RouteFamily, prefixes []*table.LookupPrefix) (rib *table.Table, v []*table.Validation, err error) {
err = s.mgmtOperation(func() error {
m := s.globalRib
id := table.GLOBAL_RIB_NAME
@@ -1573,6 +1820,7 @@ func (s *BgpServer) GetRib(addr string, family bgp.RouteFamily, prefixes []*tabl
return fmt.Errorf("address family: %s not supported", af)
}
rib, err = tbl.Select(table.TableSelectOption{ID: id, AS: as, LookupPrefixes: prefixes})
+ v = s.validateTable(rib)
return err
}, true)
return
@@ -1604,7 +1852,7 @@ func (s *BgpServer) GetVrfRib(name string, family bgp.RouteFamily, prefixes []*t
return
}
-func (s *BgpServer) GetAdjRib(addr string, family bgp.RouteFamily, in bool, prefixes []*table.LookupPrefix) (rib *table.Table, err error) {
+func (s *BgpServer) GetAdjRib(addr string, family bgp.RouteFamily, in bool, prefixes []*table.LookupPrefix) (rib *table.Table, v []*table.Validation, err error) {
err = s.mgmtOperation(func() error {
peer, ok := s.neighborMap[addr]
if !ok {
@@ -1618,10 +1866,11 @@ func (s *BgpServer) GetAdjRib(addr string, family bgp.RouteFamily, in bool, pref
adjRib = peer.adjRibIn
} else {
adjRib = table.NewAdjRib(id, peer.configuredRFlist())
- accepted, _ := peer.getBestFromLocal(peer.configuredRFlist())
+ accepted, _ := s.getBestFromLocal(peer, peer.configuredRFlist())
adjRib.Update(accepted)
}
rib, err = adjRib.Select(family, false, table.TableSelectOption{ID: id, AS: as, LookupPrefixes: prefixes})
+ v = s.validateTable(rib)
return err
}, true)
return
@@ -1662,7 +1911,7 @@ func (s *BgpServer) GetAdjRibInfo(addr string, family bgp.RouteFamily, in bool)
adjRib = peer.adjRibIn
} else {
adjRib = table.NewAdjRib(peer.ID(), peer.configuredRFlist())
- accepted, _ := peer.getBestFromLocal(peer.configuredRFlist())
+ accepted, _ := s.getBestFromLocal(peer, peer.configuredRFlist())
adjRib.Update(accepted)
}
info, err = adjRib.TableInfo(family)
@@ -1687,7 +1936,7 @@ func (s *BgpServer) GetNeighbor(address string, getAdvertised bool) (l []*config
if address != "" && address != k && address != peer.fsm.pConf.Config.NeighborInterface {
continue
}
- l = append(l, peer.ToConfig(getAdvertised))
+ l = append(l, s.ToConfig(peer, getAdvertised))
}
return nil
}, false)
@@ -2259,32 +2508,6 @@ func (s *BgpServer) DisableMrt(c *config.MrtConfig) error {
}, false)
}
-func (s *BgpServer) ValidateRib(prefix string) error {
- return s.mgmtOperation(func() error {
- for _, rf := range s.globalRib.GetRFlist() {
- if t, ok := s.globalRib.Tables[rf]; ok {
- dsts := t.GetDestinations()
- if prefix != "" {
- addr, _, _ := net.ParseCIDR(prefix)
- var nlri bgp.AddrPrefixInterface
- if addr.To16() == nil {
- nlri, _ = bgp.NewPrefixFromRouteFamily(bgp.AFI_IP, bgp.SAFI_UNICAST, prefix)
- } else {
- nlri, _ = bgp.NewPrefixFromRouteFamily(bgp.AFI_IP6, bgp.SAFI_UNICAST, prefix)
- }
- if dst := t.GetDestination(nlri); dst != nil {
- dsts = map[string]*table.Destination{nlri.String(): dst}
- }
- }
- for _, dst := range dsts {
- s.roaManager.validate(dst.GetAllKnownPathList())
- }
- }
- }
- return nil
- }, true)
-}
-
func (s *BgpServer) GetRpki() (l []*config.RpkiServer, err error) {
err = s.mgmtOperation(func() error {
l = s.roaManager.GetServers()
@@ -2531,7 +2754,7 @@ func (w *Watcher) Generate(t WatchEventType) error {
}()
l := make([]*config.Neighbor, 0, len(w.s.neighborMap))
for _, peer := range w.s.neighborMap {
- l = append(l, peer.ToConfig(false))
+ l = append(l, w.s.ToConfig(peer, false))
}
w.notify(&WatchEventTable{PathList: pathList, Neighbor: l})
default:
@@ -2635,7 +2858,7 @@ func (s *BgpServer) Watch(opts ...WatchOption) (w *Watcher) {
if peer.fsm.state != bgp.BGP_FSM_ESTABLISHED {
continue
}
- configNeighbor := peer.ToConfig(false)
+ configNeighbor := w.s.ToConfig(peer, false)
for _, rf := range peer.configuredRFlist() {
_, y := peer.fsm.capMap[bgp.BGP_CAP_FOUR_OCTET_AS_NUMBER]
l, _ := peer.fsm.LocalHostPort()
@@ -2688,7 +2911,7 @@ func (s *BgpServer) Watch(opts ...WatchOption) (w *Watcher) {
// create copy which can be access to without mutex
var configNeighbor *config.Neighbor
if peer, ok := s.neighborMap[peerInfo.Address.String()]; ok {
- configNeighbor = peer.ToConfig(false)
+ configNeighbor = w.s.ToConfig(peer, false)
}
for _, path := range paths {
msgs := table.CreateUpdateMsgFromPaths([]*table.Path{path})
diff --git a/server/server_test.go b/server/server_test.go
index 0bbd4070..7f854e73 100644
--- a/server/server_test.go
+++ b/server/server_test.go
@@ -162,7 +162,7 @@ func TestMonitor(test *testing.T) {
}
for {
// Waits for the initial route will be advertised.
- rib, err := s.GetRib("", bgp.RF_IPv4_UC, nil)
+ rib, _, err := s.GetRib("", bgp.RF_IPv4_UC, nil)
if err != nil {
log.Fatal(err)
}
@@ -407,7 +407,8 @@ func TestFilterpathWithRejectPolicy(t *testing.T) {
path1 := table.NewPath(pi1, nlri, false, pa1, time.Now(), false)
new, old := process(rib2, []*table.Path{path1})
assert.Equal(t, new, path1)
- path2 := p2.filterpath(new, old)
+ s := NewBgpServer()
+ path2 := s.filterpath(p2, new, old)
if addCommunity {
assert.True(t, path2.IsWithdraw)
} else {
diff --git a/server/zclient.go b/server/zclient.go
index 12f5357f..eb3e0b81 100644
--- a/server/zclient.go
+++ b/server/zclient.go
@@ -471,7 +471,7 @@ func (z *zebraClient) loop() {
Tables: make(map[bgp.RouteFamily]*table.Table),
}
for _, rf := range rfListFromNexthopUpdateBody(body) {
- rib, err := z.server.GetRib("", rf, nil)
+ rib, _, err := z.server.GetRib("", rf, nil)
if err != nil {
log.Errorf("failed to get global rib by family %s", rf.String())
continue
diff --git a/table/destination.go b/table/destination.go
index 84183002..e063d89d 100644
--- a/table/destination.go
+++ b/table/destination.go
@@ -968,7 +968,7 @@ func (d *Destination) MarshalJSON() ([]byte, error) {
return json.Marshal(d.GetAllKnownPathList())
}
-func (old *Destination) Select(option ...DestinationSelectOption) *Destination {
+func (d *Destination) Select(option ...DestinationSelectOption) *Destination {
id := GLOBAL_RIB_NAME
var vrf *Vrf
adj := false
@@ -989,9 +989,10 @@ func (old *Destination) Select(option ...DestinationSelectOption) *Destination {
}
var paths []*Path
if adj {
- paths = old.knownPathList
+ paths = make([]*Path, len(d.knownPathList))
+ copy(paths, d.knownPathList)
} else {
- paths = old.GetKnownPathList(id, as)
+ paths = d.GetKnownPathList(id, as)
if vrf != nil {
ps := make([]*Path, 0, len(paths))
for _, p := range paths {
@@ -1022,12 +1023,7 @@ func (old *Destination) Select(option ...DestinationSelectOption) *Destination {
}
}
}
- new := NewDestination(old.nlri, 0)
- for _, path := range paths {
- p := path.Clone(path.IsWithdraw)
- new.knownPathList = append(new.knownPathList, p)
- }
- return new
+ return NewDestination(d.nlri, 0, paths...)
}
type destinations []*Destination
diff --git a/table/policy.go b/table/policy.go
index 299a976b..e42c3462 100644
--- a/table/policy.go
+++ b/table/policy.go
@@ -35,7 +35,8 @@ import (
)
type PolicyOptions struct {
- Info *PeerInfo
+ Info *PeerInfo
+ ValidationResult *Validation
}
type DefinedType int
@@ -1712,8 +1713,11 @@ func (c *RpkiValidationCondition) Type() ConditionType {
return CONDITION_RPKI
}
-func (c *RpkiValidationCondition) Evaluate(path *Path, _ *PolicyOptions) bool {
- return c.result == path.ValidationStatus()
+func (c *RpkiValidationCondition) Evaluate(path *Path, options *PolicyOptions) bool {
+ if options != nil && options.ValidationResult != nil {
+ return c.result == options.ValidationResult.Status
+ }
+ return false
}
func (c *RpkiValidationCondition) Set() DefinedSet {