diff options
author | FUJITA Tomonori <fujita.tomonori@lab.ntt.co.jp> | 2018-07-07 13:48:38 +0900 |
---|---|---|
committer | FUJITA Tomonori <fujita.tomonori@lab.ntt.co.jp> | 2018-07-07 20:44:25 +0900 |
commit | c4775c42510d1f1ddd55036dc19e982712fa6a0b (patch) | |
tree | 6ec8b61d4338c809e239e3003a2d32d480898e22 /internal/pkg/table | |
parent | b3079759aa13172fcb548a83da9a9653d8d5fed4 (diff) |
follow Standard Go Project Layout
https://github.com/golang-standards/project-layout
Now you can see clearly what are private and public library code.
Signed-off-by: FUJITA Tomonori <fujita.tomonori@lab.ntt.co.jp>
Diffstat (limited to 'internal/pkg/table')
-rw-r--r-- | internal/pkg/table/adj.go | 186 | ||||
-rw-r--r-- | internal/pkg/table/adj_test.go | 52 | ||||
-rw-r--r-- | internal/pkg/table/destination.go | 1041 | ||||
-rw-r--r-- | internal/pkg/table/destination_test.go | 442 | ||||
-rw-r--r-- | internal/pkg/table/message.go | 502 | ||||
-rw-r--r-- | internal/pkg/table/message_test.go | 663 | ||||
-rw-r--r-- | internal/pkg/table/path.go | 1179 | ||||
-rw-r--r-- | internal/pkg/table/path_test.go | 365 | ||||
-rw-r--r-- | internal/pkg/table/policy.go | 3994 | ||||
-rw-r--r-- | internal/pkg/table/policy_test.go | 3140 | ||||
-rw-r--r-- | internal/pkg/table/roa.go | 60 | ||||
-rw-r--r-- | internal/pkg/table/table.go | 451 | ||||
-rw-r--r-- | internal/pkg/table/table_manager.go | 356 | ||||
-rw-r--r-- | internal/pkg/table/table_manager_test.go | 2282 | ||||
-rw-r--r-- | internal/pkg/table/table_test.go | 180 | ||||
-rw-r--r-- | internal/pkg/table/vrf.go | 53 |
16 files changed, 14946 insertions, 0 deletions
diff --git a/internal/pkg/table/adj.go b/internal/pkg/table/adj.go new file mode 100644 index 00000000..95fbf6af --- /dev/null +++ b/internal/pkg/table/adj.go @@ -0,0 +1,186 @@ +// Copyright (C) 2015 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + "fmt" + + "github.com/osrg/gobgp/pkg/packet/bgp" +) + +type AdjRib struct { + accepted map[bgp.RouteFamily]int + table map[bgp.RouteFamily]map[string]*Path +} + +func NewAdjRib(rfList []bgp.RouteFamily) *AdjRib { + table := make(map[bgp.RouteFamily]map[string]*Path) + for _, rf := range rfList { + table[rf] = make(map[string]*Path) + } + return &AdjRib{ + table: table, + accepted: make(map[bgp.RouteFamily]int), + } +} + +func (adj *AdjRib) Update(pathList []*Path) { + for _, path := range pathList { + if path == nil || path.IsEOR() { + continue + } + rf := path.GetRouteFamily() + key := fmt.Sprintf("%d:%s", path.GetNlri().PathIdentifier(), path.getPrefix()) + + old, found := adj.table[rf][key] + if path.IsWithdraw { + if found { + delete(adj.table[rf], key) + if !old.IsAsLooped() { + adj.accepted[rf]-- + } + } + } else { + if found { + if old.IsAsLooped() && !path.IsAsLooped() { + adj.accepted[rf]++ + } else if !old.IsAsLooped() && path.IsAsLooped() { + adj.accepted[rf]-- + } + } else { + if !path.IsAsLooped() { + adj.accepted[rf]++ + } + } + if found && old.Equal(path) { + path.setTimestamp(old.GetTimestamp()) + } + adj.table[rf][key] = path + } + } +} + +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.IsAsLooped() { + continue + } + pathList = append(pathList, rr) + } + } + return pathList +} + +func (adj *AdjRib) Count(rfList []bgp.RouteFamily) int { + count := 0 + for _, rf := range rfList { + if table, ok := adj.table[rf]; ok { + count += len(table) + } + } + return count +} + +func (adj *AdjRib) Accepted(rfList []bgp.RouteFamily) int { + count := 0 + for _, rf := range rfList { + if n, ok := adj.accepted[rf]; ok { + count += n + } + } + return count +} + +func (adj *AdjRib) Drop(rfList []bgp.RouteFamily) { + for _, rf := range rfList { + if _, ok := adj.table[rf]; ok { + adj.table[rf] = make(map[string]*Path) + adj.accepted[rf] = 0 + } + } +} + +func (adj *AdjRib) DropStale(rfList []bgp.RouteFamily) []*Path { + pathList := make([]*Path, 0, adj.Count(rfList)) + for _, rf := range rfList { + if table, ok := adj.table[rf]; ok { + for k, p := range table { + if p.IsStale() { + delete(table, k) + if !p.IsAsLooped() { + adj.accepted[rf]-- + } + pathList = append(pathList, p.Clone(true)) + } + } + } + } + return pathList +} + +func (adj *AdjRib) StaleAll(rfList []bgp.RouteFamily) []*Path { + pathList := make([]*Path, 0) + for _, rf := range rfList { + if table, ok := adj.table[rf]; ok { + l := make([]*Path, 0, len(table)) + for k, p := range table { + n := p.Clone(false) + n.MarkStale(true) + table[k] = n + l = append(l, n) + } + if len(l) > 0 { + pathList = append(pathList, l...) + } + } + } + return pathList +} + +func (adj *AdjRib) Select(family bgp.RouteFamily, accepted bool, option ...TableSelectOption) (*Table, error) { + m := make(map[string][]*Path) + pl := adj.PathList([]bgp.RouteFamily{family}, accepted) + for _, path := range pl { + key := path.GetNlri().String() + if _, y := m[key]; y { + m[key] = append(m[key], path) + } else { + m[key] = []*Path{path} + } + } + d := make([]*Destination, 0, len(pl)) + for _, l := range m { + d = append(d, NewDestination(l[0].GetNlri(), 0, l...)) + } + tbl := NewTable(family, d...) + option = append(option, TableSelectOption{adj: true}) + return tbl.Select(option...) +} + +func (adj *AdjRib) TableInfo(family bgp.RouteFamily) (*TableInfo, error) { + if _, ok := adj.table[family]; !ok { + return nil, fmt.Errorf("%s unsupported", family) + } + c := adj.Count([]bgp.RouteFamily{family}) + a := adj.Accepted([]bgp.RouteFamily{family}) + return &TableInfo{ + NumDestination: c, + NumPath: c, + NumAccepted: a, + }, nil +} diff --git a/internal/pkg/table/adj_test.go b/internal/pkg/table/adj_test.go new file mode 100644 index 00000000..ac4fc5a0 --- /dev/null +++ b/internal/pkg/table/adj_test.go @@ -0,0 +1,52 @@ +// Copyright (C) 2018 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + "testing" + "time" + + "github.com/osrg/gobgp/pkg/packet/bgp" + + "github.com/stretchr/testify/assert" +) + +func TestStaleAll(t *testing.T) { + pi := &PeerInfo{} + attrs := []bgp.PathAttributeInterface{bgp.NewPathAttributeOrigin(0)} + + nlri1 := bgp.NewIPAddrPrefix(24, "20.20.20.0") + nlri1.SetPathIdentifier(1) + p1 := NewPath(pi, nlri1, false, attrs, time.Now(), false) + nlri2 := bgp.NewIPAddrPrefix(24, "20.20.20.0") + nlri2.SetPathIdentifier(2) + p2 := NewPath(pi, nlri2, false, attrs, time.Now(), false) + family := p1.GetRouteFamily() + families := []bgp.RouteFamily{family} + + adj := NewAdjRib(families) + adj.Update([]*Path{p1, p2}) + assert.Equal(t, len(adj.table[family]), 2) + + adj.StaleAll(families) + + for _, p := range adj.table[family] { + assert.True(t, p.IsStale()) + } + + adj.DropStale(families) + assert.Equal(t, len(adj.table[family]), 0) +} diff --git a/internal/pkg/table/destination.go b/internal/pkg/table/destination.go new file mode 100644 index 00000000..fa61572e --- /dev/null +++ b/internal/pkg/table/destination.go @@ -0,0 +1,1041 @@ +// Copyright (C) 2014 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + "bytes" + "encoding/binary" + "encoding/json" + "fmt" + "net" + "sort" + + "github.com/osrg/gobgp/internal/pkg/config" + "github.com/osrg/gobgp/pkg/packet/bgp" + + log "github.com/sirupsen/logrus" +) + +var SelectionOptions config.RouteSelectionOptionsConfig +var UseMultiplePaths config.UseMultiplePathsConfig + +type BestPathReason uint8 + +const ( + BPR_UNKNOWN BestPathReason = iota + BPR_DISABLED + BPR_ONLY_PATH + BPR_REACHABLE_NEXT_HOP + BPR_HIGHEST_WEIGHT + BPR_LOCAL_PREF + BPR_LOCAL_ORIGIN + BPR_ASPATH + BPR_ORIGIN + BPR_MED + BPR_ASN + BPR_IGP_COST + BPR_ROUTER_ID + BPR_OLDER + BPR_NON_LLGR_STALE +) + +var BestPathReasonStringMap = map[BestPathReason]string{ + BPR_UNKNOWN: "Unknown", + BPR_DISABLED: "Bestpath selection disabled", + BPR_ONLY_PATH: "Only Path", + BPR_REACHABLE_NEXT_HOP: "Reachable Next Hop", + BPR_HIGHEST_WEIGHT: "Highest Weight", + BPR_LOCAL_PREF: "Local Pref", + BPR_LOCAL_ORIGIN: "Local Origin", + BPR_ASPATH: "AS Path", + BPR_ORIGIN: "Origin", + BPR_MED: "MED", + BPR_ASN: "ASN", + BPR_IGP_COST: "IGP Cost", + BPR_ROUTER_ID: "Router ID", + BPR_OLDER: "Older", + BPR_NON_LLGR_STALE: "no LLGR Stale", +} + +func (r *BestPathReason) String() string { + return BestPathReasonStringMap[*r] +} + +func IpToRadixkey(b []byte, max uint8) string { + var buffer bytes.Buffer + for i := 0; i < len(b) && i < int(max); i++ { + fmt.Fprintf(&buffer, "%08b", b[i]) + } + return buffer.String()[:max] +} + +func CidrToRadixkey(cidr string) string { + _, n, _ := net.ParseCIDR(cidr) + ones, _ := n.Mask.Size() + return IpToRadixkey(n.IP, uint8(ones)) +} + +func AddrToRadixkey(addr bgp.AddrPrefixInterface) string { + var ( + ip net.IP + size uint8 + ) + switch T := addr.(type) { + case *bgp.IPAddrPrefix: + mask := net.CIDRMask(int(T.Length), net.IPv4len*8) + ip, size = T.Prefix.Mask(mask).To4(), uint8(T.Length) + case *bgp.IPv6AddrPrefix: + mask := net.CIDRMask(int(T.Length), net.IPv6len*8) + ip, size = T.Prefix.Mask(mask).To16(), uint8(T.Length) + default: + return CidrToRadixkey(addr.String()) + } + return IpToRadixkey(ip, size) +} + +type PeerInfo struct { + AS uint32 + ID net.IP + LocalAS uint32 + LocalID net.IP + Address net.IP + LocalAddress net.IP + RouteReflectorClient bool + RouteReflectorClusterID net.IP + MultihopTtl uint8 + Confederation bool +} + +func (lhs *PeerInfo) Equal(rhs *PeerInfo) bool { + if lhs == rhs { + return true + } + + if rhs == nil { + return false + } + + if (lhs.AS == rhs.AS) && lhs.ID.Equal(rhs.ID) && lhs.LocalID.Equal(rhs.LocalID) && lhs.Address.Equal(rhs.Address) { + return true + } + return false +} + +func (i *PeerInfo) String() string { + if i.Address == nil { + return "local" + } + s := bytes.NewBuffer(make([]byte, 0, 64)) + s.WriteString(fmt.Sprintf("{ %s | ", i.Address)) + s.WriteString(fmt.Sprintf("as: %d", i.AS)) + s.WriteString(fmt.Sprintf(", id: %s", i.ID)) + if i.RouteReflectorClient { + s.WriteString(fmt.Sprintf(", cluster-id: %s", i.RouteReflectorClusterID)) + } + s.WriteString(" }") + return s.String() +} + +func NewPeerInfo(g *config.Global, p *config.Neighbor) *PeerInfo { + clusterID := net.ParseIP(string(p.RouteReflector.State.RouteReflectorClusterId)).To4() + // exclude zone info + naddr, _ := net.ResolveIPAddr("ip", p.State.NeighborAddress) + return &PeerInfo{ + AS: p.Config.PeerAs, + LocalAS: g.Config.As, + LocalID: net.ParseIP(g.Config.RouterId).To4(), + RouteReflectorClient: p.RouteReflector.Config.RouteReflectorClient, + Address: naddr.IP, + RouteReflectorClusterID: clusterID, + MultihopTtl: p.EbgpMultihop.Config.MultihopTtl, + Confederation: p.IsConfederationMember(g), + } +} + +type Destination struct { + routeFamily bgp.RouteFamily + nlri bgp.AddrPrefixInterface + knownPathList []*Path + localIdMap *Bitmap +} + +func NewDestination(nlri bgp.AddrPrefixInterface, mapSize int, known ...*Path) *Destination { + d := &Destination{ + routeFamily: bgp.AfiSafiToRouteFamily(nlri.AFI(), nlri.SAFI()), + nlri: nlri, + knownPathList: known, + localIdMap: NewBitmap(mapSize), + } + // the id zero means id is not allocated yet. + if mapSize != 0 { + d.localIdMap.Flag(0) + } + return d +} + +func (dd *Destination) Family() bgp.RouteFamily { + return dd.routeFamily +} + +func (dd *Destination) setRouteFamily(routeFamily bgp.RouteFamily) { + dd.routeFamily = routeFamily +} + +func (dd *Destination) GetNlri() bgp.AddrPrefixInterface { + return dd.nlri +} + +func (dd *Destination) setNlri(nlri bgp.AddrPrefixInterface) { + dd.nlri = nlri +} + +func (dd *Destination) GetAllKnownPathList() []*Path { + return dd.knownPathList +} + +func rsFilter(id string, as uint32, path *Path) bool { + isASLoop := func(as uint32, path *Path) bool { + for _, v := range path.GetAsList() { + if as == v { + return true + } + } + return false + } + + if id != GLOBAL_RIB_NAME && (path.GetSource().Address.String() == id || isASLoop(as, path)) { + return true + } + return false +} + +func (dd *Destination) GetKnownPathList(id string, as uint32) []*Path { + list := make([]*Path, 0, len(dd.knownPathList)) + for _, p := range dd.knownPathList { + if rsFilter(id, as, p) { + continue + } + list = append(list, p) + } + return list +} + +func getBestPath(id string, as uint32, pathList []*Path) *Path { + for _, p := range pathList { + if rsFilter(id, as, p) { + continue + } + return p + } + return nil +} + +func (dd *Destination) GetBestPath(id string, as uint32) *Path { + p := getBestPath(id, as, dd.knownPathList) + if p == nil || p.IsNexthopInvalid { + return nil + } + return p +} + +func (dd *Destination) GetMultiBestPath(id string) []*Path { + return getMultiBestPath(id, dd.knownPathList) +} + +// Calculates best-path among known paths for this destination. +// +// 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(newPath *Path) *Update { + oldKnownPathList := make([]*Path, len(dest.knownPathList)) + copy(oldKnownPathList, dest.knownPathList) + + if newPath.IsWithdraw { + p := dest.explicitWithdraw(newPath) + if p != nil { + if id := p.GetNlri().PathLocalIdentifier(); id != 0 { + dest.localIdMap.Unflag(uint(id)) + } + } + } else { + dest.implicitWithdraw(newPath) + dest.knownPathList = append(dest.knownPathList, newPath) + } + + for _, path := range dest.knownPathList { + if path.GetNlri().PathLocalIdentifier() == 0 { + id, err := dest.localIdMap.FindandSetZeroBit() + if err != nil { + dest.localIdMap.Expand() + id, _ = dest.localIdMap.FindandSetZeroBit() + } + path.GetNlri().SetPathLocalIdentifier(uint32(id)) + } + } + // Compute new best path + dest.computeKnownBestPath() + + l := make([]*Path, len(dest.knownPathList)) + copy(l, dest.knownPathList) + return &Update{ + KnownPathList: l, + OldKnownPathList: oldKnownPathList, + } +} + +// Removes withdrawn paths. +// +// Note: +// We may have disproportionate number of withdraws compared to know paths +// since not all paths get installed into the table due to bgp policy and +// we can receive withdraws for such paths and withdrawals may not be +// stopped by the same policies. +// +func (dest *Destination) explicitWithdraw(withdraw *Path) *Path { + log.WithFields(log.Fields{ + "Topic": "Table", + "Key": dest.GetNlri().String(), + }).Debug("Removing withdrawals") + + // If we have some withdrawals and no know-paths, it means it is safe to + // delete these withdraws. + if len(dest.knownPathList) == 0 { + log.WithFields(log.Fields{ + "Topic": "Table", + "Key": dest.GetNlri().String(), + }).Debug("Found withdrawals for path(s) that did not get installed") + return nil + } + + // Match all withdrawals from destination paths. + isFound := -1 + for i, path := range dest.knownPathList { + // We have a match if the source and path-id are same. + if path.GetSource().Equal(withdraw.GetSource()) && path.GetNlri().PathIdentifier() == withdraw.GetNlri().PathIdentifier() { + isFound = i + withdraw.GetNlri().SetPathLocalIdentifier(path.GetNlri().PathLocalIdentifier()) + } + } + + // We do no have any match for this withdraw. + if isFound == -1 { + log.WithFields(log.Fields{ + "Topic": "Table", + "Key": dest.GetNlri().String(), + "Path": withdraw, + }).Warn("No matching path for withdraw found, may be path was not installed into table") + return nil + } else { + p := dest.knownPathList[isFound] + dest.knownPathList = append(dest.knownPathList[:isFound], dest.knownPathList[isFound+1:]...) + return p + } +} + +// Identifies which of known paths are old and removes them. +// +// Known paths will no longer have paths whose new version is present in +// new paths. +func (dest *Destination) implicitWithdraw(newPath *Path) { + found := -1 + for i, path := range dest.knownPathList { + if newPath.NoImplicitWithdraw() { + continue + } + // Here we just check if source is same and not check if path + // version num. as newPaths are implicit withdrawal of old + // paths and when doing RouteRefresh (not EnhancedRouteRefresh) + // we get same paths again. + if newPath.GetSource().Equal(path.GetSource()) && newPath.GetNlri().PathIdentifier() == path.GetNlri().PathIdentifier() { + log.WithFields(log.Fields{ + "Topic": "Table", + "Key": dest.GetNlri().String(), + "Path": path, + }).Debug("Implicit withdrawal of old path, since we have learned new path from the same peer") + + found = i + newPath.GetNlri().SetPathLocalIdentifier(path.GetNlri().PathLocalIdentifier()) + break + } + } + if found != -1 { + dest.knownPathList = append(dest.knownPathList[:found], dest.knownPathList[found+1:]...) + } +} + +func (dest *Destination) computeKnownBestPath() (*Path, BestPathReason, error) { + if SelectionOptions.DisableBestPathSelection { + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debug("computeKnownBestPath skipped") + return nil, BPR_DISABLED, nil + } + + // If we do not have any paths to this destination, then we do not have + // new best path. + if len(dest.knownPathList) == 0 { + return nil, BPR_UNKNOWN, nil + } + + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debugf("computeKnownBestPath knownPathList: %d", len(dest.knownPathList)) + + // We pick the first path as current best path. This helps in breaking + // tie between two new paths learned in one cycle for which best-path + // calculation steps lead to tie. + if len(dest.knownPathList) == 1 { + // If the first path has the invalidated next-hop, which evaluated by + // IGP, returns no path with the reason of the next-hop reachability. + if dest.knownPathList[0].IsNexthopInvalid { + return nil, BPR_REACHABLE_NEXT_HOP, nil + } + return dest.knownPathList[0], BPR_ONLY_PATH, nil + } + dest.sort() + newBest := dest.knownPathList[0] + // If the first path has the invalidated next-hop, which evaluated by IGP, + // returns no path with the reason of the next-hop reachability. + if dest.knownPathList[0].IsNexthopInvalid { + return nil, BPR_REACHABLE_NEXT_HOP, nil + } + return newBest, newBest.reason, nil +} + +func (dst *Destination) sort() { + sort.SliceStable(dst.knownPathList, func(i, j int) bool { + //Compares given paths and returns best path. + // + //Parameters: + // -`path1`: first path to compare + // -`path2`: second path to compare + // + // Best path processing will involve following steps: + // 1. Select a path with a reachable next hop. + // 2. Select the path with the highest weight. + // 3. If path weights are the same, select the path with the highest + // local preference value. + // 4. Prefer locally originated routes (network routes, redistributed + // routes, or aggregated routes) over received routes. + // 5. Select the route with the shortest AS-path length. + // 6. If all paths have the same AS-path length, select the path based + // on origin: IGP is preferred over EGP; EGP is preferred over + // Incomplete. + // 7. If the origins are the same, select the path with lowest MED + // value. + // 8. If the paths have the same MED values, select the path learned + // via EBGP over one learned via IBGP. + // 9. Select the route with the lowest IGP cost to the next hop. + // 10. Select the route received from the peer with the lowest BGP + // router ID. + // + // Returns None if best-path among given paths cannot be computed else best + // path. + // Assumes paths from NC has source equal to None. + // + + path1 := dst.knownPathList[i] + path2 := dst.knownPathList[j] + + var better *Path + reason := BPR_UNKNOWN + + // draft-uttaro-idr-bgp-persistence-02 + if better == nil { + better = compareByLLGRStaleCommunity(path1, path2) + reason = BPR_NON_LLGR_STALE + } + // Follow best path calculation algorithm steps. + // compare by reachability + if better == nil { + better = compareByReachableNexthop(path1, path2) + reason = BPR_REACHABLE_NEXT_HOP + } + if better == nil { + better = compareByHighestWeight(path1, path2) + reason = BPR_HIGHEST_WEIGHT + } + if better == nil { + better = compareByLocalPref(path1, path2) + reason = BPR_LOCAL_PREF + } + if better == nil { + better = compareByLocalOrigin(path1, path2) + reason = BPR_LOCAL_ORIGIN + } + if better == nil { + better = compareByASPath(path1, path2) + reason = BPR_ASPATH + } + if better == nil { + better = compareByOrigin(path1, path2) + reason = BPR_ORIGIN + } + if better == nil { + better = compareByMED(path1, path2) + reason = BPR_MED + } + if better == nil { + better = compareByASNumber(path1, path2) + reason = BPR_ASN + } + if better == nil { + better = compareByIGPCost(path1, path2) + reason = BPR_IGP_COST + } + if better == nil { + better = compareByAge(path1, path2) + reason = BPR_OLDER + } + if better == nil { + var e error = nil + better, e = compareByRouterID(path1, path2) + if e != nil { + log.WithFields(log.Fields{ + "Topic": "Table", + "Error": e, + }).Error("Could not get best path by comparing router ID") + } + reason = BPR_ROUTER_ID + } + if better == nil { + reason = BPR_UNKNOWN + better = path1 + } + + better.reason = reason + + return better == path1 + }) +} + +type Update struct { + KnownPathList []*Path + OldKnownPathList []*Path +} + +func getMultiBestPath(id string, pathList []*Path) []*Path { + list := make([]*Path, 0, len(pathList)) + var best *Path + for _, p := range pathList { + if !p.IsNexthopInvalid { + if best == nil { + best = p + list = append(list, p) + } else if best.Compare(p) == 0 { + list = append(list, p) + } + } + } + return list +} + +func (u *Update) GetWithdrawnPath() []*Path { + if len(u.KnownPathList) == len(u.OldKnownPathList) { + return nil + } + + l := make([]*Path, 0, len(u.OldKnownPathList)) + + for _, p := range u.OldKnownPathList { + y := func() bool { + for _, old := range u.KnownPathList { + if p == old { + return true + } + } + return false + }() + if !y { + l = append(l, p.Clone(true)) + } + } + return l +} + +func (u *Update) GetChanges(id string, as uint32, peerDown bool) (*Path, *Path, []*Path) { + best, old := func(id string) (*Path, *Path) { + old := getBestPath(id, as, u.OldKnownPathList) + best := getBestPath(id, as, u.KnownPathList) + if best != nil && best.Equal(old) { + // RFC4684 3.2. Intra-AS VPN Route Distribution + // When processing RT membership NLRIs received from internal iBGP + // peers, it is necessary to consider all available iBGP paths for a + // given RT prefix, for building the outbound route filter, and not just + // the best path. + if best.GetRouteFamily() == bgp.RF_RTC_UC { + return best, old + } + // For BGP Nexthop Tracking, checks if the nexthop reachability + // was changed or not. + if best.IsNexthopInvalid != old.IsNexthopInvalid { + // If the nexthop of the best path became unreachable, we need + // to withdraw that path. + if best.IsNexthopInvalid { + return best.Clone(true), old + } + return best, old + } + return nil, old + } + if best == nil { + if old == nil { + return nil, nil + } + if peerDown { + // withdraws were generated by peer + // down so paths are not in knowpath + // or adjin. + old.IsWithdraw = true + return old, old + } + return old.Clone(true), old + } + return best, old + }(id) + + var multi []*Path + + if id == GLOBAL_RIB_NAME && UseMultiplePaths.Enabled { + diff := func(lhs, rhs []*Path) bool { + if len(lhs) != len(rhs) { + return true + } + for idx, l := range lhs { + if !l.Equal(rhs[idx]) { + return true + } + } + return false + } + oldM := getMultiBestPath(id, u.OldKnownPathList) + newM := getMultiBestPath(id, u.KnownPathList) + if diff(oldM, newM) { + multi = newM + if len(newM) == 0 { + multi = []*Path{best} + } + } + } + return best, old, multi +} + +func compareByLLGRStaleCommunity(path1, path2 *Path) *Path { + p1 := path1.IsLLGRStale() + p2 := path2.IsLLGRStale() + if p1 == p2 { + return nil + } else if p1 { + return path2 + } + return path1 +} + +func compareByReachableNexthop(path1, path2 *Path) *Path { + // Compares given paths and selects best path based on reachable next-hop. + // + // If no path matches this criteria, return nil. + // For BGP Nexthop Tracking, evaluates next-hop is validated by IGP. + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debugf("enter compareByReachableNexthop -- path1: %s, path2: %s", path1, path2) + + if path1.IsNexthopInvalid && !path2.IsNexthopInvalid { + return path2 + } else if !path1.IsNexthopInvalid && path2.IsNexthopInvalid { + return path1 + } + + return nil +} + +func compareByHighestWeight(path1, path2 *Path) *Path { + // Selects a path with highest weight. + // + // Weight is BGPS specific parameter. It is local to the router on which it + // is configured. + // Return: + // nil if best path among given paths cannot be decided, else best path. + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debugf("enter compareByHighestWeight -- path1: %s, path2: %s", path1, path2) + return nil +} + +func compareByLocalPref(path1, path2 *Path) *Path { + // Selects a path with highest local-preference. + // + // Unlike the weight attribute, which is only relevant to the local + // router, local preference is an attribute that routers exchange in the + // same AS. Highest local-pref is preferred. If we cannot decide, + // we return None. + // + // # Default local-pref values is 100 + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debug("enter compareByLocalPref") + localPref1, _ := path1.GetLocalPref() + localPref2, _ := path2.GetLocalPref() + // Highest local-preference value is preferred. + if localPref1 > localPref2 { + return path1 + } else if localPref1 < localPref2 { + return path2 + } else { + return nil + } +} + +func compareByLocalOrigin(path1, path2 *Path) *Path { + + // Select locally originating path as best path. + // Locally originating routes are network routes, redistributed routes, + // or aggregated routes. + // Returns None if given paths have same source. + // + // If both paths are from same sources we cannot compare them here. + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debug("enter compareByLocalOrigin") + if path1.GetSource().Equal(path2.GetSource()) { + return nil + } + + // Here we consider prefix from NC as locally originating static route. + // Hence it is preferred. + if path1.IsLocal() { + return path1 + } + + if path2.IsLocal() { + return path2 + } + return nil +} + +func compareByASPath(path1, path2 *Path) *Path { + // Calculated the best-paths by comparing as-path lengths. + // + // Shortest as-path length is preferred. If both path have same lengths, + // we return None. + if SelectionOptions.IgnoreAsPathLength { + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debug("compareByASPath -- skip") + return nil + } + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debug("enter compareByASPath") + attribute1 := path1.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + attribute2 := path2.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + + // With addpath support, we could compare paths from API don't + // AS_PATH. No need to warn here. + if !path1.IsLocal() && !path2.IsLocal() && (attribute1 == nil || attribute2 == nil) { + log.WithFields(log.Fields{ + "Topic": "Table", + "Key": "compareByASPath", + "ASPath1": attribute1, + "ASPath2": attribute2, + }).Warn("can't compare ASPath because it's not present") + } + + l1 := path1.GetAsPathLen() + l2 := path2.GetAsPathLen() + + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debugf("compareByASPath -- l1: %d, l2: %d", l1, l2) + if l1 > l2 { + return path2 + } else if l1 < l2 { + return path1 + } else { + return nil + } +} + +func compareByOrigin(path1, path2 *Path) *Path { + // Select the best path based on origin attribute. + // + // IGP is preferred over EGP; EGP is preferred over Incomplete. + // If both paths have same origin, we return None. + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debug("enter compareByOrigin") + attribute1 := path1.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + attribute2 := path2.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + + if attribute1 == nil || attribute2 == nil { + log.WithFields(log.Fields{ + "Topic": "Table", + "Key": "compareByOrigin", + "Origin1": attribute1, + "Origin2": attribute2, + }).Error("can't compare origin because it's not present") + return nil + } + + origin1 := attribute1.(*bgp.PathAttributeOrigin).Value + origin2 := attribute2.(*bgp.PathAttributeOrigin).Value + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debugf("compareByOrigin -- origin1: %d, origin2: %d", origin1, origin2) + + // If both paths have same origins + if origin1 == origin2 { + return nil + } else if origin1 < origin2 { + return path1 + } else { + return path2 + } +} + +func compareByMED(path1, path2 *Path) *Path { + // Select the path based with lowest MED value. + // + // If both paths have same MED, return None. + // By default, a route that arrives with no MED value is treated as if it + // had a MED of 0, the most preferred value. + // RFC says lower MED is preferred over higher MED value. + // compare MED among not only same AS path but also all path, + // like bgp always-compare-med + + isInternal := func() bool { return path1.GetAsPathLen() == 0 && path2.GetAsPathLen() == 0 }() + + isSameAS := func() bool { + firstAS := func(path *Path) uint32 { + if asPath := path.GetAsPath(); asPath != nil { + for _, v := range asPath.Value { + segType := v.GetType() + asList := v.GetAS() + if len(asList) == 0 { + continue + } + switch segType { + case bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SET, bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SEQ: + continue + } + return asList[0] + } + } + return 0 + } + return firstAS(path1) != 0 && firstAS(path1) == firstAS(path2) + }() + + if SelectionOptions.AlwaysCompareMed || isInternal || isSameAS { + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debug("enter compareByMED") + getMed := func(path *Path) uint32 { + attribute := path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + if attribute == nil { + return 0 + } + med := attribute.(*bgp.PathAttributeMultiExitDisc).Value + return med + } + + med1 := getMed(path1) + med2 := getMed(path2) + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debugf("compareByMED -- med1: %d, med2: %d", med1, med2) + if med1 == med2 { + return nil + } else if med1 < med2 { + return path1 + } + return path2 + } else { + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debugf("skip compareByMED %v %v %v", SelectionOptions.AlwaysCompareMed, isInternal, isSameAS) + return nil + } +} + +func compareByASNumber(path1, path2 *Path) *Path { + + //Select the path based on source (iBGP/eBGP) peer. + // + //eBGP path is preferred over iBGP. If both paths are from same kind of + //peers, return None. + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debug("enter compareByASNumber") + + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debugf("compareByASNumber -- p1Asn: %d, p2Asn: %d", path1.GetSource().AS, path2.GetSource().AS) + // Path from confederation member should be treated as internal (IBGP learned) path. + isIBGP1 := path1.GetSource().Confederation || path1.IsIBGP() + isIBGP2 := path2.GetSource().Confederation || path2.IsIBGP() + // If one path is from ibgp peer and another is from ebgp peer, take the ebgp path. + if isIBGP1 != isIBGP2 { + if isIBGP1 { + return path2 + } + return path1 + } + + // If both paths are from ebgp or ibpg peers, we cannot decide. + return nil +} + +func compareByIGPCost(path1, path2 *Path) *Path { + // Select the route with the lowest IGP cost to the next hop. + // + // Return None if igp cost is same. + // Currently BGPS has no concept of IGP and IGP cost. + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debugf("enter compareByIGPCost -- path1: %v, path2: %v", path1, path2) + return nil +} + +func compareByRouterID(path1, path2 *Path) (*Path, error) { + // Select the route received from the peer with the lowest BGP router ID. + // + // If both paths are eBGP paths, then we do not do any tie breaking, i.e we do + // not pick best-path based on this criteria. + // RFC: http://tools.ietf.org/html/rfc5004 + // We pick best path between two iBGP paths as usual. + log.WithFields(log.Fields{ + "Topic": "Table", + }).Debug("enter compareByRouterID") + + // If both paths are from NC we have same router Id, hence cannot compare. + if path1.IsLocal() && path2.IsLocal() { + return nil, nil + } + + // If both paths are from eBGP peers, then according to RFC we need + // not tie break using router id. + if !SelectionOptions.ExternalCompareRouterId && !path1.IsIBGP() && !path2.IsIBGP() { + return nil, nil + } + + if !SelectionOptions.ExternalCompareRouterId && path1.IsIBGP() != path2.IsIBGP() { + return nil, fmt.Errorf("This method does not support comparing ebgp with ibgp path") + } + + // At least one path is not coming from NC, so we get local bgp id. + id1 := binary.BigEndian.Uint32(path1.GetSource().ID) + id2 := binary.BigEndian.Uint32(path2.GetSource().ID) + + // If both router ids are same/equal we cannot decide. + // This case is possible since router ids are arbitrary. + if id1 == id2 { + return nil, nil + } else if id1 < id2 { + return path1, nil + } else { + return path2, nil + } +} + +func compareByAge(path1, path2 *Path) *Path { + if !path1.IsIBGP() && !path2.IsIBGP() && !SelectionOptions.ExternalCompareRouterId { + age1 := path1.GetTimestamp().UnixNano() + age2 := path2.GetTimestamp().UnixNano() + if age1 == age2 { + return nil + } else if age1 < age2 { + return path1 + } + return path2 + } + return nil +} + +func (dest *Destination) String() string { + return fmt.Sprintf("Destination NLRI: %s", dest.nlri.String()) +} + +type DestinationSelectOption struct { + ID string + AS uint32 + VRF *Vrf + adj bool + Best bool + MultiPath bool +} + +func (d *Destination) MarshalJSON() ([]byte, error) { + return json.Marshal(d.GetAllKnownPathList()) +} + +func (d *Destination) Select(option ...DestinationSelectOption) *Destination { + id := GLOBAL_RIB_NAME + var vrf *Vrf + adj := false + best := false + mp := false + as := uint32(0) + for _, o := range option { + if o.ID != "" { + id = o.ID + } + if o.VRF != nil { + vrf = o.VRF + } + adj = o.adj + best = o.Best + mp = o.MultiPath + as = o.AS + } + var paths []*Path + if adj { + paths = make([]*Path, len(d.knownPathList)) + copy(paths, d.knownPathList) + } else { + paths = d.GetKnownPathList(id, as) + if vrf != nil { + ps := make([]*Path, 0, len(paths)) + for _, p := range paths { + if CanImportToVrf(vrf, p) { + ps = append(ps, p.ToLocal()) + } + } + paths = ps + } + if len(paths) == 0 { + return nil + } + if best { + if !mp { + paths = []*Path{paths[0]} + } else { + ps := make([]*Path, 0, len(paths)) + var best *Path + for _, p := range paths { + if best == nil { + best = p + ps = append(ps, p) + } else if best.Compare(p) == 0 { + ps = append(ps, p) + } + } + paths = ps + } + } + } + return NewDestination(d.nlri, 0, paths...) +} diff --git a/internal/pkg/table/destination_test.go b/internal/pkg/table/destination_test.go new file mode 100644 index 00000000..110278fb --- /dev/null +++ b/internal/pkg/table/destination_test.go @@ -0,0 +1,442 @@ +// Copyright (C) 2014 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + //"fmt" + "fmt" + "net" + "testing" + "time" + + "github.com/osrg/gobgp/pkg/packet/bgp" + + "github.com/stretchr/testify/assert" +) + +func TestDestinationNewIPv4(t *testing.T) { + peerD := DestCreatePeer() + pathD := DestCreatePath(peerD) + ipv4d := NewDestination(pathD[0].GetNlri(), 0) + assert.NotNil(t, ipv4d) +} +func TestDestinationNewIPv6(t *testing.T) { + peerD := DestCreatePeer() + pathD := DestCreatePath(peerD) + ipv6d := NewDestination(pathD[0].GetNlri(), 0) + assert.NotNil(t, ipv6d) +} + +func TestDestinationSetRouteFamily(t *testing.T) { + dd := &Destination{} + dd.setRouteFamily(bgp.RF_IPv4_UC) + rf := dd.Family() + assert.Equal(t, rf, bgp.RF_IPv4_UC) +} +func TestDestinationGetRouteFamily(t *testing.T) { + dd := &Destination{} + dd.setRouteFamily(bgp.RF_IPv6_UC) + rf := dd.Family() + assert.Equal(t, rf, bgp.RF_IPv6_UC) +} +func TestDestinationSetNlri(t *testing.T) { + dd := &Destination{} + nlri := bgp.NewIPAddrPrefix(24, "13.2.3.1") + dd.setNlri(nlri) + r_nlri := dd.GetNlri() + assert.Equal(t, r_nlri, nlri) +} +func TestDestinationGetNlri(t *testing.T) { + dd := &Destination{} + nlri := bgp.NewIPAddrPrefix(24, "10.110.123.1") + dd.setNlri(nlri) + r_nlri := dd.GetNlri() + assert.Equal(t, r_nlri, nlri) +} + +func TestCalculate2(t *testing.T) { + + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAs4PathParam(2, []uint32{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := bgp.NewIPAddrPrefix(24, "10.10.0.0") + + // peer1 sends normal update message 10.10.0.0/24 + update1 := bgp.NewBGPUpdateMessage(nil, pathAttributes, []*bgp.IPAddrPrefix{nlri}) + peer1 := &PeerInfo{AS: 1, Address: net.IP{1, 1, 1, 1}} + path1 := ProcessMessage(update1, peer1, time.Now())[0] + + d := NewDestination(nlri, 0) + d.Calculate(path1) + + // suppose peer2 sends grammaatically correct but semantically flawed update message + // which has a withdrawal nlri not advertised before + update2 := bgp.NewBGPUpdateMessage([]*bgp.IPAddrPrefix{nlri}, pathAttributes, nil) + peer2 := &PeerInfo{AS: 2, Address: net.IP{2, 2, 2, 2}} + path2 := ProcessMessage(update2, peer2, time.Now())[0] + assert.Equal(t, path2.IsWithdraw, true) + + d.Calculate(path2) + + // we have a path from peer1 here + assert.Equal(t, len(d.knownPathList), 1) + + // after that, new update with the same nlri comes from peer2 + update3 := bgp.NewBGPUpdateMessage(nil, pathAttributes, []*bgp.IPAddrPrefix{nlri}) + path3 := ProcessMessage(update3, peer2, time.Now())[0] + assert.Equal(t, path3.IsWithdraw, false) + + d.Calculate(path3) + + // this time, we have paths from peer1 and peer2 + assert.Equal(t, len(d.knownPathList), 2) + + // now peer3 sends normal update message 10.10.0.0/24 + peer3 := &PeerInfo{AS: 3, Address: net.IP{3, 3, 3, 3}} + update4 := bgp.NewBGPUpdateMessage(nil, pathAttributes, []*bgp.IPAddrPrefix{nlri}) + path4 := ProcessMessage(update4, peer3, time.Now())[0] + + d.Calculate(path4) + + // we must have paths from peer1, peer2 and peer3 + assert.Equal(t, len(d.knownPathList), 3) +} + +func TestMedTieBreaker(t *testing.T) { + nlri := bgp.NewIPAddrPrefix(24, "10.10.0.0") + + p0 := func() *Path { + aspath := bgp.NewPathAttributeAsPath([]bgp.AsPathParamInterface{bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65001, 65002}), bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65003, 65004})}) + attrs := []bgp.PathAttributeInterface{aspath, bgp.NewPathAttributeMultiExitDisc(0)} + return NewPath(nil, nlri, false, attrs, time.Now(), false) + }() + + p1 := func() *Path { + aspath := bgp.NewPathAttributeAsPath([]bgp.AsPathParamInterface{bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65001, 65002}), bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65003, 65005})}) + attrs := []bgp.PathAttributeInterface{aspath, bgp.NewPathAttributeMultiExitDisc(10)} + return NewPath(nil, nlri, false, attrs, time.Now(), false) + }() + + // same AS + assert.Equal(t, compareByMED(p0, p1), p0) + + p2 := func() *Path { + aspath := bgp.NewPathAttributeAsPath([]bgp.AsPathParamInterface{bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65003})}) + attrs := []bgp.PathAttributeInterface{aspath, bgp.NewPathAttributeMultiExitDisc(10)} + return NewPath(nil, nlri, false, attrs, time.Now(), false) + }() + + // different AS + assert.Equal(t, compareByMED(p0, p2), (*Path)(nil)) + + p3 := func() *Path { + aspath := bgp.NewPathAttributeAsPath([]bgp.AsPathParamInterface{bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SEQ, []uint32{65003, 65004}), bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65001, 65003})}) + attrs := []bgp.PathAttributeInterface{aspath, bgp.NewPathAttributeMultiExitDisc(0)} + return NewPath(nil, nlri, false, attrs, time.Now(), false) + }() + + p4 := func() *Path { + aspath := bgp.NewPathAttributeAsPath([]bgp.AsPathParamInterface{bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65001, 65002}), bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SEQ, []uint32{65005, 65006})}) + attrs := []bgp.PathAttributeInterface{aspath, bgp.NewPathAttributeMultiExitDisc(10)} + return NewPath(nil, nlri, false, attrs, time.Now(), false) + }() + + // ignore confed + assert.Equal(t, compareByMED(p3, p4), p3) + + p5 := func() *Path { + attrs := []bgp.PathAttributeInterface{bgp.NewPathAttributeMultiExitDisc(0)} + return NewPath(nil, nlri, false, attrs, time.Now(), false) + }() + + p6 := func() *Path { + attrs := []bgp.PathAttributeInterface{bgp.NewPathAttributeMultiExitDisc(10)} + return NewPath(nil, nlri, false, attrs, time.Now(), false) + }() + + // no aspath + assert.Equal(t, compareByMED(p5, p6), p5) +} + +func TestTimeTieBreaker(t *testing.T) { + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAs4PathParam(2, []uint32{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := bgp.NewIPAddrPrefix(24, "10.10.0.0") + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, []*bgp.IPAddrPrefix{nlri}) + peer1 := &PeerInfo{AS: 2, LocalAS: 1, Address: net.IP{1, 1, 1, 1}, ID: net.IP{1, 1, 1, 1}} + path1 := ProcessMessage(updateMsg, peer1, time.Now())[0] + + peer2 := &PeerInfo{AS: 2, LocalAS: 1, Address: net.IP{2, 2, 2, 2}, ID: net.IP{2, 2, 2, 2}} // weaker router-id + path2 := ProcessMessage(updateMsg, peer2, time.Now().Add(-1*time.Hour))[0] // older than path1 + + d := NewDestination(nlri, 0) + d.Calculate(path1) + d.Calculate(path2) + + assert.Equal(t, len(d.knownPathList), 2) + assert.Equal(t, true, d.GetBestPath("", 0).GetSource().ID.Equal(net.IP{2, 2, 2, 2})) // path from peer2 win + + // this option disables tie breaking by age + SelectionOptions.ExternalCompareRouterId = true + d = NewDestination(nlri, 0) + d.Calculate(path1) + d.Calculate(path2) + + assert.Equal(t, len(d.knownPathList), 2) + assert.Equal(t, true, d.GetBestPath("", 0).GetSource().ID.Equal(net.IP{1, 1, 1, 1})) // path from peer1 win +} + +func DestCreatePeer() []*PeerInfo { + peerD1 := &PeerInfo{AS: 65000} + peerD2 := &PeerInfo{AS: 65001} + peerD3 := &PeerInfo{AS: 65002} + peerD := []*PeerInfo{peerD1, peerD2, peerD3} + return peerD +} + +func DestCreatePath(peerD []*PeerInfo) []*Path { + bgpMsgD1 := updateMsgD1() + bgpMsgD2 := updateMsgD2() + bgpMsgD3 := updateMsgD3() + pathD := make([]*Path, 3) + for i, msg := range []*bgp.BGPMessage{bgpMsgD1, bgpMsgD2, bgpMsgD3} { + updateMsgD := msg.Body.(*bgp.BGPUpdate) + nlriList := updateMsgD.NLRI + pathAttributes := updateMsgD.PathAttributes + nlri_info := nlriList[0] + pathD[i] = NewPath(peerD[i], nlri_info, false, pathAttributes, time.Now(), false) + } + return pathD +} + +func updateMsgD1() *bgp.BGPMessage { + + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65000})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.50.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg.Body.(*bgp.BGPUpdate)) + return updateMsg +} + +func updateMsgD2() *bgp.BGPMessage { + + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65100})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.100.1") + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "20.20.20.0")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg.Body.(*bgp.BGPUpdate)) + return updateMsg +} +func updateMsgD3() *bgp.BGPMessage { + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65100})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.150.1") + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "30.30.30.0")} + w1 := bgp.NewIPAddrPrefix(23, "40.40.40.0") + withdrawnRoutes := []*bgp.IPAddrPrefix{w1} + updateMsg := bgp.NewBGPUpdateMessage(withdrawnRoutes, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg.Body.(*bgp.BGPUpdate)) + return updateMsg +} + +func TestRadixkey(t *testing.T) { + assert.Equal(t, "000010100000001100100000", CidrToRadixkey("10.3.32.0/24")) + assert.Equal(t, "000010100000001100100000", IpToRadixkey(net.ParseIP("10.3.32.0").To4(), 24)) + assert.Equal(t, "000010100000001100100000", IpToRadixkey(net.ParseIP("10.3.32.0").To4(), 24)) + assert.Equal(t, CidrToRadixkey("::ffff:0.0.0.0/96")+"000010100000001100100000", CidrToRadixkey("::ffff:10.3.32.0/120")) +} + +func TestIpToRadixkey(t *testing.T) { + for i := byte(0); i < 255; i += 3 { + for y := byte(1); y < 128; y *= 2 { + ip := net.IPv4(i, i+2, i+3, i-y) + for n := uint8(16); n <= 32; n += 2 { + exp := CidrToRadixkey(fmt.Sprintf("%v/%d", ip.To4(), n)) + got := IpToRadixkey(ip.To4(), n) + if exp != got { + t.Fatalf(`exp %v; got %v`, exp, got) + } + } + for n := uint8(116); n <= 128; n += 2 { + exp := CidrToRadixkey(fmt.Sprintf("::ffff:%v/%d", ip.To16(), n)) + got := IpToRadixkey(ip.To16(), n) + if exp != got { + t.Fatalf(`exp %v; got %v`, exp, got) + } + } + } + } +} + +func TestMultipath(t *testing.T) { + UseMultiplePaths.Enabled = true + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAs4PathParam(2, []uint32{65000})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.150.1") + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + peer1 := &PeerInfo{AS: 1, Address: net.IP{1, 1, 1, 1}, ID: net.IP{1, 1, 1, 1}} + path1 := ProcessMessage(updateMsg, peer1, time.Now())[0] + peer2 := &PeerInfo{AS: 2, Address: net.IP{2, 2, 2, 2}, ID: net.IP{2, 2, 2, 2}} + + med = bgp.NewPathAttributeMultiExitDisc(100) + nexthop = bgp.NewPathAttributeNextHop("192.168.150.2") + pathAttributes = []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + updateMsg = bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path2 := ProcessMessage(updateMsg, peer2, time.Now())[0] + + d := NewDestination(nlri[0], 0) + d.Calculate(path2) + + best, old, multi := d.Calculate(path1).GetChanges(GLOBAL_RIB_NAME, 0, false) + assert.NotNil(t, best) + assert.Equal(t, old, path2) + assert.Equal(t, len(multi), 2) + assert.Equal(t, len(d.GetKnownPathList(GLOBAL_RIB_NAME, 0)), 2) + + path3 := path2.Clone(true) + dd := d.Calculate(path3) + best, old, multi = dd.GetChanges(GLOBAL_RIB_NAME, 0, false) + assert.Nil(t, best) + assert.Equal(t, old, path1) + assert.Equal(t, len(multi), 1) + assert.Equal(t, len(d.GetKnownPathList(GLOBAL_RIB_NAME, 0)), 1) + + peer3 := &PeerInfo{AS: 3, Address: net.IP{3, 3, 3, 3}, ID: net.IP{3, 3, 3, 3}} + med = bgp.NewPathAttributeMultiExitDisc(50) + nexthop = bgp.NewPathAttributeNextHop("192.168.150.3") + pathAttributes = []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + updateMsg = bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path4 := ProcessMessage(updateMsg, peer3, time.Now())[0] + dd = d.Calculate(path4) + best, _, multi = dd.GetChanges(GLOBAL_RIB_NAME, 0, false) + assert.NotNil(t, best) + assert.Equal(t, len(multi), 1) + assert.Equal(t, len(d.GetKnownPathList(GLOBAL_RIB_NAME, 0)), 2) + + nexthop = bgp.NewPathAttributeNextHop("192.168.150.2") + pathAttributes = []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + updateMsg = bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path5 := ProcessMessage(updateMsg, peer2, time.Now())[0] + best, _, multi = d.Calculate(path5).GetChanges(GLOBAL_RIB_NAME, 0, false) + assert.NotNil(t, best) + assert.Equal(t, len(multi), 2) + assert.Equal(t, len(d.GetKnownPathList(GLOBAL_RIB_NAME, 0)), 3) + + UseMultiplePaths.Enabled = false +} + +func TestIdMap(t *testing.T) { + d := NewDestination(bgp.NewIPAddrPrefix(24, "10.10.0.101"), 64) + for i := 0; ; i++ { + if id, err := d.localIdMap.FindandSetZeroBit(); err == nil { + assert.Equal(t, uint(i+1), id) + } else { + assert.Equal(t, i, 63) + break + } + } + d.localIdMap.Expand() + for i := 0; i < 64; i++ { + id, _ := d.localIdMap.FindandSetZeroBit() + assert.Equal(t, id, uint(64+i)) + } + _, err := d.localIdMap.FindandSetZeroBit() + assert.NotNil(t, err) +} + +func TestGetWithdrawnPath(t *testing.T) { + attrs := []bgp.PathAttributeInterface{ + bgp.NewPathAttributeOrigin(0), + } + p1 := NewPath(nil, bgp.NewIPAddrPrefix(24, "13.2.3.0"), false, attrs, time.Now(), false) + p2 := NewPath(nil, bgp.NewIPAddrPrefix(24, "13.2.4.0"), false, attrs, time.Now(), false) + p3 := NewPath(nil, bgp.NewIPAddrPrefix(24, "13.2.5.0"), false, attrs, time.Now(), false) + + u := &Update{ + KnownPathList: []*Path{p2}, + OldKnownPathList: []*Path{p1, p2, p3}, + } + + l := u.GetWithdrawnPath() + assert.Equal(t, len(l), 2) + assert.Equal(t, l[0].GetNlri(), p1.GetNlri()) +} diff --git a/internal/pkg/table/message.go b/internal/pkg/table/message.go new file mode 100644 index 00000000..31b90596 --- /dev/null +++ b/internal/pkg/table/message.go @@ -0,0 +1,502 @@ +// Copyright (C) 2014 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + "bytes" + "reflect" + + "github.com/osrg/gobgp/pkg/packet/bgp" + log "github.com/sirupsen/logrus" +) + +func UpdatePathAttrs2ByteAs(msg *bgp.BGPUpdate) error { + ps := msg.PathAttributes + msg.PathAttributes = make([]bgp.PathAttributeInterface, len(ps)) + copy(msg.PathAttributes, ps) + var asAttr *bgp.PathAttributeAsPath + idx := 0 + for i, attr := range msg.PathAttributes { + if a, ok := attr.(*bgp.PathAttributeAsPath); ok { + asAttr = a + idx = i + break + } + } + + if asAttr == nil { + return nil + } + + as4Params := make([]*bgp.As4PathParam, 0, len(asAttr.Value)) + as2Params := make([]bgp.AsPathParamInterface, 0, len(asAttr.Value)) + mkAs4 := false + for _, param := range asAttr.Value { + segType := param.GetType() + asList := param.GetAS() + as2Path := make([]uint16, 0, len(asList)) + for _, as := range asList { + if as > (1<<16)-1 { + mkAs4 = true + as2Path = append(as2Path, bgp.AS_TRANS) + } else { + as2Path = append(as2Path, uint16(as)) + } + } + as2Params = append(as2Params, bgp.NewAsPathParam(segType, as2Path)) + + // RFC 6793 4.2.2 Generating Updates + // + // Whenever the AS path information contains the AS_CONFED_SEQUENCE or + // AS_CONFED_SET path segment, the NEW BGP speaker MUST exclude such + // path segments from the AS4_PATH attribute being constructed. + switch segType { + case bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SEQ, bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SET: + // pass + default: + if as4param, ok := param.(*bgp.As4PathParam); ok { + as4Params = append(as4Params, as4param) + } + } + } + msg.PathAttributes[idx] = bgp.NewPathAttributeAsPath(as2Params) + if mkAs4 { + msg.PathAttributes = append(msg.PathAttributes, bgp.NewPathAttributeAs4Path(as4Params)) + } + return nil +} + +func UpdatePathAttrs4ByteAs(msg *bgp.BGPUpdate) error { + var asAttr *bgp.PathAttributeAsPath + var as4Attr *bgp.PathAttributeAs4Path + asAttrPos := 0 + as4AttrPos := 0 + for i, attr := range msg.PathAttributes { + switch attr.(type) { + case *bgp.PathAttributeAsPath: + asAttr = attr.(*bgp.PathAttributeAsPath) + for j, param := range asAttr.Value { + as2Param, ok := param.(*bgp.AsPathParam) + if ok { + asPath := make([]uint32, 0, len(as2Param.AS)) + for _, as := range as2Param.AS { + asPath = append(asPath, uint32(as)) + } + as4Param := bgp.NewAs4PathParam(as2Param.Type, asPath) + asAttr.Value[j] = as4Param + } + } + asAttrPos = i + msg.PathAttributes[i] = asAttr + case *bgp.PathAttributeAs4Path: + as4AttrPos = i + as4Attr = attr.(*bgp.PathAttributeAs4Path) + } + } + + if as4Attr != nil { + msg.PathAttributes = append(msg.PathAttributes[:as4AttrPos], msg.PathAttributes[as4AttrPos+1:]...) + } + + if asAttr == nil || as4Attr == nil { + return nil + } + + asLen := 0 + asConfedLen := 0 + asParams := make([]bgp.AsPathParamInterface, 0, len(asAttr.Value)) + for _, param := range asAttr.Value { + asLen += param.ASLen() + switch param.GetType() { + case bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SET: + asConfedLen++ + case bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SEQ: + asConfedLen += len(param.GetAS()) + } + asParams = append(asParams, param) + } + + as4Len := 0 + as4Params := make([]bgp.AsPathParamInterface, 0, len(as4Attr.Value)) + if as4Attr != nil { + for _, p := range as4Attr.Value { + // RFC 6793 6. Error Handling + // + // the path segment types AS_CONFED_SEQUENCE and AS_CONFED_SET [RFC5065] + // MUST NOT be carried in the AS4_PATH attribute of an UPDATE message. + // A NEW BGP speaker that receives these path segment types in the AS4_PATH + // attribute of an UPDATE message from an OLD BGP speaker MUST discard + // these path segments, adjust the relevant attribute fields accordingly, + // and continue processing the UPDATE message. + // This case SHOULD be logged locally for analysis. + switch p.Type { + case bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SEQ, bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SET: + typ := "CONFED_SEQ" + if p.Type == bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SET { + typ = "CONFED_SET" + } + log.WithFields(log.Fields{ + "Topic": "Table", + }).Warnf("AS4_PATH contains %s segment %s. ignore", typ, p.String()) + continue + } + as4Len += p.ASLen() + as4Params = append(as4Params, p) + } + } + + if asLen+asConfedLen < as4Len { + log.WithFields(log.Fields{ + "Topic": "Table", + }).Warn("AS4_PATH is longer than AS_PATH. ignore AS4_PATH") + return nil + } + + keepNum := asLen + asConfedLen - as4Len + + newParams := make([]bgp.AsPathParamInterface, 0, len(asAttr.Value)) + for _, param := range asParams { + if keepNum-param.ASLen() >= 0 { + newParams = append(newParams, param) + keepNum -= param.ASLen() + } else { + // only SEQ param reaches here + newParams = append(newParams, bgp.NewAs4PathParam(param.GetType(), param.GetAS()[:keepNum])) + keepNum = 0 + } + + if keepNum <= 0 { + break + } + } + + for _, param := range as4Params { + lastParam := newParams[len(newParams)-1] + lastParamAS := lastParam.GetAS() + paramType := param.GetType() + paramAS := param.GetAS() + if paramType == lastParam.GetType() && paramType == bgp.BGP_ASPATH_ATTR_TYPE_SEQ { + if len(lastParamAS)+len(paramAS) > 255 { + newParams[len(newParams)-1] = bgp.NewAs4PathParam(paramType, append(lastParamAS, paramAS[:255-len(lastParamAS)]...)) + newParams = append(newParams, bgp.NewAs4PathParam(paramType, paramAS[255-len(lastParamAS):])) + } else { + newParams[len(newParams)-1] = bgp.NewAs4PathParam(paramType, append(lastParamAS, paramAS...)) + } + } else { + newParams = append(newParams, param) + } + } + + newIntfParams := make([]bgp.AsPathParamInterface, 0, len(asAttr.Value)) + newIntfParams = append(newIntfParams, newParams...) + + msg.PathAttributes[asAttrPos] = bgp.NewPathAttributeAsPath(newIntfParams) + return nil +} + +func UpdatePathAggregator2ByteAs(msg *bgp.BGPUpdate) { + as := uint32(0) + var addr string + for i, attr := range msg.PathAttributes { + switch attr.(type) { + case *bgp.PathAttributeAggregator: + agg := attr.(*bgp.PathAttributeAggregator) + addr = agg.Value.Address.String() + if agg.Value.AS > (1<<16)-1 { + as = agg.Value.AS + msg.PathAttributes[i] = bgp.NewPathAttributeAggregator(uint16(bgp.AS_TRANS), addr) + } else { + msg.PathAttributes[i] = bgp.NewPathAttributeAggregator(uint16(agg.Value.AS), addr) + } + } + } + if as != 0 { + msg.PathAttributes = append(msg.PathAttributes, bgp.NewPathAttributeAs4Aggregator(as, addr)) + } +} + +func UpdatePathAggregator4ByteAs(msg *bgp.BGPUpdate) error { + var aggAttr *bgp.PathAttributeAggregator + var agg4Attr *bgp.PathAttributeAs4Aggregator + agg4AttrPos := 0 + for i, attr := range msg.PathAttributes { + switch attr.(type) { + case *bgp.PathAttributeAggregator: + attr := attr.(*bgp.PathAttributeAggregator) + if attr.Value.Askind == reflect.Uint16 { + aggAttr = attr + aggAttr.Value.Askind = reflect.Uint32 + } + case *bgp.PathAttributeAs4Aggregator: + agg4Attr = attr.(*bgp.PathAttributeAs4Aggregator) + agg4AttrPos = i + } + } + if aggAttr == nil && agg4Attr == nil { + return nil + } + + if aggAttr == nil && agg4Attr != nil { + return bgp.NewMessageError(bgp.BGP_ERROR_UPDATE_MESSAGE_ERROR, bgp.BGP_ERROR_SUB_MALFORMED_ATTRIBUTE_LIST, nil, "AS4 AGGREGATOR attribute exists, but AGGREGATOR doesn't") + } + + if agg4Attr != nil { + msg.PathAttributes = append(msg.PathAttributes[:agg4AttrPos], msg.PathAttributes[agg4AttrPos+1:]...) + aggAttr.Value.AS = agg4Attr.Value.AS + } + return nil +} + +type cage struct { + attrsBytes []byte + paths []*Path +} + +func newCage(b []byte, path *Path) *cage { + return &cage{ + attrsBytes: b, + paths: []*Path{path}, + } +} + +type packerInterface interface { + add(*Path) + pack(options ...*bgp.MarshallingOption) []*bgp.BGPMessage +} + +type packer struct { + eof bool + family bgp.RouteFamily + total uint32 +} + +type packerMP struct { + packer + paths []*Path + withdrawals []*Path +} + +func (p *packerMP) add(path *Path) { + p.packer.total++ + + if path.IsEOR() { + p.packer.eof = true + return + } + + if path.IsWithdraw { + p.withdrawals = append(p.withdrawals, path) + return + } + + p.paths = append(p.paths, path) +} + +func createMPReachMessage(path *Path) *bgp.BGPMessage { + oattrs := path.GetPathAttrs() + attrs := make([]bgp.PathAttributeInterface, 0, len(oattrs)) + for _, a := range oattrs { + if a.GetType() == bgp.BGP_ATTR_TYPE_MP_REACH_NLRI { + attrs = append(attrs, bgp.NewPathAttributeMpReachNLRI(path.GetNexthop().String(), []bgp.AddrPrefixInterface{path.GetNlri()})) + } else { + attrs = append(attrs, a) + } + } + return bgp.NewBGPUpdateMessage(nil, attrs, nil) +} + +func (p *packerMP) pack(options ...*bgp.MarshallingOption) []*bgp.BGPMessage { + msgs := make([]*bgp.BGPMessage, 0, p.packer.total) + + for _, path := range p.withdrawals { + nlris := []bgp.AddrPrefixInterface{path.GetNlri()} + msgs = append(msgs, bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{bgp.NewPathAttributeMpUnreachNLRI(nlris)}, nil)) + } + + for _, path := range p.paths { + msgs = append(msgs, createMPReachMessage(path)) + } + + if p.eof { + msgs = append(msgs, bgp.NewEndOfRib(p.family)) + } + return msgs +} + +func newPackerMP(f bgp.RouteFamily) *packerMP { + return &packerMP{ + packer: packer{ + family: f, + }, + withdrawals: make([]*Path, 0), + paths: make([]*Path, 0), + } +} + +type packerV4 struct { + packer + hashmap map[uint32][]*cage + mpPaths []*Path + withdrawals []*Path +} + +func (p *packerV4) add(path *Path) { + p.packer.total++ + + if path.IsEOR() { + p.packer.eof = true + return + } + + if path.IsWithdraw { + p.withdrawals = append(p.withdrawals, path) + return + } + + if path.GetNexthop().To4() == nil { + // RFC 5549 + p.mpPaths = append(p.mpPaths, path) + return + } + + key := path.GetHash() + attrsB := bytes.NewBuffer(make([]byte, 0)) + for _, v := range path.GetPathAttrs() { + b, _ := v.Serialize() + attrsB.Write(b) + } + + if cages, y := p.hashmap[key]; y { + added := false + for _, c := range cages { + if bytes.Equal(c.attrsBytes, attrsB.Bytes()) { + c.paths = append(c.paths, path) + added = true + break + } + } + if !added { + p.hashmap[key] = append(p.hashmap[key], newCage(attrsB.Bytes(), path)) + } + } else { + p.hashmap[key] = []*cage{newCage(attrsB.Bytes(), path)} + } +} + +func (p *packerV4) pack(options ...*bgp.MarshallingOption) []*bgp.BGPMessage { + split := func(max int, paths []*Path) ([]*bgp.IPAddrPrefix, []*Path) { + nlris := make([]*bgp.IPAddrPrefix, 0, max) + i := 0 + if max > len(paths) { + max = len(paths) + } + for ; i < max; i++ { + nlris = append(nlris, paths[i].GetNlri().(*bgp.IPAddrPrefix)) + } + return nlris, paths[i:] + } + addpathNLRILen := 0 + if bgp.IsAddPathEnabled(false, p.packer.family, options) { + addpathNLRILen = 4 + } + // Header + Update (WithdrawnRoutesLen + + // TotalPathAttributeLen + attributes + maxlen of NLRI). + // the max size of NLRI is 5bytes (plus 4bytes with addpath enabled) + maxNLRIs := func(attrsLen int) int { + return (bgp.BGP_MAX_MESSAGE_LENGTH - (19 + 2 + 2 + attrsLen)) / (5 + addpathNLRILen) + } + + loop := func(attrsLen int, paths []*Path, cb func([]*bgp.IPAddrPrefix)) { + max := maxNLRIs(attrsLen) + var nlris []*bgp.IPAddrPrefix + for { + nlris, paths = split(max, paths) + if len(nlris) == 0 { + break + } + cb(nlris) + } + } + + msgs := make([]*bgp.BGPMessage, 0, p.packer.total) + + loop(0, p.withdrawals, func(nlris []*bgp.IPAddrPrefix) { + msgs = append(msgs, bgp.NewBGPUpdateMessage(nlris, nil, nil)) + }) + + for _, cages := range p.hashmap { + for _, c := range cages { + paths := c.paths + + attrs := paths[0].GetPathAttrs() + attrsLen := 0 + for _, a := range attrs { + attrsLen += a.Len() + } + + loop(attrsLen, paths, func(nlris []*bgp.IPAddrPrefix) { + msgs = append(msgs, bgp.NewBGPUpdateMessage(nil, attrs, nlris)) + }) + } + } + + for _, path := range p.mpPaths { + msgs = append(msgs, createMPReachMessage(path)) + } + + if p.eof { + msgs = append(msgs, bgp.NewEndOfRib(p.family)) + } + return msgs +} + +func newPackerV4(f bgp.RouteFamily) *packerV4 { + return &packerV4{ + packer: packer{ + family: f, + }, + hashmap: make(map[uint32][]*cage), + withdrawals: make([]*Path, 0), + mpPaths: make([]*Path, 0), + } +} + +func newPacker(f bgp.RouteFamily) packerInterface { + switch f { + case bgp.RF_IPv4_UC: + return newPackerV4(bgp.RF_IPv4_UC) + default: + return newPackerMP(f) + } +} + +func CreateUpdateMsgFromPaths(pathList []*Path, options ...*bgp.MarshallingOption) []*bgp.BGPMessage { + msgs := make([]*bgp.BGPMessage, 0, len(pathList)) + + m := make(map[bgp.RouteFamily]packerInterface) + for _, path := range pathList { + f := path.GetRouteFamily() + if _, y := m[f]; !y { + m[f] = newPacker(f) + } + m[f].add(path) + } + + for _, p := range m { + msgs = append(msgs, p.pack(options...)...) + } + return msgs +} diff --git a/internal/pkg/table/message_test.go b/internal/pkg/table/message_test.go new file mode 100644 index 00000000..28a380fe --- /dev/null +++ b/internal/pkg/table/message_test.go @@ -0,0 +1,663 @@ +// Copyright (C) 2014 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + "fmt" + "reflect" + "testing" + "time" + + "github.com/osrg/gobgp/pkg/packet/bgp" + "github.com/stretchr/testify/assert" +) + +// before: +// as-path : 65000, 4000, 400000, 300000, 40001 +// expected result: +// as-path : 65000, 4000, 23456, 23456, 40001 +// as4-path : 65000, 4000, 400000, 300000, 40001 +func TestAsPathAs2Trans1(t *testing.T) { + as := []uint32{65000, 4000, 400000, 300000, 40001} + params := []bgp.AsPathParamInterface{bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as)} + aspath := bgp.NewPathAttributeAsPath(params) + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{aspath}, nil).Body.(*bgp.BGPUpdate) + UpdatePathAttrs2ByteAs(msg) + assert.Equal(t, len(msg.PathAttributes), 2) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.AsPathParam).AS), 5) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.AsPathParam).AS[0], uint16(65000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.AsPathParam).AS[1], uint16(4000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.AsPathParam).AS[2], uint16(bgp.AS_TRANS)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.AsPathParam).AS[3], uint16(bgp.AS_TRANS)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.AsPathParam).AS[4], uint16(40001)) + assert.Equal(t, len(msg.PathAttributes[1].(*bgp.PathAttributeAs4Path).Value), 1) + assert.Equal(t, len(msg.PathAttributes[1].(*bgp.PathAttributeAs4Path).Value[0].AS), 5) + assert.Equal(t, msg.PathAttributes[1].(*bgp.PathAttributeAs4Path).Value[0].AS[0], uint32(65000)) + assert.Equal(t, msg.PathAttributes[1].(*bgp.PathAttributeAs4Path).Value[0].AS[1], uint32(4000)) + assert.Equal(t, msg.PathAttributes[1].(*bgp.PathAttributeAs4Path).Value[0].AS[2], uint32(400000)) + assert.Equal(t, msg.PathAttributes[1].(*bgp.PathAttributeAs4Path).Value[0].AS[3], uint32(300000)) + assert.Equal(t, msg.PathAttributes[1].(*bgp.PathAttributeAs4Path).Value[0].AS[4], uint32(40001)) +} + +// before: +// as-path : 65000, 4000, 40000, 30000, 40001 +// expected result: +// as-path : 65000, 4000, 40000, 30000, 40001 +func TestAsPathAs2Trans2(t *testing.T) { + as := []uint32{65000, 4000, 40000, 30000, 40001} + params := []bgp.AsPathParamInterface{bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as)} + aspath := bgp.NewPathAttributeAsPath(params) + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{aspath}, nil).Body.(*bgp.BGPUpdate) + UpdatePathAttrs2ByteAs(msg) + assert.Equal(t, len(msg.PathAttributes), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.AsPathParam).AS), 5) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.AsPathParam).AS[0], uint16(65000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.AsPathParam).AS[1], uint16(4000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.AsPathParam).AS[2], uint16(40000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.AsPathParam).AS[3], uint16(30000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.AsPathParam).AS[4], uint16(40001)) +} + +// before: +// as-path : 65000, 4000, 23456, 23456, 40001 +// as4-path : 400000, 300000, 40001 +// expected result: +// as-path : 65000, 4000, 400000, 300000, 40001 +func TestAsPathAs4Trans1(t *testing.T) { + as := []uint16{65000, 4000, bgp.AS_TRANS, bgp.AS_TRANS, 40001} + params := []bgp.AsPathParamInterface{bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as)} + aspath := bgp.NewPathAttributeAsPath(params) + + as4 := []uint32{400000, 300000, 40001} + param4s := []*bgp.As4PathParam{bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as4)} + as4path := bgp.NewPathAttributeAs4Path(param4s) + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{aspath, as4path}, nil).Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(msg) + assert.Equal(t, len(msg.PathAttributes), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS), 5) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[0], uint32(65000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[1], uint32(4000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[2], uint32(400000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[3], uint32(300000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[4], uint32(40001)) +} + +// before: +// as-path : 65000, 4000, {10, 20, 30}, 23456, 23456, 40001 +// as4-path : 400000, 300000, 40001 +// expected result: +// as-path : 65000, 4000, {10, 20, 30}, 400000, 300000, 40001 +func TestAsPathAs4Trans2(t *testing.T) { + as1 := []uint16{65000, 4000} + param1 := bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as1) + as2 := []uint16{10, 20, 30} + param2 := bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SET, as2) + as3 := []uint16{bgp.AS_TRANS, bgp.AS_TRANS, 40001} + param3 := bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as3) + params := []bgp.AsPathParamInterface{param1, param2, param3} + aspath := bgp.NewPathAttributeAsPath(params) + + as4 := []uint32{400000, 300000, 40001} + param4s := []*bgp.As4PathParam{bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as4)} + as4path := bgp.NewPathAttributeAs4Path(param4s) + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{aspath, as4path}, nil).Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(msg) + assert.Equal(t, len(msg.PathAttributes), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value), 3) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS), 2) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[0], uint32(65000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[1], uint32(4000)) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[1].(*bgp.As4PathParam).AS), 3) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[1].(*bgp.As4PathParam).AS[0], uint32(10)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[1].(*bgp.As4PathParam).AS[1], uint32(20)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[1].(*bgp.As4PathParam).AS[2], uint32(30)) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[2].(*bgp.As4PathParam).AS), 3) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[2].(*bgp.As4PathParam).AS[0], uint32(400000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[2].(*bgp.As4PathParam).AS[1], uint32(300000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[2].(*bgp.As4PathParam).AS[2], uint32(40001)) +} + +// before: +// as-path : 65000, 4000, {10, 20, 30}, 23456, 23456, 40001 +// as4-path : 3000, 400000, 300000, 40001 +// expected result: +// as-path : 65000, 4000, 3000, 400000, 300000, 40001 +func TestAsPathAs4Trans3(t *testing.T) { + as1 := []uint16{65000, 4000} + param1 := bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as1) + as2 := []uint16{10, 20, 30} + param2 := bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SET, as2) + as3 := []uint16{bgp.AS_TRANS, bgp.AS_TRANS, 40001} + param3 := bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as3) + params := []bgp.AsPathParamInterface{param1, param2, param3} + aspath := bgp.NewPathAttributeAsPath(params) + + as4 := []uint32{3000, 400000, 300000, 40001} + param4s := []*bgp.As4PathParam{bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as4)} + as4path := bgp.NewPathAttributeAs4Path(param4s) + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{aspath, as4path}, nil).Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(msg) + assert.Equal(t, len(msg.PathAttributes), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS), 6) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[0], uint32(65000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[1], uint32(4000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[2], uint32(3000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[3], uint32(400000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[4], uint32(300000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[5], uint32(40001)) +} + +// before: +// as-path : 65000, 4000, 23456, 23456, 40001 +// as4-path : 400000, 300000, 40001, {10, 20, 30} +// expected result: +// as-path : 65000, 400000, 300000, 40001, {10, 20, 30} +func TestAsPathAs4Trans4(t *testing.T) { + as := []uint16{65000, 4000, bgp.AS_TRANS, bgp.AS_TRANS, 40001} + params := []bgp.AsPathParamInterface{bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as)} + aspath := bgp.NewPathAttributeAsPath(params) + + as4 := []uint32{400000, 300000, 40001} + as4param1 := bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as4) + as5 := []uint32{10, 20, 30} + as4param2 := bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SET, as5) + param4s := []*bgp.As4PathParam{as4param1, as4param2} + as4path := bgp.NewPathAttributeAs4Path(param4s) + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{aspath, as4path}, nil).Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(msg) + assert.Equal(t, len(msg.PathAttributes), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value), 2) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS), 4) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[0], uint32(65000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[1], uint32(400000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[2], uint32(300000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[3], uint32(40001)) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[1].(*bgp.As4PathParam).AS), 3) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[1].(*bgp.As4PathParam).AS[0], uint32(10)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[1].(*bgp.As4PathParam).AS[1], uint32(20)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[1].(*bgp.As4PathParam).AS[2], uint32(30)) +} + +// before: +// as-path : 65000, 4000, 23456, 23456, 40001 +// as4-path : {10, 20, 30} 400000, 300000, 40001 +// expected result: +// as-path : 65000, {10, 20, 30}, 400000, 300000, 40001 +func TestAsPathAs4Trans5(t *testing.T) { + as := []uint16{65000, 4000, bgp.AS_TRANS, bgp.AS_TRANS, 40001} + params := []bgp.AsPathParamInterface{bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as)} + aspath := bgp.NewPathAttributeAsPath(params) + + as4 := []uint32{400000, 300000, 40001} + as4param1 := bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as4) + as5 := []uint32{10, 20, 30} + as4param2 := bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SET, as5) + param4s := []*bgp.As4PathParam{as4param2, as4param1} + as4path := bgp.NewPathAttributeAs4Path(param4s) + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{aspath, as4path}, nil).Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(msg) + assert.Equal(t, len(msg.PathAttributes), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value), 3) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS), 1) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[0], uint32(65000)) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[1].(*bgp.As4PathParam).AS), 3) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[1].(*bgp.As4PathParam).AS[0], uint32(10)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[1].(*bgp.As4PathParam).AS[1], uint32(20)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[1].(*bgp.As4PathParam).AS[2], uint32(30)) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[2].(*bgp.As4PathParam).AS), 3) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[2].(*bgp.As4PathParam).AS[0], uint32(400000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[2].(*bgp.As4PathParam).AS[1], uint32(300000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[2].(*bgp.As4PathParam).AS[2], uint32(40001)) +} + +// before: +// as-path : 65000, 4000, 23456, 23456, 40001 +// as4-path : 100000, 65000, 4000, 400000, 300000, 40001 +// expected result: +// as-path : 65000, 4000, 23456, 23456, 40001 +func TestAsPathAs4TransInvalid1(t *testing.T) { + as := []uint16{65000, 4000, bgp.AS_TRANS, bgp.AS_TRANS, 40001} + params := []bgp.AsPathParamInterface{bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as)} + aspath := bgp.NewPathAttributeAsPath(params) + + as4 := []uint32{100000, 65000, 4000, 400000, 300000, 40001} + as4param1 := bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as4) + param4s := []*bgp.As4PathParam{as4param1} + as4path := bgp.NewPathAttributeAs4Path(param4s) + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{aspath, as4path}, nil).Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(msg) + assert.Equal(t, len(msg.PathAttributes), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS), 5) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[0], uint32(65000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[1], uint32(4000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[2], uint32(bgp.AS_TRANS)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[3], uint32(bgp.AS_TRANS)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[4], uint32(40001)) +} + +// before: +// as-path : 65000, 4000, 23456, 23456, 40001 +// as4-path : 300000, 40001 +// expected result: +// as-path : 65000, 4000, 23456, 300000, 40001 +func TestAsPathAs4TransInvalid2(t *testing.T) { + as := []uint16{65000, 4000, bgp.AS_TRANS, bgp.AS_TRANS, 40001} + params := []bgp.AsPathParamInterface{bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as)} + aspath := bgp.NewPathAttributeAsPath(params) + + as4 := []uint32{300000, 40001} + as4param1 := bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as4) + param4s := []*bgp.As4PathParam{as4param1} + as4path := bgp.NewPathAttributeAs4Path(param4s) + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{aspath, as4path}, nil).Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(msg) + assert.Equal(t, len(msg.PathAttributes), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS), 5) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[0], uint32(65000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[1], uint32(4000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[2], uint32(bgp.AS_TRANS)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[3], uint32(300000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[4], uint32(40001)) +} + +// before: +// as-path : 65000, 4000, 23456, 23456, 40001 +// as4-path : nil +// expected result: +// as-path : 65000, 4000, 23456, 23456, 40001 +func TestAsPathAs4TransInvalid3(t *testing.T) { + as := []uint16{65000, 4000, bgp.AS_TRANS, bgp.AS_TRANS, 40001} + params := []bgp.AsPathParamInterface{bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as)} + aspath := bgp.NewPathAttributeAsPath(params) + + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{aspath}, nil).Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(msg) + assert.Equal(t, len(msg.PathAttributes), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS), 5) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[0], uint32(65000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[1], uint32(4000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[2], uint32(bgp.AS_TRANS)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[3], uint32(bgp.AS_TRANS)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[4], uint32(40001)) +} + +// before: +// as-path : 65000, 4000, 23456, 23456, 40001 +// as4-path : empty +// expected result: +// as-path : 65000, 4000, 23456, 23456, 40001 +func TestAsPathAs4TransInvalid4(t *testing.T) { + as := []uint16{65000, 4000, bgp.AS_TRANS, bgp.AS_TRANS, 40001} + params := []bgp.AsPathParamInterface{bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as)} + aspath := bgp.NewPathAttributeAsPath(params) + + as4 := []uint32{} + as4param1 := bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as4) + param4s := []*bgp.As4PathParam{as4param1} + as4path := bgp.NewPathAttributeAs4Path(param4s) + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{aspath, as4path}, nil).Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(msg) + assert.Equal(t, len(msg.PathAttributes), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value), 1) + assert.Equal(t, len(msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS), 5) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[0], uint32(65000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[1], uint32(4000)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[2], uint32(bgp.AS_TRANS)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[3], uint32(bgp.AS_TRANS)) + assert.Equal(t, msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value[0].(*bgp.As4PathParam).AS[4], uint32(40001)) +} + +func TestASPathAs4TransMultipleParams(t *testing.T) { + as1 := []uint16{17676, 2914, 174, 50607} + as2 := []uint16{bgp.AS_TRANS, bgp.AS_TRANS} + params := []bgp.AsPathParamInterface{bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as1), bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as2)} + aspath := bgp.NewPathAttributeAsPath(params) + + as41 := []uint32{2914, 174, 50607} + as42 := []uint32{198035, 198035} + as4param1 := bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as41) + as4param2 := bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as42) + param4s := []*bgp.As4PathParam{as4param1, as4param2} + as4path := bgp.NewPathAttributeAs4Path(param4s) + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{aspath, as4path}, nil).Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(msg) + for _, param := range msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value { + p := param.(*bgp.As4PathParam) + assert.Equal(t, p.Num, uint8(len(p.AS))) + } +} + +func TestASPathAs4TransMultipleLargeParams(t *testing.T) { + as1 := make([]uint16, 0, 255) + for i := 0; i < 255-5; i++ { + as1 = append(as1, uint16(i+1)) + } + as1 = append(as1, []uint16{17676, 2914, 174, 50607}...) + as2 := []uint16{bgp.AS_TRANS, bgp.AS_TRANS} + params := []bgp.AsPathParamInterface{bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as1), bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as2)} + aspath := bgp.NewPathAttributeAsPath(params) + + as41 := []uint32{2914, 174, 50607} + as42 := []uint32{198035, 198035} + as4param1 := bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as41) + as4param2 := bgp.NewAs4PathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, as42) + param4s := []*bgp.As4PathParam{as4param1, as4param2} + as4path := bgp.NewPathAttributeAs4Path(param4s) + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{aspath, as4path}, nil).Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(msg) + for _, param := range msg.PathAttributes[0].(*bgp.PathAttributeAsPath).Value { + p := param.(*bgp.As4PathParam) + assert.Equal(t, p.Num, uint8(len(p.AS))) + } +} + +func TestAggregator4BytesASes(t *testing.T) { + getAggr := func(msg *bgp.BGPUpdate) *bgp.PathAttributeAggregator { + for _, attr := range msg.PathAttributes { + switch attr.(type) { + case *bgp.PathAttributeAggregator: + return attr.(*bgp.PathAttributeAggregator) + } + } + return nil + } + + getAggr4 := func(msg *bgp.BGPUpdate) *bgp.PathAttributeAs4Aggregator { + for _, attr := range msg.PathAttributes { + switch attr.(type) { + case *bgp.PathAttributeAs4Aggregator: + return attr.(*bgp.PathAttributeAs4Aggregator) + } + } + return nil + } + + addr := "192.168.0.1" + as4 := uint32(100000) + as := uint32(1000) + msg := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{bgp.NewPathAttributeAggregator(as4, addr)}, nil).Body.(*bgp.BGPUpdate) + + // 4byte capable to 4byte capable for 4 bytes AS + assert.Equal(t, UpdatePathAggregator4ByteAs(msg), nil) + assert.Equal(t, getAggr(msg).Value.AS, as4) + assert.Equal(t, getAggr(msg).Value.Address.String(), addr) + + // 4byte capable to 2byte capable for 4 bytes AS + UpdatePathAggregator2ByteAs(msg) + assert.Equal(t, getAggr(msg).Value.AS, uint32(bgp.AS_TRANS)) + assert.Equal(t, getAggr(msg).Value.Askind, reflect.Uint16) + assert.Equal(t, getAggr4(msg).Value.AS, as4) + assert.Equal(t, getAggr4(msg).Value.Address.String(), addr) + + msg = bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{bgp.NewPathAttributeAggregator(uint16(bgp.AS_TRANS), addr), bgp.NewPathAttributeAs4Aggregator(as4, addr)}, nil).Body.(*bgp.BGPUpdate) + assert.Equal(t, getAggr(msg).Value.AS, uint32(bgp.AS_TRANS)) + assert.Equal(t, getAggr(msg).Value.Askind, reflect.Uint16) + + // non 4byte capable to 4byte capable for 4 bytes AS + assert.Equal(t, UpdatePathAggregator4ByteAs(msg), nil) + assert.Equal(t, getAggr(msg).Value.AS, as4) + assert.Equal(t, getAggr(msg).Value.Askind, reflect.Uint32) + assert.Equal(t, getAggr(msg).Value.Address.String(), addr) + assert.Equal(t, getAggr4(msg), (*bgp.PathAttributeAs4Aggregator)(nil)) + + // non 4byte capable to non 4byte capable for 4 bytes AS + UpdatePathAggregator2ByteAs(msg) + assert.Equal(t, getAggr(msg).Value.AS, uint32(bgp.AS_TRANS)) + assert.Equal(t, getAggr(msg).Value.Askind, reflect.Uint16) + assert.Equal(t, getAggr4(msg).Value.AS, as4) + assert.Equal(t, getAggr4(msg).Value.Address.String(), addr) + + msg = bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{bgp.NewPathAttributeAggregator(uint32(as), addr)}, nil).Body.(*bgp.BGPUpdate) + // 4byte capable to 4byte capable for 2 bytes AS + assert.Equal(t, getAggr(msg).Value.AS, as) + assert.Equal(t, getAggr(msg).Value.Askind, reflect.Uint32) + assert.Equal(t, UpdatePathAggregator4ByteAs(msg), nil) + assert.Equal(t, getAggr(msg).Value.AS, as) + assert.Equal(t, getAggr(msg).Value.Askind, reflect.Uint32) + + // 4byte capable to non 4byte capable for 2 bytes AS + UpdatePathAggregator2ByteAs(msg) + assert.Equal(t, getAggr4(msg), (*bgp.PathAttributeAs4Aggregator)(nil)) + assert.Equal(t, getAggr(msg).Value.Askind, reflect.Uint16) + assert.Equal(t, getAggr(msg).Value.AS, as) + + msg = bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{bgp.NewPathAttributeAggregator(uint16(as), addr)}, nil).Body.(*bgp.BGPUpdate) + // non 4byte capable to 4byte capable for 2 bytes AS + assert.Equal(t, getAggr(msg).Value.AS, as) + assert.Equal(t, getAggr(msg).Value.Askind, reflect.Uint16) + assert.Equal(t, UpdatePathAggregator4ByteAs(msg), nil) + + assert.Equal(t, getAggr(msg).Value.AS, as) + assert.Equal(t, getAggr(msg).Value.Askind, reflect.Uint32) + + // non 4byte capable to non 4byte capable for 2 bytes AS + UpdatePathAggregator2ByteAs(msg) + assert.Equal(t, getAggr(msg).Value.AS, as) + assert.Equal(t, getAggr(msg).Value.Askind, reflect.Uint16) + assert.Equal(t, getAggr4(msg), (*bgp.PathAttributeAs4Aggregator)(nil)) +} + +func TestBMP(t *testing.T) { + aspath1 := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{1000000}), + bgp.NewAs4PathParam(1, []uint32{1000001, 1002}), + bgp.NewAs4PathParam(2, []uint32{1003, 100004}), + } + mp_nlri := []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(100, + "fe80:1234:1234:5667:8967:af12:8912:1023")} + + p := []bgp.PathAttributeInterface{ + bgp.NewPathAttributeOrigin(3), + bgp.NewPathAttributeAsPath(aspath1), + bgp.NewPathAttributeMpUnreachNLRI(mp_nlri), + } + w := []*bgp.IPAddrPrefix{} + n := []*bgp.IPAddrPrefix{} + + msg := bgp.NewBGPUpdateMessage(w, p, n) + pList := ProcessMessage(msg, peerR1(), time.Now()) + CreateUpdateMsgFromPaths(pList) +} + +func unreachIndex(msgs []*bgp.BGPMessage) int { + for i, _ := range msgs { + for _, a := range msgs[i].Body.(*bgp.BGPUpdate).PathAttributes { + if a.GetType() == bgp.BGP_ATTR_TYPE_MP_UNREACH_NLRI { + return i + } + } + } + // should not be here + return -1 +} + +func TestMixedMPReachMPUnreach(t *testing.T) { + aspath1 := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{100}), + } + nlri1 := []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(32, "2222::")} + nlri2 := []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(32, "1111::")} + + p := []bgp.PathAttributeInterface{ + bgp.NewPathAttributeOrigin(0), + bgp.NewPathAttributeAsPath(aspath1), + bgp.NewPathAttributeMpReachNLRI("1::1", nlri1), + bgp.NewPathAttributeMpUnreachNLRI(nlri2), + } + msg := bgp.NewBGPUpdateMessage(nil, p, nil) + pList := ProcessMessage(msg, peerR1(), time.Now()) + assert.Equal(t, len(pList), 2) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.Equal(t, pList[1].IsWithdraw, true) + msgs := CreateUpdateMsgFromPaths(pList) + assert.Equal(t, len(msgs), 2) + + uIndex := unreachIndex(msgs) + rIndex := 0 + if uIndex == 0 { + rIndex = 1 + } + assert.Equal(t, len(msgs[uIndex].Body.(*bgp.BGPUpdate).PathAttributes), 1) + assert.Equal(t, len(msgs[rIndex].Body.(*bgp.BGPUpdate).PathAttributes), 3) +} + +func TestMixedNLRIAndMPUnreach(t *testing.T) { + aspath1 := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{100}), + } + nlri1 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.0.0.0")} + nlri2 := []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(32, "1111::")} + + p := []bgp.PathAttributeInterface{ + bgp.NewPathAttributeOrigin(0), + bgp.NewPathAttributeAsPath(aspath1), + bgp.NewPathAttributeNextHop("1.1.1.1"), + bgp.NewPathAttributeMpUnreachNLRI(nlri2), + } + msg := bgp.NewBGPUpdateMessage(nil, p, nlri1) + pList := ProcessMessage(msg, peerR1(), time.Now()) + + assert.Equal(t, len(pList), 2) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.Equal(t, pList[1].IsWithdraw, true) + msgs := CreateUpdateMsgFromPaths(pList) + assert.Equal(t, len(msgs), 2) + + uIndex := unreachIndex(msgs) + rIndex := 0 + if uIndex == 0 { + rIndex = 1 + } + assert.Equal(t, len(msgs[uIndex].Body.(*bgp.BGPUpdate).PathAttributes), 1) + assert.Equal(t, len(msgs[rIndex].Body.(*bgp.BGPUpdate).PathAttributes), 3) +} + +func TestMergeV4NLRIs(t *testing.T) { + aspath1 := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{100}), + } + attrs := []bgp.PathAttributeInterface{ + bgp.NewPathAttributeOrigin(0), + bgp.NewPathAttributeAsPath(aspath1), + bgp.NewPathAttributeNextHop("1.1.1.1"), + } + + nr := 1024 + paths := make([]*Path, 0, nr) + addrs := make([]string, 0, nr) + for i := 0; i < nr; i++ { + addrs = append(addrs, fmt.Sprintf("1.1.%d.%d", i>>8&0xff, i&0xff)) + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(32, addrs[i])} + msg := bgp.NewBGPUpdateMessage(nil, attrs, nlri) + paths = append(paths, ProcessMessage(msg, peerR1(), time.Now())...) + } + msgs := CreateUpdateMsgFromPaths(paths) + assert.Equal(t, len(msgs), 2) + + l := make([]*bgp.IPAddrPrefix, 0, nr) + for _, msg := range msgs { + u := msg.Body.(*bgp.BGPUpdate) + assert.Equal(t, len(u.PathAttributes), 3) + l = append(l, u.NLRI...) + } + + assert.Equal(t, len(l), nr) + for i, addr := range addrs { + assert.Equal(t, addr, l[i].Prefix.String()) + } + for _, msg := range msgs { + d, _ := msg.Serialize() + assert.True(t, len(d) < bgp.BGP_MAX_MESSAGE_LENGTH) + } +} + +func TestNotMergeV4NLRIs(t *testing.T) { + paths := make([]*Path, 0, 2) + + aspath1 := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{100}), + } + attrs1 := []bgp.PathAttributeInterface{ + bgp.NewPathAttributeOrigin(0), + bgp.NewPathAttributeAsPath(aspath1), + bgp.NewPathAttributeNextHop("1.1.1.1"), + } + nlri1 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(32, "1.1.1.1")} + paths = append(paths, ProcessMessage(bgp.NewBGPUpdateMessage(nil, attrs1, nlri1), peerR1(), time.Now())...) + + attrs2 := []bgp.PathAttributeInterface{ + bgp.NewPathAttributeOrigin(0), + bgp.NewPathAttributeAsPath(aspath1), + bgp.NewPathAttributeNextHop("2.2.2.2"), + } + nlri2 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(32, "2.2.2.2")} + paths = append(paths, ProcessMessage(bgp.NewBGPUpdateMessage(nil, attrs2, nlri2), peerR1(), time.Now())...) + + assert.NotEmpty(t, paths[0].GetHash(), paths[1].GetHash()) + + msgs := CreateUpdateMsgFromPaths(paths) + assert.Equal(t, len(msgs), 2) + + paths[1].SetHash(paths[0].GetHash()) + msgs = CreateUpdateMsgFromPaths(paths) + assert.Equal(t, len(msgs), 2) +} + +func TestMergeV4Withdraw(t *testing.T) { + nr := 1024 + paths := make([]*Path, 0, nr) + addrs := make([]string, 0, nr) + for i := 0; i < nr; i++ { + addrs = append(addrs, fmt.Sprintf("1.1.%d.%d", i>>8&0xff, i&0xff)) + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(32, addrs[i])} + // use different attribute for each nlri + aspath1 := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{uint32(i)}), + } + attrs := []bgp.PathAttributeInterface{ + bgp.NewPathAttributeOrigin(0), + bgp.NewPathAttributeAsPath(aspath1), + bgp.NewPathAttributeNextHop("1.1.1.1"), + } + msg := bgp.NewBGPUpdateMessage(nlri, attrs, nil) + paths = append(paths, ProcessMessage(msg, peerR1(), time.Now())...) + } + msgs := CreateUpdateMsgFromPaths(paths) + assert.Equal(t, len(msgs), 2) + + l := make([]*bgp.IPAddrPrefix, 0, nr) + for _, msg := range msgs { + u := msg.Body.(*bgp.BGPUpdate) + assert.Equal(t, len(u.PathAttributes), 0) + l = append(l, u.WithdrawnRoutes...) + } + assert.Equal(t, len(l), nr) + for i, addr := range addrs { + assert.Equal(t, addr, l[i].Prefix.String()) + } + + for _, msg := range msgs { + d, _ := msg.Serialize() + assert.True(t, len(d) < bgp.BGP_MAX_MESSAGE_LENGTH) + } +} diff --git a/internal/pkg/table/path.go b/internal/pkg/table/path.go new file mode 100644 index 00000000..14bbe6ae --- /dev/null +++ b/internal/pkg/table/path.go @@ -0,0 +1,1179 @@ +// Copyright (C) 2014 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + "bytes" + "encoding/json" + "fmt" + "math" + "net" + "sort" + "time" + + "github.com/osrg/gobgp/internal/pkg/config" + "github.com/osrg/gobgp/pkg/packet/bgp" + + log "github.com/sirupsen/logrus" +) + +const ( + DEFAULT_LOCAL_PREF = 100 +) + +type Bitmap struct { + bitmap []uint64 +} + +func (b *Bitmap) Flag(i uint) { + b.bitmap[i/64] |= 1 << uint(i%64) +} + +func (b *Bitmap) Unflag(i uint) { + b.bitmap[i/64] &^= 1 << uint(i%64) +} + +func (b *Bitmap) GetFlag(i uint) bool { + return b.bitmap[i/64]&(1<<uint(i%64)) > 0 +} + +func (b *Bitmap) FindandSetZeroBit() (uint, error) { + for i := 0; i < len(b.bitmap); i++ { + if b.bitmap[i] == math.MaxUint64 { + continue + } + // replace this with TrailingZero64() when gobgp drops go 1.8 support. + for j := 0; j < 64; j++ { + v := ^b.bitmap[i] + if v&(1<<uint64(j)) > 0 { + r := i*64 + j + b.Flag(uint(r)) + return uint(r), nil + } + } + } + return 0, fmt.Errorf("no space") +} + +func (b *Bitmap) Expand() { + old := b.bitmap + new := make([]uint64, len(old)+1) + for i := 0; i < len(old); i++ { + new[i] = old[i] + } + b.bitmap = new +} + +func NewBitmap(size int) *Bitmap { + b := &Bitmap{} + if size != 0 { + b.bitmap = make([]uint64, (size+64-1)/64) + } + return b +} + +type originInfo struct { + nlri bgp.AddrPrefixInterface + source *PeerInfo + timestamp int64 + validation *Validation + noImplicitWithdraw bool + isFromExternal bool + eor bool + stale bool +} + +type RpkiValidationReasonType string + +const ( + RPKI_VALIDATION_REASON_TYPE_NONE RpkiValidationReasonType = "none" + RPKI_VALIDATION_REASON_TYPE_AS RpkiValidationReasonType = "as" + RPKI_VALIDATION_REASON_TYPE_LENGTH RpkiValidationReasonType = "length" +) + +var RpkiValidationReasonTypeToIntMap = map[RpkiValidationReasonType]int{ + RPKI_VALIDATION_REASON_TYPE_NONE: 0, + RPKI_VALIDATION_REASON_TYPE_AS: 1, + RPKI_VALIDATION_REASON_TYPE_LENGTH: 2, +} + +func (v RpkiValidationReasonType) ToInt() int { + i, ok := RpkiValidationReasonTypeToIntMap[v] + if !ok { + return -1 + } + return i +} + +var IntToRpkiValidationReasonTypeMap = map[int]RpkiValidationReasonType{ + 0: RPKI_VALIDATION_REASON_TYPE_NONE, + 1: RPKI_VALIDATION_REASON_TYPE_AS, + 2: RPKI_VALIDATION_REASON_TYPE_LENGTH, +} + +type Validation struct { + Status config.RpkiValidationResultType + Reason RpkiValidationReasonType + Matched []*ROA + UnmatchedAs []*ROA + UnmatchedLength []*ROA +} + +type Path struct { + info *originInfo + parent *Path + pathAttrs []bgp.PathAttributeInterface + dels []bgp.BGPAttrType + attrsHash uint32 + aslooped bool + reason BestPathReason + + // For BGP Nexthop Tracking, this field shows if nexthop is invalidated by IGP. + IsNexthopInvalid bool + IsWithdraw bool +} + +func NewPath(source *PeerInfo, nlri bgp.AddrPrefixInterface, isWithdraw bool, pattrs []bgp.PathAttributeInterface, timestamp time.Time, noImplicitWithdraw bool) *Path { + if !isWithdraw && pattrs == nil { + log.WithFields(log.Fields{ + "Topic": "Table", + "Key": nlri.String(), + }).Error("Need to provide path attributes for non-withdrawn path.") + return nil + } + + return &Path{ + info: &originInfo{ + nlri: nlri, + source: source, + timestamp: timestamp.Unix(), + noImplicitWithdraw: noImplicitWithdraw, + }, + IsWithdraw: isWithdraw, + pathAttrs: pattrs, + } +} + +func NewEOR(family bgp.RouteFamily) *Path { + afi, safi := bgp.RouteFamilyToAfiSafi(family) + nlri, _ := bgp.NewPrefixFromRouteFamily(afi, safi) + return &Path{ + info: &originInfo{ + nlri: nlri, + eor: true, + }, + } +} + +func (path *Path) IsEOR() bool { + if path.info != nil && path.info.eor { + return true + } + return false +} + +func cloneAsPath(asAttr *bgp.PathAttributeAsPath) *bgp.PathAttributeAsPath { + newASparams := make([]bgp.AsPathParamInterface, len(asAttr.Value)) + for i, param := range asAttr.Value { + asList := param.GetAS() + as := make([]uint32, len(asList)) + copy(as, asList) + newASparams[i] = bgp.NewAs4PathParam(param.GetType(), as) + } + return bgp.NewPathAttributeAsPath(newASparams) +} + +func UpdatePathAttrs(global *config.Global, peer *config.Neighbor, info *PeerInfo, original *Path) *Path { + if peer.RouteServer.Config.RouteServerClient { + return original + } + path := original.Clone(original.IsWithdraw) + + for _, a := range path.GetPathAttrs() { + if _, y := bgp.PathAttrFlags[a.GetType()]; !y { + if a.GetFlags()&bgp.BGP_ATTR_FLAG_TRANSITIVE == 0 { + path.delPathAttr(a.GetType()) + } + } else { + switch a.GetType() { + case bgp.BGP_ATTR_TYPE_CLUSTER_LIST, bgp.BGP_ATTR_TYPE_ORIGINATOR_ID: + if !(peer.State.PeerType == config.PEER_TYPE_INTERNAL && peer.RouteReflector.Config.RouteReflectorClient) { + // send these attributes to only rr clients + path.delPathAttr(a.GetType()) + } + } + } + } + + localAddress := info.LocalAddress + nexthop := path.GetNexthop() + if peer.State.PeerType == config.PEER_TYPE_EXTERNAL { + // NEXTHOP handling + if !path.IsLocal() || nexthop.IsUnspecified() { + path.SetNexthop(localAddress) + } + + // remove-private-as handling + path.RemovePrivateAS(peer.Config.LocalAs, peer.State.RemovePrivateAs) + + // AS_PATH handling + confed := peer.IsConfederationMember(global) + path.PrependAsn(peer.Config.LocalAs, 1, confed) + if !confed { + path.removeConfedAs() + } + + // MED Handling + if med := path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC); med != nil && !path.IsLocal() { + path.delPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + } + + } else if peer.State.PeerType == config.PEER_TYPE_INTERNAL { + // NEXTHOP handling for iBGP + // if the path generated locally set local address as nexthop. + // if not, don't modify it. + // TODO: NEXT-HOP-SELF support + if path.IsLocal() && nexthop.IsUnspecified() { + path.SetNexthop(localAddress) + } + + // AS_PATH handling for iBGP + // if the path has AS_PATH path attribute, don't modify it. + // if not, attach *empty* AS_PATH path attribute. + if nh := path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH); nh == nil { + path.PrependAsn(0, 0, false) + } + + // For iBGP peers we are required to send local-pref attribute + // for connected or local prefixes. + // We set default local-pref 100. + if pref := path.getPathAttr(bgp.BGP_ATTR_TYPE_LOCAL_PREF); pref == nil { + path.setPathAttr(bgp.NewPathAttributeLocalPref(DEFAULT_LOCAL_PREF)) + } + + // RFC4456: BGP Route Reflection + // 8. Avoiding Routing Information Loops + info := path.GetSource() + if peer.RouteReflector.Config.RouteReflectorClient { + // This attribute will carry the BGP Identifier of the originator of the route in the local AS. + // A BGP speaker SHOULD NOT create an ORIGINATOR_ID attribute if one already exists. + // + // RFC4684 3.2 Intra-AS VPN Route Distribution + // When advertising RT membership NLRI to a route-reflector client, + // the Originator attribute shall be set to the router-id of the + // advertiser, and the Next-hop attribute shall be set of the local + // address for that session. + if path.GetRouteFamily() == bgp.RF_RTC_UC { + path.SetNexthop(localAddress) + path.setPathAttr(bgp.NewPathAttributeOriginatorId(info.LocalID.String())) + } else if path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGINATOR_ID) == nil { + if path.IsLocal() { + path.setPathAttr(bgp.NewPathAttributeOriginatorId(global.Config.RouterId)) + } else { + path.setPathAttr(bgp.NewPathAttributeOriginatorId(info.ID.String())) + } + } + // When an RR reflects a route, it MUST prepend the local CLUSTER_ID to the CLUSTER_LIST. + // If the CLUSTER_LIST is empty, it MUST create a new one. + clusterID := string(peer.RouteReflector.State.RouteReflectorClusterId) + if p := path.getPathAttr(bgp.BGP_ATTR_TYPE_CLUSTER_LIST); p == nil { + path.setPathAttr(bgp.NewPathAttributeClusterList([]string{clusterID})) + } else { + clusterList := p.(*bgp.PathAttributeClusterList) + newClusterList := make([]string, 0, len(clusterList.Value)) + for _, ip := range clusterList.Value { + newClusterList = append(newClusterList, ip.String()) + } + path.setPathAttr(bgp.NewPathAttributeClusterList(append([]string{clusterID}, newClusterList...))) + } + } + + } else { + log.WithFields(log.Fields{ + "Topic": "Peer", + "Key": peer.State.NeighborAddress, + }).Warnf("invalid peer type: %d", peer.State.PeerType) + } + return path +} + +func (path *Path) GetTimestamp() time.Time { + return time.Unix(path.OriginInfo().timestamp, 0) +} + +func (path *Path) setTimestamp(t time.Time) { + path.OriginInfo().timestamp = t.Unix() +} + +func (path *Path) IsLocal() bool { + return path.GetSource().Address == nil +} + +func (path *Path) IsIBGP() bool { + return path.GetSource().AS == path.GetSource().LocalAS +} + +// create new PathAttributes +func (path *Path) Clone(isWithdraw bool) *Path { + return &Path{ + parent: path, + IsWithdraw: isWithdraw, + IsNexthopInvalid: path.IsNexthopInvalid, + attrsHash: path.attrsHash, + } +} + +func (path *Path) root() *Path { + p := path + for p.parent != nil { + p = p.parent + } + return p +} + +func (path *Path) OriginInfo() *originInfo { + return path.root().info +} + +func (path *Path) NoImplicitWithdraw() bool { + return path.OriginInfo().noImplicitWithdraw +} + +func (path *Path) Validation() *Validation { + return path.OriginInfo().validation +} + +func (path *Path) ValidationStatus() config.RpkiValidationResultType { + if v := path.OriginInfo().validation; v != nil { + return v.Status + } else { + return config.RPKI_VALIDATION_RESULT_TYPE_NONE + } +} + +func (path *Path) SetValidation(v *Validation) { + path.OriginInfo().validation = v +} + +func (path *Path) IsFromExternal() bool { + return path.OriginInfo().isFromExternal +} + +func (path *Path) SetIsFromExternal(y bool) { + path.OriginInfo().isFromExternal = y +} + +func (path *Path) GetRouteFamily() bgp.RouteFamily { + return bgp.AfiSafiToRouteFamily(path.OriginInfo().nlri.AFI(), path.OriginInfo().nlri.SAFI()) +} + +func (path *Path) SetSource(source *PeerInfo) { + path.OriginInfo().source = source +} +func (path *Path) GetSource() *PeerInfo { + return path.OriginInfo().source +} + +func (path *Path) MarkStale(s bool) { + path.OriginInfo().stale = s +} + +func (path *Path) IsStale() bool { + return path.OriginInfo().stale +} + +func (path *Path) IsAsLooped() bool { + return path.aslooped +} + +func (path *Path) SetAsLooped(y bool) { + path.aslooped = y +} + +func (path *Path) IsLLGRStale() bool { + for _, c := range path.GetCommunities() { + if c == uint32(bgp.COMMUNITY_LLGR_STALE) { + return true + } + } + return false +} + +func (path *Path) GetSourceAs() uint32 { + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + if attr != nil { + asPathParam := attr.(*bgp.PathAttributeAsPath).Value + if len(asPathParam) == 0 { + return 0 + } + asList := asPathParam[len(asPathParam)-1].GetAS() + if len(asList) == 0 { + return 0 + } + return asList[len(asList)-1] + } + return 0 +} + +func (path *Path) GetNexthop() net.IP { + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + if attr != nil { + return attr.(*bgp.PathAttributeNextHop).Value + } + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + if attr != nil { + return attr.(*bgp.PathAttributeMpReachNLRI).Nexthop + } + return net.IP{} +} + +func (path *Path) SetNexthop(nexthop net.IP) { + if path.GetRouteFamily() == bgp.RF_IPv4_UC && nexthop.To4() == nil { + path.delPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + mpreach := bgp.NewPathAttributeMpReachNLRI(nexthop.String(), []bgp.AddrPrefixInterface{path.GetNlri()}) + path.setPathAttr(mpreach) + return + } + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + if attr != nil { + path.setPathAttr(bgp.NewPathAttributeNextHop(nexthop.String())) + } + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + if attr != nil { + oldNlri := attr.(*bgp.PathAttributeMpReachNLRI) + path.setPathAttr(bgp.NewPathAttributeMpReachNLRI(nexthop.String(), oldNlri.Value)) + } +} + +func (path *Path) GetNlri() bgp.AddrPrefixInterface { + return path.OriginInfo().nlri +} + +type PathAttrs []bgp.PathAttributeInterface + +func (a PathAttrs) Len() int { + return len(a) +} + +func (a PathAttrs) Swap(i, j int) { + a[i], a[j] = a[j], a[i] +} + +func (a PathAttrs) Less(i, j int) bool { + return a[i].GetType() < a[j].GetType() +} + +func (path *Path) GetPathAttrs() []bgp.PathAttributeInterface { + deleted := NewBitmap(math.MaxUint8) + modified := make(map[uint]bgp.PathAttributeInterface) + p := path + for { + for _, t := range p.dels { + deleted.Flag(uint(t)) + } + if p.parent == nil { + list := PathAttrs(make([]bgp.PathAttributeInterface, 0, len(p.pathAttrs))) + // we assume that the original pathAttrs are + // in order, that is, other bgp speakers send + // attributes in order. + for _, a := range p.pathAttrs { + typ := uint(a.GetType()) + if m, ok := modified[typ]; ok { + list = append(list, m) + delete(modified, typ) + } else if !deleted.GetFlag(typ) { + list = append(list, a) + } + } + if len(modified) > 0 { + // Huh, some attributes were newly + // added. So we need to sort... + for _, m := range modified { + list = append(list, m) + } + sort.Sort(list) + } + return list + } else { + for _, a := range p.pathAttrs { + typ := uint(a.GetType()) + if _, ok := modified[typ]; !deleted.GetFlag(typ) && !ok { + modified[typ] = a + } + } + } + p = p.parent + } +} + +func (path *Path) getPathAttr(typ bgp.BGPAttrType) bgp.PathAttributeInterface { + p := path + for { + for _, t := range p.dels { + if t == typ { + return nil + } + } + for _, a := range p.pathAttrs { + if a.GetType() == typ { + return a + } + } + if p.parent == nil { + return nil + } + p = p.parent + } +} + +func (path *Path) setPathAttr(a bgp.PathAttributeInterface) { + if len(path.pathAttrs) == 0 { + path.pathAttrs = []bgp.PathAttributeInterface{a} + } else { + for i, b := range path.pathAttrs { + if a.GetType() == b.GetType() { + path.pathAttrs[i] = a + return + } + } + path.pathAttrs = append(path.pathAttrs, a) + } +} + +func (path *Path) delPathAttr(typ bgp.BGPAttrType) { + if len(path.dels) == 0 { + path.dels = []bgp.BGPAttrType{typ} + } else { + path.dels = append(path.dels, typ) + } +} + +// return Path's string representation +func (path *Path) String() string { + s := bytes.NewBuffer(make([]byte, 0, 64)) + if path.IsEOR() { + s.WriteString(fmt.Sprintf("{ %s EOR | src: %s }", path.GetRouteFamily(), path.GetSource())) + return s.String() + } + s.WriteString(fmt.Sprintf("{ %s | ", path.getPrefix())) + s.WriteString(fmt.Sprintf("src: %s", path.GetSource())) + s.WriteString(fmt.Sprintf(", nh: %s", path.GetNexthop())) + if path.IsNexthopInvalid { + s.WriteString(" (not reachable)") + } + if path.IsWithdraw { + s.WriteString(", withdraw") + } + s.WriteString(" }") + return s.String() +} + +func (path *Path) getPrefix() string { + return path.GetNlri().String() +} + +func (path *Path) GetAsPath() *bgp.PathAttributeAsPath { + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + if attr != nil { + return attr.(*bgp.PathAttributeAsPath) + } + return nil +} + +// GetAsPathLen returns the number of AS_PATH +func (path *Path) GetAsPathLen() int { + + var length int = 0 + if aspath := path.GetAsPath(); aspath != nil { + for _, as := range aspath.Value { + length += as.ASLen() + } + } + return length +} + +func (path *Path) GetAsString() string { + s := bytes.NewBuffer(make([]byte, 0, 64)) + if aspath := path.GetAsPath(); aspath != nil { + return bgp.AsPathString(aspath) + } + return s.String() +} + +func (path *Path) GetAsList() []uint32 { + return path.getAsListOfSpecificType(true, true) + +} + +func (path *Path) GetAsSeqList() []uint32 { + return path.getAsListOfSpecificType(true, false) + +} + +func (path *Path) getAsListOfSpecificType(getAsSeq, getAsSet bool) []uint32 { + asList := []uint32{} + if aspath := path.GetAsPath(); aspath != nil { + for _, param := range aspath.Value { + segType := param.GetType() + if getAsSeq && segType == bgp.BGP_ASPATH_ATTR_TYPE_SEQ { + asList = append(asList, param.GetAS()...) + continue + } + if getAsSet && segType == bgp.BGP_ASPATH_ATTR_TYPE_SET { + asList = append(asList, param.GetAS()...) + } else { + asList = append(asList, 0) + } + } + } + return asList +} + +func (path *Path) GetLabelString() string { + return bgp.LabelString(path.GetNlri()) +} + +// PrependAsn prepends AS number. +// This function updates the AS_PATH attribute as follows. +// (If the peer is in the confederation member AS, +// replace AS_SEQUENCE in the following sentence with AS_CONFED_SEQUENCE.) +// 1) if the first path segment of the AS_PATH is of type +// AS_SEQUENCE, the local system prepends the specified AS num as +// the last element of the sequence (put it in the left-most +// position with respect to the position of octets in the +// protocol message) the specified number of times. +// If the act of prepending will cause an overflow in the AS_PATH +// segment (i.e., more than 255 ASes), +// it SHOULD prepend a new segment of type AS_SEQUENCE +// and prepend its own AS number to this new segment. +// +// 2) if the first path segment of the AS_PATH is of other than type +// AS_SEQUENCE, the local system prepends a new path segment of type +// AS_SEQUENCE to the AS_PATH, including the specified AS number in +// that segment. +// +// 3) if the AS_PATH is empty, the local system creates a path +// segment of type AS_SEQUENCE, places the specified AS number +// into that segment, and places that segment into the AS_PATH. +func (path *Path) PrependAsn(asn uint32, repeat uint8, confed bool) { + var segType uint8 + if confed { + segType = bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SEQ + } else { + segType = bgp.BGP_ASPATH_ATTR_TYPE_SEQ + } + + original := path.GetAsPath() + + asns := make([]uint32, repeat) + for i := range asns { + asns[i] = asn + } + + var asPath *bgp.PathAttributeAsPath + if original == nil { + asPath = bgp.NewPathAttributeAsPath([]bgp.AsPathParamInterface{}) + } else { + asPath = cloneAsPath(original) + } + + if len(asPath.Value) > 0 { + param := asPath.Value[0] + asList := param.GetAS() + if param.GetType() == segType { + if int(repeat)+len(asList) > 255 { + repeat = uint8(255 - len(asList)) + } + newAsList := append(asns[:int(repeat)], asList...) + asPath.Value[0] = bgp.NewAs4PathParam(segType, newAsList) + asns = asns[int(repeat):] + } + } + + if len(asns) > 0 { + p := bgp.NewAs4PathParam(segType, asns) + asPath.Value = append([]bgp.AsPathParamInterface{p}, asPath.Value...) + } + path.setPathAttr(asPath) +} + +func isPrivateAS(as uint32) bool { + return (64512 <= as && as <= 65534) || (4200000000 <= as && as <= 4294967294) +} + +func (path *Path) RemovePrivateAS(localAS uint32, option config.RemovePrivateAsOption) { + original := path.GetAsPath() + if original == nil { + return + } + switch option { + case config.REMOVE_PRIVATE_AS_OPTION_ALL, config.REMOVE_PRIVATE_AS_OPTION_REPLACE: + newASParams := make([]bgp.AsPathParamInterface, 0, len(original.Value)) + for _, param := range original.Value { + asList := param.GetAS() + newASParam := make([]uint32, 0, len(asList)) + for _, as := range asList { + if isPrivateAS(as) { + if option == config.REMOVE_PRIVATE_AS_OPTION_REPLACE { + newASParam = append(newASParam, localAS) + } + } else { + newASParam = append(newASParam, as) + } + } + if len(newASParam) > 0 { + newASParams = append(newASParams, bgp.NewAs4PathParam(param.GetType(), newASParam)) + } + } + path.setPathAttr(bgp.NewPathAttributeAsPath(newASParams)) + } +} + +func (path *Path) removeConfedAs() { + original := path.GetAsPath() + if original == nil { + return + } + newAsParams := make([]bgp.AsPathParamInterface, 0, len(original.Value)) + for _, param := range original.Value { + switch param.GetType() { + case bgp.BGP_ASPATH_ATTR_TYPE_SEQ, bgp.BGP_ASPATH_ATTR_TYPE_SET: + newAsParams = append(newAsParams, param) + } + } + path.setPathAttr(bgp.NewPathAttributeAsPath(newAsParams)) +} + +func (path *Path) ReplaceAS(localAS, peerAS uint32) *Path { + original := path.GetAsPath() + if original == nil { + return path + } + newASParams := make([]bgp.AsPathParamInterface, 0, len(original.Value)) + changed := false + for _, param := range original.Value { + segType := param.GetType() + asList := param.GetAS() + newASParam := make([]uint32, 0, len(asList)) + for _, as := range asList { + if as == peerAS { + as = localAS + changed = true + } + newASParam = append(newASParam, as) + } + newASParams = append(newASParams, bgp.NewAs4PathParam(segType, newASParam)) + } + if changed { + path = path.Clone(path.IsWithdraw) + path.setPathAttr(bgp.NewPathAttributeAsPath(newASParams)) + } + return path +} + +func (path *Path) GetCommunities() []uint32 { + communityList := []uint32{} + if attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_COMMUNITIES); attr != nil { + communities := attr.(*bgp.PathAttributeCommunities) + communityList = append(communityList, communities.Value...) + } + return communityList +} + +// SetCommunities adds or replaces communities with new ones. +// If the length of communities is 0 and doReplace is true, it clears communities. +func (path *Path) SetCommunities(communities []uint32, doReplace bool) { + + if len(communities) == 0 && doReplace { + // clear communities + path.delPathAttr(bgp.BGP_ATTR_TYPE_COMMUNITIES) + return + } + + newList := make([]uint32, 0) + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_COMMUNITIES) + if attr != nil { + c := attr.(*bgp.PathAttributeCommunities) + if doReplace { + newList = append(newList, communities...) + } else { + newList = append(newList, c.Value...) + newList = append(newList, communities...) + } + } else { + newList = append(newList, communities...) + } + path.setPathAttr(bgp.NewPathAttributeCommunities(newList)) + +} + +// RemoveCommunities removes specific communities. +// If the length of communities is 0, it does nothing. +// If all communities are removed, it removes Communities path attribute itself. +func (path *Path) RemoveCommunities(communities []uint32) int { + + if len(communities) == 0 { + // do nothing + return 0 + } + + find := func(val uint32) bool { + for _, com := range communities { + if com == val { + return true + } + } + return false + } + + count := 0 + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_COMMUNITIES) + if attr != nil { + newList := make([]uint32, 0) + c := attr.(*bgp.PathAttributeCommunities) + + for _, value := range c.Value { + if find(value) { + count += 1 + } else { + newList = append(newList, value) + } + } + + if len(newList) != 0 { + path.setPathAttr(bgp.NewPathAttributeCommunities(newList)) + } else { + path.delPathAttr(bgp.BGP_ATTR_TYPE_COMMUNITIES) + } + } + return count +} + +func (path *Path) GetExtCommunities() []bgp.ExtendedCommunityInterface { + eCommunityList := make([]bgp.ExtendedCommunityInterface, 0) + if attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_EXTENDED_COMMUNITIES); attr != nil { + eCommunities := attr.(*bgp.PathAttributeExtendedCommunities).Value + eCommunityList = append(eCommunityList, eCommunities...) + } + return eCommunityList +} + +func (path *Path) SetExtCommunities(exts []bgp.ExtendedCommunityInterface, doReplace bool) { + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_EXTENDED_COMMUNITIES) + if attr != nil { + l := attr.(*bgp.PathAttributeExtendedCommunities).Value + if doReplace { + l = exts + } else { + l = append(l, exts...) + } + path.setPathAttr(bgp.NewPathAttributeExtendedCommunities(l)) + } else { + path.setPathAttr(bgp.NewPathAttributeExtendedCommunities(exts)) + } +} + +func (path *Path) GetLargeCommunities() []*bgp.LargeCommunity { + if a := path.getPathAttr(bgp.BGP_ATTR_TYPE_LARGE_COMMUNITY); a != nil { + v := a.(*bgp.PathAttributeLargeCommunities).Values + ret := make([]*bgp.LargeCommunity, 0, len(v)) + ret = append(ret, v...) + return ret + } + return nil +} + +func (path *Path) SetLargeCommunities(cs []*bgp.LargeCommunity, doReplace bool) { + a := path.getPathAttr(bgp.BGP_ATTR_TYPE_LARGE_COMMUNITY) + if a == nil || doReplace { + path.setPathAttr(bgp.NewPathAttributeLargeCommunities(cs)) + } else { + l := a.(*bgp.PathAttributeLargeCommunities).Values + path.setPathAttr(bgp.NewPathAttributeLargeCommunities(append(l, cs...))) + } +} + +func (path *Path) GetMed() (uint32, error) { + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + if attr == nil { + return 0, fmt.Errorf("no med path attr") + } + return attr.(*bgp.PathAttributeMultiExitDisc).Value, nil +} + +// SetMed replace, add or subtraction med with new ones. +func (path *Path) SetMed(med int64, doReplace bool) error { + parseMed := func(orgMed uint32, med int64, doReplace bool) (*bgp.PathAttributeMultiExitDisc, error) { + if doReplace { + return bgp.NewPathAttributeMultiExitDisc(uint32(med)), nil + } + + medVal := int64(orgMed) + med + if medVal < 0 { + return nil, fmt.Errorf("med value invalid. it's underflow threshold: %v", medVal) + } else if medVal > int64(math.MaxUint32) { + return nil, fmt.Errorf("med value invalid. it's overflow threshold: %v", medVal) + } + + return bgp.NewPathAttributeMultiExitDisc(uint32(int64(orgMed) + med)), nil + } + + m := uint32(0) + if attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC); attr != nil { + m = attr.(*bgp.PathAttributeMultiExitDisc).Value + } + newMed, err := parseMed(m, med, doReplace) + if err != nil { + return err + } + path.setPathAttr(newMed) + return nil +} + +func (path *Path) RemoveLocalPref() { + if path.getPathAttr(bgp.BGP_ATTR_TYPE_LOCAL_PREF) != nil { + path.delPathAttr(bgp.BGP_ATTR_TYPE_LOCAL_PREF) + } +} + +func (path *Path) GetOriginatorID() net.IP { + if attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGINATOR_ID); attr != nil { + return attr.(*bgp.PathAttributeOriginatorId).Value + } + return nil +} + +func (path *Path) GetClusterList() []net.IP { + if attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_CLUSTER_LIST); attr != nil { + return attr.(*bgp.PathAttributeClusterList).Value + } + return nil +} + +func (path *Path) GetOrigin() (uint8, error) { + if attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN); attr != nil { + return attr.(*bgp.PathAttributeOrigin).Value, nil + } + return 0, fmt.Errorf("no origin path attr") +} + +func (path *Path) GetLocalPref() (uint32, error) { + lp := uint32(DEFAULT_LOCAL_PREF) + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_LOCAL_PREF) + if attr != nil { + lp = attr.(*bgp.PathAttributeLocalPref).Value + } + return lp, nil +} + +func (lhs *Path) Equal(rhs *Path) bool { + if rhs == nil { + return false + } + + if lhs.GetSource() != rhs.GetSource() { + return false + } + + pattrs := func(arg []bgp.PathAttributeInterface) []byte { + ret := make([]byte, 0) + for _, a := range arg { + aa, _ := a.Serialize() + ret = append(ret, aa...) + } + return ret + } + return bytes.Equal(pattrs(lhs.GetPathAttrs()), pattrs(rhs.GetPathAttrs())) +} + +func (path *Path) MarshalJSON() ([]byte, error) { + return json.Marshal(struct { + Nlri bgp.AddrPrefixInterface `json:"nlri"` + PathAttrs []bgp.PathAttributeInterface `json:"attrs"` + Age int64 `json:"age"` + Withdrawal bool `json:"withdrawal,omitempty"` + Validation string `json:"validation,omitempty"` + SourceID net.IP `json:"source-id,omitempty"` + NeighborIP net.IP `json:"neighbor-ip,omitempty"` + Stale bool `json:"stale,omitempty"` + UUID string `json:"uuid,omitempty"` + ID uint32 `json:"id,omitempty"` + }{ + Nlri: path.GetNlri(), + PathAttrs: path.GetPathAttrs(), + Age: path.GetTimestamp().Unix(), + Withdrawal: path.IsWithdraw, + Validation: string(path.ValidationStatus()), + SourceID: path.GetSource().ID, + NeighborIP: path.GetSource().Address, + Stale: path.IsStale(), + ID: path.GetNlri().PathIdentifier(), + }) +} + +func (lhs *Path) Compare(rhs *Path) int { + if lhs.IsLocal() && !rhs.IsLocal() { + return 1 + } else if !lhs.IsLocal() && rhs.IsLocal() { + return -1 + } + + if !lhs.IsIBGP() && rhs.IsIBGP() { + return 1 + } else if lhs.IsIBGP() && !rhs.IsIBGP() { + return -1 + } + + lp1, _ := lhs.GetLocalPref() + lp2, _ := rhs.GetLocalPref() + if lp1 != lp2 { + return int(lp1 - lp2) + } + + l1 := lhs.GetAsPathLen() + l2 := rhs.GetAsPathLen() + if l1 != l2 { + return int(l2 - l1) + } + + o1, _ := lhs.GetOrigin() + o2, _ := rhs.GetOrigin() + if o1 != o2 { + return int(o2 - o1) + } + + m1, _ := lhs.GetMed() + m2, _ := rhs.GetMed() + return int(m2 - m1) +} + +func (v *Vrf) ToGlobalPath(path *Path) error { + nlri := path.GetNlri() + switch rf := path.GetRouteFamily(); rf { + case bgp.RF_IPv4_UC: + n := nlri.(*bgp.IPAddrPrefix) + pathIdentifier := path.GetNlri().PathIdentifier() + path.OriginInfo().nlri = bgp.NewLabeledVPNIPAddrPrefix(n.Length, n.Prefix.String(), *bgp.NewMPLSLabelStack(0), v.Rd) + path.GetNlri().SetPathIdentifier(pathIdentifier) + case bgp.RF_IPv6_UC: + n := nlri.(*bgp.IPv6AddrPrefix) + pathIdentifier := path.GetNlri().PathIdentifier() + path.OriginInfo().nlri = bgp.NewLabeledVPNIPv6AddrPrefix(n.Length, n.Prefix.String(), *bgp.NewMPLSLabelStack(0), v.Rd) + path.GetNlri().SetPathIdentifier(pathIdentifier) + case bgp.RF_EVPN: + n := nlri.(*bgp.EVPNNLRI) + switch n.RouteType { + case bgp.EVPN_ROUTE_TYPE_MAC_IP_ADVERTISEMENT: + n.RouteTypeData.(*bgp.EVPNMacIPAdvertisementRoute).RD = v.Rd + case bgp.EVPN_INCLUSIVE_MULTICAST_ETHERNET_TAG: + n.RouteTypeData.(*bgp.EVPNMulticastEthernetTagRoute).RD = v.Rd + } + default: + return fmt.Errorf("unsupported route family for vrf: %s", rf) + } + path.SetExtCommunities(v.ExportRt, false) + return nil +} + +func (p *Path) ToGlobal(vrf *Vrf) *Path { + nlri := p.GetNlri() + nh := p.GetNexthop() + pathId := nlri.PathIdentifier() + switch rf := p.GetRouteFamily(); rf { + case bgp.RF_IPv4_UC: + n := nlri.(*bgp.IPAddrPrefix) + nlri = bgp.NewLabeledVPNIPAddrPrefix(n.Length, n.Prefix.String(), *bgp.NewMPLSLabelStack(0), vrf.Rd) + nlri.SetPathIdentifier(pathId) + case bgp.RF_IPv6_UC: + n := nlri.(*bgp.IPv6AddrPrefix) + nlri = bgp.NewLabeledVPNIPv6AddrPrefix(n.Length, n.Prefix.String(), *bgp.NewMPLSLabelStack(0), vrf.Rd) + nlri.SetPathIdentifier(pathId) + case bgp.RF_EVPN: + n := nlri.(*bgp.EVPNNLRI) + switch n.RouteType { + case bgp.EVPN_ROUTE_TYPE_MAC_IP_ADVERTISEMENT: + old := n.RouteTypeData.(*bgp.EVPNMacIPAdvertisementRoute) + new := &bgp.EVPNMacIPAdvertisementRoute{ + RD: vrf.Rd, + ESI: old.ESI, + ETag: old.ETag, + MacAddressLength: old.MacAddressLength, + MacAddress: old.MacAddress, + IPAddressLength: old.IPAddressLength, + IPAddress: old.IPAddress, + Labels: old.Labels, + } + nlri = bgp.NewEVPNNLRI(n.RouteType, new) + case bgp.EVPN_INCLUSIVE_MULTICAST_ETHERNET_TAG: + old := n.RouteTypeData.(*bgp.EVPNMulticastEthernetTagRoute) + new := &bgp.EVPNMulticastEthernetTagRoute{ + RD: vrf.Rd, + ETag: old.ETag, + IPAddressLength: old.IPAddressLength, + IPAddress: old.IPAddress, + } + nlri = bgp.NewEVPNNLRI(n.RouteType, new) + } + default: + return p + } + path := NewPath(p.OriginInfo().source, nlri, p.IsWithdraw, p.GetPathAttrs(), p.GetTimestamp(), false) + path.SetExtCommunities(vrf.ExportRt, false) + path.delPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + path.setPathAttr(bgp.NewPathAttributeMpReachNLRI(nh.String(), []bgp.AddrPrefixInterface{nlri})) + path.IsNexthopInvalid = p.IsNexthopInvalid + return path +} + +func (p *Path) ToLocal() *Path { + nlri := p.GetNlri() + f := p.GetRouteFamily() + pathId := nlri.PathLocalIdentifier() + switch f { + case bgp.RF_IPv4_VPN: + n := nlri.(*bgp.LabeledVPNIPAddrPrefix) + _, c, _ := net.ParseCIDR(n.IPPrefix()) + ones, _ := c.Mask.Size() + nlri = bgp.NewIPAddrPrefix(uint8(ones), c.IP.String()) + nlri.SetPathLocalIdentifier(pathId) + case bgp.RF_IPv6_VPN: + n := nlri.(*bgp.LabeledVPNIPv6AddrPrefix) + _, c, _ := net.ParseCIDR(n.IPPrefix()) + ones, _ := c.Mask.Size() + nlri = bgp.NewIPv6AddrPrefix(uint8(ones), c.IP.String()) + nlri.SetPathLocalIdentifier(pathId) + default: + return p + } + path := NewPath(p.OriginInfo().source, nlri, p.IsWithdraw, p.GetPathAttrs(), p.GetTimestamp(), false) + path.delPathAttr(bgp.BGP_ATTR_TYPE_EXTENDED_COMMUNITIES) + + if f == bgp.RF_IPv4_VPN { + nh := path.GetNexthop() + path.delPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + path.setPathAttr(bgp.NewPathAttributeNextHop(nh.String())) + } + path.IsNexthopInvalid = p.IsNexthopInvalid + return path +} + +func (p *Path) SetHash(v uint32) { + p.attrsHash = v +} + +func (p *Path) GetHash() uint32 { + return p.attrsHash +} diff --git a/internal/pkg/table/path_test.go b/internal/pkg/table/path_test.go new file mode 100644 index 00000000..449c4a8e --- /dev/null +++ b/internal/pkg/table/path_test.go @@ -0,0 +1,365 @@ +// path_test.go +package table + +import ( + "testing" + "time" + + "github.com/osrg/gobgp/internal/pkg/config" + "github.com/osrg/gobgp/pkg/packet/bgp" + + "github.com/stretchr/testify/assert" +) + +func TestPathNewIPv4(t *testing.T) { + peerP := PathCreatePeer() + pathP := PathCreatePath(peerP) + ipv4p := NewPath(pathP[0].GetSource(), pathP[0].GetNlri(), true, pathP[0].GetPathAttrs(), time.Now(), false) + assert.NotNil(t, ipv4p) +} + +func TestPathNewIPv6(t *testing.T) { + peerP := PathCreatePeer() + pathP := PathCreatePath(peerP) + ipv6p := NewPath(pathP[0].GetSource(), pathP[0].GetNlri(), true, pathP[0].GetPathAttrs(), time.Now(), false) + assert.NotNil(t, ipv6p) +} + +func TestPathGetNlri(t *testing.T) { + nlri := bgp.NewIPAddrPrefix(24, "13.2.3.2") + pd := &Path{ + info: &originInfo{ + nlri: nlri, + }, + } + r_nlri := pd.GetNlri() + assert.Equal(t, r_nlri, nlri) +} + +func TestPathCreatePath(t *testing.T) { + peerP := PathCreatePeer() + msg := updateMsgP1() + updateMsgP := msg.Body.(*bgp.BGPUpdate) + nlriList := updateMsgP.NLRI + pathAttributes := updateMsgP.PathAttributes + nlri_info := nlriList[0] + path := NewPath(peerP[0], nlri_info, false, pathAttributes, time.Now(), false) + assert.NotNil(t, path) + +} + +func TestPathGetPrefix(t *testing.T) { + peerP := PathCreatePeer() + pathP := PathCreatePath(peerP) + prefix := "10.10.10.0/24" + r_prefix := pathP[0].getPrefix() + assert.Equal(t, r_prefix, prefix) +} + +func TestPathGetAttribute(t *testing.T) { + peerP := PathCreatePeer() + pathP := PathCreatePath(peerP) + nh := "192.168.50.1" + pa := pathP[0].getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + r_nh := pa.(*bgp.PathAttributeNextHop).Value.String() + assert.Equal(t, r_nh, nh) +} + +func TestASPathLen(t *testing.T) { + assert := assert.New(t) + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint16{65001, 65002, 65003, 65004, 65004, 65004, 65004, 65004, 65005}), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SET, []uint16{65001, 65002, 65003, 65004, 65005}), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SEQ, []uint16{65100, 65101, 65102}), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SET, []uint16{65100, 65101})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.50.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpmsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + update := bgpmsg.Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(update) + peer := PathCreatePeer() + p := NewPath(peer[0], update.NLRI[0], false, update.PathAttributes, time.Now(), false) + assert.Equal(10, p.GetAsPathLen()) +} + +func TestPathPrependAsnToExistingSeqAttr(t *testing.T) { + assert := assert.New(t) + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint16{65001, 65002, 65003, 65004, 65005}), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SET, []uint16{65001, 65002, 65003, 65004, 65005}), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SEQ, []uint16{65100, 65101, 65102}), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SET, []uint16{65100, 65101})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.50.1") + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpmsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + update := bgpmsg.Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(update) + peer := PathCreatePeer() + p := NewPath(peer[0], update.NLRI[0], false, update.PathAttributes, time.Now(), false) + + p.PrependAsn(65000, 1, false) + assert.Equal([]uint32{65000, 65001, 65002, 65003, 65004, 65005, 0, 0, 0}, p.GetAsSeqList()) +} + +func TestPathPrependAsnToNewAsPathAttr(t *testing.T) { + assert := assert.New(t) + origin := bgp.NewPathAttributeOrigin(0) + nexthop := bgp.NewPathAttributeNextHop("192.168.50.1") + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + nexthop, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpmsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + update := bgpmsg.Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(update) + peer := PathCreatePeer() + p := NewPath(peer[0], update.NLRI[0], false, update.PathAttributes, time.Now(), false) + + asn := uint32(65000) + p.PrependAsn(asn, 1, false) + assert.Equal([]uint32{asn}, p.GetAsSeqList()) +} + +func TestPathPrependAsnToNewAsPathSeq(t *testing.T) { + assert := assert.New(t) + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SET, []uint16{65001, 65002, 65003, 65004, 65005}), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SEQ, []uint16{65100, 65101, 65102}), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SET, []uint16{65100, 65101})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.50.1") + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpmsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + update := bgpmsg.Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(update) + peer := PathCreatePeer() + p := NewPath(peer[0], update.NLRI[0], false, update.PathAttributes, time.Now(), false) + + asn := uint32(65000) + p.PrependAsn(asn, 1, false) + assert.Equal([]uint32{asn, 0, 0, 0}, p.GetAsSeqList()) +} + +func TestPathPrependAsnToEmptyAsPathAttr(t *testing.T) { + assert := assert.New(t) + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint16{}), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SET, []uint16{65001, 65002, 65003, 65004, 65005}), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SEQ, []uint16{65100, 65101, 65102}), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SET, []uint16{65100, 65101})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.50.1") + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpmsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + update := bgpmsg.Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(update) + peer := PathCreatePeer() + p := NewPath(peer[0], update.NLRI[0], false, update.PathAttributes, time.Now(), false) + + asn := uint32(65000) + p.PrependAsn(asn, 1, false) + assert.Equal([]uint32{asn, 0, 0, 0}, p.GetAsSeqList()) +} + +func TestPathPrependAsnToFullPathAttr(t *testing.T) { + assert := assert.New(t) + origin := bgp.NewPathAttributeOrigin(0) + + asns := make([]uint16, 255) + for i, _ := range asns { + asns[i] = 65000 + uint16(i) + } + + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, asns), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_SET, []uint16{65001, 65002, 65003, 65004, 65005}), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SEQ, []uint16{65100, 65101, 65102}), + bgp.NewAsPathParam(bgp.BGP_ASPATH_ATTR_TYPE_CONFED_SET, []uint16{65100, 65101})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.50.1") + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpmsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + update := bgpmsg.Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(update) + peer := PathCreatePeer() + p := NewPath(peer[0], update.NLRI[0], false, update.PathAttributes, time.Now(), false) + + expected := []uint32{65000, 65000} + for _, v := range asns { + expected = append(expected, uint32(v)) + } + p.PrependAsn(65000, 2, false) + assert.Equal(append(expected, []uint32{0, 0, 0}...), p.GetAsSeqList()) +} + +func TestGetPathAttrs(t *testing.T) { + paths := PathCreatePath(PathCreatePeer()) + path0 := paths[0] + path1 := path0.Clone(false) + path1.delPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + path2 := path1.Clone(false) + path2.setPathAttr(bgp.NewPathAttributeNextHop("192.168.50.1")) + assert.NotNil(t, path2.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP)) +} + +func PathCreatePeer() []*PeerInfo { + peerP1 := &PeerInfo{AS: 65000} + peerP2 := &PeerInfo{AS: 65001} + peerP3 := &PeerInfo{AS: 65002} + peerP := []*PeerInfo{peerP1, peerP2, peerP3} + return peerP +} + +func PathCreatePath(peerP []*PeerInfo) []*Path { + bgpMsgP1 := updateMsgP1() + bgpMsgP2 := updateMsgP2() + bgpMsgP3 := updateMsgP3() + pathP := make([]*Path, 3) + for i, msg := range []*bgp.BGPMessage{bgpMsgP1, bgpMsgP2, bgpMsgP3} { + updateMsgP := msg.Body.(*bgp.BGPUpdate) + nlriList := updateMsgP.NLRI + pathAttributes := updateMsgP.PathAttributes + nlri_info := nlriList[0] + pathP[i] = NewPath(peerP[i], nlri_info, false, pathAttributes, time.Now(), false) + } + return pathP +} + +func updateMsgP1() *bgp.BGPMessage { + + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65000})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.50.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + return bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) +} + +func updateMsgP2() *bgp.BGPMessage { + + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65100})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.100.1") + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "20.20.20.0")} + return bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) +} + +func updateMsgP3() *bgp.BGPMessage { + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65100})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.150.1") + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "30.30.30.0")} + w1 := bgp.NewIPAddrPrefix(23, "40.40.40.0") + withdrawnRoutes := []*bgp.IPAddrPrefix{w1} + return bgp.NewBGPUpdateMessage(withdrawnRoutes, pathAttributes, nlri) +} + +func TestRemovePrivateAS(t *testing.T) { + aspathParam := []bgp.AsPathParamInterface{bgp.NewAs4PathParam(2, []uint32{64512, 64513, 1, 2})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nlri := bgp.NewIPAddrPrefix(24, "30.30.30.0") + path := NewPath(nil, nlri, false, []bgp.PathAttributeInterface{aspath}, time.Now(), false) + path.RemovePrivateAS(10, config.REMOVE_PRIVATE_AS_OPTION_ALL) + list := path.GetAsList() + assert.Equal(t, len(list), 2) + assert.Equal(t, list[0], uint32(1)) + assert.Equal(t, list[1], uint32(2)) + + path = NewPath(nil, nlri, false, []bgp.PathAttributeInterface{aspath}, time.Now(), false) + path.RemovePrivateAS(10, config.REMOVE_PRIVATE_AS_OPTION_REPLACE) + list = path.GetAsList() + assert.Equal(t, len(list), 4) + assert.Equal(t, list[0], uint32(10)) + assert.Equal(t, list[1], uint32(10)) + assert.Equal(t, list[2], uint32(1)) + assert.Equal(t, list[3], uint32(2)) +} + +func TestReplaceAS(t *testing.T) { + aspathParam := []bgp.AsPathParamInterface{bgp.NewAs4PathParam(2, []uint32{64512, 64513, 1, 2})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nlri := bgp.NewIPAddrPrefix(24, "30.30.30.0") + path := NewPath(nil, nlri, false, []bgp.PathAttributeInterface{aspath}, time.Now(), false) + path = path.ReplaceAS(10, 1) + list := path.GetAsList() + assert.Equal(t, len(list), 4) + assert.Equal(t, list[0], uint32(64512)) + assert.Equal(t, list[1], uint32(64513)) + assert.Equal(t, list[2], uint32(10)) + assert.Equal(t, list[3], uint32(2)) +} diff --git a/internal/pkg/table/policy.go b/internal/pkg/table/policy.go new file mode 100644 index 00000000..23c2110c --- /dev/null +++ b/internal/pkg/table/policy.go @@ -0,0 +1,3994 @@ +// Copyright (C) 2014-2016 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + "bytes" + "encoding/json" + "fmt" + "net" + "reflect" + "regexp" + "sort" + "strconv" + "strings" + "sync" + + "github.com/osrg/gobgp/internal/pkg/config" + "github.com/osrg/gobgp/pkg/packet/bgp" + + radix "github.com/armon/go-radix" + log "github.com/sirupsen/logrus" +) + +type PolicyOptions struct { + Info *PeerInfo + ValidationResult *Validation + OldNextHop net.IP +} + +type DefinedType int + +const ( + DEFINED_TYPE_PREFIX DefinedType = iota + DEFINED_TYPE_NEIGHBOR + DEFINED_TYPE_TAG + DEFINED_TYPE_AS_PATH + DEFINED_TYPE_COMMUNITY + DEFINED_TYPE_EXT_COMMUNITY + DEFINED_TYPE_LARGE_COMMUNITY + DEFINED_TYPE_NEXT_HOP +) + +type RouteType int + +const ( + ROUTE_TYPE_NONE RouteType = iota + ROUTE_TYPE_ACCEPT + ROUTE_TYPE_REJECT +) + +func (t RouteType) String() string { + switch t { + case ROUTE_TYPE_NONE: + return "continue" + case ROUTE_TYPE_ACCEPT: + return "accept" + case ROUTE_TYPE_REJECT: + return "reject" + } + return fmt.Sprintf("unknown(%d)", t) +} + +type PolicyDirection int + +const ( + POLICY_DIRECTION_NONE PolicyDirection = iota + POLICY_DIRECTION_IN + POLICY_DIRECTION_IMPORT + POLICY_DIRECTION_EXPORT +) + +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 ( + MATCH_OPTION_ANY MatchOption = iota + MATCH_OPTION_ALL + MATCH_OPTION_INVERT +) + +func (o MatchOption) String() string { + switch o { + case MATCH_OPTION_ANY: + return "any" + case MATCH_OPTION_ALL: + return "all" + case MATCH_OPTION_INVERT: + return "invert" + default: + return fmt.Sprintf("MatchOption(%d)", o) + } +} + +func (o MatchOption) ConvertToMatchSetOptionsRestrictedType() config.MatchSetOptionsRestrictedType { + switch o { + case MATCH_OPTION_ANY: + return config.MATCH_SET_OPTIONS_RESTRICTED_TYPE_ANY + case MATCH_OPTION_INVERT: + return config.MATCH_SET_OPTIONS_RESTRICTED_TYPE_INVERT + } + return "unknown" +} + +type MedActionType int + +const ( + MED_ACTION_MOD MedActionType = iota + MED_ACTION_REPLACE +) + +var CommunityOptionNameMap = map[config.BgpSetCommunityOptionType]string{ + config.BGP_SET_COMMUNITY_OPTION_TYPE_ADD: "add", + config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE: "remove", + config.BGP_SET_COMMUNITY_OPTION_TYPE_REPLACE: "replace", +} + +var CommunityOptionValueMap = map[string]config.BgpSetCommunityOptionType{ + CommunityOptionNameMap[config.BGP_SET_COMMUNITY_OPTION_TYPE_ADD]: config.BGP_SET_COMMUNITY_OPTION_TYPE_ADD, + CommunityOptionNameMap[config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE]: config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE, + CommunityOptionNameMap[config.BGP_SET_COMMUNITY_OPTION_TYPE_REPLACE]: config.BGP_SET_COMMUNITY_OPTION_TYPE_REPLACE, +} + +type ConditionType int + +const ( + CONDITION_PREFIX ConditionType = iota + CONDITION_NEIGHBOR + CONDITION_AS_PATH + CONDITION_COMMUNITY + CONDITION_EXT_COMMUNITY + CONDITION_AS_PATH_LENGTH + CONDITION_RPKI + CONDITION_ROUTE_TYPE + CONDITION_LARGE_COMMUNITY + CONDITION_NEXT_HOP + CONDITION_AFI_SAFI_IN +) + +type ActionType int + +const ( + ACTION_ROUTING ActionType = iota + ACTION_COMMUNITY + ACTION_EXT_COMMUNITY + ACTION_MED + ACTION_AS_PATH_PREPEND + ACTION_NEXTHOP + ACTION_LOCAL_PREF + ACTION_LARGE_COMMUNITY +) + +func NewMatchOption(c interface{}) (MatchOption, error) { + switch t := c.(type) { + case config.MatchSetOptionsType: + t = t.DefaultAsNeeded() + switch t { + case config.MATCH_SET_OPTIONS_TYPE_ANY: + return MATCH_OPTION_ANY, nil + case config.MATCH_SET_OPTIONS_TYPE_ALL: + return MATCH_OPTION_ALL, nil + case config.MATCH_SET_OPTIONS_TYPE_INVERT: + return MATCH_OPTION_INVERT, nil + } + case config.MatchSetOptionsRestrictedType: + t = t.DefaultAsNeeded() + switch t { + case config.MATCH_SET_OPTIONS_RESTRICTED_TYPE_ANY: + return MATCH_OPTION_ANY, nil + case config.MATCH_SET_OPTIONS_RESTRICTED_TYPE_INVERT: + return MATCH_OPTION_INVERT, nil + } + } + return MATCH_OPTION_ANY, fmt.Errorf("invalid argument to create match option: %v", c) +} + +type AttributeComparison int + +const ( + // "== comparison" + ATTRIBUTE_EQ AttributeComparison = iota + // ">= comparison" + ATTRIBUTE_GE + // "<= comparison" + ATTRIBUTE_LE +) + +func (c AttributeComparison) String() string { + switch c { + case ATTRIBUTE_EQ: + return "=" + case ATTRIBUTE_GE: + return ">=" + case ATTRIBUTE_LE: + return "<=" + } + return "?" +} + +const ( + ASPATH_REGEXP_MAGIC = "(^|[,{}() ]|$)" +) + +type DefinedSet interface { + Type() DefinedType + Name() string + Append(DefinedSet) error + Remove(DefinedSet) error + Replace(DefinedSet) error + String() string + List() []string +} + +type DefinedSetMap map[DefinedType]map[string]DefinedSet + +type DefinedSetList []DefinedSet + +func (l DefinedSetList) Len() int { + return len(l) +} + +func (l DefinedSetList) Swap(i, j int) { + l[i], l[j] = l[j], l[i] +} + +func (l DefinedSetList) Less(i, j int) bool { + if l[i].Type() != l[j].Type() { + return l[i].Type() < l[j].Type() + } + return l[i].Name() < l[j].Name() +} + +type Prefix struct { + Prefix *net.IPNet + AddressFamily bgp.RouteFamily + MasklengthRangeMax uint8 + MasklengthRangeMin uint8 +} + +func (p *Prefix) Match(path *Path) bool { + rf := path.GetRouteFamily() + if rf != p.AddressFamily { + return false + } + + var pAddr net.IP + var pMasklen uint8 + switch rf { + case bgp.RF_IPv4_UC: + pAddr = path.GetNlri().(*bgp.IPAddrPrefix).Prefix + pMasklen = path.GetNlri().(*bgp.IPAddrPrefix).Length + case bgp.RF_IPv6_UC: + pAddr = path.GetNlri().(*bgp.IPv6AddrPrefix).Prefix + pMasklen = path.GetNlri().(*bgp.IPv6AddrPrefix).Length + default: + return false + } + + return (p.MasklengthRangeMin <= pMasklen && pMasklen <= p.MasklengthRangeMax) && p.Prefix.Contains(pAddr) +} + +func (lhs *Prefix) Equal(rhs *Prefix) bool { + if lhs == rhs { + return true + } + if rhs == nil { + return false + } + return lhs.Prefix.String() == rhs.Prefix.String() && lhs.MasklengthRangeMin == rhs.MasklengthRangeMin && lhs.MasklengthRangeMax == rhs.MasklengthRangeMax +} + +func (p *Prefix) PrefixString() string { + isZeros := func(p net.IP) bool { + for i := 0; i < len(p); i++ { + if p[i] != 0 { + return false + } + } + return true + } + + ip := p.Prefix.IP + if p.AddressFamily == bgp.RF_IPv6_UC && isZeros(ip[0:10]) && ip[10] == 0xff && ip[11] == 0xff { + m, _ := p.Prefix.Mask.Size() + return fmt.Sprintf("::FFFF:%s/%d", ip.To16(), m) + } + return p.Prefix.String() +} + +var _regexpPrefixRange = regexp.MustCompile(`(\d+)\.\.(\d+)`) + +func NewPrefix(c config.Prefix) (*Prefix, error) { + _, prefix, err := net.ParseCIDR(c.IpPrefix) + if err != nil { + return nil, err + } + + rf := bgp.RF_IPv4_UC + if strings.Contains(c.IpPrefix, ":") { + rf = bgp.RF_IPv6_UC + } + p := &Prefix{ + Prefix: prefix, + AddressFamily: rf, + } + maskRange := c.MasklengthRange + + if maskRange == "" { + l, _ := prefix.Mask.Size() + maskLength := uint8(l) + p.MasklengthRangeMax = maskLength + p.MasklengthRangeMin = maskLength + return p, nil + } + + elems := _regexpPrefixRange.FindStringSubmatch(maskRange) + if len(elems) != 3 { + log.WithFields(log.Fields{ + "Topic": "Policy", + "Type": "Prefix", + "MaskRangeFormat": maskRange, + }).Warn("mask length range format is invalid.") + return nil, fmt.Errorf("mask length range format is invalid") + } + + // we've already checked the range is sane by regexp + min, _ := strconv.ParseUint(elems[1], 10, 8) + max, _ := strconv.ParseUint(elems[2], 10, 8) + p.MasklengthRangeMin = uint8(min) + p.MasklengthRangeMax = uint8(max) + return p, nil +} + +type PrefixSet struct { + name string + tree *radix.Tree + family bgp.RouteFamily +} + +func (s *PrefixSet) Name() string { + return s.name +} + +func (s *PrefixSet) Type() DefinedType { + return DEFINED_TYPE_PREFIX +} + +func (lhs *PrefixSet) Append(arg DefinedSet) error { + rhs, ok := arg.(*PrefixSet) + if !ok { + return fmt.Errorf("type cast failed") + } + // if either is empty, family can be ignored. + if lhs.tree.Len() != 0 && rhs.tree.Len() != 0 { + _, w, _ := lhs.tree.Minimum() + l := w.([]*Prefix) + _, v, _ := rhs.tree.Minimum() + r := v.([]*Prefix) + if l[0].AddressFamily != r[0].AddressFamily { + return fmt.Errorf("can't append different family") + } + } + rhs.tree.Walk(func(key string, v interface{}) bool { + w, ok := lhs.tree.Get(key) + if ok { + r := v.([]*Prefix) + l := w.([]*Prefix) + lhs.tree.Insert(key, append(l, r...)) + } else { + lhs.tree.Insert(key, v) + } + return false + }) + _, w, _ := lhs.tree.Minimum() + lhs.family = w.([]*Prefix)[0].AddressFamily + return nil +} + +func (lhs *PrefixSet) Remove(arg DefinedSet) error { + rhs, ok := arg.(*PrefixSet) + if !ok { + return fmt.Errorf("type cast failed") + } + rhs.tree.Walk(func(key string, v interface{}) bool { + w, ok := lhs.tree.Get(key) + if !ok { + return false + } + r := v.([]*Prefix) + l := w.([]*Prefix) + new := make([]*Prefix, 0, len(l)) + for _, lp := range l { + delete := false + for _, rp := range r { + if lp.Equal(rp) { + delete = true + break + } + } + if !delete { + new = append(new, lp) + } + } + if len(new) == 0 { + lhs.tree.Delete(key) + } else { + lhs.tree.Insert(key, new) + } + return false + }) + return nil +} + +func (lhs *PrefixSet) Replace(arg DefinedSet) error { + rhs, ok := arg.(*PrefixSet) + if !ok { + return fmt.Errorf("type cast failed") + } + lhs.tree = rhs.tree + lhs.family = rhs.family + return nil +} + +func (s *PrefixSet) List() []string { + var list []string + s.tree.Walk(func(s string, v interface{}) bool { + ps := v.([]*Prefix) + for _, p := range ps { + list = append(list, fmt.Sprintf("%s %d..%d", p.PrefixString(), p.MasklengthRangeMin, p.MasklengthRangeMax)) + } + return false + }) + return list +} + +func (s *PrefixSet) ToConfig() *config.PrefixSet { + list := make([]config.Prefix, 0, s.tree.Len()) + s.tree.Walk(func(s string, v interface{}) bool { + ps := v.([]*Prefix) + for _, p := range ps { + list = append(list, config.Prefix{IpPrefix: p.PrefixString(), MasklengthRange: fmt.Sprintf("%d..%d", p.MasklengthRangeMin, p.MasklengthRangeMax)}) + } + return false + }) + return &config.PrefixSet{ + PrefixSetName: s.name, + PrefixList: list, + } +} + +func (s *PrefixSet) String() string { + return strings.Join(s.List(), "\n") +} + +func (s *PrefixSet) MarshalJSON() ([]byte, error) { + return json.Marshal(s.ToConfig()) +} + +func NewPrefixSetFromApiStruct(name string, prefixes []*Prefix) (*PrefixSet, error) { + if name == "" { + return nil, fmt.Errorf("empty prefix set name") + } + tree := radix.New() + var family bgp.RouteFamily + for i, x := range prefixes { + if i == 0 { + family = x.AddressFamily + } else if family != x.AddressFamily { + return nil, fmt.Errorf("multiple families") + } + key := CidrToRadixkey(x.Prefix.String()) + d, ok := tree.Get(key) + if ok { + ps := d.([]*Prefix) + tree.Insert(key, append(ps, x)) + } else { + tree.Insert(key, []*Prefix{x}) + } + } + return &PrefixSet{ + name: name, + tree: tree, + family: family, + }, nil +} + +func NewPrefixSet(c config.PrefixSet) (*PrefixSet, error) { + name := c.PrefixSetName + if name == "" { + if len(c.PrefixList) == 0 { + return nil, nil + } + return nil, fmt.Errorf("empty prefix set name") + } + tree := radix.New() + var family bgp.RouteFamily + for i, x := range c.PrefixList { + y, err := NewPrefix(x) + if err != nil { + return nil, err + } + if i == 0 { + family = y.AddressFamily + } else if family != y.AddressFamily { + return nil, fmt.Errorf("multiple families") + } + key := CidrToRadixkey(y.Prefix.String()) + d, ok := tree.Get(key) + if ok { + ps := d.([]*Prefix) + tree.Insert(key, append(ps, y)) + } else { + tree.Insert(key, []*Prefix{y}) + } + } + return &PrefixSet{ + name: name, + tree: tree, + family: family, + }, nil +} + +type NextHopSet struct { + list []net.IPNet +} + +func (s *NextHopSet) Name() string { + return "NextHopSet: NO NAME" +} + +func (s *NextHopSet) Type() DefinedType { + return DEFINED_TYPE_NEXT_HOP +} + +func (lhs *NextHopSet) Append(arg DefinedSet) error { + rhs, ok := arg.(*NextHopSet) + if !ok { + return fmt.Errorf("type cast failed") + } + lhs.list = append(lhs.list, rhs.list...) + return nil +} + +func (lhs *NextHopSet) Remove(arg DefinedSet) error { + rhs, ok := arg.(*NextHopSet) + if !ok { + return fmt.Errorf("type cast failed") + } + ps := make([]net.IPNet, 0, len(lhs.list)) + for _, x := range lhs.list { + found := false + for _, y := range rhs.list { + if x.String() == y.String() { + found = true + break + } + } + if !found { + ps = append(ps, x) + } + } + lhs.list = ps + return nil +} + +func (lhs *NextHopSet) Replace(arg DefinedSet) error { + rhs, ok := arg.(*NextHopSet) + if !ok { + return fmt.Errorf("type cast failed") + } + lhs.list = rhs.list + return nil +} + +func (s *NextHopSet) List() []string { + list := make([]string, 0, len(s.list)) + for _, n := range s.list { + list = append(list, n.String()) + } + return list +} + +func (s *NextHopSet) ToConfig() []string { + return s.List() +} + +func (s *NextHopSet) String() string { + return "[ " + strings.Join(s.List(), ", ") + " ]" +} + +func (s *NextHopSet) MarshalJSON() ([]byte, error) { + return json.Marshal(s.ToConfig()) +} + +func NewNextHopSetFromApiStruct(name string, list []net.IPNet) (*NextHopSet, error) { + return &NextHopSet{ + list: list, + }, nil +} + +func NewNextHopSet(c []string) (*NextHopSet, error) { + list := make([]net.IPNet, 0, len(c)) + for _, x := range c { + _, cidr, err := net.ParseCIDR(x) + if err != nil { + addr := net.ParseIP(x) + if addr == nil { + return nil, fmt.Errorf("invalid address or prefix: %s", x) + } + mask := net.CIDRMask(32, 32) + if addr.To4() == nil { + mask = net.CIDRMask(128, 128) + } + cidr = &net.IPNet{ + IP: addr, + Mask: mask, + } + } + list = append(list, *cidr) + } + return &NextHopSet{ + list: list, + }, nil +} + +type NeighborSet struct { + name string + list []net.IPNet +} + +func (s *NeighborSet) Name() string { + return s.name +} + +func (s *NeighborSet) Type() DefinedType { + return DEFINED_TYPE_NEIGHBOR +} + +func (lhs *NeighborSet) Append(arg DefinedSet) error { + rhs, ok := arg.(*NeighborSet) + if !ok { + return fmt.Errorf("type cast failed") + } + lhs.list = append(lhs.list, rhs.list...) + return nil +} + +func (lhs *NeighborSet) Remove(arg DefinedSet) error { + rhs, ok := arg.(*NeighborSet) + if !ok { + return fmt.Errorf("type cast failed") + } + ps := make([]net.IPNet, 0, len(lhs.list)) + for _, x := range lhs.list { + found := false + for _, y := range rhs.list { + if x.String() == y.String() { + found = true + break + } + } + if !found { + ps = append(ps, x) + } + } + lhs.list = ps + return nil +} + +func (lhs *NeighborSet) Replace(arg DefinedSet) error { + rhs, ok := arg.(*NeighborSet) + if !ok { + return fmt.Errorf("type cast failed") + } + lhs.list = rhs.list + return nil +} + +func (s *NeighborSet) List() []string { + list := make([]string, 0, len(s.list)) + for _, n := range s.list { + list = append(list, n.String()) + } + return list +} + +func (s *NeighborSet) ToConfig() *config.NeighborSet { + return &config.NeighborSet{ + NeighborSetName: s.name, + NeighborInfoList: s.List(), + } +} + +func (s *NeighborSet) String() string { + return strings.Join(s.List(), "\n") +} + +func (s *NeighborSet) MarshalJSON() ([]byte, error) { + return json.Marshal(s.ToConfig()) +} + +func NewNeighborSetFromApiStruct(name string, list []net.IPNet) (*NeighborSet, error) { + return &NeighborSet{ + name: name, + list: list, + }, nil +} + +func NewNeighborSet(c config.NeighborSet) (*NeighborSet, error) { + name := c.NeighborSetName + if name == "" { + if len(c.NeighborInfoList) == 0 { + return nil, nil + } + return nil, fmt.Errorf("empty neighbor set name") + } + list := make([]net.IPNet, 0, len(c.NeighborInfoList)) + for _, x := range c.NeighborInfoList { + _, cidr, err := net.ParseCIDR(x) + if err != nil { + addr := net.ParseIP(x) + if addr == nil { + return nil, fmt.Errorf("invalid address or prefix: %s", x) + } + mask := net.CIDRMask(32, 32) + if addr.To4() == nil { + mask = net.CIDRMask(128, 128) + } + cidr = &net.IPNet{ + IP: addr, + Mask: mask, + } + } + list = append(list, *cidr) + } + return &NeighborSet{ + name: name, + list: list, + }, nil +} + +type singleAsPathMatchMode int + +const ( + INCLUDE singleAsPathMatchMode = iota + LEFT_MOST + ORIGIN + ONLY +) + +type singleAsPathMatch struct { + asn uint32 + mode singleAsPathMatchMode +} + +func (lhs *singleAsPathMatch) Equal(rhs *singleAsPathMatch) bool { + return lhs.asn == rhs.asn && lhs.mode == rhs.mode +} + +func (lhs *singleAsPathMatch) String() string { + switch lhs.mode { + case INCLUDE: + return fmt.Sprintf("_%d_", lhs.asn) + case LEFT_MOST: + return fmt.Sprintf("^%d_", lhs.asn) + case ORIGIN: + return fmt.Sprintf("_%d$", lhs.asn) + case ONLY: + return fmt.Sprintf("^%d$", lhs.asn) + } + return "" +} + +func (m *singleAsPathMatch) Match(aspath []uint32) bool { + if len(aspath) == 0 { + return false + } + switch m.mode { + case INCLUDE: + for _, asn := range aspath { + if m.asn == asn { + return true + } + } + case LEFT_MOST: + if m.asn == aspath[0] { + return true + } + case ORIGIN: + if m.asn == aspath[len(aspath)-1] { + return true + } + case ONLY: + if len(aspath) == 1 && m.asn == aspath[0] { + return true + } + } + return false +} + +var ( + _regexpLeftMostRe = regexp.MustCompile(`$\^([0-9]+)_^`) + _regexpOriginRe = regexp.MustCompile(`^_([0-9]+)\$$`) + _regexpIncludeRe = regexp.MustCompile("^_([0-9]+)_$") + _regexpOnlyRe = regexp.MustCompile(`^\^([0-9]+)\$$`) +) + +func NewSingleAsPathMatch(arg string) *singleAsPathMatch { + switch { + case _regexpLeftMostRe.MatchString(arg): + asn, _ := strconv.ParseUint(_regexpLeftMostRe.FindStringSubmatch(arg)[1], 10, 32) + return &singleAsPathMatch{ + asn: uint32(asn), + mode: LEFT_MOST, + } + case _regexpOriginRe.MatchString(arg): + asn, _ := strconv.ParseUint(_regexpOriginRe.FindStringSubmatch(arg)[1], 10, 32) + return &singleAsPathMatch{ + asn: uint32(asn), + mode: ORIGIN, + } + case _regexpIncludeRe.MatchString(arg): + asn, _ := strconv.ParseUint(_regexpIncludeRe.FindStringSubmatch(arg)[1], 10, 32) + return &singleAsPathMatch{ + asn: uint32(asn), + mode: INCLUDE, + } + case _regexpOnlyRe.MatchString(arg): + asn, _ := strconv.ParseUint(_regexpOnlyRe.FindStringSubmatch(arg)[1], 10, 32) + return &singleAsPathMatch{ + asn: uint32(asn), + mode: ONLY, + } + } + return nil +} + +type AsPathSet struct { + typ DefinedType + name string + list []*regexp.Regexp + singleList []*singleAsPathMatch +} + +func (s *AsPathSet) Name() string { + return s.name +} + +func (s *AsPathSet) Type() DefinedType { + return s.typ +} + +func (lhs *AsPathSet) Append(arg DefinedSet) error { + if lhs.Type() != arg.Type() { + return fmt.Errorf("can't append to different type of defined-set") + } + lhs.list = append(lhs.list, arg.(*AsPathSet).list...) + lhs.singleList = append(lhs.singleList, arg.(*AsPathSet).singleList...) + return nil +} + +func (lhs *AsPathSet) Remove(arg DefinedSet) error { + if lhs.Type() != arg.Type() { + return fmt.Errorf("can't append to different type of defined-set") + } + newList := make([]*regexp.Regexp, 0, len(lhs.list)) + for _, x := range lhs.list { + found := false + for _, y := range arg.(*AsPathSet).list { + if x.String() == y.String() { + found = true + break + } + } + if !found { + newList = append(newList, x) + } + } + lhs.list = newList + newSingleList := make([]*singleAsPathMatch, 0, len(lhs.singleList)) + for _, x := range lhs.singleList { + found := false + for _, y := range arg.(*AsPathSet).singleList { + if x.Equal(y) { + found = true + break + } + } + if !found { + newSingleList = append(newSingleList, x) + } + } + lhs.singleList = newSingleList + return nil +} + +func (lhs *AsPathSet) Replace(arg DefinedSet) error { + rhs, ok := arg.(*AsPathSet) + if !ok { + return fmt.Errorf("type cast failed") + } + lhs.list = rhs.list + lhs.singleList = rhs.singleList + return nil +} + +func (s *AsPathSet) List() []string { + list := make([]string, 0, len(s.list)+len(s.singleList)) + for _, exp := range s.singleList { + list = append(list, exp.String()) + } + for _, exp := range s.list { + list = append(list, exp.String()) + } + return list +} + +func (s *AsPathSet) ToConfig() *config.AsPathSet { + return &config.AsPathSet{ + AsPathSetName: s.name, + AsPathList: s.List(), + } +} + +func (s *AsPathSet) String() string { + return strings.Join(s.List(), "\n") +} + +func (s *AsPathSet) MarshalJSON() ([]byte, error) { + return json.Marshal(s.ToConfig()) +} + +func NewAsPathSet(c config.AsPathSet) (*AsPathSet, error) { + name := c.AsPathSetName + if name == "" { + if len(c.AsPathList) == 0 { + return nil, nil + } + return nil, fmt.Errorf("empty as-path set name") + } + list := make([]*regexp.Regexp, 0, len(c.AsPathList)) + singleList := make([]*singleAsPathMatch, 0, len(c.AsPathList)) + for _, x := range c.AsPathList { + if s := NewSingleAsPathMatch(x); s != nil { + singleList = append(singleList, s) + } else { + exp, err := regexp.Compile(strings.Replace(x, "_", ASPATH_REGEXP_MAGIC, -1)) + if err != nil { + return nil, fmt.Errorf("invalid regular expression: %s", x) + } + list = append(list, exp) + } + } + return &AsPathSet{ + typ: DEFINED_TYPE_AS_PATH, + name: name, + list: list, + singleList: singleList, + }, nil +} + +type regExpSet struct { + typ DefinedType + name string + list []*regexp.Regexp +} + +func (s *regExpSet) Name() string { + return s.name +} + +func (s *regExpSet) Type() DefinedType { + return s.typ +} + +func (lhs *regExpSet) Append(arg DefinedSet) error { + if lhs.Type() != arg.Type() { + return fmt.Errorf("can't append to different type of defined-set") + } + var list []*regexp.Regexp + switch lhs.Type() { + case DEFINED_TYPE_AS_PATH: + list = arg.(*AsPathSet).list + case DEFINED_TYPE_COMMUNITY: + list = arg.(*CommunitySet).list + case DEFINED_TYPE_EXT_COMMUNITY: + list = arg.(*ExtCommunitySet).list + case DEFINED_TYPE_LARGE_COMMUNITY: + list = arg.(*LargeCommunitySet).list + default: + return fmt.Errorf("invalid defined-set type: %d", lhs.Type()) + } + lhs.list = append(lhs.list, list...) + return nil +} + +func (lhs *regExpSet) Remove(arg DefinedSet) error { + if lhs.Type() != arg.Type() { + return fmt.Errorf("can't append to different type of defined-set") + } + var list []*regexp.Regexp + switch lhs.Type() { + case DEFINED_TYPE_AS_PATH: + list = arg.(*AsPathSet).list + case DEFINED_TYPE_COMMUNITY: + list = arg.(*CommunitySet).list + case DEFINED_TYPE_EXT_COMMUNITY: + list = arg.(*ExtCommunitySet).list + case DEFINED_TYPE_LARGE_COMMUNITY: + list = arg.(*LargeCommunitySet).list + default: + return fmt.Errorf("invalid defined-set type: %d", lhs.Type()) + } + ps := make([]*regexp.Regexp, 0, len(lhs.list)) + for _, x := range lhs.list { + found := false + for _, y := range list { + if x.String() == y.String() { + found = true + break + } + } + if !found { + ps = append(ps, x) + } + } + lhs.list = ps + return nil +} + +func (lhs *regExpSet) Replace(arg DefinedSet) error { + switch c := arg.(type) { + case *CommunitySet: + lhs.list = c.list + case *ExtCommunitySet: + lhs.list = c.list + case *LargeCommunitySet: + lhs.list = c.list + default: + return fmt.Errorf("type cast failed") + } + return nil +} + +type CommunitySet struct { + regExpSet +} + +func (s *CommunitySet) List() []string { + list := make([]string, 0, len(s.list)) + for _, exp := range s.list { + list = append(list, exp.String()) + } + return list +} + +func (s *CommunitySet) ToConfig() *config.CommunitySet { + return &config.CommunitySet{ + CommunitySetName: s.name, + CommunityList: s.List(), + } +} + +func (s *CommunitySet) String() string { + return strings.Join(s.List(), "\n") +} + +func (s *CommunitySet) MarshalJSON() ([]byte, error) { + return json.Marshal(s.ToConfig()) +} + +var _regexpCommunity = regexp.MustCompile(`(\d+):(\d+)`) + +func ParseCommunity(arg string) (uint32, error) { + i, err := strconv.ParseUint(arg, 10, 32) + if err == nil { + return uint32(i), nil + } + + elems := _regexpCommunity.FindStringSubmatch(arg) + if len(elems) == 3 { + fst, _ := strconv.ParseUint(elems[1], 10, 16) + snd, _ := strconv.ParseUint(elems[2], 10, 16) + return uint32(fst<<16 | snd), nil + } + for i, v := range bgp.WellKnownCommunityNameMap { + if arg == v { + return uint32(i), nil + } + } + return 0, fmt.Errorf("failed to parse %s as community", arg) +} + +func ParseExtCommunity(arg string) (bgp.ExtendedCommunityInterface, error) { + var subtype bgp.ExtendedCommunityAttrSubType + var value string + elems := strings.SplitN(arg, ":", 2) + + isValidationState := func(s string) bool { + s = strings.ToLower(s) + r := s == bgp.VALIDATION_STATE_VALID.String() + r = r || s == bgp.VALIDATION_STATE_NOT_FOUND.String() + return r || s == bgp.VALIDATION_STATE_INVALID.String() + } + if len(elems) < 2 && (len(elems) < 1 && !isValidationState(elems[0])) { + return nil, fmt.Errorf("invalid ext-community (rt|soo):<value> | valid | not-found | invalid") + } + if isValidationState(elems[0]) { + subtype = bgp.EC_SUBTYPE_ORIGIN_VALIDATION + value = elems[0] + } else { + switch strings.ToLower(elems[0]) { + case "rt": + subtype = bgp.EC_SUBTYPE_ROUTE_TARGET + case "soo": + subtype = bgp.EC_SUBTYPE_ROUTE_ORIGIN + default: + return nil, fmt.Errorf("invalid ext-community (rt|soo):<value> | valid | not-found | invalid") + } + value = elems[1] + } + return bgp.ParseExtendedCommunity(subtype, value) +} + +var _regexpCommunity2 = regexp.MustCompile(`(\d+.)*\d+:\d+`) + +func ParseCommunityRegexp(arg string) (*regexp.Regexp, error) { + i, err := strconv.ParseUint(arg, 10, 32) + if err == nil { + return regexp.Compile(fmt.Sprintf("^%d:%d$", i>>16, i&0x0000ffff)) + } + + if _regexpCommunity2.MatchString(arg) { + return regexp.Compile(fmt.Sprintf("^%s$", arg)) + } + + for i, v := range bgp.WellKnownCommunityNameMap { + if strings.Replace(strings.ToLower(arg), "_", "-", -1) == v { + return regexp.Compile(fmt.Sprintf("^%d:%d$", i>>16, i&0x0000ffff)) + } + } + + return regexp.Compile(arg) +} + +func ParseExtCommunityRegexp(arg string) (bgp.ExtendedCommunityAttrSubType, *regexp.Regexp, error) { + var subtype bgp.ExtendedCommunityAttrSubType + elems := strings.SplitN(arg, ":", 2) + if len(elems) < 2 { + return subtype, nil, fmt.Errorf("invalid ext-community format([rt|soo]:<value>)") + } + switch strings.ToLower(elems[0]) { + case "rt": + subtype = bgp.EC_SUBTYPE_ROUTE_TARGET + case "soo": + subtype = bgp.EC_SUBTYPE_ROUTE_ORIGIN + default: + return subtype, nil, fmt.Errorf("unknown ext-community subtype. rt, soo is supported") + } + exp, err := ParseCommunityRegexp(elems[1]) + return subtype, exp, err +} + +func NewCommunitySet(c config.CommunitySet) (*CommunitySet, error) { + name := c.CommunitySetName + if name == "" { + if len(c.CommunityList) == 0 { + return nil, nil + } + return nil, fmt.Errorf("empty community set name") + } + list := make([]*regexp.Regexp, 0, len(c.CommunityList)) + for _, x := range c.CommunityList { + exp, err := ParseCommunityRegexp(x) + if err != nil { + return nil, err + } + list = append(list, exp) + } + return &CommunitySet{ + regExpSet: regExpSet{ + typ: DEFINED_TYPE_COMMUNITY, + name: name, + list: list, + }, + }, nil +} + +type ExtCommunitySet struct { + regExpSet + subtypeList []bgp.ExtendedCommunityAttrSubType +} + +func (s *ExtCommunitySet) List() []string { + list := make([]string, 0, len(s.list)) + f := func(idx int, arg string) string { + switch s.subtypeList[idx] { + case bgp.EC_SUBTYPE_ROUTE_TARGET: + return fmt.Sprintf("rt:%s", arg) + case bgp.EC_SUBTYPE_ROUTE_ORIGIN: + return fmt.Sprintf("soo:%s", arg) + case bgp.EC_SUBTYPE_ORIGIN_VALIDATION: + return arg + default: + return fmt.Sprintf("%d:%s", s.subtypeList[idx], arg) + } + } + for idx, exp := range s.list { + list = append(list, f(idx, exp.String())) + } + return list +} + +func (s *ExtCommunitySet) ToConfig() *config.ExtCommunitySet { + return &config.ExtCommunitySet{ + ExtCommunitySetName: s.name, + ExtCommunityList: s.List(), + } +} + +func (s *ExtCommunitySet) String() string { + return strings.Join(s.List(), "\n") +} + +func (s *ExtCommunitySet) MarshalJSON() ([]byte, error) { + return json.Marshal(s.ToConfig()) +} + +func NewExtCommunitySet(c config.ExtCommunitySet) (*ExtCommunitySet, error) { + name := c.ExtCommunitySetName + if name == "" { + if len(c.ExtCommunityList) == 0 { + return nil, nil + } + return nil, fmt.Errorf("empty ext-community set name") + } + list := make([]*regexp.Regexp, 0, len(c.ExtCommunityList)) + subtypeList := make([]bgp.ExtendedCommunityAttrSubType, 0, len(c.ExtCommunityList)) + for _, x := range c.ExtCommunityList { + subtype, exp, err := ParseExtCommunityRegexp(x) + if err != nil { + return nil, err + } + list = append(list, exp) + subtypeList = append(subtypeList, subtype) + } + return &ExtCommunitySet{ + regExpSet: regExpSet{ + typ: DEFINED_TYPE_EXT_COMMUNITY, + name: name, + list: list, + }, + subtypeList: subtypeList, + }, nil +} + +func (s *ExtCommunitySet) Append(arg DefinedSet) error { + err := s.regExpSet.Append(arg) + if err != nil { + return err + } + sList := arg.(*ExtCommunitySet).subtypeList + s.subtypeList = append(s.subtypeList, sList...) + return nil +} + +type LargeCommunitySet struct { + regExpSet +} + +func (s *LargeCommunitySet) List() []string { + list := make([]string, 0, len(s.list)) + for _, exp := range s.list { + list = append(list, exp.String()) + } + return list +} + +func (s *LargeCommunitySet) ToConfig() *config.LargeCommunitySet { + return &config.LargeCommunitySet{ + LargeCommunitySetName: s.name, + LargeCommunityList: s.List(), + } +} + +func (s *LargeCommunitySet) String() string { + return strings.Join(s.List(), "\n") +} + +func (s *LargeCommunitySet) MarshalJSON() ([]byte, error) { + return json.Marshal(s.ToConfig()) +} + +var _regexpCommunityLarge = regexp.MustCompile(`\d+:\d+:\d+`) + +func ParseLargeCommunityRegexp(arg string) (*regexp.Regexp, error) { + if _regexpCommunityLarge.MatchString(arg) { + return regexp.Compile(fmt.Sprintf("^%s$", arg)) + } + exp, err := regexp.Compile(arg) + if err != nil { + return nil, fmt.Errorf("invalid large-community format: %v", err) + } + + return exp, nil +} + +func NewLargeCommunitySet(c config.LargeCommunitySet) (*LargeCommunitySet, error) { + name := c.LargeCommunitySetName + if name == "" { + if len(c.LargeCommunityList) == 0 { + return nil, nil + } + return nil, fmt.Errorf("empty large community set name") + } + list := make([]*regexp.Regexp, 0, len(c.LargeCommunityList)) + for _, x := range c.LargeCommunityList { + exp, err := ParseLargeCommunityRegexp(x) + if err != nil { + return nil, err + } + list = append(list, exp) + } + return &LargeCommunitySet{ + regExpSet: regExpSet{ + typ: DEFINED_TYPE_LARGE_COMMUNITY, + name: name, + list: list, + }, + }, nil +} + +type Condition interface { + Name() string + Type() ConditionType + Evaluate(*Path, *PolicyOptions) bool + Set() DefinedSet +} + +type NextHopCondition struct { + set *NextHopSet +} + +func (c *NextHopCondition) Type() ConditionType { + return CONDITION_NEXT_HOP +} + +func (c *NextHopCondition) Set() DefinedSet { + return c.set +} + +func (c *NextHopCondition) Name() string { return "" } + +func (c *NextHopCondition) String() string { + return c.set.String() +} + +// compare next-hop ipaddress of this condition and source address of path +// and, subsequent comparisons are skipped if that matches the conditions. +// If NextHopSet's length is zero, return true. +func (c *NextHopCondition) Evaluate(path *Path, options *PolicyOptions) bool { + if len(c.set.list) == 0 { + log.WithFields(log.Fields{ + "Topic": "Policy", + }).Debug("NextHop doesn't have elements") + return true + } + + nexthop := path.GetNexthop() + + // In cases where we advertise routes from iBGP to eBGP, we want to filter + // on the "original" nexthop. The current paths' nexthop has already been + // set and is ready to be advertised as per: + // https://tools.ietf.org/html/rfc4271#section-5.1.3 + if options != nil && options.OldNextHop != nil && + !options.OldNextHop.IsUnspecified() && !options.OldNextHop.Equal(nexthop) { + nexthop = options.OldNextHop + } + + if nexthop == nil { + return false + } + + for _, n := range c.set.list { + if n.Contains(nexthop) { + return true + } + } + + return false +} + +func NewNextHopCondition(c []string) (*NextHopCondition, error) { + if len(c) == 0 { + return nil, nil + } + + list, err := NewNextHopSet(c) + if err != nil { + return nil, nil + } + + return &NextHopCondition{ + set: list, + }, nil +} + +type PrefixCondition struct { + set *PrefixSet + option MatchOption +} + +func (c *PrefixCondition) Type() ConditionType { + return CONDITION_PREFIX +} + +func (c *PrefixCondition) Set() DefinedSet { + return c.set +} + +func (c *PrefixCondition) Option() MatchOption { + return c.option +} + +// compare prefixes in this condition and nlri of path and +// subsequent comparison is skipped if that matches the conditions. +// If PrefixList's length is zero, return true. +func (c *PrefixCondition) Evaluate(path *Path, _ *PolicyOptions) bool { + var key string + var masklen uint8 + keyf := func(ip net.IP, ones int) string { + var buffer bytes.Buffer + for i := 0; i < len(ip) && i < ones; i++ { + buffer.WriteString(fmt.Sprintf("%08b", ip[i])) + } + return buffer.String()[:ones] + } + family := path.GetRouteFamily() + switch family { + case bgp.RF_IPv4_UC: + masklen = path.GetNlri().(*bgp.IPAddrPrefix).Length + key = keyf(path.GetNlri().(*bgp.IPAddrPrefix).Prefix, int(masklen)) + case bgp.RF_IPv6_UC: + masklen = path.GetNlri().(*bgp.IPv6AddrPrefix).Length + key = keyf(path.GetNlri().(*bgp.IPv6AddrPrefix).Prefix, int(masklen)) + default: + return false + } + if family != c.set.family { + return false + } + + result := false + _, ps, ok := c.set.tree.LongestPrefix(key) + if ok { + for _, p := range ps.([]*Prefix) { + if p.MasklengthRangeMin <= masklen && masklen <= p.MasklengthRangeMax { + result = true + break + } + } + } + + if c.option == MATCH_OPTION_INVERT { + result = !result + } + + return result +} + +func (c *PrefixCondition) Name() string { return c.set.name } + +func NewPrefixCondition(c config.MatchPrefixSet) (*PrefixCondition, error) { + if c.PrefixSet == "" { + return nil, nil + } + o, err := NewMatchOption(c.MatchSetOptions) + if err != nil { + return nil, err + } + return &PrefixCondition{ + set: &PrefixSet{ + name: c.PrefixSet, + }, + option: o, + }, nil +} + +type NeighborCondition struct { + set *NeighborSet + option MatchOption +} + +func (c *NeighborCondition) Type() ConditionType { + return CONDITION_NEIGHBOR +} + +func (c *NeighborCondition) Set() DefinedSet { + return c.set +} + +func (c *NeighborCondition) Option() MatchOption { + return c.option +} + +// compare neighbor ipaddress of this condition and source address of path +// and, subsequent comparisons are skipped if that matches the conditions. +// If NeighborList's length is zero, return true. +func (c *NeighborCondition) Evaluate(path *Path, options *PolicyOptions) bool { + if len(c.set.list) == 0 { + log.WithFields(log.Fields{ + "Topic": "Policy", + }).Debug("NeighborList doesn't have elements") + return true + } + + neighbor := path.GetSource().Address + if options != nil && options.Info != nil && options.Info.Address != nil { + neighbor = options.Info.Address + } + + if neighbor == nil { + return false + } + result := false + for _, n := range c.set.list { + if n.Contains(neighbor) { + result = true + break + } + } + + if c.option == MATCH_OPTION_INVERT { + result = !result + } + + return result +} + +func (c *NeighborCondition) Name() string { return c.set.name } + +func NewNeighborCondition(c config.MatchNeighborSet) (*NeighborCondition, error) { + if c.NeighborSet == "" { + return nil, nil + } + o, err := NewMatchOption(c.MatchSetOptions) + if err != nil { + return nil, err + } + return &NeighborCondition{ + set: &NeighborSet{ + name: c.NeighborSet, + }, + option: o, + }, nil +} + +type AsPathCondition struct { + set *AsPathSet + option MatchOption +} + +func (c *AsPathCondition) Type() ConditionType { + return CONDITION_AS_PATH +} + +func (c *AsPathCondition) Set() DefinedSet { + return c.set +} + +func (c *AsPathCondition) Option() MatchOption { + return c.option +} + +func (c *AsPathCondition) Evaluate(path *Path, _ *PolicyOptions) bool { + if len(c.set.singleList) > 0 { + aspath := path.GetAsSeqList() + for _, m := range c.set.singleList { + result := m.Match(aspath) + if c.option == MATCH_OPTION_ALL && !result { + return false + } + if c.option == MATCH_OPTION_ANY && result { + return true + } + if c.option == MATCH_OPTION_INVERT && result { + return false + } + } + } + if len(c.set.list) > 0 { + aspath := path.GetAsString() + for _, r := range c.set.list { + result := r.MatchString(aspath) + if c.option == MATCH_OPTION_ALL && !result { + return false + } + if c.option == MATCH_OPTION_ANY && result { + return true + } + if c.option == MATCH_OPTION_INVERT && result { + return false + } + } + } + if c.option == MATCH_OPTION_ANY { + return false + } + return true +} + +func (c *AsPathCondition) Name() string { return c.set.name } + +func NewAsPathCondition(c config.MatchAsPathSet) (*AsPathCondition, error) { + if c.AsPathSet == "" { + return nil, nil + } + o, err := NewMatchOption(c.MatchSetOptions) + if err != nil { + return nil, err + } + return &AsPathCondition{ + set: &AsPathSet{ + name: c.AsPathSet, + }, + option: o, + }, nil +} + +type CommunityCondition struct { + set *CommunitySet + option MatchOption +} + +func (c *CommunityCondition) Type() ConditionType { + return CONDITION_COMMUNITY +} + +func (c *CommunityCondition) Set() DefinedSet { + return c.set +} + +func (c *CommunityCondition) Option() MatchOption { + return c.option +} + +func (c *CommunityCondition) Evaluate(path *Path, _ *PolicyOptions) bool { + cs := path.GetCommunities() + result := false + for _, x := range c.set.list { + result = false + for _, y := range cs { + if x.MatchString(fmt.Sprintf("%d:%d", y>>16, y&0x0000ffff)) { + result = true + break + } + } + if c.option == MATCH_OPTION_ALL && !result { + break + } + if (c.option == MATCH_OPTION_ANY || c.option == MATCH_OPTION_INVERT) && result { + break + } + } + if c.option == MATCH_OPTION_INVERT { + result = !result + } + return result +} + +func (c *CommunityCondition) Name() string { return c.set.name } + +func NewCommunityCondition(c config.MatchCommunitySet) (*CommunityCondition, error) { + if c.CommunitySet == "" { + return nil, nil + } + o, err := NewMatchOption(c.MatchSetOptions) + if err != nil { + return nil, err + } + return &CommunityCondition{ + set: &CommunitySet{ + regExpSet: regExpSet{ + name: c.CommunitySet, + }, + }, + option: o, + }, nil +} + +type ExtCommunityCondition struct { + set *ExtCommunitySet + option MatchOption +} + +func (c *ExtCommunityCondition) Type() ConditionType { + return CONDITION_EXT_COMMUNITY +} + +func (c *ExtCommunityCondition) Set() DefinedSet { + return c.set +} + +func (c *ExtCommunityCondition) Option() MatchOption { + return c.option +} + +func (c *ExtCommunityCondition) Evaluate(path *Path, _ *PolicyOptions) bool { + es := path.GetExtCommunities() + result := false + for _, x := range es { + result = false + typ, subtype := x.GetTypes() + // match only with transitive community. see RFC7153 + if typ >= 0x3f { + continue + } + for idx, y := range c.set.list { + if subtype == c.set.subtypeList[idx] && y.MatchString(x.String()) { + result = true + break + } + } + if c.option == MATCH_OPTION_ALL && !result { + break + } + if c.option == MATCH_OPTION_ANY && result { + break + } + } + if c.option == MATCH_OPTION_INVERT { + result = !result + } + return result +} + +func (c *ExtCommunityCondition) Name() string { return c.set.name } + +func NewExtCommunityCondition(c config.MatchExtCommunitySet) (*ExtCommunityCondition, error) { + if c.ExtCommunitySet == "" { + return nil, nil + } + o, err := NewMatchOption(c.MatchSetOptions) + if err != nil { + return nil, err + } + return &ExtCommunityCondition{ + set: &ExtCommunitySet{ + regExpSet: regExpSet{ + name: c.ExtCommunitySet, + }, + }, + option: o, + }, nil +} + +type LargeCommunityCondition struct { + set *LargeCommunitySet + option MatchOption +} + +func (c *LargeCommunityCondition) Type() ConditionType { + return CONDITION_LARGE_COMMUNITY +} + +func (c *LargeCommunityCondition) Set() DefinedSet { + return c.set +} + +func (c *LargeCommunityCondition) Option() MatchOption { + return c.option +} + +func (c *LargeCommunityCondition) Evaluate(path *Path, _ *PolicyOptions) bool { + result := false + cs := path.GetLargeCommunities() + for _, x := range c.set.list { + result = false + for _, y := range cs { + if x.MatchString(y.String()) { + result = true + break + } + } + if c.option == MATCH_OPTION_ALL && !result { + break + } + if (c.option == MATCH_OPTION_ANY || c.option == MATCH_OPTION_INVERT) && result { + break + } + } + if c.option == MATCH_OPTION_INVERT { + result = !result + } + return result +} + +func (c *LargeCommunityCondition) Name() string { return c.set.name } + +func NewLargeCommunityCondition(c config.MatchLargeCommunitySet) (*LargeCommunityCondition, error) { + if c.LargeCommunitySet == "" { + return nil, nil + } + o, err := NewMatchOption(c.MatchSetOptions) + if err != nil { + return nil, err + } + return &LargeCommunityCondition{ + set: &LargeCommunitySet{ + regExpSet: regExpSet{ + name: c.LargeCommunitySet, + }, + }, + option: o, + }, nil +} + +type AsPathLengthCondition struct { + length uint32 + operator AttributeComparison +} + +func (c *AsPathLengthCondition) Type() ConditionType { + return CONDITION_AS_PATH_LENGTH +} + +// compare AS_PATH length in the message's AS_PATH attribute with +// the one in condition. +func (c *AsPathLengthCondition) Evaluate(path *Path, _ *PolicyOptions) bool { + + length := uint32(path.GetAsPathLen()) + result := false + switch c.operator { + case ATTRIBUTE_EQ: + result = c.length == length + case ATTRIBUTE_GE: + result = c.length <= length + case ATTRIBUTE_LE: + result = c.length >= length + } + + return result +} + +func (c *AsPathLengthCondition) Set() DefinedSet { + return nil +} + +func (c *AsPathLengthCondition) Name() string { return "" } + +func (c *AsPathLengthCondition) String() string { + return fmt.Sprintf("%s%d", c.operator, c.length) +} + +func NewAsPathLengthCondition(c config.AsPathLength) (*AsPathLengthCondition, error) { + if c.Value == 0 && c.Operator == "" { + return nil, nil + } + var op AttributeComparison + if i := c.Operator.ToInt(); i < 0 { + return nil, fmt.Errorf("invalid as path length operator: %s", c.Operator) + } else { + // take mod 3 because we have extended openconfig attribute-comparison + // for simple configuration. see config.AttributeComparison definition + op = AttributeComparison(i % 3) + } + return &AsPathLengthCondition{ + length: c.Value, + operator: op, + }, nil +} + +type RpkiValidationCondition struct { + result config.RpkiValidationResultType +} + +func (c *RpkiValidationCondition) Type() ConditionType { + return CONDITION_RPKI +} + +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 { + return nil +} + +func (c *RpkiValidationCondition) Name() string { return "" } + +func (c *RpkiValidationCondition) String() string { + return string(c.result) +} + +func NewRpkiValidationCondition(c config.RpkiValidationResultType) (*RpkiValidationCondition, error) { + if c == config.RpkiValidationResultType("") || c == config.RPKI_VALIDATION_RESULT_TYPE_NONE { + return nil, nil + } + return &RpkiValidationCondition{ + result: c, + }, nil +} + +type RouteTypeCondition struct { + typ config.RouteType +} + +func (c *RouteTypeCondition) Type() ConditionType { + return CONDITION_ROUTE_TYPE +} + +func (c *RouteTypeCondition) Evaluate(path *Path, _ *PolicyOptions) bool { + switch c.typ { + case config.ROUTE_TYPE_LOCAL: + return path.IsLocal() + case config.ROUTE_TYPE_INTERNAL: + return !path.IsLocal() && path.IsIBGP() + case config.ROUTE_TYPE_EXTERNAL: + return !path.IsLocal() && !path.IsIBGP() + } + return false +} + +func (c *RouteTypeCondition) Set() DefinedSet { + return nil +} + +func (c *RouteTypeCondition) Name() string { return "" } + +func (c *RouteTypeCondition) String() string { + return string(c.typ) +} + +func NewRouteTypeCondition(c config.RouteType) (*RouteTypeCondition, error) { + if string(c) == "" || c == config.ROUTE_TYPE_NONE { + return nil, nil + } + if err := c.Validate(); err != nil { + return nil, err + } + return &RouteTypeCondition{ + typ: c, + }, nil +} + +type AfiSafiInCondition struct { + routeFamilies []bgp.RouteFamily +} + +func (c *AfiSafiInCondition) Type() ConditionType { + return CONDITION_AFI_SAFI_IN +} + +func (c *AfiSafiInCondition) Evaluate(path *Path, _ *PolicyOptions) bool { + for _, rf := range c.routeFamilies { + if path.GetRouteFamily() == rf { + return true + } + } + return false +} + +func (c *AfiSafiInCondition) Set() DefinedSet { + return nil +} + +func (c *AfiSafiInCondition) Name() string { return "" } + +func (c *AfiSafiInCondition) String() string { + tmp := make([]string, 0, len(c.routeFamilies)) + for _, afiSafi := range c.routeFamilies { + tmp = append(tmp, afiSafi.String()) + } + return strings.Join(tmp, " ") +} + +func NewAfiSafiInCondition(afiSafInConfig []config.AfiSafiType) (*AfiSafiInCondition, error) { + if afiSafInConfig == nil { + return nil, nil + } + + routeFamilies := make([]bgp.RouteFamily, 0, len(afiSafInConfig)) + for _, afiSafiValue := range afiSafInConfig { + if err := afiSafiValue.Validate(); err != nil { + return nil, err + } + rf, err := bgp.GetRouteFamily(string(afiSafiValue)) + if err != nil { + return nil, err + } + routeFamilies = append(routeFamilies, rf) + } + return &AfiSafiInCondition{ + routeFamilies: routeFamilies, + }, nil +} + +type Action interface { + Type() ActionType + Apply(*Path, *PolicyOptions) *Path + String() string +} + +type RoutingAction struct { + AcceptRoute bool +} + +func (a *RoutingAction) Type() ActionType { + return ACTION_ROUTING +} + +func (a *RoutingAction) Apply(path *Path, _ *PolicyOptions) *Path { + if a.AcceptRoute { + return path + } + return nil +} + +func (a *RoutingAction) String() string { + action := "reject" + if a.AcceptRoute { + action = "accept" + } + return action +} + +func NewRoutingAction(c config.RouteDisposition) (*RoutingAction, error) { + var accept bool + switch c { + case config.RouteDisposition(""), config.ROUTE_DISPOSITION_NONE: + return nil, nil + case config.ROUTE_DISPOSITION_ACCEPT_ROUTE: + accept = true + case config.ROUTE_DISPOSITION_REJECT_ROUTE: + accept = false + default: + return nil, fmt.Errorf("invalid route disposition") + } + return &RoutingAction{ + AcceptRoute: accept, + }, nil +} + +type CommunityAction struct { + action config.BgpSetCommunityOptionType + list []uint32 + removeList []*regexp.Regexp +} + +func RegexpRemoveCommunities(path *Path, exps []*regexp.Regexp) { + comms := path.GetCommunities() + newComms := make([]uint32, 0, len(comms)) + for _, comm := range comms { + c := fmt.Sprintf("%d:%d", comm>>16, comm&0x0000ffff) + match := false + for _, exp := range exps { + if exp.MatchString(c) { + match = true + break + } + } + if !match { + newComms = append(newComms, comm) + } + } + path.SetCommunities(newComms, true) +} + +func RegexpRemoveExtCommunities(path *Path, exps []*regexp.Regexp, subtypes []bgp.ExtendedCommunityAttrSubType) { + comms := path.GetExtCommunities() + newComms := make([]bgp.ExtendedCommunityInterface, 0, len(comms)) + for _, comm := range comms { + match := false + typ, subtype := comm.GetTypes() + // match only with transitive community. see RFC7153 + if typ >= 0x3f { + continue + } + for idx, exp := range exps { + if subtype == subtypes[idx] && exp.MatchString(comm.String()) { + match = true + break + } + } + if !match { + newComms = append(newComms, comm) + } + } + path.SetExtCommunities(newComms, true) +} + +func RegexpRemoveLargeCommunities(path *Path, exps []*regexp.Regexp) { + comms := path.GetLargeCommunities() + newComms := make([]*bgp.LargeCommunity, 0, len(comms)) + for _, comm := range comms { + c := comm.String() + match := false + for _, exp := range exps { + if exp.MatchString(c) { + match = true + break + } + } + if !match { + newComms = append(newComms, comm) + } + } + path.SetLargeCommunities(newComms, true) +} + +func (a *CommunityAction) Type() ActionType { + return ACTION_COMMUNITY +} + +func (a *CommunityAction) Apply(path *Path, _ *PolicyOptions) *Path { + switch a.action { + case config.BGP_SET_COMMUNITY_OPTION_TYPE_ADD: + path.SetCommunities(a.list, false) + case config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE: + RegexpRemoveCommunities(path, a.removeList) + case config.BGP_SET_COMMUNITY_OPTION_TYPE_REPLACE: + path.SetCommunities(a.list, true) + } + return path +} + +func (a *CommunityAction) ToConfig() *config.SetCommunity { + cs := make([]string, 0, len(a.list)+len(a.removeList)) + for _, comm := range a.list { + c := fmt.Sprintf("%d:%d", comm>>16, comm&0x0000ffff) + cs = append(cs, c) + } + for _, exp := range a.removeList { + cs = append(cs, exp.String()) + } + return &config.SetCommunity{ + Options: string(a.action), + SetCommunityMethod: config.SetCommunityMethod{CommunitiesList: cs}, + } +} + +func (a *CommunityAction) MarshalJSON() ([]byte, error) { + return json.Marshal(a.ToConfig()) +} + +// TODO: this is not efficient use of regexp, probably slow +var _regexpCommunityReplaceString = regexp.MustCompile(`[\^\$]`) + +func (a *CommunityAction) String() string { + list := a.ToConfig().SetCommunityMethod.CommunitiesList + l := _regexpCommunityReplaceString.ReplaceAllString(strings.Join(list, ", "), "") + return fmt.Sprintf("%s[%s]", a.action, l) +} + +func NewCommunityAction(c config.SetCommunity) (*CommunityAction, error) { + a, ok := CommunityOptionValueMap[strings.ToLower(c.Options)] + if !ok { + if len(c.SetCommunityMethod.CommunitiesList) == 0 { + return nil, nil + } + return nil, fmt.Errorf("invalid option name: %s", c.Options) + } + var list []uint32 + var removeList []*regexp.Regexp + if a == config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE { + removeList = make([]*regexp.Regexp, 0, len(c.SetCommunityMethod.CommunitiesList)) + } else { + list = make([]uint32, 0, len(c.SetCommunityMethod.CommunitiesList)) + } + for _, x := range c.SetCommunityMethod.CommunitiesList { + if a == config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE { + exp, err := ParseCommunityRegexp(x) + if err != nil { + return nil, err + } + removeList = append(removeList, exp) + } else { + comm, err := ParseCommunity(x) + if err != nil { + return nil, err + } + list = append(list, comm) + } + } + return &CommunityAction{ + action: a, + list: list, + removeList: removeList, + }, nil +} + +type ExtCommunityAction struct { + action config.BgpSetCommunityOptionType + list []bgp.ExtendedCommunityInterface + removeList []*regexp.Regexp + subtypeList []bgp.ExtendedCommunityAttrSubType +} + +func (a *ExtCommunityAction) Type() ActionType { + return ACTION_EXT_COMMUNITY +} + +func (a *ExtCommunityAction) Apply(path *Path, _ *PolicyOptions) *Path { + switch a.action { + case config.BGP_SET_COMMUNITY_OPTION_TYPE_ADD: + path.SetExtCommunities(a.list, false) + case config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE: + RegexpRemoveExtCommunities(path, a.removeList, a.subtypeList) + case config.BGP_SET_COMMUNITY_OPTION_TYPE_REPLACE: + path.SetExtCommunities(a.list, true) + } + return path +} + +func (a *ExtCommunityAction) ToConfig() *config.SetExtCommunity { + cs := make([]string, 0, len(a.list)+len(a.removeList)) + f := func(idx int, arg string) string { + switch a.subtypeList[idx] { + case bgp.EC_SUBTYPE_ROUTE_TARGET: + return fmt.Sprintf("rt:%s", arg) + case bgp.EC_SUBTYPE_ROUTE_ORIGIN: + return fmt.Sprintf("soo:%s", arg) + case bgp.EC_SUBTYPE_ORIGIN_VALIDATION: + return arg + default: + return fmt.Sprintf("%d:%s", a.subtypeList[idx], arg) + } + } + for idx, c := range a.list { + cs = append(cs, f(idx, c.String())) + } + for idx, exp := range a.removeList { + cs = append(cs, f(idx, exp.String())) + } + return &config.SetExtCommunity{ + Options: string(a.action), + SetExtCommunityMethod: config.SetExtCommunityMethod{ + CommunitiesList: cs, + }, + } +} + +func (a *ExtCommunityAction) String() string { + list := a.ToConfig().SetExtCommunityMethod.CommunitiesList + l := _regexpCommunityReplaceString.ReplaceAllString(strings.Join(list, ", "), "") + return fmt.Sprintf("%s[%s]", a.action, l) +} + +func (a *ExtCommunityAction) MarshalJSON() ([]byte, error) { + return json.Marshal(a.ToConfig()) +} + +func NewExtCommunityAction(c config.SetExtCommunity) (*ExtCommunityAction, error) { + a, ok := CommunityOptionValueMap[strings.ToLower(c.Options)] + if !ok { + if len(c.SetExtCommunityMethod.CommunitiesList) == 0 { + return nil, nil + } + return nil, fmt.Errorf("invalid option name: %s", c.Options) + } + var list []bgp.ExtendedCommunityInterface + var removeList []*regexp.Regexp + subtypeList := make([]bgp.ExtendedCommunityAttrSubType, 0, len(c.SetExtCommunityMethod.CommunitiesList)) + if a == config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE { + removeList = make([]*regexp.Regexp, 0, len(c.SetExtCommunityMethod.CommunitiesList)) + } else { + list = make([]bgp.ExtendedCommunityInterface, 0, len(c.SetExtCommunityMethod.CommunitiesList)) + } + for _, x := range c.SetExtCommunityMethod.CommunitiesList { + if a == config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE { + subtype, exp, err := ParseExtCommunityRegexp(x) + if err != nil { + return nil, err + } + removeList = append(removeList, exp) + subtypeList = append(subtypeList, subtype) + } else { + comm, err := ParseExtCommunity(x) + if err != nil { + return nil, err + } + list = append(list, comm) + _, subtype := comm.GetTypes() + subtypeList = append(subtypeList, subtype) + } + } + return &ExtCommunityAction{ + action: a, + list: list, + removeList: removeList, + subtypeList: subtypeList, + }, nil +} + +type LargeCommunityAction struct { + action config.BgpSetCommunityOptionType + list []*bgp.LargeCommunity + removeList []*regexp.Regexp +} + +func (a *LargeCommunityAction) Type() ActionType { + return ACTION_LARGE_COMMUNITY +} + +func (a *LargeCommunityAction) Apply(path *Path, _ *PolicyOptions) *Path { + switch a.action { + case config.BGP_SET_COMMUNITY_OPTION_TYPE_ADD: + path.SetLargeCommunities(a.list, false) + case config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE: + RegexpRemoveLargeCommunities(path, a.removeList) + case config.BGP_SET_COMMUNITY_OPTION_TYPE_REPLACE: + path.SetLargeCommunities(a.list, true) + } + return path +} + +func (a *LargeCommunityAction) ToConfig() *config.SetLargeCommunity { + cs := make([]string, 0, len(a.list)+len(a.removeList)) + for _, comm := range a.list { + cs = append(cs, comm.String()) + } + for _, exp := range a.removeList { + cs = append(cs, exp.String()) + } + return &config.SetLargeCommunity{ + SetLargeCommunityMethod: config.SetLargeCommunityMethod{CommunitiesList: cs}, + Options: config.BgpSetCommunityOptionType(a.action), + } +} + +func (a *LargeCommunityAction) String() string { + list := a.ToConfig().SetLargeCommunityMethod.CommunitiesList + l := _regexpCommunityReplaceString.ReplaceAllString(strings.Join(list, ", "), "") + return fmt.Sprintf("%s[%s]", a.action, l) +} + +func (a *LargeCommunityAction) MarshalJSON() ([]byte, error) { + return json.Marshal(a.ToConfig()) +} + +func NewLargeCommunityAction(c config.SetLargeCommunity) (*LargeCommunityAction, error) { + a, ok := CommunityOptionValueMap[strings.ToLower(string(c.Options))] + if !ok { + if len(c.SetLargeCommunityMethod.CommunitiesList) == 0 { + return nil, nil + } + return nil, fmt.Errorf("invalid option name: %s", c.Options) + } + var list []*bgp.LargeCommunity + var removeList []*regexp.Regexp + if a == config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE { + removeList = make([]*regexp.Regexp, 0, len(c.SetLargeCommunityMethod.CommunitiesList)) + } else { + list = make([]*bgp.LargeCommunity, 0, len(c.SetLargeCommunityMethod.CommunitiesList)) + } + for _, x := range c.SetLargeCommunityMethod.CommunitiesList { + if a == config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE { + exp, err := ParseLargeCommunityRegexp(x) + if err != nil { + return nil, err + } + removeList = append(removeList, exp) + } else { + comm, err := bgp.ParseLargeCommunity(x) + if err != nil { + return nil, err + } + list = append(list, comm) + } + } + return &LargeCommunityAction{ + action: a, + list: list, + removeList: removeList, + }, nil + +} + +type MedAction struct { + value int64 + action MedActionType +} + +func (a *MedAction) Type() ActionType { + return ACTION_MED +} + +func (a *MedAction) Apply(path *Path, _ *PolicyOptions) *Path { + var err error + switch a.action { + case MED_ACTION_MOD: + err = path.SetMed(a.value, false) + case MED_ACTION_REPLACE: + err = path.SetMed(a.value, true) + } + + if err != nil { + log.WithFields(log.Fields{ + "Topic": "Policy", + "Type": "Med Action", + "Error": err, + }).Warn("Could not set Med on path") + } + return path +} + +func (a *MedAction) ToConfig() config.BgpSetMedType { + if a.action == MED_ACTION_MOD && a.value > 0 { + return config.BgpSetMedType(fmt.Sprintf("+%d", a.value)) + } + return config.BgpSetMedType(fmt.Sprintf("%d", a.value)) +} + +func (a *MedAction) String() string { + return string(a.ToConfig()) +} + +func (a *MedAction) MarshalJSON() ([]byte, error) { + return json.Marshal(a.ToConfig()) +} + +var _regexpParseMedAction = regexp.MustCompile(`^(\+|\-)?(\d+)$`) + +func NewMedAction(c config.BgpSetMedType) (*MedAction, error) { + if string(c) == "" { + return nil, nil + } + + elems := _regexpParseMedAction.FindStringSubmatch(string(c)) + if len(elems) != 3 { + return nil, fmt.Errorf("invalid med action format") + } + action := MED_ACTION_REPLACE + switch elems[1] { + case "+", "-": + action = MED_ACTION_MOD + } + value, _ := strconv.ParseInt(string(c), 10, 64) + return &MedAction{ + value: value, + action: action, + }, nil +} + +func NewMedActionFromApiStruct(action MedActionType, value int64) *MedAction { + return &MedAction{action: action, value: value} +} + +type LocalPrefAction struct { + value uint32 +} + +func (a *LocalPrefAction) Type() ActionType { + return ACTION_LOCAL_PREF +} + +func (a *LocalPrefAction) Apply(path *Path, _ *PolicyOptions) *Path { + path.setPathAttr(bgp.NewPathAttributeLocalPref(a.value)) + return path +} + +func (a *LocalPrefAction) ToConfig() uint32 { + return a.value +} + +func (a *LocalPrefAction) String() string { + return fmt.Sprintf("%d", a.value) +} + +func (a *LocalPrefAction) MarshalJSON() ([]byte, error) { + return json.Marshal(a.ToConfig()) +} + +func NewLocalPrefAction(value uint32) (*LocalPrefAction, error) { + if value == 0 { + return nil, nil + } + return &LocalPrefAction{ + value: value, + }, nil +} + +type AsPathPrependAction struct { + asn uint32 + useLeftMost bool + repeat uint8 +} + +func (a *AsPathPrependAction) Type() ActionType { + return ACTION_AS_PATH_PREPEND +} + +func (a *AsPathPrependAction) Apply(path *Path, option *PolicyOptions) *Path { + var asn uint32 + if a.useLeftMost { + aspath := path.GetAsSeqList() + if len(aspath) == 0 { + log.WithFields(log.Fields{ + "Topic": "Policy", + "Type": "AsPathPrepend Action", + }).Warn("aspath length is zero.") + return path + } + asn = aspath[0] + if asn == 0 { + log.WithFields(log.Fields{ + "Topic": "Policy", + "Type": "AsPathPrepend Action", + }).Warn("left-most ASN is not seq") + return path + } + } else { + asn = a.asn + } + + confed := option != nil && option.Info.Confederation + path.PrependAsn(asn, a.repeat, confed) + + return path +} + +func (a *AsPathPrependAction) ToConfig() *config.SetAsPathPrepend { + return &config.SetAsPathPrepend{ + RepeatN: uint8(a.repeat), + As: func() string { + if a.useLeftMost { + return "last-as" + } + return fmt.Sprintf("%d", a.asn) + }(), + } +} + +func (a *AsPathPrependAction) String() string { + c := a.ToConfig() + return fmt.Sprintf("prepend %s %d times", c.As, c.RepeatN) +} + +func (a *AsPathPrependAction) MarshalJSON() ([]byte, error) { + return json.Marshal(a.ToConfig()) +} + +// NewAsPathPrependAction creates AsPathPrependAction object. +// If ASN cannot be parsed, nil will be returned. +func NewAsPathPrependAction(action config.SetAsPathPrepend) (*AsPathPrependAction, error) { + a := &AsPathPrependAction{ + repeat: action.RepeatN, + } + switch action.As { + case "": + if a.repeat == 0 { + return nil, nil + } + return nil, fmt.Errorf("specify as to prepend") + case "last-as": + a.useLeftMost = true + default: + asn, err := strconv.ParseUint(action.As, 10, 32) + if err != nil { + return nil, fmt.Errorf("As number string invalid") + } + a.asn = uint32(asn) + } + return a, nil +} + +type NexthopAction struct { + value net.IP + self bool +} + +func (a *NexthopAction) Type() ActionType { + return ACTION_NEXTHOP +} + +func (a *NexthopAction) Apply(path *Path, options *PolicyOptions) *Path { + if a.self { + if options != nil && options.Info != nil && options.Info.LocalAddress != nil { + path.SetNexthop(options.Info.LocalAddress) + } + return path + } + path.SetNexthop(a.value) + return path +} + +func (a *NexthopAction) ToConfig() config.BgpNextHopType { + if a.self { + return config.BgpNextHopType("self") + } + return config.BgpNextHopType(a.value.String()) +} + +func (a *NexthopAction) String() string { + return string(a.ToConfig()) +} + +func (a *NexthopAction) MarshalJSON() ([]byte, error) { + return json.Marshal(a.ToConfig()) +} + +func NewNexthopAction(c config.BgpNextHopType) (*NexthopAction, error) { + switch strings.ToLower(string(c)) { + case "": + return nil, nil + case "self": + return &NexthopAction{ + self: true, + }, nil + } + addr := net.ParseIP(string(c)) + if addr == nil { + return nil, fmt.Errorf("invalid ip address format: %s", string(c)) + } + return &NexthopAction{ + value: addr, + }, nil +} + +type Statement struct { + Name string + Conditions []Condition + RouteAction Action + ModActions []Action +} + +// evaluate each condition in the statement according to MatchSetOptions +func (s *Statement) Evaluate(p *Path, options *PolicyOptions) bool { + for _, c := range s.Conditions { + if !c.Evaluate(p, options) { + return false + } + } + return true +} + +func (s *Statement) Apply(path *Path, options *PolicyOptions) (RouteType, *Path) { + result := s.Evaluate(path, options) + if result { + if len(s.ModActions) != 0 { + // apply all modification actions + path = path.Clone(path.IsWithdraw) + for _, action := range s.ModActions { + path = action.Apply(path, options) + } + } + //Routing action + if s.RouteAction == nil || reflect.ValueOf(s.RouteAction).IsNil() { + return ROUTE_TYPE_NONE, path + } + p := s.RouteAction.Apply(path, options) + if p == nil { + return ROUTE_TYPE_REJECT, path + } + return ROUTE_TYPE_ACCEPT, path + } + return ROUTE_TYPE_NONE, path +} + +func (s *Statement) ToConfig() *config.Statement { + return &config.Statement{ + Name: s.Name, + Conditions: func() config.Conditions { + cond := config.Conditions{} + for _, c := range s.Conditions { + switch c.(type) { + case *PrefixCondition: + v := c.(*PrefixCondition) + cond.MatchPrefixSet = config.MatchPrefixSet{PrefixSet: v.set.Name(), MatchSetOptions: v.option.ConvertToMatchSetOptionsRestrictedType()} + case *NeighborCondition: + v := c.(*NeighborCondition) + cond.MatchNeighborSet = config.MatchNeighborSet{NeighborSet: v.set.Name(), MatchSetOptions: v.option.ConvertToMatchSetOptionsRestrictedType()} + case *AsPathLengthCondition: + v := c.(*AsPathLengthCondition) + cond.BgpConditions.AsPathLength = config.AsPathLength{Operator: config.IntToAttributeComparisonMap[int(v.operator)], Value: v.length} + case *AsPathCondition: + v := c.(*AsPathCondition) + cond.BgpConditions.MatchAsPathSet = config.MatchAsPathSet{AsPathSet: v.set.Name(), MatchSetOptions: config.IntToMatchSetOptionsTypeMap[int(v.option)]} + case *CommunityCondition: + v := c.(*CommunityCondition) + cond.BgpConditions.MatchCommunitySet = config.MatchCommunitySet{CommunitySet: v.set.Name(), MatchSetOptions: config.IntToMatchSetOptionsTypeMap[int(v.option)]} + case *ExtCommunityCondition: + v := c.(*ExtCommunityCondition) + cond.BgpConditions.MatchExtCommunitySet = config.MatchExtCommunitySet{ExtCommunitySet: v.set.Name(), MatchSetOptions: config.IntToMatchSetOptionsTypeMap[int(v.option)]} + case *LargeCommunityCondition: + v := c.(*LargeCommunityCondition) + cond.BgpConditions.MatchLargeCommunitySet = config.MatchLargeCommunitySet{LargeCommunitySet: v.set.Name(), MatchSetOptions: config.IntToMatchSetOptionsTypeMap[int(v.option)]} + case *NextHopCondition: + v := c.(*NextHopCondition) + cond.BgpConditions.NextHopInList = v.set.List() + case *RpkiValidationCondition: + v := c.(*RpkiValidationCondition) + cond.BgpConditions.RpkiValidationResult = v.result + case *RouteTypeCondition: + v := c.(*RouteTypeCondition) + cond.BgpConditions.RouteType = v.typ + case *AfiSafiInCondition: + v := c.(*AfiSafiInCondition) + res := make([]config.AfiSafiType, 0, len(v.routeFamilies)) + for _, rf := range v.routeFamilies { + res = append(res, config.AfiSafiType(rf.String())) + } + cond.BgpConditions.AfiSafiInList = res + } + } + return cond + }(), + Actions: func() config.Actions { + act := config.Actions{} + if s.RouteAction != nil && !reflect.ValueOf(s.RouteAction).IsNil() { + a := s.RouteAction.(*RoutingAction) + if a.AcceptRoute { + act.RouteDisposition = config.ROUTE_DISPOSITION_ACCEPT_ROUTE + } else { + act.RouteDisposition = config.ROUTE_DISPOSITION_REJECT_ROUTE + } + } else { + act.RouteDisposition = config.ROUTE_DISPOSITION_NONE + } + for _, a := range s.ModActions { + switch a.(type) { + case *AsPathPrependAction: + act.BgpActions.SetAsPathPrepend = *a.(*AsPathPrependAction).ToConfig() + case *CommunityAction: + act.BgpActions.SetCommunity = *a.(*CommunityAction).ToConfig() + case *ExtCommunityAction: + act.BgpActions.SetExtCommunity = *a.(*ExtCommunityAction).ToConfig() + case *LargeCommunityAction: + act.BgpActions.SetLargeCommunity = *a.(*LargeCommunityAction).ToConfig() + case *MedAction: + act.BgpActions.SetMed = a.(*MedAction).ToConfig() + case *LocalPrefAction: + act.BgpActions.SetLocalPref = a.(*LocalPrefAction).ToConfig() + case *NexthopAction: + act.BgpActions.SetNextHop = a.(*NexthopAction).ToConfig() + } + } + return act + }(), + } +} + +func (s *Statement) MarshalJSON() ([]byte, error) { + return json.Marshal(s.ToConfig()) +} + +type opType int + +const ( + ADD opType = iota + REMOVE + REPLACE +) + +func (lhs *Statement) mod(op opType, rhs *Statement) error { + cs := make([]Condition, len(lhs.Conditions)) + copy(cs, lhs.Conditions) + ra := lhs.RouteAction + as := make([]Action, len(lhs.ModActions)) + copy(as, lhs.ModActions) + for _, x := range rhs.Conditions { + var c Condition + i := 0 + for idx, y := range lhs.Conditions { + if x.Type() == y.Type() { + c = y + i = idx + break + } + } + switch op { + case ADD: + if c != nil { + return fmt.Errorf("condition %d is already set", x.Type()) + } + if cs == nil { + cs = make([]Condition, 0, len(rhs.Conditions)) + } + cs = append(cs, x) + case REMOVE: + if c == nil { + return fmt.Errorf("condition %d is not set", x.Type()) + } + cs = append(cs[:i], cs[i+1:]...) + if len(cs) == 0 { + cs = nil + } + case REPLACE: + if c == nil { + return fmt.Errorf("condition %d is not set", x.Type()) + } + cs[i] = x + } + } + if rhs.RouteAction != nil && !reflect.ValueOf(rhs.RouteAction).IsNil() { + switch op { + case ADD: + if lhs.RouteAction != nil && !reflect.ValueOf(lhs.RouteAction).IsNil() { + return fmt.Errorf("route action is already set") + } + ra = rhs.RouteAction + case REMOVE: + if lhs.RouteAction == nil || reflect.ValueOf(lhs.RouteAction).IsNil() { + return fmt.Errorf("route action is not set") + } + ra = nil + case REPLACE: + if lhs.RouteAction == nil || reflect.ValueOf(lhs.RouteAction).IsNil() { + return fmt.Errorf("route action is not set") + } + ra = rhs.RouteAction + } + } + for _, x := range rhs.ModActions { + var a Action + i := 0 + for idx, y := range lhs.ModActions { + if x.Type() == y.Type() { + a = y + i = idx + break + } + } + switch op { + case ADD: + if a != nil { + return fmt.Errorf("action %d is already set", x.Type()) + } + if as == nil { + as = make([]Action, 0, len(rhs.ModActions)) + } + as = append(as, x) + case REMOVE: + if a == nil { + return fmt.Errorf("action %d is not set", x.Type()) + } + as = append(as[:i], as[i+1:]...) + if len(as) == 0 { + as = nil + } + case REPLACE: + if a == nil { + return fmt.Errorf("action %d is not set", x.Type()) + } + as[i] = x + } + } + lhs.Conditions = cs + lhs.RouteAction = ra + lhs.ModActions = as + return nil +} + +func (lhs *Statement) Add(rhs *Statement) error { + return lhs.mod(ADD, rhs) +} + +func (lhs *Statement) Remove(rhs *Statement) error { + return lhs.mod(REMOVE, rhs) +} + +func (lhs *Statement) Replace(rhs *Statement) error { + return lhs.mod(REPLACE, rhs) +} + +func NewStatement(c config.Statement) (*Statement, error) { + if c.Name == "" { + return nil, fmt.Errorf("empty statement name") + } + var ra Action + var as []Action + var cs []Condition + var err error + cfs := []func() (Condition, error){ + func() (Condition, error) { + return NewPrefixCondition(c.Conditions.MatchPrefixSet) + }, + func() (Condition, error) { + return NewNeighborCondition(c.Conditions.MatchNeighborSet) + }, + func() (Condition, error) { + return NewAsPathLengthCondition(c.Conditions.BgpConditions.AsPathLength) + }, + func() (Condition, error) { + return NewRpkiValidationCondition(c.Conditions.BgpConditions.RpkiValidationResult) + }, + func() (Condition, error) { + return NewRouteTypeCondition(c.Conditions.BgpConditions.RouteType) + }, + func() (Condition, error) { + return NewAsPathCondition(c.Conditions.BgpConditions.MatchAsPathSet) + }, + func() (Condition, error) { + return NewCommunityCondition(c.Conditions.BgpConditions.MatchCommunitySet) + }, + func() (Condition, error) { + return NewExtCommunityCondition(c.Conditions.BgpConditions.MatchExtCommunitySet) + }, + func() (Condition, error) { + return NewLargeCommunityCondition(c.Conditions.BgpConditions.MatchLargeCommunitySet) + }, + func() (Condition, error) { + return NewNextHopCondition(c.Conditions.BgpConditions.NextHopInList) + }, + func() (Condition, error) { + return NewAfiSafiInCondition(c.Conditions.BgpConditions.AfiSafiInList) + }, + } + cs = make([]Condition, 0, len(cfs)) + for _, f := range cfs { + c, err := f() + if err != nil { + return nil, err + } + if !reflect.ValueOf(c).IsNil() { + cs = append(cs, c) + } + } + ra, err = NewRoutingAction(c.Actions.RouteDisposition) + if err != nil { + return nil, err + } + afs := []func() (Action, error){ + func() (Action, error) { + return NewCommunityAction(c.Actions.BgpActions.SetCommunity) + }, + func() (Action, error) { + return NewExtCommunityAction(c.Actions.BgpActions.SetExtCommunity) + }, + func() (Action, error) { + return NewLargeCommunityAction(c.Actions.BgpActions.SetLargeCommunity) + }, + func() (Action, error) { + return NewMedAction(c.Actions.BgpActions.SetMed) + }, + func() (Action, error) { + return NewLocalPrefAction(c.Actions.BgpActions.SetLocalPref) + }, + func() (Action, error) { + return NewAsPathPrependAction(c.Actions.BgpActions.SetAsPathPrepend) + }, + func() (Action, error) { + return NewNexthopAction(c.Actions.BgpActions.SetNextHop) + }, + } + as = make([]Action, 0, len(afs)) + for _, f := range afs { + a, err := f() + if err != nil { + return nil, err + } + if !reflect.ValueOf(a).IsNil() { + as = append(as, a) + } + } + return &Statement{ + Name: c.Name, + Conditions: cs, + RouteAction: ra, + ModActions: as, + }, nil +} + +type Policy struct { + Name string + Statements []*Statement +} + +// Compare path with a policy's condition in stored order in the policy. +// If a condition match, then this function stops evaluation and +// subsequent conditions are skipped. +func (p *Policy) Apply(path *Path, options *PolicyOptions) (RouteType, *Path) { + for _, stmt := range p.Statements { + var result RouteType + result, path = stmt.Apply(path, options) + if result != ROUTE_TYPE_NONE { + return result, path + } + } + return ROUTE_TYPE_NONE, path +} + +func (p *Policy) ToConfig() *config.PolicyDefinition { + ss := make([]config.Statement, 0, len(p.Statements)) + for _, s := range p.Statements { + ss = append(ss, *s.ToConfig()) + } + return &config.PolicyDefinition{ + Name: p.Name, + Statements: ss, + } +} + +func (p *Policy) FillUp(m map[string]*Statement) error { + stmts := make([]*Statement, 0, len(p.Statements)) + for _, x := range p.Statements { + y, ok := m[x.Name] + if !ok { + return fmt.Errorf("not found statement %s", x.Name) + } + stmts = append(stmts, y) + } + p.Statements = stmts + return nil +} + +func (lhs *Policy) Add(rhs *Policy) error { + lhs.Statements = append(lhs.Statements, rhs.Statements...) + return nil +} + +func (lhs *Policy) Remove(rhs *Policy) error { + stmts := make([]*Statement, 0, len(lhs.Statements)) + for _, x := range lhs.Statements { + found := false + for _, y := range rhs.Statements { + if x.Name == y.Name { + found = true + break + } + } + if !found { + stmts = append(stmts, x) + } + } + lhs.Statements = stmts + return nil +} + +func (lhs *Policy) Replace(rhs *Policy) error { + lhs.Statements = rhs.Statements + return nil +} + +func (p *Policy) MarshalJSON() ([]byte, error) { + return json.Marshal(p.ToConfig()) +} + +func NewPolicy(c config.PolicyDefinition) (*Policy, error) { + if c.Name == "" { + return nil, fmt.Errorf("empty policy name") + } + var st []*Statement + stmts := c.Statements + if len(stmts) != 0 { + st = make([]*Statement, 0, len(stmts)) + for idx, stmt := range stmts { + if stmt.Name == "" { + stmt.Name = fmt.Sprintf("%s_stmt%d", c.Name, idx) + } + s, err := NewStatement(stmt) + if err != nil { + return nil, err + } + st = append(st, s) + } + } + return &Policy{ + Name: c.Name, + Statements: st, + }, nil +} + +type Policies []*Policy + +func (p Policies) Len() int { + return len(p) +} + +func (p Policies) Swap(i, j int) { + p[i], p[j] = p[j], p[i] +} + +func (p Policies) Less(i, j int) bool { + return p[i].Name < p[j].Name +} + +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 + mu sync.RWMutex +} + +func (r *RoutingPolicy) ApplyPolicy(id string, dir PolicyDirection, before *Path, options *PolicyOptions) *Path { + r.mu.RLock() + defer r.mu.RUnlock() + + if before == nil { + return nil + } + + if before.IsWithdraw { + return before + } + result := ROUTE_TYPE_NONE + after := before + for _, p := range r.getPolicy(id, dir) { + result, after = p.Apply(after, options) + 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) { + var names []string + var cdef config.DefaultPolicyType + def := ROUTE_TYPE_ACCEPT + c := a.Config + switch dir { + case POLICY_DIRECTION_IN: + names = c.InPolicyList + cdef = c.DefaultInPolicy + case POLICY_DIRECTION_IMPORT: + names = c.ImportPolicyList + cdef = c.DefaultImportPolicy + case POLICY_DIRECTION_EXPORT: + names = c.ExportPolicyList + cdef = c.DefaultExportPolicy + default: + return nil, def, fmt.Errorf("invalid policy direction") + } + if cdef == config.DEFAULT_POLICY_TYPE_REJECT_ROUTE { + def = ROUTE_TYPE_REJECT + } + ps := make([]*Policy, 0, len(names)) + seen := make(map[string]bool) + for _, name := range names { + p, ok := r.policyMap[name] + if !ok { + return nil, def, fmt.Errorf("not found policy %s", name) + } + if seen[name] { + return nil, def, fmt.Errorf("duplicated policy %s", name) + } + seen[name] = true + ps = append(ps, p) + } + return ps, def, nil +} + +func (r *RoutingPolicy) validateCondition(v Condition) (err error) { + switch v.Type() { + case CONDITION_PREFIX: + m := r.definedSetMap[DEFINED_TYPE_PREFIX] + if i, ok := m[v.Name()]; !ok { + return fmt.Errorf("not found prefix set %s", v.Name()) + } else { + c := v.(*PrefixCondition) + c.set = i.(*PrefixSet) + } + case CONDITION_NEIGHBOR: + m := r.definedSetMap[DEFINED_TYPE_NEIGHBOR] + if i, ok := m[v.Name()]; !ok { + return fmt.Errorf("not found neighbor set %s", v.Name()) + } else { + c := v.(*NeighborCondition) + c.set = i.(*NeighborSet) + } + case CONDITION_AS_PATH: + m := r.definedSetMap[DEFINED_TYPE_AS_PATH] + if i, ok := m[v.Name()]; !ok { + return fmt.Errorf("not found as path set %s", v.Name()) + } else { + c := v.(*AsPathCondition) + c.set = i.(*AsPathSet) + } + case CONDITION_COMMUNITY: + m := r.definedSetMap[DEFINED_TYPE_COMMUNITY] + if i, ok := m[v.Name()]; !ok { + return fmt.Errorf("not found community set %s", v.Name()) + } else { + c := v.(*CommunityCondition) + c.set = i.(*CommunitySet) + } + case CONDITION_EXT_COMMUNITY: + m := r.definedSetMap[DEFINED_TYPE_EXT_COMMUNITY] + if i, ok := m[v.Name()]; !ok { + return fmt.Errorf("not found ext-community set %s", v.Name()) + } else { + c := v.(*ExtCommunityCondition) + c.set = i.(*ExtCommunitySet) + } + case CONDITION_LARGE_COMMUNITY: + m := r.definedSetMap[DEFINED_TYPE_LARGE_COMMUNITY] + if i, ok := m[v.Name()]; !ok { + return fmt.Errorf("not found large-community set %s", v.Name()) + } else { + c := v.(*LargeCommunityCondition) + c.set = i.(*LargeCommunitySet) + } + case CONDITION_NEXT_HOP: + case CONDITION_AFI_SAFI_IN: + case CONDITION_AS_PATH_LENGTH: + case CONDITION_RPKI: + } + return nil +} + +func (r *RoutingPolicy) inUse(d DefinedSet) bool { + name := d.Name() + for _, p := range r.policyMap { + for _, s := range p.Statements { + for _, c := range s.Conditions { + if c.Set() != nil && c.Set().Name() == name { + return true + } + } + } + } + return false +} + +func (r *RoutingPolicy) statementInUse(x *Statement) bool { + for _, p := range r.policyMap { + for _, y := range p.Statements { + if x.Name == y.Name { + return true + } + } + } + return false +} + +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 { + y, err := NewPrefixSet(x) + if err != nil { + return err + } + if y == nil { + return fmt.Errorf("empty prefix set") + } + dmap[DEFINED_TYPE_PREFIX][y.Name()] = y + } + dmap[DEFINED_TYPE_NEIGHBOR] = make(map[string]DefinedSet) + for _, x := range d.NeighborSets { + y, err := NewNeighborSet(x) + if err != nil { + return err + } + if y == nil { + return fmt.Errorf("empty neighbor set") + } + dmap[DEFINED_TYPE_NEIGHBOR][y.Name()] = y + } + // dmap[DEFINED_TYPE_TAG] = make(map[string]DefinedSet) + // for _, x := range c.DefinedSets.TagSets{ + // y, err := NewTagSet(x) + // if err != nil { + // return nil, err + // } + // dmap[DEFINED_TYPE_TAG][y.Name()] = y + // } + bd := c.DefinedSets.BgpDefinedSets + dmap[DEFINED_TYPE_AS_PATH] = make(map[string]DefinedSet) + for _, x := range bd.AsPathSets { + y, err := NewAsPathSet(x) + if err != nil { + return err + } + if y == nil { + return fmt.Errorf("empty as path set") + } + dmap[DEFINED_TYPE_AS_PATH][y.Name()] = y + } + dmap[DEFINED_TYPE_COMMUNITY] = make(map[string]DefinedSet) + for _, x := range bd.CommunitySets { + y, err := NewCommunitySet(x) + if err != nil { + return err + } + if y == nil { + return fmt.Errorf("empty community set") + } + dmap[DEFINED_TYPE_COMMUNITY][y.Name()] = y + } + dmap[DEFINED_TYPE_EXT_COMMUNITY] = make(map[string]DefinedSet) + for _, x := range bd.ExtCommunitySets { + y, err := NewExtCommunitySet(x) + if err != nil { + return err + } + if y == nil { + return fmt.Errorf("empty ext-community set") + } + dmap[DEFINED_TYPE_EXT_COMMUNITY][y.Name()] = y + } + dmap[DEFINED_TYPE_LARGE_COMMUNITY] = make(map[string]DefinedSet) + for _, x := range bd.LargeCommunitySets { + y, err := NewLargeCommunitySet(x) + if err != nil { + return err + } + if y == nil { + return fmt.Errorf("empty large-community set") + } + dmap[DEFINED_TYPE_LARGE_COMMUNITY][y.Name()] = y + } + + pmap := make(map[string]*Policy) + smap := make(map[string]*Statement) + for _, x := range c.PolicyDefinitions { + y, err := NewPolicy(x) + if err != nil { + return err + } + if _, ok := pmap[y.Name]; ok { + return fmt.Errorf("duplicated policy name. policy name must be unique.") + } + pmap[y.Name] = y + for _, s := range y.Statements { + _, ok := smap[s.Name] + if ok { + return fmt.Errorf("duplicated statement name. statement name must be unique.") + } + smap[s.Name] = s + } + } + + // hacky + oldMap := r.definedSetMap + r.definedSetMap = dmap + for _, y := range pmap { + for _, s := range y.Statements { + for _, c := range s.Conditions { + if err := r.validateCondition(c); err != nil { + r.definedSetMap = oldMap + return err + } + } + } + } + + r.definedSetMap = dmap + r.policyMap = pmap + r.statementMap = smap + r.assignmentMap = make(map[string]*Assignment) + // allow all routes coming in and going out by default + r.setDefaultPolicy(GLOBAL_RIB_NAME, POLICY_DIRECTION_IMPORT, ROUTE_TYPE_ACCEPT) + r.setDefaultPolicy(GLOBAL_RIB_NAME, POLICY_DIRECTION_EXPORT, ROUTE_TYPE_ACCEPT) + return nil +} + +func (r *RoutingPolicy) GetDefinedSet(typ DefinedType, name string) (*config.DefinedSets, error) { + r.mu.RLock() + + set, ok := r.definedSetMap[typ] + if !ok { + return nil, fmt.Errorf("invalid defined-set type: %d", typ) + } + + var dl DefinedSetList + for _, s := range set { + dl = append(dl, s) + } + r.mu.RUnlock() + + sort.Sort(dl) + + sets := &config.DefinedSets{ + PrefixSets: make([]config.PrefixSet, 0), + NeighborSets: make([]config.NeighborSet, 0), + BgpDefinedSets: config.BgpDefinedSets{ + CommunitySets: make([]config.CommunitySet, 0), + ExtCommunitySets: make([]config.ExtCommunitySet, 0), + LargeCommunitySets: make([]config.LargeCommunitySet, 0), + AsPathSets: make([]config.AsPathSet, 0), + }, + } + for _, s := range dl { + if name != "" && s.Name() != name { + continue + } + switch s.(type) { + case *PrefixSet: + sets.PrefixSets = append(sets.PrefixSets, *s.(*PrefixSet).ToConfig()) + case *NeighborSet: + sets.NeighborSets = append(sets.NeighborSets, *s.(*NeighborSet).ToConfig()) + case *CommunitySet: + sets.BgpDefinedSets.CommunitySets = append(sets.BgpDefinedSets.CommunitySets, *s.(*CommunitySet).ToConfig()) + case *ExtCommunitySet: + sets.BgpDefinedSets.ExtCommunitySets = append(sets.BgpDefinedSets.ExtCommunitySets, *s.(*ExtCommunitySet).ToConfig()) + case *LargeCommunitySet: + sets.BgpDefinedSets.LargeCommunitySets = append(sets.BgpDefinedSets.LargeCommunitySets, *s.(*LargeCommunitySet).ToConfig()) + case *AsPathSet: + sets.BgpDefinedSets.AsPathSets = append(sets.BgpDefinedSets.AsPathSets, *s.(*AsPathSet).ToConfig()) + } + } + return sets, nil +} + +func (r *RoutingPolicy) AddDefinedSet(s DefinedSet) error { + r.mu.Lock() + defer r.mu.Unlock() + + if m, ok := r.definedSetMap[s.Type()]; !ok { + return fmt.Errorf("invalid defined-set type: %d", s.Type()) + } else { + if d, ok := m[s.Name()]; ok { + if err := d.Append(s); err != nil { + return err + } + } else { + m[s.Name()] = s + } + } + return nil +} + +func (r *RoutingPolicy) DeleteDefinedSet(a DefinedSet, all bool) (err error) { + r.mu.Lock() + defer r.mu.Unlock() + + if m, ok := r.definedSetMap[a.Type()]; !ok { + err = fmt.Errorf("invalid defined-set type: %d", a.Type()) + } else { + d, ok := m[a.Name()] + if !ok { + return fmt.Errorf("not found defined-set: %s", a.Name()) + } + if all { + if r.inUse(d) { + err = fmt.Errorf("can't delete. defined-set %s is in use", a.Name()) + } else { + delete(m, a.Name()) + } + } else { + err = d.Remove(a) + } + } + return err +} + +func (r *RoutingPolicy) ReplaceDefinedSet(a DefinedSet) (err error) { + r.mu.Lock() + defer r.mu.Unlock() + + if m, ok := r.definedSetMap[a.Type()]; !ok { + err = fmt.Errorf("invalid defined-set type: %d", a.Type()) + } else { + if d, ok := m[a.Name()]; !ok { + err = fmt.Errorf("not found defined-set: %s", a.Name()) + } else { + err = d.Replace(a) + } + } + return err +} + +func (r *RoutingPolicy) GetStatement() []*config.Statement { + r.mu.RLock() + defer r.mu.RUnlock() + + l := make([]*config.Statement, 0, len(r.statementMap)) + for _, st := range r.statementMap { + l = append(l, st.ToConfig()) + } + return l +} + +func (r *RoutingPolicy) AddStatement(st *Statement) (err error) { + r.mu.Lock() + defer r.mu.Unlock() + + for _, c := range st.Conditions { + if err = r.validateCondition(c); err != nil { + return + } + } + m := r.statementMap + name := st.Name + if d, ok := m[name]; ok { + err = d.Add(st) + } else { + m[name] = st + } + + return err +} + +func (r *RoutingPolicy) DeleteStatement(st *Statement, all bool) (err error) { + r.mu.Lock() + defer r.mu.Unlock() + + m := r.statementMap + name := st.Name + if d, ok := m[name]; ok { + if all { + if r.statementInUse(d) { + err = fmt.Errorf("can't delete. statement %s is in use", name) + } else { + delete(m, name) + } + } else { + err = d.Remove(st) + } + } else { + err = fmt.Errorf("not found statement: %s", name) + } + return err +} + +func (r *RoutingPolicy) ReplaceStatement(st *Statement) (err error) { + r.mu.Lock() + defer r.mu.Unlock() + + for _, c := range st.Conditions { + if err = r.validateCondition(c); err != nil { + return + } + } + m := r.statementMap + name := st.Name + if d, ok := m[name]; ok { + err = d.Replace(st) + } else { + err = fmt.Errorf("not found statement: %s", name) + } + return err +} + +func (r *RoutingPolicy) GetAllPolicy() []*config.PolicyDefinition { + r.mu.RLock() + + var ps Policies + for _, p := range r.policyMap { + ps = append(ps, p) + } + r.mu.RUnlock() + + sort.Sort(ps) + + l := make([]*config.PolicyDefinition, 0, len(ps)) + for _, p := range ps { + l = append(l, p.ToConfig()) + } + return l +} + +func (r *RoutingPolicy) AddPolicy(x *Policy, refer bool) (err error) { + r.mu.Lock() + defer r.mu.Unlock() + + for _, st := range x.Statements { + for _, c := range st.Conditions { + if err = r.validateCondition(c); err != nil { + return + } + } + } + + pMap := r.policyMap + sMap := r.statementMap + name := x.Name + y, ok := pMap[name] + if refer { + err = x.FillUp(sMap) + } else { + for _, st := range x.Statements { + if _, ok := sMap[st.Name]; ok { + err = fmt.Errorf("statement %s already defined", st.Name) + return + } + sMap[st.Name] = st + } + } + if ok { + err = y.Add(x) + } else { + pMap[name] = x + } + + return err +} + +func (r *RoutingPolicy) DeletePolicy(x *Policy, all, preserve bool, activeId []string) (err error) { + r.mu.Lock() + defer r.mu.Unlock() + + pMap := r.policyMap + sMap := r.statementMap + name := x.Name + y, ok := pMap[name] + if !ok { + err = fmt.Errorf("not found policy: %s", name) + return + } + inUse := func(ids []string) bool { + for _, id := range ids { + for _, dir := range []PolicyDirection{POLICY_DIRECTION_IN, POLICY_DIRECTION_EXPORT, POLICY_DIRECTION_EXPORT} { + for _, y := range r.getPolicy(id, dir) { + if x.Name == y.Name { + return true + } + } + } + } + return false + } + + if all { + if inUse(activeId) { + err = fmt.Errorf("can't delete. policy %s is in use", name) + return + } + log.WithFields(log.Fields{ + "Topic": "Policy", + "Key": name, + }).Debug("delete policy") + delete(pMap, name) + } else { + err = y.Remove(x) + } + if err == nil && !preserve { + for _, st := range y.Statements { + if !r.statementInUse(st) { + log.WithFields(log.Fields{ + "Topic": "Policy", + "Key": st.Name, + }).Debug("delete unused statement") + delete(sMap, st.Name) + } + } + } + return err +} + +func (r *RoutingPolicy) ReplacePolicy(x *Policy, refer, preserve bool) (err error) { + r.mu.Lock() + defer r.mu.Unlock() + + for _, st := range x.Statements { + for _, c := range st.Conditions { + if err = r.validateCondition(c); err != nil { + return + } + } + } + + pMap := r.policyMap + sMap := r.statementMap + name := x.Name + y, ok := pMap[name] + if !ok { + err = fmt.Errorf("not found policy: %s", name) + return + } + if refer { + if err = x.FillUp(sMap); err != nil { + return + } + } else { + for _, st := range x.Statements { + if _, ok := sMap[st.Name]; ok { + err = fmt.Errorf("statement %s already defined", st.Name) + return + } + sMap[st.Name] = st + } + } + + ys := y.Statements + err = y.Replace(x) + if err == nil && !preserve { + for _, st := range ys { + if !r.statementInUse(st) { + log.WithFields(log.Fields{ + "Topic": "Policy", + "Key": st.Name, + }).Debug("delete unused statement") + delete(sMap, st.Name) + } + } + } + return err +} + +func (r *RoutingPolicy) GetPolicyAssignment(id string, dir PolicyDirection) (RouteType, []*config.PolicyDefinition, error) { + r.mu.RLock() + defer r.mu.RUnlock() + + rt := r.getDefaultPolicy(id, dir) + + ps := r.getPolicy(id, dir) + l := make([]*config.PolicyDefinition, 0, len(ps)) + for _, p := range ps { + l = append(l, p.ToConfig()) + } + return rt, l, nil +} + +func (r *RoutingPolicy) AddPolicyAssignment(id string, dir PolicyDirection, policies []*config.PolicyDefinition, def RouteType) (err error) { + r.mu.Lock() + defer r.mu.Unlock() + + ps := make([]*Policy, 0, len(policies)) + seen := make(map[string]bool) + for _, x := range policies { + p, ok := r.policyMap[x.Name] + if !ok { + err = fmt.Errorf("not found policy %s", x.Name) + return + } + if seen[x.Name] { + err = fmt.Errorf("duplicated policy %s", x.Name) + return + } + seen[x.Name] = true + ps = append(ps, p) + } + cur := r.getPolicy(id, dir) + if cur == nil { + err = r.setPolicy(id, dir, ps) + } else { + seen = make(map[string]bool) + ps = append(cur, ps...) + for _, x := range ps { + if seen[x.Name] { + err = fmt.Errorf("duplicated policy %s", x.Name) + return + } + seen[x.Name] = true + } + err = r.setPolicy(id, dir, ps) + } + if err == nil && def != ROUTE_TYPE_NONE { + err = r.setDefaultPolicy(id, dir, def) + } + return err +} + +func (r *RoutingPolicy) DeletePolicyAssignment(id string, dir PolicyDirection, policies []*config.PolicyDefinition, all bool) (err error) { + r.mu.Lock() + defer r.mu.Unlock() + + ps := make([]*Policy, 0, len(policies)) + seen := make(map[string]bool) + for _, x := range policies { + p, ok := r.policyMap[x.Name] + if !ok { + err = fmt.Errorf("not found policy %s", x.Name) + return + } + if seen[x.Name] { + err = fmt.Errorf("duplicated policy %s", x.Name) + return + } + seen[x.Name] = true + ps = append(ps, p) + } + cur := r.getPolicy(id, dir) + + if all { + err = r.setPolicy(id, dir, nil) + if err != nil { + return + } + err = r.setDefaultPolicy(id, dir, ROUTE_TYPE_NONE) + } else { + l := len(cur) - len(ps) + if l < 0 { + // try to remove more than the assigned policies... + l = len(cur) + } + n := make([]*Policy, 0, l) + for _, y := range cur { + found := false + for _, x := range ps { + if x.Name == y.Name { + found = true + break + } + } + if !found { + n = append(n, y) + } + } + err = r.setPolicy(id, dir, n) + } + return err +} + +func (r *RoutingPolicy) ReplacePolicyAssignment(id string, dir PolicyDirection, policies []*config.PolicyDefinition, def RouteType) (err error) { + r.mu.Lock() + defer r.mu.Unlock() + + ps := make([]*Policy, 0, len(policies)) + seen := make(map[string]bool) + for _, x := range policies { + p, ok := r.policyMap[x.Name] + if !ok { + err = fmt.Errorf("not found policy %s", x.Name) + return + } + if seen[x.Name] { + err = fmt.Errorf("duplicated policy %s", x.Name) + return + } + seen[x.Name] = true + ps = append(ps, p) + } + r.getPolicy(id, dir) + err = r.setPolicy(id, dir, ps) + if err == nil && def != ROUTE_TYPE_NONE { + err = r.setDefaultPolicy(id, dir, def) + } + return err +} + +func (r *RoutingPolicy) Reset(rp *config.RoutingPolicy, ap map[string]config.ApplyPolicy) error { + r.mu.Lock() + defer r.mu.Unlock() + + if rp != nil { + if err := r.reload(*rp); err != nil { + log.WithFields(log.Fields{ + "Topic": "Policy", + }).Errorf("failed to create routing policy: %s", err) + return err + } + } + + for id, c := range ap { + for _, dir := range []PolicyDirection{POLICY_DIRECTION_IN, POLICY_DIRECTION_IMPORT, POLICY_DIRECTION_EXPORT} { + ps, def, err := r.getAssignmentFromConfig(dir, c) + if err != nil { + log.WithFields(log.Fields{ + "Topic": "Policy", + "Dir": dir, + }).Errorf("failed to get policy info: %s", err) + continue + } + r.setDefaultPolicy(id, dir, def) + r.setPolicy(id, dir, ps) + } + } + return nil +} + +func NewRoutingPolicy() *RoutingPolicy { + return &RoutingPolicy{ + 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 { + f := func(arg []bgp.ExtendedCommunityInterface) []string { + ret := make([]string, 0, len(arg)) + for _, a := range arg { + ret = append(ret, fmt.Sprintf("RT:%s", a.String())) + } + return ret + } + set, _ := NewExtCommunitySet(config.ExtCommunitySet{ + ExtCommunitySetName: v.Name, + ExtCommunityList: f(v.ImportRt), + }) + matchSet := config.MatchExtCommunitySet{ + ExtCommunitySet: v.Name, + MatchSetOptions: config.MATCH_SET_OPTIONS_TYPE_ANY, + } + c, _ := NewExtCommunityCondition(matchSet) + c.set = set + return c.Evaluate(path, nil) +} + +type PolicyAssignment struct { + Name string + Type PolicyDirection + Policies []*Policy + Default RouteType +} diff --git a/internal/pkg/table/policy_test.go b/internal/pkg/table/policy_test.go new file mode 100644 index 00000000..7f1a1dd9 --- /dev/null +++ b/internal/pkg/table/policy_test.go @@ -0,0 +1,3140 @@ +// Copyright (C) 2014,2015 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + "fmt" + "math" + "net" + "strconv" + "strings" + "testing" + "time" + + "github.com/osrg/gobgp/internal/pkg/config" + "github.com/osrg/gobgp/pkg/packet/bgp" + + log "github.com/sirupsen/logrus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestPrefixCalcurateNoRange(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.0")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // test + pl1, _ := NewPrefix(config.Prefix{IpPrefix: "10.10.0.0/24", MasklengthRange: ""}) + match1 := pl1.Match(path) + assert.Equal(t, true, match1) + pl2, _ := NewPrefix(config.Prefix{IpPrefix: "10.10.0.0/23", MasklengthRange: ""}) + match2 := pl2.Match(path) + assert.Equal(t, false, match2) + pl3, _ := NewPrefix(config.Prefix{IpPrefix: "10.10.0.0/16", MasklengthRange: "21..24"}) + match3 := pl3.Match(path) + assert.Equal(t, true, match3) +} + +func TestPrefixCalcurateAddress(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // test + pl1, _ := NewPrefix(config.Prefix{IpPrefix: "10.11.0.0/16", MasklengthRange: "21..24"}) + match1 := pl1.Match(path) + assert.Equal(t, false, match1) + pl2, _ := NewPrefix(config.Prefix{IpPrefix: "10.10.0.0/16", MasklengthRange: "21..24"}) + match2 := pl2.Match(path) + assert.Equal(t, true, match2) +} + +func TestPrefixCalcurateLength(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // test + pl1, _ := NewPrefix(config.Prefix{IpPrefix: "10.10.64.0/24", MasklengthRange: "21..24"}) + match1 := pl1.Match(path) + assert.Equal(t, false, match1) + pl2, _ := NewPrefix(config.Prefix{IpPrefix: "10.10.64.0/16", MasklengthRange: "21..24"}) + match2 := pl2.Match(path) + assert.Equal(t, true, match2) +} + +func TestPrefixCalcurateLengthRange(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // test + pl1, _ := NewPrefix(config.Prefix{IpPrefix: "10.10.0.0/16", MasklengthRange: "21..23"}) + match1 := pl1.Match(path) + assert.Equal(t, false, match1) + pl2, _ := NewPrefix(config.Prefix{IpPrefix: "10.10.0.0/16", MasklengthRange: "25..26"}) + match2 := pl2.Match(path) + assert.Equal(t, false, match2) + pl3, _ := NewPrefix(config.Prefix{IpPrefix: "10.10.0.0/16", MasklengthRange: "21..24"}) + match3 := pl3.Match(path) + assert.Equal(t, true, match3) +} + +func TestPrefixCalcurateNoRangeIPv6(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("2001::192:168:50:1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + mpnlri := []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")} + mpreach := bgp.NewPathAttributeMpReachNLRI("2001::192:168:50:1", mpnlri) + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{mpreach, origin, aspath, med} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nil) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // test + pl1, _ := NewPrefix(config.Prefix{IpPrefix: "2001:123:123::/48", MasklengthRange: ""}) + match1 := pl1.Match(path) + assert.Equal(t, false, match1) + pl2, _ := NewPrefix(config.Prefix{IpPrefix: "2001:123:123:1::/64", MasklengthRange: ""}) + match2 := pl2.Match(path) + assert.Equal(t, true, match2) + pl3, _ := NewPrefix(config.Prefix{IpPrefix: "2001:123:123::/48", MasklengthRange: "64..80"}) + match3 := pl3.Match(path) + assert.Equal(t, true, match3) +} + +func TestPrefixCalcurateAddressIPv6(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("2001::192:168:50:1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + mpnlri := []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")} + mpreach := bgp.NewPathAttributeMpReachNLRI("2001::192:168:50:1", mpnlri) + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{mpreach, origin, aspath, med} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nil) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // test + pl1, _ := NewPrefix(config.Prefix{IpPrefix: "2001:123:128::/48", MasklengthRange: "64..80"}) + match1 := pl1.Match(path) + assert.Equal(t, false, match1) + pl2, _ := NewPrefix(config.Prefix{IpPrefix: "2001:123:123::/48", MasklengthRange: "64..80"}) + match2 := pl2.Match(path) + assert.Equal(t, true, match2) +} + +func TestPrefixCalcurateLengthIPv6(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("2001::192:168:50:1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + mpnlri := []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")} + mpreach := bgp.NewPathAttributeMpReachNLRI("2001::192:168:50:1", mpnlri) + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{mpreach, origin, aspath, med} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nil) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // test + pl1, _ := NewPrefix(config.Prefix{IpPrefix: "2001:123:123:64::/64", MasklengthRange: "64..80"}) + match1 := pl1.Match(path) + assert.Equal(t, false, match1) + pl2, _ := NewPrefix(config.Prefix{IpPrefix: "2001:123:123:64::/48", MasklengthRange: "64..80"}) + match2 := pl2.Match(path) + assert.Equal(t, true, match2) +} + +func TestPrefixCalcurateLengthRangeIPv6(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("2001::192:168:50:1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + mpnlri := []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")} + mpreach := bgp.NewPathAttributeMpReachNLRI("2001::192:168:50:1", mpnlri) + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{mpreach, origin, aspath, med} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nil) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // test + pl1, _ := NewPrefix(config.Prefix{IpPrefix: "2001:123:123::/48", MasklengthRange: "62..63"}) + match1 := pl1.Match(path) + assert.Equal(t, false, match1) + pl2, _ := NewPrefix(config.Prefix{IpPrefix: "2001:123:123::/48", MasklengthRange: "65..66"}) + match2 := pl2.Match(path) + assert.Equal(t, false, match2) + pl3, _ := NewPrefix(config.Prefix{IpPrefix: "2001:123:123::/48", MasklengthRange: "63..65"}) + match3 := pl3.Match(path) + assert.Equal(t, true, match3) +} + +func TestPolicyNotMatch(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + ps := createPrefixSet("ps1", "10.3.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + s := createStatement("statement1", "ps1", "ns1", false) + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + pType, newPath := r.policyMap["pd1"].Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_NONE, pType) + assert.Equal(t, newPath, path) +} + +func TestPolicyMatchAndReject(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + s := createStatement("statement1", "ps1", "ns1", false) + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + pType, newPath := r.policyMap["pd1"].Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_REJECT, pType) + assert.Equal(t, newPath, path) +} + +func TestPolicyMatchAndAccept(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + s := createStatement("statement1", "ps1", "ns1", true) + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + pType, newPath := r.policyMap["pd1"].Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.Equal(t, path, newPath) +} + +func TestPolicyRejectOnlyPrefixSet(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.1.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.1.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.1.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path1 := ProcessMessage(updateMsg, peer, time.Now())[0] + + peer = &PeerInfo{AS: 65002, Address: net.ParseIP("10.0.2.2")} + origin = bgp.NewPathAttributeOrigin(0) + aspathParam = []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65002})} + aspath = bgp.NewPathAttributeAsPath(aspathParam) + nexthop = bgp.NewPathAttributeNextHop("10.0.2.2") + med = bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes = []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri = []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.9.2.102")} + updateMsg = bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path2 := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + ps := createPrefixSet("ps1", "10.10.1.0/16", "21..24") + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + + s := createStatement("statement1", "ps1", "", false) + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + pType, newPath := p.Apply(path1, nil) + assert.Equal(t, ROUTE_TYPE_REJECT, pType) + assert.Equal(t, newPath, path1) + + pType2, newPath2 := p.Apply(path2, nil) + assert.Equal(t, ROUTE_TYPE_NONE, pType2) + assert.Equal(t, newPath2, path2) +} + +func TestPolicyRejectOnlyNeighborSet(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.1.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.1.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.1.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path1 := ProcessMessage(updateMsg, peer, time.Now())[0] + + peer = &PeerInfo{AS: 65002, Address: net.ParseIP("10.0.2.2")} + origin = bgp.NewPathAttributeOrigin(0) + aspathParam = []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65002})} + aspath = bgp.NewPathAttributeAsPath(aspathParam) + nexthop = bgp.NewPathAttributeNextHop("10.0.2.2") + med = bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes = []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri = []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.2.102")} + updateMsg = bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path2 := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + ns := createNeighborSet("ns1", "10.0.1.1") + ds := config.DefinedSets{} + ds.NeighborSets = []config.NeighborSet{ns} + + s := createStatement("statement1", "", "ns1", false) + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + pType, newPath := r.policyMap["pd1"].Apply(path1, nil) + assert.Equal(t, ROUTE_TYPE_REJECT, pType) + assert.Equal(t, newPath, path1) + + pType2, newPath2 := r.policyMap["pd1"].Apply(path2, nil) + assert.Equal(t, ROUTE_TYPE_NONE, pType2) + assert.Equal(t, newPath2, path2) +} + +func TestPolicyDifferentRoutefamilyOfPathAndPolicy(t *testing.T) { + // create path ipv4 + peerIPv4 := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + originIPv4 := bgp.NewPathAttributeOrigin(0) + aspathParamIPv4 := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspathIPv4 := bgp.NewPathAttributeAsPath(aspathParamIPv4) + nexthopIPv4 := bgp.NewPathAttributeNextHop("10.0.0.1") + medIPv4 := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributesIPv4 := []bgp.PathAttributeInterface{originIPv4, aspathIPv4, nexthopIPv4, medIPv4} + nlriIPv4 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsgIPv4 := bgp.NewBGPUpdateMessage(nil, pathAttributesIPv4, nlriIPv4) + pathIPv4 := ProcessMessage(updateMsgIPv4, peerIPv4, time.Now())[0] + // create path ipv6 + peerIPv6 := &PeerInfo{AS: 65001, Address: net.ParseIP("2001::192:168:50:1")} + originIPv6 := bgp.NewPathAttributeOrigin(0) + aspathParamIPv6 := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspathIPv6 := bgp.NewPathAttributeAsPath(aspathParamIPv6) + mpnlriIPv6 := []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")} + mpreachIPv6 := bgp.NewPathAttributeMpReachNLRI("2001::192:168:50:1", mpnlriIPv6) + medIPv6 := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributesIPv6 := []bgp.PathAttributeInterface{mpreachIPv6, originIPv6, aspathIPv6, medIPv6} + updateMsgIPv6 := bgp.NewBGPUpdateMessage(nil, pathAttributesIPv6, nil) + pathIPv6 := ProcessMessage(updateMsgIPv6, peerIPv6, time.Now())[0] + // create policy + psIPv4 := createPrefixSet("psIPv4", "10.10.0.0/16", "21..24") + nsIPv4 := createNeighborSet("nsIPv4", "10.0.0.1") + + psIPv6 := createPrefixSet("psIPv6", "2001:123:123::/48", "64..80") + nsIPv6 := createNeighborSet("nsIPv6", "2001::192:168:50:1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{psIPv4, psIPv6} + ds.NeighborSets = []config.NeighborSet{nsIPv4, nsIPv6} + + stIPv4 := createStatement("statement1", "psIPv4", "nsIPv4", false) + stIPv6 := createStatement("statement2", "psIPv6", "nsIPv6", false) + + pd := createPolicyDefinition("pd1", stIPv4, stIPv6) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + pType1, newPath1 := p.Apply(pathIPv4, nil) + assert.Equal(t, ROUTE_TYPE_REJECT, pType1) + assert.Equal(t, newPath1, pathIPv4) + + pType2, newPath2 := p.Apply(pathIPv6, nil) + assert.Equal(t, ROUTE_TYPE_REJECT, pType2) + assert.Equal(t, newPath2, pathIPv6) +} + +func TestAsPathLengthConditionEvaluate(t *testing.T) { + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(2, []uint16{65001, 65000, 65004, 65005}), + bgp.NewAsPathParam(1, []uint16{65001, 65000, 65004, 65005}), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg.Body.(*bgp.BGPUpdate)) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create match condition + asPathLength := config.AsPathLength{ + Operator: "eq", + Value: 5, + } + c, _ := NewAsPathLengthCondition(asPathLength) + + // test + assert.Equal(t, true, c.Evaluate(path, nil)) + + // create match condition + asPathLength = config.AsPathLength{ + Operator: "ge", + Value: 3, + } + c, _ = NewAsPathLengthCondition(asPathLength) + + // test + assert.Equal(t, true, c.Evaluate(path, nil)) + + // create match condition + asPathLength = config.AsPathLength{ + Operator: "le", + Value: 3, + } + c, _ = NewAsPathLengthCondition(asPathLength) + + // test + assert.Equal(t, false, c.Evaluate(path, nil)) +} + +func TestPolicyMatchAndAcceptNextHop(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + s := createStatement("statement1", "ps1", "ns1", true) + s.Conditions.BgpConditions.NextHopInList = []string{"10.0.0.1/32"} + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + pType, newPath := r.policyMap["pd1"].Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.Equal(t, newPath, path) +} + +func TestPolicyMatchAndRejectNextHop(t *testing.T) { + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + s := createStatement("statement1", "ps1", "ns1", true) + s.Conditions.BgpConditions.NextHopInList = []string{"10.0.0.12"} + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + pType, newPath := r.policyMap["pd1"].Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_NONE, pType) + assert.Equal(t, newPath, path) +} + +func TestAsPathLengthConditionWithOtherCondition(t *testing.T) { + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(2, []uint16{65001, 65000, 65004, 65004, 65005}), + bgp.NewAsPathParam(1, []uint16{65001, 65000, 65004, 65005}), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg.Body.(*bgp.BGPUpdate)) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + ps := createPrefixSet("ps1", "10.10.1.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + // create match condition + asPathLength := config.AsPathLength{ + Operator: "le", + Value: 10, + } + + s := createStatement("statement1", "ps1", "ns1", false) + s.Conditions.BgpConditions.AsPathLength = asPathLength + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_REJECT, pType) + assert.Equal(t, newPath, path) + +} + +func TestAs4PathLengthConditionEvaluate(t *testing.T) { + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{ + createAs4Value("65001.1"), + createAs4Value("65000.1"), + createAs4Value("65004.1"), + createAs4Value("65005.1"), + }), + bgp.NewAs4PathParam(1, []uint32{ + createAs4Value("65001.1"), + createAs4Value("65000.1"), + createAs4Value("65004.1"), + createAs4Value("65005.1"), + }), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg.Body.(*bgp.BGPUpdate)) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create match condition + asPathLength := config.AsPathLength{ + Operator: "eq", + Value: 5, + } + c, _ := NewAsPathLengthCondition(asPathLength) + + // test + assert.Equal(t, true, c.Evaluate(path, nil)) + + // create match condition + asPathLength = config.AsPathLength{ + Operator: "ge", + Value: 3, + } + c, _ = NewAsPathLengthCondition(asPathLength) + + // test + assert.Equal(t, true, c.Evaluate(path, nil)) + + // create match condition + asPathLength = config.AsPathLength{ + Operator: "le", + Value: 3, + } + c, _ = NewAsPathLengthCondition(asPathLength) + + // test + assert.Equal(t, false, c.Evaluate(path, nil)) +} + +func addPolicy(r *RoutingPolicy, x *Policy) { + for _, s := range x.Statements { + for _, c := range s.Conditions { + r.validateCondition(c) + } + } +} + +func TestAs4PathLengthConditionWithOtherCondition(t *testing.T) { + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{ + createAs4Value("65001.1"), + createAs4Value("65000.1"), + createAs4Value("65004.1"), + createAs4Value("65004.1"), + createAs4Value("65005.1"), + }), + bgp.NewAs4PathParam(1, []uint32{ + createAs4Value("65001.1"), + createAs4Value("65000.1"), + createAs4Value("65004.1"), + createAs4Value("65005.1"), + }), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg.Body.(*bgp.BGPUpdate)) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + ps := createPrefixSet("ps1", "10.10.1.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + // create match condition + asPathLength := config.AsPathLength{ + Operator: "le", + Value: 10, + } + + s := createStatement("statement1", "ps1", "ns1", false) + s.Conditions.BgpConditions.AsPathLength = asPathLength + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + r.reload(pl) + p, _ := NewPolicy(pl.PolicyDefinitions[0]) + addPolicy(r, p) + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_REJECT, pType) + assert.Equal(t, newPath, path) + +} + +func TestAsPathConditionEvaluate(t *testing.T) { + + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam1 := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(2, []uint16{65001, 65000, 65010, 65004, 65005}), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam1) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg1 := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg1.Body.(*bgp.BGPUpdate)) + path1 := ProcessMessage(updateMsg1, peer, time.Now())[0] + + aspathParam2 := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(2, []uint16{65010}), + } + aspath2 := bgp.NewPathAttributeAsPath(aspathParam2) + pathAttributes = []bgp.PathAttributeInterface{origin, aspath2, nexthop, med} + updateMsg2 := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg2.Body.(*bgp.BGPUpdate)) + path2 := ProcessMessage(updateMsg2, peer, time.Now())[0] + + // create match condition + asPathSet1 := config.AsPathSet{ + AsPathSetName: "asset1", + AsPathList: []string{"^65001"}, + } + + asPathSet2 := config.AsPathSet{ + AsPathSetName: "asset2", + AsPathList: []string{"65005$"}, + } + + asPathSet3 := config.AsPathSet{ + AsPathSetName: "asset3", + AsPathList: []string{"65004", "65005$"}, + } + + asPathSet4 := config.AsPathSet{ + AsPathSetName: "asset4", + AsPathList: []string{"65000$"}, + } + + asPathSet5 := config.AsPathSet{ + AsPathSetName: "asset5", + AsPathList: []string{"65010"}, + } + + asPathSet6 := config.AsPathSet{ + AsPathSetName: "asset6", + AsPathList: []string{"^65010$"}, + } + + m := make(map[string]DefinedSet) + for _, s := range []config.AsPathSet{asPathSet1, asPathSet2, asPathSet3, + asPathSet4, asPathSet5, asPathSet6} { + a, _ := NewAsPathSet(s) + m[s.AsPathSetName] = a + } + + createAspathC := func(name string, option config.MatchSetOptionsType) *AsPathCondition { + matchSet := config.MatchAsPathSet{} + matchSet.AsPathSet = name + matchSet.MatchSetOptions = option + p, _ := NewAsPathCondition(matchSet) + if v, ok := m[name]; ok { + p.set = v.(*AsPathSet) + } + return p + } + + p1 := createAspathC("asset1", config.MATCH_SET_OPTIONS_TYPE_ANY) + p2 := createAspathC("asset2", config.MATCH_SET_OPTIONS_TYPE_ANY) + p3 := createAspathC("asset3", config.MATCH_SET_OPTIONS_TYPE_ANY) + p4 := createAspathC("asset4", config.MATCH_SET_OPTIONS_TYPE_ANY) + p5 := createAspathC("asset5", config.MATCH_SET_OPTIONS_TYPE_ANY) + p6 := createAspathC("asset6", config.MATCH_SET_OPTIONS_TYPE_ANY) + p7 := createAspathC("asset3", config.MATCH_SET_OPTIONS_TYPE_ALL) + p8 := createAspathC("asset3", config.MATCH_SET_OPTIONS_TYPE_INVERT) + + // test + assert.Equal(t, true, p1.Evaluate(path1, nil)) + assert.Equal(t, true, p2.Evaluate(path1, nil)) + assert.Equal(t, true, p3.Evaluate(path1, nil)) + assert.Equal(t, false, p4.Evaluate(path1, nil)) + assert.Equal(t, true, p5.Evaluate(path1, nil)) + assert.Equal(t, false, p6.Evaluate(path1, nil)) + assert.Equal(t, true, p6.Evaluate(path2, nil)) + assert.Equal(t, true, p7.Evaluate(path1, nil)) + assert.Equal(t, true, p8.Evaluate(path2, nil)) +} + +func TestMultipleAsPathConditionEvaluate(t *testing.T) { + + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam1 := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(2, []uint16{65001, 65000, 54000, 65004, 65005}), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam1) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg1 := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg1.Body.(*bgp.BGPUpdate)) + path1 := ProcessMessage(updateMsg1, peer, time.Now())[0] + + // create match condition + asPathSet1 := config.AsPathSet{ + AsPathSetName: "asset1", + AsPathList: []string{"^65001_65000"}, + } + + asPathSet2 := config.AsPathSet{ + AsPathSetName: "asset2", + AsPathList: []string{"65004_65005$"}, + } + + asPathSet3 := config.AsPathSet{ + AsPathSetName: "asset3", + AsPathList: []string{"65001_65000_54000"}, + } + + asPathSet4 := config.AsPathSet{ + AsPathSetName: "asset4", + AsPathList: []string{"54000_65004_65005"}, + } + + asPathSet5 := config.AsPathSet{ + AsPathSetName: "asset5", + AsPathList: []string{"^65001 65000 54000 65004 65005$"}, + } + + asPathSet6 := config.AsPathSet{ + AsPathSetName: "asset6", + AsPathList: []string{".*_[0-9]+_65005"}, + } + + asPathSet7 := config.AsPathSet{ + AsPathSetName: "asset7", + AsPathList: []string{".*_5[0-9]+_[0-9]+"}, + } + + asPathSet8 := config.AsPathSet{ + AsPathSetName: "asset8", + AsPathList: []string{"6[0-9]+_6[0-9]+_5[0-9]+"}, + } + + asPathSet9 := config.AsPathSet{ + AsPathSetName: "asset9", + AsPathList: []string{"6[0-9]+__6[0-9]+"}, + } + + m := make(map[string]DefinedSet) + for _, s := range []config.AsPathSet{asPathSet1, asPathSet2, asPathSet3, + asPathSet4, asPathSet5, asPathSet6, asPathSet7, asPathSet8, asPathSet9} { + a, _ := NewAsPathSet(s) + m[s.AsPathSetName] = a + } + + createAspathC := func(name string, option config.MatchSetOptionsType) *AsPathCondition { + matchSet := config.MatchAsPathSet{} + matchSet.AsPathSet = name + matchSet.MatchSetOptions = option + p, _ := NewAsPathCondition(matchSet) + if v, ok := m[name]; ok { + p.set = v.(*AsPathSet) + } + return p + } + + p1 := createAspathC("asset1", config.MATCH_SET_OPTIONS_TYPE_ANY) + p2 := createAspathC("asset2", config.MATCH_SET_OPTIONS_TYPE_ANY) + p3 := createAspathC("asset3", config.MATCH_SET_OPTIONS_TYPE_ANY) + p4 := createAspathC("asset4", config.MATCH_SET_OPTIONS_TYPE_ANY) + p5 := createAspathC("asset5", config.MATCH_SET_OPTIONS_TYPE_ANY) + p6 := createAspathC("asset6", config.MATCH_SET_OPTIONS_TYPE_ANY) + p7 := createAspathC("asset7", config.MATCH_SET_OPTIONS_TYPE_ANY) + p8 := createAspathC("asset8", config.MATCH_SET_OPTIONS_TYPE_ANY) + p9 := createAspathC("asset9", config.MATCH_SET_OPTIONS_TYPE_ANY) + + // test + assert.Equal(t, true, p1.Evaluate(path1, nil)) + assert.Equal(t, true, p2.Evaluate(path1, nil)) + assert.Equal(t, true, p3.Evaluate(path1, nil)) + assert.Equal(t, true, p4.Evaluate(path1, nil)) + assert.Equal(t, true, p5.Evaluate(path1, nil)) + assert.Equal(t, true, p6.Evaluate(path1, nil)) + assert.Equal(t, true, p7.Evaluate(path1, nil)) + assert.Equal(t, true, p8.Evaluate(path1, nil)) + assert.Equal(t, false, p9.Evaluate(path1, nil)) +} + +func TestAsPathCondition(t *testing.T) { + type astest struct { + path *Path + result bool + } + + makeTest := func(asPathAttrType uint8, ases []uint32, result bool) astest { + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(asPathAttrType, ases), + } + pathAttributes := []bgp.PathAttributeInterface{bgp.NewPathAttributeAsPath(aspathParam)} + p := NewPath(nil, nil, false, pathAttributes, time.Time{}, false) + return astest{ + path: p, + result: result, + } + } + + tests := make(map[string][]astest) + + tests["^(100_)+(200_)+$"] = []astest{ + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{100, 200}, true), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{100, 100, 200}, true), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{100, 100, 200, 200}, true), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{100, 100, 200, 200, 300}, false), + } + + aslen255 := func() []uint32 { + r := make([]uint32, 255) + for i := 0; i < 255; i++ { + r[i] = 1 + } + return r + }() + tests["^([0-9]+_){0,255}$"] = []astest{ + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, aslen255, true), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, append(aslen255, 1), false), + } + + tests["(_7521)$"] = []astest{ + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{7521}, true), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{1000, 7521}, true), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{7521, 1000}, false), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{1000, 7521, 100}, false), + } + + tests["^65001( |_.*_)65535$"] = []astest{ + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65001, 65535}, true), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65001, 65001, 65535}, true), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65001, 65002, 65003, 65535}, true), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65001, 65534}, false), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65002, 65535}, false), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65002, 65001, 65535}, false), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65001, 65535, 65002}, false), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{650019, 65535}, false), + makeTest(bgp.BGP_ASPATH_ATTR_TYPE_SEQ, []uint32{65001, 165535}, false), + } + + for k, v := range tests { + s, _ := NewAsPathSet(config.AsPathSet{ + AsPathSetName: k, + AsPathList: []string{k}, + }) + c, _ := NewAsPathCondition(config.MatchAsPathSet{ + AsPathSet: k, + MatchSetOptions: config.MATCH_SET_OPTIONS_TYPE_ANY, + }) + c.set = s + for _, a := range v { + result := c.Evaluate(a.path, nil) + if a.result != result { + log.WithFields(log.Fields{ + "EXP": k, + "ASSTR": a.path.GetAsString(), + "Expected": a.result, + "Result": result, + }).Fatal("failed") + } + } + } +} + +func TestAsPathConditionWithOtherCondition(t *testing.T) { + + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(1, []uint16{65001, 65000, 65004, 65005}), + bgp.NewAsPathParam(2, []uint16{65001, 65000, 65004, 65004, 65005}), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg.Body.(*bgp.BGPUpdate)) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + asPathSet := config.AsPathSet{ + AsPathSetName: "asset1", + AsPathList: []string{"65005$"}, + } + + ps := createPrefixSet("ps1", "10.10.1.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + ds.BgpDefinedSets.AsPathSets = []config.AsPathSet{asPathSet} + + s := createStatement("statement1", "ps1", "ns1", false) + s.Conditions.BgpConditions.MatchAsPathSet.AsPathSet = "asset1" + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_REJECT, pType) + assert.Equal(t, newPath, path) + +} + +func TestAs4PathConditionEvaluate(t *testing.T) { + + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam1 := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{ + createAs4Value("65001.1"), + createAs4Value("65000.1"), + createAs4Value("65010.1"), + createAs4Value("65004.1"), + createAs4Value("65005.1"), + })} + + aspath := bgp.NewPathAttributeAsPath(aspathParam1) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg1 := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg1.Body.(*bgp.BGPUpdate)) + path1 := ProcessMessage(updateMsg1, peer, time.Now())[0] + + aspathParam2 := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{ + createAs4Value("65010.1"), + }), + } + aspath2 := bgp.NewPathAttributeAsPath(aspathParam2) + pathAttributes = []bgp.PathAttributeInterface{origin, aspath2, nexthop, med} + updateMsg2 := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg2.Body.(*bgp.BGPUpdate)) + path2 := ProcessMessage(updateMsg2, peer, time.Now())[0] + + // create match condition + asPathSet1 := config.AsPathSet{ + AsPathSetName: "asset1", + AsPathList: []string{fmt.Sprintf("^%d", createAs4Value("65001.1"))}, + } + + asPathSet2 := config.AsPathSet{ + AsPathSetName: "asset2", + AsPathList: []string{fmt.Sprintf("%d$", createAs4Value("65005.1"))}, + } + + asPathSet3 := config.AsPathSet{ + AsPathSetName: "asset3", + AsPathList: []string{ + fmt.Sprintf("%d", createAs4Value("65004.1")), + fmt.Sprintf("%d$", createAs4Value("65005.1")), + }, + } + + asPathSet4 := config.AsPathSet{ + AsPathSetName: "asset4", + AsPathList: []string{ + fmt.Sprintf("%d$", createAs4Value("65000.1")), + }, + } + + asPathSet5 := config.AsPathSet{ + AsPathSetName: "asset5", + AsPathList: []string{ + fmt.Sprintf("%d", createAs4Value("65010.1")), + }, + } + + asPathSet6 := config.AsPathSet{ + AsPathSetName: "asset6", + AsPathList: []string{ + fmt.Sprintf("%d$", createAs4Value("65010.1")), + }, + } + + m := make(map[string]DefinedSet) + for _, s := range []config.AsPathSet{asPathSet1, asPathSet2, asPathSet3, + asPathSet4, asPathSet5, asPathSet6} { + a, _ := NewAsPathSet(s) + m[s.AsPathSetName] = a + } + + createAspathC := func(name string, option config.MatchSetOptionsType) *AsPathCondition { + matchSet := config.MatchAsPathSet{} + matchSet.AsPathSet = name + matchSet.MatchSetOptions = option + p, _ := NewAsPathCondition(matchSet) + if v, ok := m[name]; ok { + p.set = v.(*AsPathSet) + } + return p + } + + p1 := createAspathC("asset1", config.MATCH_SET_OPTIONS_TYPE_ANY) + p2 := createAspathC("asset2", config.MATCH_SET_OPTIONS_TYPE_ANY) + p3 := createAspathC("asset3", config.MATCH_SET_OPTIONS_TYPE_ANY) + p4 := createAspathC("asset4", config.MATCH_SET_OPTIONS_TYPE_ANY) + p5 := createAspathC("asset5", config.MATCH_SET_OPTIONS_TYPE_ANY) + p6 := createAspathC("asset6", config.MATCH_SET_OPTIONS_TYPE_ANY) + + p7 := createAspathC("asset3", config.MATCH_SET_OPTIONS_TYPE_ALL) + p8 := createAspathC("asset3", config.MATCH_SET_OPTIONS_TYPE_INVERT) + + // test + assert.Equal(t, true, p1.Evaluate(path1, nil)) + assert.Equal(t, true, p2.Evaluate(path1, nil)) + assert.Equal(t, true, p3.Evaluate(path1, nil)) + assert.Equal(t, false, p4.Evaluate(path1, nil)) + assert.Equal(t, true, p5.Evaluate(path1, nil)) + assert.Equal(t, false, p6.Evaluate(path1, nil)) + assert.Equal(t, true, p6.Evaluate(path2, nil)) + + assert.Equal(t, true, p7.Evaluate(path1, nil)) + assert.Equal(t, true, p8.Evaluate(path2, nil)) +} + +func TestMultipleAs4PathConditionEvaluate(t *testing.T) { + + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam1 := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{ + createAs4Value("65001.1"), + createAs4Value("65000.1"), + createAs4Value("54000.1"), + createAs4Value("65004.1"), + createAs4Value("65005.1"), + }), + } + + aspath := bgp.NewPathAttributeAsPath(aspathParam1) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg1 := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg1.Body.(*bgp.BGPUpdate)) + path1 := ProcessMessage(updateMsg1, peer, time.Now())[0] + + // create match condition + asPathSet1 := config.AsPathSet{ + AsPathSetName: "asset1", + AsPathList: []string{ + fmt.Sprintf("^%d_%d", createAs4Value("65001.1"), createAs4Value("65000.1")), + }, + } + + asPathSet2 := config.AsPathSet{ + AsPathSetName: "asset2", + AsPathList: []string{ + fmt.Sprintf("%d_%d$", createAs4Value("65004.1"), createAs4Value("65005.1")), + }, + } + + asPathSet3 := config.AsPathSet{ + AsPathSetName: "asset3", + AsPathList: []string{ + fmt.Sprintf("%d_%d_%d", createAs4Value("65001.1"), createAs4Value("65000.1"), createAs4Value("54000.1")), + }, + } + + asPathSet4 := config.AsPathSet{ + AsPathSetName: "asset4", + AsPathList: []string{ + fmt.Sprintf("%d_%d_%d", createAs4Value("54000.1"), createAs4Value("65004.1"), createAs4Value("65005.1")), + }, + } + + asPathSet5 := config.AsPathSet{ + AsPathSetName: "asset5", + AsPathList: []string{ + fmt.Sprintf("^%d %d %d %d %d$", createAs4Value("65001.1"), createAs4Value("65000.1"), createAs4Value("54000.1"), createAs4Value("65004.1"), createAs4Value("65005.1")), + }, + } + + asPathSet6 := config.AsPathSet{ + AsPathSetName: "asset6", + AsPathList: []string{ + fmt.Sprintf(".*_[0-9]+_%d", createAs4Value("65005.1")), + }, + } + + asPathSet7 := config.AsPathSet{ + AsPathSetName: "asset7", + AsPathList: []string{".*_3[0-9]+_[0-9]+"}, + } + + asPathSet8 := config.AsPathSet{ + AsPathSetName: "asset8", + AsPathList: []string{"4[0-9]+_4[0-9]+_3[0-9]+"}, + } + + asPathSet9 := config.AsPathSet{ + AsPathSetName: "asset9", + AsPathList: []string{"4[0-9]+__4[0-9]+"}, + } + + m := make(map[string]DefinedSet) + for _, s := range []config.AsPathSet{asPathSet1, asPathSet2, asPathSet3, + asPathSet4, asPathSet5, asPathSet6, asPathSet7, asPathSet8, asPathSet9} { + a, _ := NewAsPathSet(s) + m[s.AsPathSetName] = a + } + + createAspathC := func(name string, option config.MatchSetOptionsType) *AsPathCondition { + matchSet := config.MatchAsPathSet{} + matchSet.AsPathSet = name + matchSet.MatchSetOptions = option + p, _ := NewAsPathCondition(matchSet) + if v, ok := m[name]; ok { + p.set = v.(*AsPathSet) + } + return p + } + + p1 := createAspathC("asset1", config.MATCH_SET_OPTIONS_TYPE_ANY) + p2 := createAspathC("asset2", config.MATCH_SET_OPTIONS_TYPE_ANY) + p3 := createAspathC("asset3", config.MATCH_SET_OPTIONS_TYPE_ANY) + p4 := createAspathC("asset4", config.MATCH_SET_OPTIONS_TYPE_ANY) + p5 := createAspathC("asset5", config.MATCH_SET_OPTIONS_TYPE_ANY) + p6 := createAspathC("asset6", config.MATCH_SET_OPTIONS_TYPE_ANY) + p7 := createAspathC("asset7", config.MATCH_SET_OPTIONS_TYPE_ANY) + p8 := createAspathC("asset8", config.MATCH_SET_OPTIONS_TYPE_ANY) + p9 := createAspathC("asset9", config.MATCH_SET_OPTIONS_TYPE_ANY) + + // test + assert.Equal(t, true, p1.Evaluate(path1, nil)) + assert.Equal(t, true, p2.Evaluate(path1, nil)) + assert.Equal(t, true, p3.Evaluate(path1, nil)) + assert.Equal(t, true, p4.Evaluate(path1, nil)) + assert.Equal(t, true, p5.Evaluate(path1, nil)) + assert.Equal(t, true, p6.Evaluate(path1, nil)) + assert.Equal(t, true, p7.Evaluate(path1, nil)) + assert.Equal(t, true, p8.Evaluate(path1, nil)) + assert.Equal(t, false, p9.Evaluate(path1, nil)) +} + +func TestAs4PathConditionWithOtherCondition(t *testing.T) { + + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(1, []uint32{ + createAs4Value("65001.1"), + createAs4Value("65000.1"), + createAs4Value("65004.1"), + createAs4Value("65005.1"), + }), + bgp.NewAs4PathParam(2, []uint32{ + createAs4Value("65001.1"), + createAs4Value("65000.1"), + createAs4Value("65004.1"), + createAs4Value("65004.1"), + createAs4Value("65005.1"), + }), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg.Body.(*bgp.BGPUpdate)) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + asPathSet := config.AsPathSet{ + AsPathSetName: "asset1", + AsPathList: []string{fmt.Sprintf("%d$", createAs4Value("65005.1"))}, + } + + ps := createPrefixSet("ps1", "10.10.1.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + ds.BgpDefinedSets.AsPathSets = []config.AsPathSet{asPathSet} + + s := createStatement("statement1", "ps1", "ns1", false) + s.Conditions.BgpConditions.MatchAsPathSet.AsPathSet = "asset1" + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + r.reload(pl) + p, _ := NewPolicy(pl.PolicyDefinitions[0]) + addPolicy(r, p) + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_REJECT, pType) + assert.Equal(t, newPath, path) + +} + +func TestAs4PathConditionEvaluateMixedWith2byteAS(t *testing.T) { + + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam1 := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{ + createAs4Value("65001.1"), + createAs4Value("65000.1"), + createAs4Value("54000.1"), + 100, + 5000, + createAs4Value("65004.1"), + createAs4Value("65005.1"), + 4000, + }), + } + + aspath := bgp.NewPathAttributeAsPath(aspathParam1) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg1 := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg1.Body.(*bgp.BGPUpdate)) + path1 := ProcessMessage(updateMsg1, peer, time.Now())[0] + + // create match condition + asPathSet1 := config.AsPathSet{ + AsPathSetName: "asset1", + AsPathList: []string{fmt.Sprintf("^%d", createAs4Value("65001.1"))}, + } + + asPathSet2 := config.AsPathSet{ + AsPathSetName: "asset2", + AsPathList: []string{"4000$"}, + } + + asPathSet3 := config.AsPathSet{ + AsPathSetName: "asset3", + AsPathList: []string{fmt.Sprintf("%d", createAs4Value("65004.1")), "4000$"}, + } + + asPathSet4 := config.AsPathSet{ + AsPathSetName: "asset4", + AsPathList: []string{fmt.Sprintf("%d_%d_%d", createAs4Value("54000.1"), 100, 5000)}, + } + + asPathSet5 := config.AsPathSet{ + AsPathSetName: "asset5", + AsPathList: []string{".*_[0-9]+_100"}, + } + + asPathSet6 := config.AsPathSet{ + AsPathSetName: "asset6", + AsPathList: []string{".*_3[0-9]+_[0]+"}, + } + + asPathSet7 := config.AsPathSet{ + AsPathSetName: "asset7", + AsPathList: []string{".*_3[0-9]+_[1]+"}, + } + + m := make(map[string]DefinedSet) + for _, s := range []config.AsPathSet{asPathSet1, asPathSet2, asPathSet3, + asPathSet4, asPathSet5, asPathSet6, asPathSet7} { + a, _ := NewAsPathSet(s) + m[s.AsPathSetName] = a + } + + createAspathC := func(name string, option config.MatchSetOptionsType) *AsPathCondition { + matchSet := config.MatchAsPathSet{} + matchSet.AsPathSet = name + matchSet.MatchSetOptions = option + p, _ := NewAsPathCondition(matchSet) + if v, ok := m[name]; ok { + p.set = v.(*AsPathSet) + } + return p + } + + p1 := createAspathC("asset1", config.MATCH_SET_OPTIONS_TYPE_ANY) + p2 := createAspathC("asset2", config.MATCH_SET_OPTIONS_TYPE_ANY) + p3 := createAspathC("asset3", config.MATCH_SET_OPTIONS_TYPE_ALL) + p4 := createAspathC("asset4", config.MATCH_SET_OPTIONS_TYPE_ANY) + p5 := createAspathC("asset5", config.MATCH_SET_OPTIONS_TYPE_ANY) + p6 := createAspathC("asset6", config.MATCH_SET_OPTIONS_TYPE_ANY) + p7 := createAspathC("asset7", config.MATCH_SET_OPTIONS_TYPE_ANY) + + // test + assert.Equal(t, true, p1.Evaluate(path1, nil)) + assert.Equal(t, true, p2.Evaluate(path1, nil)) + assert.Equal(t, true, p3.Evaluate(path1, nil)) + assert.Equal(t, true, p4.Evaluate(path1, nil)) + assert.Equal(t, true, p5.Evaluate(path1, nil)) + assert.Equal(t, false, p6.Evaluate(path1, nil)) + assert.Equal(t, true, p7.Evaluate(path1, nil)) + +} + +func TestCommunityConditionEvaluate(t *testing.T) { + + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam1 := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(2, []uint16{65001, 65000, 65004, 65005}), + bgp.NewAsPathParam(1, []uint16{65001, 65010, 65004, 65005}), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam1) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + communities := bgp.NewPathAttributeCommunities([]uint32{ + stringToCommunityValue("65001:100"), + stringToCommunityValue("65001:200"), + stringToCommunityValue("65001:300"), + stringToCommunityValue("65001:400"), + 0x00000000, + 0xFFFFFF01, + 0xFFFFFF02, + 0xFFFFFF03}) + + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med, communities} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg1 := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg1.Body.(*bgp.BGPUpdate)) + path1 := ProcessMessage(updateMsg1, peer, time.Now())[0] + + communities2 := bgp.NewPathAttributeCommunities([]uint32{ + stringToCommunityValue("65001:100"), + stringToCommunityValue("65001:200"), + stringToCommunityValue("65001:300"), + stringToCommunityValue("65001:400")}) + + pathAttributes2 := []bgp.PathAttributeInterface{origin, aspath, nexthop, med, communities2} + updateMsg2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nlri) + UpdatePathAttrs4ByteAs(updateMsg2.Body.(*bgp.BGPUpdate)) + path2 := ProcessMessage(updateMsg2, peer, time.Now())[0] + + // create match condition + comSet1 := config.CommunitySet{ + CommunitySetName: "comset1", + CommunityList: []string{"65001:10", "65001:50", "65001:100"}, + } + + comSet2 := config.CommunitySet{ + CommunitySetName: "comset2", + CommunityList: []string{"65001:200"}, + } + + comSet3 := config.CommunitySet{ + CommunitySetName: "comset3", + CommunityList: []string{"4259905936"}, + } + + comSet4 := config.CommunitySet{ + CommunitySetName: "comset4", + CommunityList: []string{"^[0-9]*:300$"}, + } + + comSet5 := config.CommunitySet{ + CommunitySetName: "comset5", + CommunityList: []string{"INTERNET"}, + } + + comSet6 := config.CommunitySet{ + CommunitySetName: "comset6", + CommunityList: []string{"NO_EXPORT"}, + } + + comSet7 := config.CommunitySet{ + CommunitySetName: "comset7", + CommunityList: []string{"NO_ADVERTISE"}, + } + + comSet8 := config.CommunitySet{ + CommunitySetName: "comset8", + CommunityList: []string{"NO_EXPORT_SUBCONFED"}, + } + + comSet9 := config.CommunitySet{ + CommunitySetName: "comset9", + CommunityList: []string{ + "65001:\\d+", + "\\d+:\\d00", + }, + } + + comSet10 := config.CommunitySet{ + CommunitySetName: "comset10", + CommunityList: []string{ + "65001:1", + "65001:2", + "65001:3", + }, + } + + m := make(map[string]DefinedSet) + + for _, c := range []config.CommunitySet{comSet1, comSet2, comSet3, + comSet4, comSet5, comSet6, comSet7, comSet8, comSet9, comSet10} { + s, _ := NewCommunitySet(c) + m[c.CommunitySetName] = s + } + + createCommunityC := func(name string, option config.MatchSetOptionsType) *CommunityCondition { + matchSet := config.MatchCommunitySet{} + matchSet.CommunitySet = name + matchSet.MatchSetOptions = option + c, _ := NewCommunityCondition(matchSet) + if v, ok := m[name]; ok { + c.set = v.(*CommunitySet) + } + return c + } + + // ANY case + p1 := createCommunityC("comset1", config.MATCH_SET_OPTIONS_TYPE_ANY) + p2 := createCommunityC("comset2", config.MATCH_SET_OPTIONS_TYPE_ANY) + p3 := createCommunityC("comset3", config.MATCH_SET_OPTIONS_TYPE_ANY) + p4 := createCommunityC("comset4", config.MATCH_SET_OPTIONS_TYPE_ANY) + p5 := createCommunityC("comset5", config.MATCH_SET_OPTIONS_TYPE_ANY) + p6 := createCommunityC("comset6", config.MATCH_SET_OPTIONS_TYPE_ANY) + p7 := createCommunityC("comset7", config.MATCH_SET_OPTIONS_TYPE_ANY) + p8 := createCommunityC("comset8", config.MATCH_SET_OPTIONS_TYPE_ANY) + + // ALL case + p9 := createCommunityC("comset9", config.MATCH_SET_OPTIONS_TYPE_ALL) + + // INVERT case + p10 := createCommunityC("comset10", config.MATCH_SET_OPTIONS_TYPE_INVERT) + + // test + assert.Equal(t, true, p1.Evaluate(path1, nil)) + assert.Equal(t, true, p2.Evaluate(path1, nil)) + assert.Equal(t, true, p3.Evaluate(path1, nil)) + assert.Equal(t, true, p4.Evaluate(path1, nil)) + assert.Equal(t, true, p5.Evaluate(path1, nil)) + assert.Equal(t, true, p6.Evaluate(path1, nil)) + assert.Equal(t, true, p7.Evaluate(path1, nil)) + assert.Equal(t, true, p8.Evaluate(path1, nil)) + assert.Equal(t, true, p9.Evaluate(path2, nil)) + assert.Equal(t, true, p10.Evaluate(path1, nil)) + +} + +func TestCommunityConditionEvaluateWithOtherCondition(t *testing.T) { + + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(1, []uint16{65001, 65000, 65004, 65005}), + bgp.NewAsPathParam(2, []uint16{65001, 65000, 65004, 65004, 65005}), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + communities := bgp.NewPathAttributeCommunities([]uint32{ + stringToCommunityValue("65001:100"), + stringToCommunityValue("65001:200"), + stringToCommunityValue("65001:300"), + stringToCommunityValue("65001:400"), + 0x00000000, + 0xFFFFFF01, + 0xFFFFFF02, + 0xFFFFFF03}) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med, communities} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg.Body.(*bgp.BGPUpdate)) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + asPathSet := config.AsPathSet{ + AsPathSetName: "asset1", + AsPathList: []string{"65005$"}, + } + + comSet1 := config.CommunitySet{ + CommunitySetName: "comset1", + CommunityList: []string{"65001:100", "65001:200", "65001:300"}, + } + + comSet2 := config.CommunitySet{ + CommunitySetName: "comset2", + CommunityList: []string{"65050:\\d+"}, + } + + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + ds.BgpDefinedSets.AsPathSets = []config.AsPathSet{asPathSet} + ds.BgpDefinedSets.CommunitySets = []config.CommunitySet{comSet1, comSet2} + + s1 := createStatement("statement1", "ps1", "ns1", false) + s1.Conditions.BgpConditions.MatchAsPathSet.AsPathSet = "asset1" + s1.Conditions.BgpConditions.MatchCommunitySet.CommunitySet = "comset1" + + s2 := createStatement("statement2", "ps1", "ns1", false) + s2.Conditions.BgpConditions.MatchAsPathSet.AsPathSet = "asset1" + s2.Conditions.BgpConditions.MatchCommunitySet.CommunitySet = "comset2" + + pd1 := createPolicyDefinition("pd1", s1) + pd2 := createPolicyDefinition("pd2", s2) + pl := createRoutingPolicy(ds, pd1, pd2) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_REJECT, pType) + assert.Equal(t, newPath, path) + + p = r.policyMap["pd2"] + pType, newPath = p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_NONE, pType) + assert.Equal(t, newPath, path) + +} + +func TestPolicyMatchAndAddCommunities(t *testing.T) { + + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + community := "65000:100" + + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetCommunity = createSetCommunity("ADD", community) + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(t, nil, newPath) + assert.Equal(t, []uint32{stringToCommunityValue(community)}, newPath.GetCommunities()) +} + +func TestPolicyMatchAndReplaceCommunities(t *testing.T) { + + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + communities := bgp.NewPathAttributeCommunities([]uint32{ + stringToCommunityValue("65001:200"), + }) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med, communities} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + community := "65000:100" + + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetCommunity = createSetCommunity("REPLACE", community) + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(t, nil, newPath) + assert.Equal(t, []uint32{stringToCommunityValue(community)}, newPath.GetCommunities()) +} + +func TestPolicyMatchAndRemoveCommunities(t *testing.T) { + + // create path + community1 := "65000:100" + community2 := "65000:200" + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + communities := bgp.NewPathAttributeCommunities([]uint32{ + stringToCommunityValue(community1), + stringToCommunityValue(community2), + }) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med, communities} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetCommunity = createSetCommunity("REMOVE", community1) + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(t, nil, newPath) + assert.Equal(t, []uint32{stringToCommunityValue(community2)}, newPath.GetCommunities()) +} + +func TestPolicyMatchAndRemoveCommunitiesRegexp(t *testing.T) { + + // create path + community1 := "65000:100" + community2 := "65000:200" + community3 := "65100:100" + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + communities := bgp.NewPathAttributeCommunities([]uint32{ + stringToCommunityValue(community1), + stringToCommunityValue(community2), + stringToCommunityValue(community3), + }) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med, communities} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetCommunity = createSetCommunity("REMOVE", ".*:100") + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(t, nil, newPath) + assert.Equal(t, []uint32{stringToCommunityValue(community2)}, newPath.GetCommunities()) +} + +func TestPolicyMatchAndRemoveCommunitiesRegexp2(t *testing.T) { + + // create path + community1 := "0:1" + community2 := "10:1" + community3 := "45686:2" + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + communities := bgp.NewPathAttributeCommunities([]uint32{ + stringToCommunityValue(community1), + stringToCommunityValue(community2), + stringToCommunityValue(community3), + }) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med, communities} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetCommunity = createSetCommunity("REMOVE", "^(0|45686):[0-9]+") + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(t, nil, newPath) + assert.Equal(t, []uint32{stringToCommunityValue(community2)}, newPath.GetCommunities()) +} + +func TestPolicyMatchAndClearCommunities(t *testing.T) { + + // create path + community1 := "65000:100" + community2 := "65000:200" + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + communities := bgp.NewPathAttributeCommunities([]uint32{ + stringToCommunityValue(community1), + stringToCommunityValue(community2), + }) + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med, communities} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + s := createStatement("statement1", "ps1", "ns1", true) + // action NULL is obsolate + s.Actions.BgpActions.SetCommunity.Options = "REPLACE" + s.Actions.BgpActions.SetCommunity.SetCommunityMethod.CommunitiesList = nil + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(t, nil, newPath) + //assert.Equal(t, []uint32{}, newPath.GetCommunities()) +} + +func TestExtCommunityConditionEvaluate(t *testing.T) { + + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam1 := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(2, []uint16{65001, 65000, 65004, 65005}), + bgp.NewAsPathParam(1, []uint16{65001, 65010, 65004, 65005}), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam1) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + eComAsSpecific1 := &bgp.TwoOctetAsSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + AS: 65001, + LocalAdmin: 200, + IsTransitive: true, + } + eComIpPrefix1 := &bgp.IPv4AddressSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + IPv4: net.ParseIP("10.0.0.1"), + LocalAdmin: 300, + IsTransitive: true, + } + eComAs4Specific1 := &bgp.FourOctetAsSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + AS: 65030000, + LocalAdmin: 200, + IsTransitive: true, + } + eComAsSpecific2 := &bgp.TwoOctetAsSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + AS: 65002, + LocalAdmin: 200, + IsTransitive: false, + } + eComIpPrefix2 := &bgp.IPv4AddressSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + IPv4: net.ParseIP("10.0.0.2"), + LocalAdmin: 300, + IsTransitive: false, + } + eComAs4Specific2 := &bgp.FourOctetAsSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + AS: 65030001, + LocalAdmin: 200, + IsTransitive: false, + } + eComAsSpecific3 := &bgp.TwoOctetAsSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_ORIGIN), + AS: 65010, + LocalAdmin: 300, + IsTransitive: true, + } + eComIpPrefix3 := &bgp.IPv4AddressSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_ORIGIN), + IPv4: net.ParseIP("10.0.10.10"), + LocalAdmin: 400, + IsTransitive: true, + } + eComAs4Specific3 := &bgp.FourOctetAsSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + AS: 65030002, + LocalAdmin: 500, + IsTransitive: true, + } + ec := []bgp.ExtendedCommunityInterface{eComAsSpecific1, eComIpPrefix1, eComAs4Specific1, eComAsSpecific2, + eComIpPrefix2, eComAs4Specific2, eComAsSpecific3, eComIpPrefix3, eComAs4Specific3} + extCommunities := bgp.NewPathAttributeExtendedCommunities(ec) + + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med, extCommunities} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg1 := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg1.Body.(*bgp.BGPUpdate)) + path1 := ProcessMessage(updateMsg1, peer, time.Now())[0] + + convUintStr := func(as uint32) string { + upper := strconv.FormatUint(uint64(as&0xFFFF0000>>16), 10) + lower := strconv.FormatUint(uint64(as&0x0000FFFF), 10) + str := fmt.Sprintf("%s.%s", upper, lower) + return str + } + + // create match condition + ecomSet1 := config.ExtCommunitySet{ + ExtCommunitySetName: "ecomSet1", + ExtCommunityList: []string{"RT:65001:200"}, + } + ecomSet2 := config.ExtCommunitySet{ + ExtCommunitySetName: "ecomSet2", + ExtCommunityList: []string{"RT:10.0.0.1:300"}, + } + ecomSet3 := config.ExtCommunitySet{ + ExtCommunitySetName: "ecomSet3", + ExtCommunityList: []string{fmt.Sprintf("RT:%s:200", convUintStr(65030000))}, + } + ecomSet4 := config.ExtCommunitySet{ + ExtCommunitySetName: "ecomSet4", + ExtCommunityList: []string{"RT:65002:200"}, + } + ecomSet5 := config.ExtCommunitySet{ + ExtCommunitySetName: "ecomSet5", + ExtCommunityList: []string{"RT:10.0.0.2:300"}, + } + ecomSet6 := config.ExtCommunitySet{ + ExtCommunitySetName: "ecomSet6", + ExtCommunityList: []string{fmt.Sprintf("RT:%s:200", convUintStr(65030001))}, + } + ecomSet7 := config.ExtCommunitySet{ + ExtCommunitySetName: "ecomSet7", + ExtCommunityList: []string{"SoO:65010:300"}, + } + ecomSet8 := config.ExtCommunitySet{ + ExtCommunitySetName: "ecomSet8", + ExtCommunityList: []string{"SoO:10.0.10.10:[0-9]+"}, + } + ecomSet9 := config.ExtCommunitySet{ + ExtCommunitySetName: "ecomSet9", + ExtCommunityList: []string{"RT:[0-9]+:[0-9]+"}, + } + ecomSet10 := config.ExtCommunitySet{ + ExtCommunitySetName: "ecomSet10", + ExtCommunityList: []string{"RT:.+:\\d00", "SoO:.+:\\d00"}, + } + + ecomSet11 := config.ExtCommunitySet{ + ExtCommunitySetName: "ecomSet11", + ExtCommunityList: []string{"RT:65001:2", "SoO:11.0.10.10:[0-9]+"}, + } + + m := make(map[string]DefinedSet) + for _, c := range []config.ExtCommunitySet{ecomSet1, ecomSet2, ecomSet3, ecomSet4, ecomSet5, ecomSet6, ecomSet7, + ecomSet8, ecomSet9, ecomSet10, ecomSet11} { + s, _ := NewExtCommunitySet(c) + m[s.Name()] = s + } + + createExtCommunityC := func(name string, option config.MatchSetOptionsType) *ExtCommunityCondition { + matchSet := config.MatchExtCommunitySet{} + matchSet.ExtCommunitySet = name + matchSet.MatchSetOptions = option + c, _ := NewExtCommunityCondition(matchSet) + if v, ok := m[name]; ok { + c.set = v.(*ExtCommunitySet) + } + + return c + } + + p1 := createExtCommunityC("ecomSet1", config.MATCH_SET_OPTIONS_TYPE_ANY) + p2 := createExtCommunityC("ecomSet2", config.MATCH_SET_OPTIONS_TYPE_ANY) + p3 := createExtCommunityC("ecomSet3", config.MATCH_SET_OPTIONS_TYPE_ANY) + p4 := createExtCommunityC("ecomSet4", config.MATCH_SET_OPTIONS_TYPE_ANY) + p5 := createExtCommunityC("ecomSet5", config.MATCH_SET_OPTIONS_TYPE_ANY) + p6 := createExtCommunityC("ecomSet6", config.MATCH_SET_OPTIONS_TYPE_ANY) + p7 := createExtCommunityC("ecomSet7", config.MATCH_SET_OPTIONS_TYPE_ANY) + p8 := createExtCommunityC("ecomSet8", config.MATCH_SET_OPTIONS_TYPE_ANY) + p9 := createExtCommunityC("ecomSet9", config.MATCH_SET_OPTIONS_TYPE_ANY) + + // ALL case + p10 := createExtCommunityC("ecomSet10", config.MATCH_SET_OPTIONS_TYPE_ALL) + + // INVERT case + p11 := createExtCommunityC("ecomSet11", config.MATCH_SET_OPTIONS_TYPE_INVERT) + + // test + assert.Equal(t, true, p1.Evaluate(path1, nil)) + assert.Equal(t, true, p2.Evaluate(path1, nil)) + assert.Equal(t, true, p3.Evaluate(path1, nil)) + assert.Equal(t, false, p4.Evaluate(path1, nil)) + assert.Equal(t, false, p5.Evaluate(path1, nil)) + assert.Equal(t, false, p6.Evaluate(path1, nil)) + assert.Equal(t, true, p7.Evaluate(path1, nil)) + assert.Equal(t, true, p8.Evaluate(path1, nil)) + assert.Equal(t, true, p9.Evaluate(path1, nil)) + assert.Equal(t, true, p10.Evaluate(path1, nil)) + assert.Equal(t, true, p11.Evaluate(path1, nil)) + +} + +func TestExtCommunityConditionEvaluateWithOtherCondition(t *testing.T) { + + // setup + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.2.1.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAsPathParam(1, []uint16{65001, 65000, 65004, 65005}), + bgp.NewAsPathParam(2, []uint16{65001, 65000, 65004, 65004, 65005}), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.2.1.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + eComAsSpecific1 := &bgp.TwoOctetAsSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + AS: 65001, + LocalAdmin: 200, + IsTransitive: true, + } + eComIpPrefix1 := &bgp.IPv4AddressSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + IPv4: net.ParseIP("10.0.0.1"), + LocalAdmin: 300, + IsTransitive: true, + } + eComAs4Specific1 := &bgp.FourOctetAsSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + AS: 65030000, + LocalAdmin: 200, + IsTransitive: true, + } + eComAsSpecific2 := &bgp.TwoOctetAsSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + AS: 65002, + LocalAdmin: 200, + IsTransitive: false, + } + eComIpPrefix2 := &bgp.IPv4AddressSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + IPv4: net.ParseIP("10.0.0.2"), + LocalAdmin: 300, + IsTransitive: false, + } + eComAs4Specific2 := &bgp.FourOctetAsSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + AS: 65030001, + LocalAdmin: 200, + IsTransitive: false, + } + eComAsSpecific3 := &bgp.TwoOctetAsSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_ORIGIN), + AS: 65010, + LocalAdmin: 300, + IsTransitive: true, + } + eComIpPrefix3 := &bgp.IPv4AddressSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_ORIGIN), + IPv4: net.ParseIP("10.0.10.10"), + LocalAdmin: 400, + IsTransitive: true, + } + eComAs4Specific3 := &bgp.FourOctetAsSpecificExtended{ + SubType: bgp.ExtendedCommunityAttrSubType(bgp.EC_SUBTYPE_ROUTE_TARGET), + AS: 65030002, + LocalAdmin: 500, + IsTransitive: true, + } + ec := []bgp.ExtendedCommunityInterface{eComAsSpecific1, eComIpPrefix1, eComAs4Specific1, eComAsSpecific2, + eComIpPrefix2, eComAs4Specific2, eComAsSpecific3, eComIpPrefix3, eComAs4Specific3} + extCommunities := bgp.NewPathAttributeExtendedCommunities(ec) + + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med, extCommunities} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + UpdatePathAttrs4ByteAs(updateMsg.Body.(*bgp.BGPUpdate)) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + asPathSet := config.AsPathSet{ + AsPathSetName: "asset1", + AsPathList: []string{"65005$"}, + } + + ecomSet1 := config.ExtCommunitySet{ + ExtCommunitySetName: "ecomSet1", + ExtCommunityList: []string{"RT:65001:201"}, + } + ecomSet2 := config.ExtCommunitySet{ + ExtCommunitySetName: "ecomSet2", + ExtCommunityList: []string{"RT:[0-9]+:[0-9]+"}, + } + + ps := createPrefixSet("ps1", "10.10.1.0/16", "21..24") + ns := createNeighborSet("ns1", "10.2.1.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + ds.BgpDefinedSets.AsPathSets = []config.AsPathSet{asPathSet} + ds.BgpDefinedSets.ExtCommunitySets = []config.ExtCommunitySet{ecomSet1, ecomSet2} + + s1 := createStatement("statement1", "ps1", "ns1", false) + s1.Conditions.BgpConditions.MatchAsPathSet.AsPathSet = "asset1" + s1.Conditions.BgpConditions.MatchExtCommunitySet.ExtCommunitySet = "ecomSet1" + + s2 := createStatement("statement2", "ps1", "ns1", false) + s2.Conditions.BgpConditions.MatchAsPathSet.AsPathSet = "asset1" + s2.Conditions.BgpConditions.MatchExtCommunitySet.ExtCommunitySet = "ecomSet2" + + pd1 := createPolicyDefinition("pd1", s1) + pd2 := createPolicyDefinition("pd2", s2) + pl := createRoutingPolicy(ds, pd1, pd2) + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_NONE, pType) + assert.Equal(t, newPath, path) + + p = r.policyMap["pd2"] + pType, newPath = p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_REJECT, pType) + assert.Equal(t, newPath, path) + +} + +func TestPolicyMatchAndReplaceMed(t *testing.T) { + + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + m := "200" + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetMed = config.BgpSetMedType(m) + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(t, nil, newPath) + v, err := newPath.GetMed() + assert.Nil(t, err) + newMed := fmt.Sprintf("%d", v) + assert.Equal(t, m, newMed) +} + +func TestPolicyMatchAndAddingMed(t *testing.T) { + + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + m := "+200" + ma := "300" + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetMed = config.BgpSetMedType(m) + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(t, nil, newPath) + + v, err := newPath.GetMed() + assert.Nil(t, err) + newMed := fmt.Sprintf("%d", v) + assert.Equal(t, ma, newMed) +} + +func TestPolicyMatchAndAddingMedOverFlow(t *testing.T) { + + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(1) + + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + m := fmt.Sprintf("+%d", uint32(math.MaxUint32)) + ma := "1" + + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetMed = config.BgpSetMedType(m) + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(t, nil, newPath) + + v, err := newPath.GetMed() + assert.Nil(t, err) + newMed := fmt.Sprintf("%d", v) + assert.Equal(t, ma, newMed) +} + +func TestPolicyMatchAndSubtractMed(t *testing.T) { + + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + m := "-50" + ma := "50" + + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetMed = config.BgpSetMedType(m) + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(t, nil, newPath) + + v, err := newPath.GetMed() + assert.Nil(t, err) + newMed := fmt.Sprintf("%d", v) + assert.Equal(t, ma, newMed) +} + +func TestPolicyMatchAndSubtractMedUnderFlow(t *testing.T) { + + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + m := "-101" + ma := "100" + + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetMed = config.BgpSetMedType(m) + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(t, nil, newPath) + + v, err := newPath.GetMed() + assert.Nil(t, err) + newMed := fmt.Sprintf("%d", v) + assert.Equal(t, ma, newMed) +} + +func TestPolicyMatchWhenPathHaveNotMed(t *testing.T) { + + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + m := "-50" + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetMed = config.BgpSetMedType(m) + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + //test + r := NewRoutingPolicy() + err := r.reload(pl) + assert.Nil(t, err) + p := r.policyMap["pd1"] + + pType, newPath := p.Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(t, nil, newPath) + + _, err = newPath.GetMed() + assert.NotNil(t, err) +} + +func TestPolicyAsPathPrepend(t *testing.T) { + + assert := assert.New(t) + + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001, 65000})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + + body := updateMsg.Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(body) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetAsPathPrepend.As = "65002" + s.Actions.BgpActions.SetAsPathPrepend.RepeatN = 10 + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + //test + r := NewRoutingPolicy() + r.reload(pl) + p := r.policyMap["pd1"] + + pType, newPath := p.Apply(path, nil) + assert.Equal(ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(nil, newPath) + assert.Equal([]uint32{65002, 65002, 65002, 65002, 65002, 65002, 65002, 65002, 65002, 65002, 65001, 65000}, newPath.GetAsSeqList()) +} + +func TestPolicyAsPathPrependLastAs(t *testing.T) { + + assert := assert.New(t) + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65002, 65001, 65000})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + + body := updateMsg.Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(body) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetAsPathPrepend.As = "last-as" + s.Actions.BgpActions.SetAsPathPrepend.RepeatN = 5 + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + //test + r := NewRoutingPolicy() + r.reload(pl) + p := r.policyMap["pd1"] + + pType, newPath := p.Apply(path, nil) + assert.Equal(ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(nil, newPath) + assert.Equal([]uint32{65002, 65002, 65002, 65002, 65002, 65002, 65001, 65000}, newPath.GetAsSeqList()) +} + +func TestPolicyAs4PathPrepend(t *testing.T) { + + assert := assert.New(t) + + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{ + createAs4Value("65001.1"), + createAs4Value("65000.1"), + }), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + + body := updateMsg.Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(body) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetAsPathPrepend.As = fmt.Sprintf("%d", createAs4Value("65002.1")) + s.Actions.BgpActions.SetAsPathPrepend.RepeatN = 10 + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + //test + r := NewRoutingPolicy() + r.reload(pl) + p, err := NewPolicy(pl.PolicyDefinitions[0]) + assert.Nil(err) + addPolicy(r, p) + + pType, newPath := p.Apply(path, nil) + assert.Equal(ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(nil, newPath) + asn := createAs4Value("65002.1") + assert.Equal([]uint32{ + asn, asn, asn, asn, asn, asn, asn, asn, asn, asn, + createAs4Value("65001.1"), + createAs4Value("65000.1"), + }, newPath.GetAsSeqList()) +} + +func TestPolicyAs4PathPrependLastAs(t *testing.T) { + + assert := assert.New(t) + // create path + peer := &PeerInfo{AS: 65001, Address: net.ParseIP("10.0.0.1")} + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{ + createAs4Value("65002.1"), + createAs4Value("65001.1"), + createAs4Value("65000.1"), + }), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + + pathAttributes := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.0.101")} + updateMsg := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + + body := updateMsg.Body.(*bgp.BGPUpdate) + UpdatePathAttrs4ByteAs(body) + path := ProcessMessage(updateMsg, peer, time.Now())[0] + + // create policy + ps := createPrefixSet("ps1", "10.10.0.0/16", "21..24") + ns := createNeighborSet("ns1", "10.0.0.1") + + ds := config.DefinedSets{} + ds.PrefixSets = []config.PrefixSet{ps} + ds.NeighborSets = []config.NeighborSet{ns} + + s := createStatement("statement1", "ps1", "ns1", true) + s.Actions.BgpActions.SetAsPathPrepend.As = "last-as" + s.Actions.BgpActions.SetAsPathPrepend.RepeatN = 5 + + pd := createPolicyDefinition("pd1", s) + pl := createRoutingPolicy(ds, pd) + //test + r := NewRoutingPolicy() + r.reload(pl) + p, _ := NewPolicy(pl.PolicyDefinitions[0]) + addPolicy(r, p) + + pType, newPath := p.Apply(path, nil) + assert.Equal(ROUTE_TYPE_ACCEPT, pType) + assert.NotEqual(nil, newPath) + asn := createAs4Value("65002.1") + assert.Equal([]uint32{ + asn, asn, asn, asn, asn, + createAs4Value("65002.1"), + createAs4Value("65001.1"), + createAs4Value("65000.1"), + }, newPath.GetAsSeqList()) +} + +func TestParseCommunityRegexp(t *testing.T) { + exp, err := ParseCommunityRegexp("65000:1") + assert.Equal(t, nil, err) + assert.Equal(t, true, exp.MatchString("65000:1")) + assert.Equal(t, false, exp.MatchString("65000:100")) +} + +func TestLocalPrefAction(t *testing.T) { + action, err := NewLocalPrefAction(10) + assert.Nil(t, err) + + nlri := bgp.NewIPAddrPrefix(24, "10.0.0.0") + + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{ + bgp.NewAs4PathParam(2, []uint32{ + createAs4Value("65002.1"), + createAs4Value("65001.1"), + createAs4Value("65000.1"), + }), + } + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + + attrs := []bgp.PathAttributeInterface{origin, aspath, nexthop, med} + + path := NewPath(nil, nlri, false, attrs, time.Now(), false) + p := action.Apply(path, nil) + assert.NotNil(t, p) + + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_LOCAL_PREF) + assert.NotNil(t, attr) + lp := attr.(*bgp.PathAttributeLocalPref) + assert.Equal(t, int(lp.Value), int(10)) +} + +func createStatement(name, psname, nsname string, accept bool) config.Statement { + c := config.Conditions{ + MatchPrefixSet: config.MatchPrefixSet{ + PrefixSet: psname, + }, + MatchNeighborSet: config.MatchNeighborSet{ + NeighborSet: nsname, + }, + } + rd := config.ROUTE_DISPOSITION_REJECT_ROUTE + if accept { + rd = config.ROUTE_DISPOSITION_ACCEPT_ROUTE + } + a := config.Actions{ + RouteDisposition: rd, + } + s := config.Statement{ + Name: name, + Conditions: c, + Actions: a, + } + return s +} + +func createSetCommunity(operation string, community ...string) config.SetCommunity { + + s := config.SetCommunity{ + SetCommunityMethod: config.SetCommunityMethod{ + CommunitiesList: community, + }, + Options: operation, + } + return s +} + +func stringToCommunityValue(comStr string) uint32 { + elem := strings.Split(comStr, ":") + asn, _ := strconv.ParseUint(elem[0], 10, 16) + val, _ := strconv.ParseUint(elem[1], 10, 16) + return uint32(asn<<16 | val) +} + +func createPolicyDefinition(defName string, stmt ...config.Statement) config.PolicyDefinition { + pd := config.PolicyDefinition{ + Name: defName, + Statements: []config.Statement(stmt), + } + return pd +} + +func createRoutingPolicy(ds config.DefinedSets, pd ...config.PolicyDefinition) config.RoutingPolicy { + pl := config.RoutingPolicy{ + DefinedSets: ds, + PolicyDefinitions: []config.PolicyDefinition(pd), + } + return pl +} + +func createPrefixSet(name string, prefix string, maskLength string) config.PrefixSet { + ps := config.PrefixSet{ + PrefixSetName: name, + PrefixList: []config.Prefix{ + config.Prefix{ + IpPrefix: prefix, + MasklengthRange: maskLength, + }}, + } + return ps +} + +func createNeighborSet(name string, addr string) config.NeighborSet { + ns := config.NeighborSet{ + NeighborSetName: name, + NeighborInfoList: []string{addr}, + } + return ns +} + +func createAs4Value(s string) uint32 { + v := strings.Split(s, ".") + upper, _ := strconv.ParseUint(v[0], 10, 16) + lower, _ := strconv.ParseUint(v[1], 10, 16) + return uint32(upper<<16 | lower) +} + +func TestPrefixSetOperation(t *testing.T) { + // tryp to create prefixset with multiple families + p1 := config.Prefix{ + IpPrefix: "0.0.0.0/0", + MasklengthRange: "0..7", + } + p2 := config.Prefix{ + IpPrefix: "0::/25", + MasklengthRange: "25..128", + } + _, err := NewPrefixSet(config.PrefixSet{ + PrefixSetName: "ps1", + PrefixList: []config.Prefix{p1, p2}, + }) + assert.NotNil(t, err) + m1, _ := NewPrefixSet(config.PrefixSet{ + PrefixSetName: "ps1", + PrefixList: []config.Prefix{p1}, + }) + m2, err := NewPrefixSet(config.PrefixSet{PrefixSetName: "ps2"}) + assert.Nil(t, err) + err = m1.Append(m2) + assert.Nil(t, err) + err = m2.Append(m1) + assert.Nil(t, err) + assert.Equal(t, bgp.RF_IPv4_UC, m2.family) + p3, _ := NewPrefix(config.Prefix{IpPrefix: "10.10.0.0/24", MasklengthRange: ""}) + p4, _ := NewPrefix(config.Prefix{IpPrefix: "0::/25", MasklengthRange: ""}) + _, err = NewPrefixSetFromApiStruct("ps3", []*Prefix{p3, p4}) + assert.NotNil(t, err) +} + +func TestPrefixSetMatch(t *testing.T) { + p1 := config.Prefix{ + IpPrefix: "0.0.0.0/0", + MasklengthRange: "0..7", + } + p2 := config.Prefix{ + IpPrefix: "0.0.0.0/0", + MasklengthRange: "25..32", + } + ps, err := NewPrefixSet(config.PrefixSet{ + PrefixSetName: "ps1", + PrefixList: []config.Prefix{p1, p2}, + }) + assert.Nil(t, err) + m := &PrefixCondition{ + set: ps, + } + + path := NewPath(nil, bgp.NewIPAddrPrefix(6, "0.0.0.0"), false, []bgp.PathAttributeInterface{}, time.Now(), false) + assert.True(t, m.Evaluate(path, nil)) + + path = NewPath(nil, bgp.NewIPAddrPrefix(10, "0.0.0.0"), false, []bgp.PathAttributeInterface{}, time.Now(), false) + assert.False(t, m.Evaluate(path, nil)) + + path = NewPath(nil, bgp.NewIPAddrPrefix(25, "0.0.0.0"), false, []bgp.PathAttributeInterface{}, time.Now(), false) + assert.True(t, m.Evaluate(path, nil)) + + path = NewPath(nil, bgp.NewIPAddrPrefix(30, "0.0.0.0"), false, []bgp.PathAttributeInterface{}, time.Now(), false) + assert.True(t, m.Evaluate(path, nil)) + + p3 := config.Prefix{ + IpPrefix: "0.0.0.0/0", + MasklengthRange: "9..10", + } + ps2, err := NewPrefixSet(config.PrefixSet{ + PrefixSetName: "ps2", + PrefixList: []config.Prefix{p3}, + }) + assert.Nil(t, err) + err = ps.Append(ps2) + assert.Nil(t, err) + + path = NewPath(nil, bgp.NewIPAddrPrefix(10, "0.0.0.0"), false, []bgp.PathAttributeInterface{}, time.Now(), false) + assert.True(t, m.Evaluate(path, nil)) + + ps3, err := NewPrefixSet(config.PrefixSet{ + PrefixSetName: "ps3", + PrefixList: []config.Prefix{p1}, + }) + assert.Nil(t, err) + err = ps.Remove(ps3) + assert.Nil(t, err) + + path = NewPath(nil, bgp.NewIPAddrPrefix(6, "0.0.0.0"), false, []bgp.PathAttributeInterface{}, time.Now(), false) + assert.False(t, m.Evaluate(path, nil)) +} + +func TestPrefixSetMatchV4withV6Prefix(t *testing.T) { + p1 := config.Prefix{ + IpPrefix: "c000::/3", + MasklengthRange: "3..128", + } + ps, err := NewPrefixSet(config.PrefixSet{ + PrefixSetName: "ps1", + PrefixList: []config.Prefix{p1}, + }) + assert.Nil(t, err) + m := &PrefixCondition{ + set: ps, + } + + path := NewPath(nil, bgp.NewIPAddrPrefix(6, "192.0.0.0"), false, []bgp.PathAttributeInterface{}, time.Now(), false) + assert.False(t, m.Evaluate(path, nil)) +} + +func TestLargeCommunityMatchAction(t *testing.T) { + coms := []*bgp.LargeCommunity{ + &bgp.LargeCommunity{ASN: 100, LocalData1: 100, LocalData2: 100}, + &bgp.LargeCommunity{ASN: 100, LocalData1: 200, LocalData2: 200}, + } + p := NewPath(nil, nil, false, []bgp.PathAttributeInterface{bgp.NewPathAttributeLargeCommunities(coms)}, time.Time{}, false) + + c := config.LargeCommunitySet{ + LargeCommunitySetName: "l0", + LargeCommunityList: []string{ + "100:100:100", + "100:300:100", + }, + } + + set, err := NewLargeCommunitySet(c) + assert.Equal(t, err, nil) + + m, err := NewLargeCommunityCondition(config.MatchLargeCommunitySet{ + LargeCommunitySet: "l0", + }) + assert.Equal(t, err, nil) + m.set = set + + assert.Equal(t, m.Evaluate(p, nil), true) + + a, err := NewLargeCommunityAction(config.SetLargeCommunity{ + SetLargeCommunityMethod: config.SetLargeCommunityMethod{ + CommunitiesList: []string{"100:100:100"}, + }, + Options: config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE, + }) + assert.Equal(t, err, nil) + p = a.Apply(p, nil) + + assert.Equal(t, m.Evaluate(p, nil), false) + + a, err = NewLargeCommunityAction(config.SetLargeCommunity{ + SetLargeCommunityMethod: config.SetLargeCommunityMethod{ + CommunitiesList: []string{ + "100:300:100", + "200:100:100", + }, + }, + Options: config.BGP_SET_COMMUNITY_OPTION_TYPE_ADD, + }) + assert.Equal(t, err, nil) + p = a.Apply(p, nil) + + assert.Equal(t, m.Evaluate(p, nil), true) + + a, err = NewLargeCommunityAction(config.SetLargeCommunity{ + SetLargeCommunityMethod: config.SetLargeCommunityMethod{ + CommunitiesList: []string{"^100:"}, + }, + Options: config.BGP_SET_COMMUNITY_OPTION_TYPE_REMOVE, + }) + assert.Equal(t, err, nil) + p = a.Apply(p, nil) + + assert.Equal(t, m.Evaluate(p, nil), false) + + c = config.LargeCommunitySet{ + LargeCommunitySetName: "l1", + LargeCommunityList: []string{ + "200:", + }, + } + + set, err = NewLargeCommunitySet(c) + assert.Equal(t, err, nil) + + m, err = NewLargeCommunityCondition(config.MatchLargeCommunitySet{ + LargeCommunitySet: "l1", + }) + assert.Equal(t, err, nil) + m.set = set + + assert.Equal(t, m.Evaluate(p, nil), true) +} + +func TestAfiSafiInMatchPath(t *testing.T) { + condition, err := NewAfiSafiInCondition([]config.AfiSafiType{config.AFI_SAFI_TYPE_L3VPN_IPV4_UNICAST, config.AFI_SAFI_TYPE_L3VPN_IPV6_UNICAST}) + require.NoError(t, err) + + rtExtCom, err := bgp.ParseExtendedCommunity(bgp.EC_SUBTYPE_ROUTE_TARGET, "100:100") + assert.NoError(t, err) + + prefixVPNv4 := bgp.NewLabeledVPNIPAddrPrefix(0, "1.1.1.0/24", *bgp.NewMPLSLabelStack(), bgp.NewRouteDistinguisherTwoOctetAS(100, 100)) + prefixVPNv6 := bgp.NewLabeledVPNIPv6AddrPrefix(0, "2001:0db8:85a3:0000:0000:8a2e:0370:7334", *bgp.NewMPLSLabelStack(), bgp.NewRouteDistinguisherTwoOctetAS(200, 200)) + prefixRTC := bgp.NewRouteTargetMembershipNLRI(100, nil) + prefixv4 := bgp.NewIPAddrPrefix(0, "1.1.1.0/24") + prefixv6 := bgp.NewIPv6AddrPrefix(0, "2001:0db8:85a3:0000:0000:8a2e:0370:7334") + + pathVPNv4 := NewPath(nil, prefixVPNv4, false, []bgp.PathAttributeInterface{bgp.NewPathAttributeExtendedCommunities([]bgp.ExtendedCommunityInterface{rtExtCom})}, time.Time{}, false) + pathVPNv6 := NewPath(nil, prefixVPNv6, false, []bgp.PathAttributeInterface{bgp.NewPathAttributeExtendedCommunities([]bgp.ExtendedCommunityInterface{rtExtCom})}, time.Time{}, false) + pathv4 := NewPath(nil, prefixv4, false, []bgp.PathAttributeInterface{}, time.Time{}, false) + pathv6 := NewPath(nil, prefixv6, false, []bgp.PathAttributeInterface{}, time.Time{}, false) + pathRTC := NewPath(nil, prefixRTC, false, []bgp.PathAttributeInterface{}, time.Time{}, false) + + type Entry struct { + path *Path + shouldMatch bool + } + + for _, entry := range []Entry{ + {pathVPNv4, true}, + {pathVPNv6, true}, + {pathv4, false}, + {pathv6, false}, + {pathRTC, false}, + } { + assert.Equal(t, condition.Evaluate(entry.path, nil), entry.shouldMatch) + } +} + +func TestMultipleStatementPolicy(t *testing.T) { + r := NewRoutingPolicy() + rp := config.RoutingPolicy{ + PolicyDefinitions: []config.PolicyDefinition{config.PolicyDefinition{ + Name: "p1", + Statements: []config.Statement{ + config.Statement{ + Actions: config.Actions{ + BgpActions: config.BgpActions{ + SetMed: "+100", + }, + }, + }, + config.Statement{ + Actions: config.Actions{ + BgpActions: config.BgpActions{ + SetLocalPref: 100, + }, + }, + }, + }, + }, + }, + } + err := r.reload(rp) + assert.Nil(t, err) + + nlri := bgp.NewIPAddrPrefix(24, "10.10.0.0") + + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65001})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("10.0.0.1") + pattrs := []bgp.PathAttributeInterface{origin, aspath, nexthop} + + path := NewPath(nil, nlri, false, pattrs, time.Now(), false) + + pType, newPath := r.policyMap["p1"].Apply(path, nil) + assert.Equal(t, ROUTE_TYPE_NONE, pType) + med, _ := newPath.GetMed() + assert.Equal(t, med, uint32(100)) + localPref, _ := newPath.GetLocalPref() + assert.Equal(t, localPref, uint32(100)) +} diff --git a/internal/pkg/table/roa.go b/internal/pkg/table/roa.go new file mode 100644 index 00000000..fe08fe54 --- /dev/null +++ b/internal/pkg/table/roa.go @@ -0,0 +1,60 @@ +// Copyright (C) 2016 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + "fmt" + "net" +) + +type IPPrefix struct { + Prefix net.IP + Length uint8 +} + +func (p *IPPrefix) String() string { + return fmt.Sprintf("%s/%d", p.Prefix, p.Length) +} + +type ROA struct { + Family int + Prefix *IPPrefix + MaxLen uint8 + AS uint32 + Src string +} + +func NewROA(family int, prefixByte []byte, prefixLen uint8, maxLen uint8, as uint32, src string) *ROA { + p := make([]byte, len(prefixByte)) + copy(p, prefixByte) + return &ROA{ + Family: family, + Prefix: &IPPrefix{ + Prefix: p, + Length: prefixLen, + }, + MaxLen: maxLen, + AS: as, + Src: src, + } +} + +func (r *ROA) Equal(roa *ROA) bool { + if r.MaxLen == roa.MaxLen && r.Src == roa.Src && r.AS == roa.AS { + return true + } + return false +} diff --git a/internal/pkg/table/table.go b/internal/pkg/table/table.go new file mode 100644 index 00000000..bcde936b --- /dev/null +++ b/internal/pkg/table/table.go @@ -0,0 +1,451 @@ +// Copyright (C) 2014 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + "fmt" + "net" + "strings" + "unsafe" + + "github.com/armon/go-radix" + "github.com/osrg/gobgp/pkg/packet/bgp" + log "github.com/sirupsen/logrus" +) + +type LookupOption uint8 + +const ( + LOOKUP_EXACT LookupOption = iota + LOOKUP_LONGER + LOOKUP_SHORTER +) + +type LookupPrefix struct { + Prefix string + LookupOption +} + +type TableSelectOption struct { + ID string + AS uint32 + LookupPrefixes []*LookupPrefix + VRF *Vrf + adj bool + Best bool + MultiPath bool +} + +type Table struct { + routeFamily bgp.RouteFamily + destinations map[string]*Destination +} + +func NewTable(rf bgp.RouteFamily, dsts ...*Destination) *Table { + t := &Table{ + routeFamily: rf, + destinations: make(map[string]*Destination), + } + for _, dst := range dsts { + t.setDestination(dst) + } + return t +} + +func (t *Table) GetRoutefamily() bgp.RouteFamily { + return t.routeFamily +} + +func (t *Table) deletePathsByVrf(vrf *Vrf) []*Path { + pathList := make([]*Path, 0) + for _, dest := range t.destinations { + for _, p := range dest.knownPathList { + var rd bgp.RouteDistinguisherInterface + nlri := p.GetNlri() + switch nlri.(type) { + case *bgp.LabeledVPNIPAddrPrefix: + rd = nlri.(*bgp.LabeledVPNIPAddrPrefix).RD + case *bgp.LabeledVPNIPv6AddrPrefix: + rd = nlri.(*bgp.LabeledVPNIPv6AddrPrefix).RD + case *bgp.EVPNNLRI: + rd = nlri.(*bgp.EVPNNLRI).RD() + default: + return pathList + } + if p.IsLocal() && vrf.Rd.String() == rd.String() { + pathList = append(pathList, p.Clone(true)) + break + } + } + } + return pathList +} + +func (t *Table) deleteRTCPathsByVrf(vrf *Vrf, vrfs map[string]*Vrf) []*Path { + pathList := make([]*Path, 0) + if t.routeFamily != bgp.RF_RTC_UC { + return pathList + } + for _, target := range vrf.ImportRt { + lhs := target.String() + for _, dest := range t.destinations { + nlri := dest.GetNlri().(*bgp.RouteTargetMembershipNLRI) + rhs := nlri.RouteTarget.String() + if lhs == rhs && isLastTargetUser(vrfs, target) { + for _, p := range dest.knownPathList { + if p.IsLocal() { + pathList = append(pathList, p.Clone(true)) + break + } + } + } + } + } + return pathList +} + +func (t *Table) deleteDestByNlri(nlri bgp.AddrPrefixInterface) *Destination { + if dst := t.GetDestination(nlri); dst != nil { + t.deleteDest(dst) + return dst + } + return nil +} + +func (t *Table) deleteDest(dest *Destination) { + destinations := t.GetDestinations() + delete(destinations, t.tableKey(dest.GetNlri())) + if len(destinations) == 0 { + t.destinations = make(map[string]*Destination) + } +} + +func (t *Table) validatePath(path *Path) { + if path == nil { + log.WithFields(log.Fields{ + "Topic": "Table", + "Key": t.routeFamily, + }).Error("path is nil") + } + if path.GetRouteFamily() != t.routeFamily { + log.WithFields(log.Fields{ + "Topic": "Table", + "Key": t.routeFamily, + "Prefix": path.GetNlri().String(), + "ReceivedRf": path.GetRouteFamily().String(), + }).Error("Invalid path. RouteFamily mismatch") + } + if attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH); attr != nil { + pathParam := attr.(*bgp.PathAttributeAsPath).Value + for _, as := range pathParam { + _, y := as.(*bgp.As4PathParam) + if !y { + log.WithFields(log.Fields{ + "Topic": "Table", + "Key": t.routeFamily, + "As": as, + }).Fatal("AsPathParam must be converted to As4PathParam") + } + } + } + if attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_AS4_PATH); attr != nil { + log.WithFields(log.Fields{ + "Topic": "Table", + "Key": t.routeFamily, + }).Fatal("AS4_PATH must be converted to AS_PATH") + } + if path.GetNlri() == nil { + log.WithFields(log.Fields{ + "Topic": "Table", + "Key": t.routeFamily, + }).Fatal("path's nlri is nil") + } +} + +func (t *Table) getOrCreateDest(nlri bgp.AddrPrefixInterface) *Destination { + dest := t.GetDestination(nlri) + // If destination for given prefix does not exist we create it. + if dest == nil { + log.WithFields(log.Fields{ + "Topic": "Table", + "Nlri": nlri, + }).Debugf("create Destination") + dest = NewDestination(nlri, 64) + t.setDestination(dest) + } + return dest +} + +func (t *Table) GetDestinations() map[string]*Destination { + return t.destinations +} +func (t *Table) setDestinations(destinations map[string]*Destination) { + t.destinations = destinations +} +func (t *Table) GetDestination(nlri bgp.AddrPrefixInterface) *Destination { + dest, ok := t.destinations[t.tableKey(nlri)] + if ok { + return dest + } else { + return nil + } +} + +func (t *Table) GetLongerPrefixDestinations(key string) ([]*Destination, error) { + results := make([]*Destination, 0, len(t.GetDestinations())) + switch t.routeFamily { + case bgp.RF_IPv4_UC, bgp.RF_IPv6_UC, bgp.RF_IPv4_MPLS, bgp.RF_IPv6_MPLS: + _, prefix, err := net.ParseCIDR(key) + if err != nil { + return nil, err + } + k := CidrToRadixkey(prefix.String()) + r := radix.New() + for _, dst := range t.GetDestinations() { + r.Insert(AddrToRadixkey(dst.nlri), dst) + } + r.WalkPrefix(k, func(s string, v interface{}) bool { + results = append(results, v.(*Destination)) + return false + }) + default: + for _, dst := range t.GetDestinations() { + results = append(results, dst) + } + } + return results, nil +} + +func (t *Table) GetEvpnDestinationsWithRouteType(typ string) ([]*Destination, error) { + var routeType uint8 + switch strings.ToLower(typ) { + case "a-d": + routeType = bgp.EVPN_ROUTE_TYPE_ETHERNET_AUTO_DISCOVERY + case "macadv": + routeType = bgp.EVPN_ROUTE_TYPE_MAC_IP_ADVERTISEMENT + case "multicast": + routeType = bgp.EVPN_INCLUSIVE_MULTICAST_ETHERNET_TAG + case "esi": + routeType = bgp.EVPN_ETHERNET_SEGMENT_ROUTE + case "prefix": + routeType = bgp.EVPN_IP_PREFIX + default: + return nil, fmt.Errorf("unsupported evpn route type: %s", typ) + } + destinations := t.GetDestinations() + results := make([]*Destination, 0, len(destinations)) + switch t.routeFamily { + case bgp.RF_EVPN: + for _, dst := range destinations { + if nlri, ok := dst.nlri.(*bgp.EVPNNLRI); !ok { + return nil, fmt.Errorf("invalid evpn nlri type detected: %T", dst.nlri) + } else if nlri.RouteType == routeType { + results = append(results, dst) + } + } + default: + for _, dst := range destinations { + results = append(results, dst) + } + } + return results, nil +} + +func (t *Table) setDestination(dst *Destination) { + t.destinations[t.tableKey(dst.nlri)] = dst +} + +func (t *Table) tableKey(nlri bgp.AddrPrefixInterface) string { + switch T := nlri.(type) { + case *bgp.IPAddrPrefix: + b := make([]byte, 5) + copy(b, T.Prefix.To4()) + b[4] = T.Length + return *(*string)(unsafe.Pointer(&b)) + case *bgp.IPv6AddrPrefix: + b := make([]byte, 17) + copy(b, T.Prefix.To16()) + b[16] = T.Length + return *(*string)(unsafe.Pointer(&b)) + } + return nlri.String() +} + +func (t *Table) Bests(id string, as uint32) []*Path { + paths := make([]*Path, 0, len(t.destinations)) + for _, dst := range t.destinations { + path := dst.GetBestPath(id, as) + if path != nil { + paths = append(paths, path) + } + } + return paths +} + +func (t *Table) MultiBests(id string) [][]*Path { + paths := make([][]*Path, 0, len(t.destinations)) + for _, dst := range t.destinations { + path := dst.GetMultiBestPath(id) + if path != nil { + paths = append(paths, path) + } + } + return paths +} + +func (t *Table) GetKnownPathList(id string, as uint32) []*Path { + paths := make([]*Path, 0, len(t.destinations)) + for _, dst := range t.destinations { + paths = append(paths, dst.GetKnownPathList(id, as)...) + } + return paths +} + +func (t *Table) Select(option ...TableSelectOption) (*Table, error) { + id := GLOBAL_RIB_NAME + var vrf *Vrf + adj := false + prefixes := make([]*LookupPrefix, 0, len(option)) + best := false + mp := false + as := uint32(0) + for _, o := range option { + if o.ID != "" { + id = o.ID + } + if o.VRF != nil { + vrf = o.VRF + } + adj = o.adj + prefixes = append(prefixes, o.LookupPrefixes...) + best = o.Best + mp = o.MultiPath + as = o.AS + } + dOption := DestinationSelectOption{ID: id, AS: as, VRF: vrf, adj: adj, Best: best, MultiPath: mp} + r := &Table{ + routeFamily: t.routeFamily, + destinations: make(map[string]*Destination), + } + + if len(prefixes) != 0 { + switch t.routeFamily { + case bgp.RF_IPv4_UC, bgp.RF_IPv6_UC: + f := func(prefixStr string) bool { + var nlri bgp.AddrPrefixInterface + if t.routeFamily == bgp.RF_IPv4_UC { + nlri, _ = bgp.NewPrefixFromRouteFamily(bgp.AFI_IP, bgp.SAFI_UNICAST, prefixStr) + } else { + nlri, _ = bgp.NewPrefixFromRouteFamily(bgp.AFI_IP6, bgp.SAFI_UNICAST, prefixStr) + } + if dst := t.GetDestination(nlri); dst != nil { + if d := dst.Select(dOption); d != nil { + r.setDestination(d) + return true + } + } + return false + } + + for _, p := range prefixes { + key := p.Prefix + switch p.LookupOption { + case LOOKUP_LONGER: + ds, err := t.GetLongerPrefixDestinations(key) + if err != nil { + return nil, err + } + for _, dst := range ds { + if d := dst.Select(dOption); d != nil { + r.setDestination(d) + } + } + case LOOKUP_SHORTER: + addr, prefix, err := net.ParseCIDR(key) + if err != nil { + return nil, err + } + ones, _ := prefix.Mask.Size() + for i := ones; i >= 0; i-- { + _, prefix, _ := net.ParseCIDR(fmt.Sprintf("%s/%d", addr.String(), i)) + f(prefix.String()) + } + default: + if host := net.ParseIP(key); host != nil { + masklen := 32 + if t.routeFamily == bgp.RF_IPv6_UC { + masklen = 128 + } + for i := masklen; i >= 0; i-- { + _, prefix, err := net.ParseCIDR(fmt.Sprintf("%s/%d", key, i)) + if err != nil { + return nil, err + } + if f(prefix.String()) { + break + } + } + } else { + f(key) + } + } + } + case bgp.RF_EVPN: + for _, p := range prefixes { + // Uses LookupPrefix.Prefix as EVPN Route Type string + ds, err := t.GetEvpnDestinationsWithRouteType(p.Prefix) + if err != nil { + return nil, err + } + for _, dst := range ds { + if d := dst.Select(dOption); d != nil { + r.setDestination(d) + } + } + } + default: + return nil, fmt.Errorf("route filtering is not supported for this family") + } + } else { + for _, dst := range t.GetDestinations() { + if d := dst.Select(dOption); d != nil { + r.setDestination(d) + } + } + } + return r, nil +} + +type TableInfo struct { + NumDestination int + NumPath int + NumAccepted int +} + +func (t *Table) Info(id string, as uint32) *TableInfo { + var numD, numP int + for _, d := range t.destinations { + ps := d.GetKnownPathList(id, as) + if len(ps) > 0 { + numD += 1 + numP += len(ps) + } + } + return &TableInfo{ + NumDestination: numD, + NumPath: numP, + } +} diff --git a/internal/pkg/table/table_manager.go b/internal/pkg/table/table_manager.go new file mode 100644 index 00000000..e10f4d6a --- /dev/null +++ b/internal/pkg/table/table_manager.go @@ -0,0 +1,356 @@ +// Copyright (C) 2014 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + "bytes" + "fmt" + "net" + "time" + + "github.com/osrg/gobgp/pkg/packet/bgp" + + farm "github.com/dgryski/go-farm" + log "github.com/sirupsen/logrus" +) + +const ( + GLOBAL_RIB_NAME = "global" +) + +func ProcessMessage(m *bgp.BGPMessage, peerInfo *PeerInfo, timestamp time.Time) []*Path { + update := m.Body.(*bgp.BGPUpdate) + + if y, f := update.IsEndOfRib(); y { + // this message has no normal updates or withdrawals. + return []*Path{NewEOR(f)} + } + + adds := make([]bgp.AddrPrefixInterface, 0, len(update.NLRI)) + for _, nlri := range update.NLRI { + adds = append(adds, nlri) + } + + dels := make([]bgp.AddrPrefixInterface, 0, len(update.WithdrawnRoutes)) + for _, nlri := range update.WithdrawnRoutes { + dels = append(dels, nlri) + } + + attrs := make([]bgp.PathAttributeInterface, 0, len(update.PathAttributes)) + var reach *bgp.PathAttributeMpReachNLRI + for _, attr := range update.PathAttributes { + switch a := attr.(type) { + case *bgp.PathAttributeMpReachNLRI: + reach = a + case *bgp.PathAttributeMpUnreachNLRI: + l := make([]bgp.AddrPrefixInterface, 0, len(a.Value)) + l = append(l, a.Value...) + dels = append(dels, l...) + default: + attrs = append(attrs, attr) + } + } + + listLen := len(adds) + len(dels) + if reach != nil { + listLen += len(reach.Value) + } + + var hash uint32 + if len(adds) > 0 || reach != nil { + total := bytes.NewBuffer(make([]byte, 0)) + for _, a := range attrs { + b, _ := a.Serialize() + total.Write(b) + } + hash = farm.Hash32(total.Bytes()) + } + + pathList := make([]*Path, 0, listLen) + for _, nlri := range adds { + p := NewPath(peerInfo, nlri, false, attrs, timestamp, false) + p.SetHash(hash) + pathList = append(pathList, p) + } + if reach != nil { + reachAttrs := make([]bgp.PathAttributeInterface, len(attrs)+1) + copy(reachAttrs, attrs) + // we sort attributes when creating a bgp message from paths + reachAttrs[len(reachAttrs)-1] = reach + + for _, nlri := range reach.Value { + p := NewPath(peerInfo, nlri, false, reachAttrs, timestamp, false) + p.SetHash(hash) + pathList = append(pathList, p) + } + } + for _, nlri := range dels { + p := NewPath(peerInfo, nlri, true, []bgp.PathAttributeInterface{}, timestamp, false) + pathList = append(pathList, p) + } + return pathList +} + +type TableManager struct { + Tables map[bgp.RouteFamily]*Table + Vrfs map[string]*Vrf + rfList []bgp.RouteFamily +} + +func NewTableManager(rfList []bgp.RouteFamily) *TableManager { + t := &TableManager{ + Tables: make(map[bgp.RouteFamily]*Table), + Vrfs: make(map[string]*Vrf), + rfList: rfList, + } + for _, rf := range rfList { + t.Tables[rf] = NewTable(rf) + } + return t +} + +func (manager *TableManager) GetRFlist() []bgp.RouteFamily { + return manager.rfList +} + +func (manager *TableManager) AddVrf(name string, id uint32, rd bgp.RouteDistinguisherInterface, importRt, exportRt []bgp.ExtendedCommunityInterface, info *PeerInfo) ([]*Path, error) { + if _, ok := manager.Vrfs[name]; ok { + return nil, fmt.Errorf("vrf %s already exists", name) + } + log.WithFields(log.Fields{ + "Topic": "Vrf", + "Key": name, + "Rd": rd, + "ImportRt": importRt, + "ExportRt": exportRt, + }).Debugf("add vrf") + manager.Vrfs[name] = &Vrf{ + Name: name, + Id: id, + Rd: rd, + ImportRt: importRt, + ExportRt: exportRt, + } + msgs := make([]*Path, 0, len(importRt)) + nexthop := "0.0.0.0" + for _, target := range importRt { + nlri := bgp.NewRouteTargetMembershipNLRI(info.AS, target) + pattr := make([]bgp.PathAttributeInterface, 0, 2) + pattr = append(pattr, bgp.NewPathAttributeOrigin(bgp.BGP_ORIGIN_ATTR_TYPE_IGP)) + pattr = append(pattr, bgp.NewPathAttributeMpReachNLRI(nexthop, []bgp.AddrPrefixInterface{nlri})) + msgs = append(msgs, NewPath(info, nlri, false, pattr, time.Now(), false)) + } + return msgs, nil +} + +func (manager *TableManager) DeleteVrf(name string) ([]*Path, error) { + if _, ok := manager.Vrfs[name]; !ok { + return nil, fmt.Errorf("vrf %s not found", name) + } + msgs := make([]*Path, 0) + vrf := manager.Vrfs[name] + for _, t := range manager.Tables { + msgs = append(msgs, t.deletePathsByVrf(vrf)...) + } + log.WithFields(log.Fields{ + "Topic": "Vrf", + "Key": vrf.Name, + "Rd": vrf.Rd, + "ImportRt": vrf.ImportRt, + "ExportRt": vrf.ExportRt, + }).Debugf("delete vrf") + delete(manager.Vrfs, name) + rtcTable := manager.Tables[bgp.RF_RTC_UC] + msgs = append(msgs, rtcTable.deleteRTCPathsByVrf(vrf, manager.Vrfs)...) + return msgs, nil +} + +func (tm *TableManager) update(newPath *Path) *Update { + t := tm.Tables[newPath.GetRouteFamily()] + t.validatePath(newPath) + dst := t.getOrCreateDest(newPath.GetNlri()) + u := dst.Calculate(newPath) + if len(dst.knownPathList) == 0 { + t.deleteDest(dst) + } + return u +} + +func (manager *TableManager) GetPathListByPeer(info *PeerInfo, rf bgp.RouteFamily) []*Path { + if t, ok := manager.Tables[rf]; ok { + pathList := make([]*Path, 0, len(t.destinations)) + for _, dst := range t.destinations { + for _, p := range dst.knownPathList { + if p.GetSource().Equal(info) { + pathList = append(pathList, p) + } + } + } + return pathList + } + return nil +} + +func (manager *TableManager) Update(newPath *Path) []*Update { + if newPath == nil || newPath.IsEOR() { + return nil + } + + // Except for a special case with EVPN, we'll have one destination. + updates := make([]*Update, 0, 1) + family := newPath.GetRouteFamily() + if _, ok := manager.Tables[family]; ok { + updates = append(updates, manager.update(newPath)) + + if family == bgp.RF_EVPN { + for _, p := range manager.handleMacMobility(newPath) { + updates = append(updates, manager.update(p)) + } + } + } + return updates +} + +// EVPN MAC MOBILITY HANDLING +// +// RFC7432 15. MAC Mobility +// +// A PE receiving a MAC/IP Advertisement route for a MAC address with a +// different Ethernet segment identifier and a higher sequence number +// than that which it had previously advertised withdraws its MAC/IP +// Advertisement route. +func (manager *TableManager) handleMacMobility(path *Path) []*Path { + pathList := make([]*Path, 0) + nlri := path.GetNlri().(*bgp.EVPNNLRI) + if path.IsWithdraw || path.IsLocal() || nlri.RouteType != bgp.EVPN_ROUTE_TYPE_MAC_IP_ADVERTISEMENT { + return nil + } + for _, path2 := range manager.GetPathList(GLOBAL_RIB_NAME, 0, []bgp.RouteFamily{bgp.RF_EVPN}) { + if !path2.IsLocal() || path2.GetNlri().(*bgp.EVPNNLRI).RouteType != bgp.EVPN_ROUTE_TYPE_MAC_IP_ADVERTISEMENT { + continue + } + f := func(p *Path) (bgp.EthernetSegmentIdentifier, net.HardwareAddr, int) { + nlri := p.GetNlri().(*bgp.EVPNNLRI) + d := nlri.RouteTypeData.(*bgp.EVPNMacIPAdvertisementRoute) + ecs := p.GetExtCommunities() + seq := -1 + for _, ec := range ecs { + if t, st := ec.GetTypes(); t == bgp.EC_TYPE_EVPN && st == bgp.EC_SUBTYPE_MAC_MOBILITY { + seq = int(ec.(*bgp.MacMobilityExtended).Sequence) + break + } + } + return d.ESI, d.MacAddress, seq + } + e1, m1, s1 := f(path) + e2, m2, s2 := f(path2) + if bytes.Equal(m1, m2) && !bytes.Equal(e1.Value, e2.Value) && s1 > s2 { + pathList = append(pathList, path2.Clone(true)) + } + } + return pathList +} + +func (manager *TableManager) tables(list ...bgp.RouteFamily) []*Table { + l := make([]*Table, 0, len(manager.Tables)) + if len(list) == 0 { + for _, v := range manager.Tables { + l = append(l, v) + } + return l + } + for _, f := range list { + if t, ok := manager.Tables[f]; ok { + l = append(l, t) + } + } + return l +} + +func (manager *TableManager) getDestinationCount(rfList []bgp.RouteFamily) int { + count := 0 + for _, t := range manager.tables(rfList...) { + count += len(t.GetDestinations()) + } + return count +} + +func (manager *TableManager) GetBestPathList(id string, as uint32, rfList []bgp.RouteFamily) []*Path { + if SelectionOptions.DisableBestPathSelection { + // Note: If best path selection disabled, there is no best path. + return nil + } + paths := make([]*Path, 0, manager.getDestinationCount(rfList)) + for _, t := range manager.tables(rfList...) { + paths = append(paths, t.Bests(id, as)...) + } + return paths +} + +func (manager *TableManager) GetBestMultiPathList(id string, rfList []bgp.RouteFamily) [][]*Path { + if !UseMultiplePaths.Enabled || SelectionOptions.DisableBestPathSelection { + // Note: If multi path not enabled or best path selection disabled, + // there is no best multi path. + return nil + } + paths := make([][]*Path, 0, manager.getDestinationCount(rfList)) + for _, t := range manager.tables(rfList...) { + paths = append(paths, t.MultiBests(id)...) + } + return paths +} + +func (manager *TableManager) GetPathList(id string, as uint32, rfList []bgp.RouteFamily) []*Path { + paths := make([]*Path, 0, manager.getDestinationCount(rfList)) + for _, t := range manager.tables(rfList...) { + paths = append(paths, t.GetKnownPathList(id, as)...) + } + return paths +} + +func (manager *TableManager) GetPathListWithNexthop(id string, rfList []bgp.RouteFamily, nexthop net.IP) []*Path { + paths := make([]*Path, 0, manager.getDestinationCount(rfList)) + for _, rf := range rfList { + if t, ok := manager.Tables[rf]; ok { + for _, path := range t.GetKnownPathList(id, 0) { + if path.GetNexthop().Equal(nexthop) { + paths = append(paths, path) + } + } + } + } + return paths +} + +func (manager *TableManager) GetDestination(path *Path) *Destination { + if path == nil { + return nil + } + family := path.GetRouteFamily() + t, ok := manager.Tables[family] + if !ok { + return nil + } + return t.GetDestination(path.GetNlri()) +} + +func (manager *TableManager) TableInfo(id string, as uint32, family bgp.RouteFamily) (*TableInfo, error) { + t, ok := manager.Tables[family] + if !ok { + return nil, fmt.Errorf("address family %s is not configured", family) + } + return t.Info(id, as), nil +} diff --git a/internal/pkg/table/table_manager_test.go b/internal/pkg/table/table_manager_test.go new file mode 100644 index 00000000..67c26c11 --- /dev/null +++ b/internal/pkg/table/table_manager_test.go @@ -0,0 +1,2282 @@ +// Copyright (C) 2014 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + _ "fmt" + "net" + "testing" + "time" + + "github.com/osrg/gobgp/pkg/packet/bgp" + + "github.com/stretchr/testify/assert" +) + +// process BGPUpdate message +// this function processes only BGPUpdate +func (manager *TableManager) ProcessUpdate(fromPeer *PeerInfo, message *bgp.BGPMessage) ([]*Path, error) { + pathList := make([]*Path, 0) + dsts := make([]*Update, 0) + for _, path := range ProcessMessage(message, fromPeer, time.Now()) { + dsts = append(dsts, manager.Update(path)...) + } + for _, d := range dsts { + b, _, _ := d.GetChanges(GLOBAL_RIB_NAME, 0, false) + pathList = append(pathList, b) + } + return pathList, nil +} + +func peerR1() *PeerInfo { + peer := &PeerInfo{ + AS: 65000, + LocalAS: 65000, + ID: net.ParseIP("10.0.0.3").To4(), + LocalID: net.ParseIP("10.0.0.1").To4(), + Address: net.ParseIP("10.0.0.1").To4(), + } + return peer +} + +func peerR2() *PeerInfo { + peer := &PeerInfo{ + AS: 65100, + LocalAS: 65000, + Address: net.ParseIP("10.0.0.2").To4(), + } + return peer +} + +func peerR3() *PeerInfo { + peer := &PeerInfo{ + AS: 65000, + LocalAS: 65000, + ID: net.ParseIP("10.0.0.2").To4(), + LocalID: net.ParseIP("10.0.0.1").To4(), + Address: net.ParseIP("10.0.0.3").To4(), + } + return peer +} + +// test best path calculation and check the result path is from R1 +func TestProcessBGPUpdate_0_select_onlypath_ipv4(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv4_UC}) + + bgpMessage := update_fromR1() + peer := peerR1() + pList, err := tm.ProcessUpdate(peer, bgpMessage) + assert.Equal(t, len(pList), 1) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv4_UC) + + // check PathAttribute + pathAttributes := bgpMessage.Body.(*bgp.BGPUpdate).PathAttributes + expectedOrigin := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedNexthopAttr := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + pathNexthop := attr.(*bgp.PathAttributeNextHop) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, 4, len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "10.10.10.0/24" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "192.168.50.1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +// test best path calculation and check the result path is from R1 +func TestProcessBGPUpdate_0_select_onlypath_ipv6(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv6_UC}) + + bgpMessage := update_fromR1_ipv6() + peer := peerR1() + pList, err := tm.ProcessUpdate(peer, bgpMessage) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv6_UC) + + // check PathAttribute + pathAttributes := bgpMessage.Body.(*bgp.BGPUpdate).PathAttributes + + expectedNexthopAttr := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + pathNexthop := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, 4, len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "2001:123:123:1::/64" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "2001::192:168:50:1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +// test: compare localpref +func TestProcessBGPUpdate_1_select_high_localpref_ipv4(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv4_UC}) + + // low localpref message + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000}) + nexthop1 := bgp.NewPathAttributeNextHop("192.168.50.1") + med1 := bgp.NewPathAttributeMultiExitDisc(0) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + origin1, aspath1, nexthop1, med1, localpref1, + } + nlri1 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nlri1) + + // high localpref message + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65100, 65000}) + nexthop2 := bgp.NewPathAttributeNextHop("192.168.50.1") + med2 := bgp.NewPathAttributeMultiExitDisc(100) + localpref2 := bgp.NewPathAttributeLocalPref(200) + + pathAttributes2 := []bgp.PathAttributeInterface{ + origin2, aspath2, nexthop2, med2, localpref2, + } + nlri2 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nlri2) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + peer2 := peerR2() + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv4_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + expectedOrigin := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedNexthopAttr := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + pathNexthop := attr.(*bgp.PathAttributeNextHop) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, len(pathAttributes2), len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "10.10.10.0/24" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "192.168.50.1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +func TestProcessBGPUpdate_1_select_high_localpref_ipv6(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv6_UC}) + + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000}) + mp_reach1 := createMpReach("2001::192:168:50:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med1 := bgp.NewPathAttributeMultiExitDisc(100) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + mp_reach1, origin1, aspath1, med1, localpref1, + } + + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nil) + + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65100, 65000}) + mp_reach2 := createMpReach("2001::192:168:100:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med2 := bgp.NewPathAttributeMultiExitDisc(100) + localpref2 := bgp.NewPathAttributeLocalPref(200) + + pathAttributes2 := []bgp.PathAttributeInterface{ + mp_reach2, origin2, aspath2, med2, localpref2, + } + + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nil) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + peer2 := peerR2() + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv6_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + + expectedNexthopAttr := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + pathNexthop := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, 5, len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "2001:123:123:1::/64" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "2001::192:168:100:1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +// test: compare localOrigin +func TestProcessBGPUpdate_2_select_local_origin_ipv4(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv4_UC}) + + // low localpref message + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000}) + nexthop1 := bgp.NewPathAttributeNextHop("192.168.50.1") + med1 := bgp.NewPathAttributeMultiExitDisc(0) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + origin1, aspath1, nexthop1, med1, localpref1, + } + nlri1 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nlri1) + + // high localpref message + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{}) + nexthop2 := bgp.NewPathAttributeNextHop("0.0.0.0") + med2 := bgp.NewPathAttributeMultiExitDisc(100) + localpref2 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes2 := []bgp.PathAttributeInterface{ + origin2, aspath2, nexthop2, med2, localpref2, + } + nlri2 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nlri2) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + var peer2 *PeerInfo = &PeerInfo{ + Address: net.ParseIP("0.0.0.0"), + } + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv4_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + expectedOrigin := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedNexthopAttr := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + pathNexthop := attr.(*bgp.PathAttributeNextHop) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, len(pathAttributes2), len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "10.10.10.0/24" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "0.0.0.0" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +func TestProcessBGPUpdate_2_select_local_origin_ipv6(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv6_UC}) + + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000}) + mp_reach1 := createMpReach("2001::192:168:50:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med1 := bgp.NewPathAttributeMultiExitDisc(100) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + mp_reach1, origin1, aspath1, med1, localpref1, + } + + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nil) + + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{}) + mp_reach2 := createMpReach("::", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med2 := bgp.NewPathAttributeMultiExitDisc(100) + localpref2 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes2 := []bgp.PathAttributeInterface{ + mp_reach2, origin2, aspath2, med2, localpref2, + } + + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nil) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + var peer2 *PeerInfo = &PeerInfo{ + Address: net.ParseIP("0.0.0.0"), + } + + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv6_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + + expectedNexthopAttr := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + pathNexthop := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, 5, len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "2001:123:123:1::/64" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "::" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +// test: compare AS_PATH +func TestProcessBGPUpdate_3_select_aspath_ipv4(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv4_UC}) + + bgpMessage1 := update_fromR2viaR1() + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + bgpMessage2 := update_fromR2() + peer2 := peerR2() + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv4_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + expectedOrigin := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedNexthopAttr := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + pathNexthop := attr.(*bgp.PathAttributeNextHop) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, 4, len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "20.20.20.0/24" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "192.168.100.1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +func TestProcessBGPUpdate_3_select_aspath_ipv6(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv6_UC}) + + bgpMessage1 := update_fromR2viaR1_ipv6() + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + bgpMessage2 := update_fromR2_ipv6() + peer2 := peerR2() + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv6_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + + expectedNexthopAttr := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + pathNexthop := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, 4, len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "2002:223:123:1::/64" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "2001::192:168:100:1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +// test: compare Origin +func TestProcessBGPUpdate_4_select_low_origin_ipv4(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv4_UC}) + + // low origin message + origin1 := bgp.NewPathAttributeOrigin(1) + aspath1 := createAsPathAttribute([]uint32{65200, 65000}) + nexthop1 := bgp.NewPathAttributeNextHop("192.168.50.1") + med1 := bgp.NewPathAttributeMultiExitDisc(100) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + origin1, aspath1, nexthop1, med1, localpref1, + } + nlri1 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nlri1) + + // high origin message + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65100, 65000}) + nexthop2 := bgp.NewPathAttributeNextHop("192.168.100.1") + med2 := bgp.NewPathAttributeMultiExitDisc(100) + localpref2 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes2 := []bgp.PathAttributeInterface{ + origin2, aspath2, nexthop2, med2, localpref2, + } + nlri2 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nlri2) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + peer2 := peerR2() + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv4_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + expectedOrigin := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedNexthopAttr := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + pathNexthop := attr.(*bgp.PathAttributeNextHop) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, len(pathAttributes2), len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "10.10.10.0/24" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "192.168.100.1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +func TestProcessBGPUpdate_4_select_low_origin_ipv6(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv6_UC}) + + origin1 := bgp.NewPathAttributeOrigin(1) + aspath1 := createAsPathAttribute([]uint32{65200, 65000}) + mp_reach1 := createMpReach("2001::192:168:50:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med1 := bgp.NewPathAttributeMultiExitDisc(100) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + mp_reach1, origin1, aspath1, med1, localpref1, + } + + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nil) + + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65100, 65000}) + mp_reach2 := createMpReach("2001::192:168:100:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med2 := bgp.NewPathAttributeMultiExitDisc(100) + localpref2 := bgp.NewPathAttributeLocalPref(200) + + pathAttributes2 := []bgp.PathAttributeInterface{ + mp_reach2, origin2, aspath2, med2, localpref2, + } + + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nil) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + peer2 := peerR2() + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv6_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + + expectedNexthopAttr := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + pathNexthop := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, 5, len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "2001:123:123:1::/64" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "2001::192:168:100:1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +// test: compare MED +func TestProcessBGPUpdate_5_select_low_med_ipv4(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv4_UC}) + + // low origin message + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65200, 65000}) + nexthop1 := bgp.NewPathAttributeNextHop("192.168.50.1") + med1 := bgp.NewPathAttributeMultiExitDisc(500) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + origin1, aspath1, nexthop1, med1, localpref1, + } + nlri1 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nlri1) + + // high origin message + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65100, 65000}) + nexthop2 := bgp.NewPathAttributeNextHop("192.168.100.1") + med2 := bgp.NewPathAttributeMultiExitDisc(100) + localpref2 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes2 := []bgp.PathAttributeInterface{ + origin2, aspath2, nexthop2, med2, localpref2, + } + nlri2 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nlri2) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + peer2 := peerR2() + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv4_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + expectedOrigin := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedNexthopAttr := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + pathNexthop := attr.(*bgp.PathAttributeNextHop) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, len(pathAttributes2), len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "10.10.10.0/24" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "192.168.100.1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +func TestProcessBGPUpdate_5_select_low_med_ipv6(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv6_UC}) + + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65200, 65000}) + mp_reach1 := createMpReach("2001::192:168:50:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med1 := bgp.NewPathAttributeMultiExitDisc(500) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + mp_reach1, origin1, aspath1, med1, localpref1, + } + + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nil) + + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65100, 65000}) + mp_reach2 := createMpReach("2001::192:168:100:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med2 := bgp.NewPathAttributeMultiExitDisc(200) + localpref2 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes2 := []bgp.PathAttributeInterface{ + mp_reach2, origin2, aspath2, med2, localpref2, + } + + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nil) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + peer2 := peerR2() + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv6_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + + expectedNexthopAttr := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + pathNexthop := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, 5, len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "2001:123:123:1::/64" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "2001::192:168:100:1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +// test: compare AS_NUMBER(prefer eBGP path) +func TestProcessBGPUpdate_6_select_ebgp_path_ipv4(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv4_UC}) + + // low origin message + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000, 65200}) + nexthop1 := bgp.NewPathAttributeNextHop("192.168.50.1") + med1 := bgp.NewPathAttributeMultiExitDisc(200) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + origin1, aspath1, nexthop1, med1, localpref1, + } + nlri1 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nlri1) + + // high origin message + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65100, 65000}) + nexthop2 := bgp.NewPathAttributeNextHop("192.168.100.1") + med2 := bgp.NewPathAttributeMultiExitDisc(200) + localpref2 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes2 := []bgp.PathAttributeInterface{ + origin2, aspath2, nexthop2, med2, localpref2, + } + nlri2 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nlri2) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + peer2 := peerR2() + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv4_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + expectedOrigin := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedNexthopAttr := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + pathNexthop := attr.(*bgp.PathAttributeNextHop) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, len(pathAttributes2), len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "10.10.10.0/24" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "192.168.100.1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +func TestProcessBGPUpdate_6_select_ebgp_path_ipv6(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv6_UC}) + + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000, 65200}) + mp_reach1 := createMpReach("2001::192:168:50:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med1 := bgp.NewPathAttributeMultiExitDisc(200) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + mp_reach1, origin1, aspath1, med1, localpref1, + } + + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nil) + + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65100, 65200}) + mp_reach2 := createMpReach("2001::192:168:100:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med2 := bgp.NewPathAttributeMultiExitDisc(200) + localpref2 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes2 := []bgp.PathAttributeInterface{ + mp_reach2, origin2, aspath2, med2, localpref2, + } + + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nil) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + peer2 := peerR2() + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv6_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + + expectedNexthopAttr := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + pathNexthop := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, 5, len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "2001:123:123:1::/64" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "2001::192:168:100:1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +// test: compare IGP cost -> N/A + +// test: compare Router ID +func TestProcessBGPUpdate_7_select_low_routerid_path_ipv4(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv4_UC}) + SelectionOptions.ExternalCompareRouterId = true + + // low origin message + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000, 65200}) + nexthop1 := bgp.NewPathAttributeNextHop("192.168.50.1") + med1 := bgp.NewPathAttributeMultiExitDisc(200) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + origin1, aspath1, nexthop1, med1, localpref1, + } + nlri1 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nlri1) + + // high origin message + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65000, 65100}) + nexthop2 := bgp.NewPathAttributeNextHop("192.168.100.1") + med2 := bgp.NewPathAttributeMultiExitDisc(200) + localpref2 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes2 := []bgp.PathAttributeInterface{ + origin2, aspath2, nexthop2, med2, localpref2, + } + nlri2 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nlri2) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + peer3 := peerR3() + pList, err = tm.ProcessUpdate(peer3, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv4_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + expectedOrigin := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedNexthopAttr := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + pathNexthop := attr.(*bgp.PathAttributeNextHop) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, len(pathAttributes2), len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "10.10.10.0/24" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "192.168.100.1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +func TestProcessBGPUpdate_7_select_low_routerid_path_ipv6(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv6_UC}) + + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000, 65200}) + mp_reach1 := createMpReach("2001::192:168:50:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med1 := bgp.NewPathAttributeMultiExitDisc(200) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + mp_reach1, origin1, aspath1, med1, localpref1, + } + + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nil) + + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65100, 65200}) + mp_reach2 := createMpReach("2001::192:168:100:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med2 := bgp.NewPathAttributeMultiExitDisc(200) + localpref2 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes2 := []bgp.PathAttributeInterface{ + mp_reach2, origin2, aspath2, med2, localpref2, + } + + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nil) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + peer3 := peerR3() + pList, err = tm.ProcessUpdate(peer3, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv6_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + + expectedNexthopAttr := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + pathNexthop := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, 5, len(path.GetPathAttrs())) + + // check destination + expectedPrefix := "2001:123:123:1::/64" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "2001::192:168:100:1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +// test: withdraw and mpunreach path +func TestProcessBGPUpdate_8_withdraw_path_ipv4(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv4_UC}) + + // path1 + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000}) + nexthop1 := bgp.NewPathAttributeNextHop("192.168.50.1") + med1 := bgp.NewPathAttributeMultiExitDisc(200) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + origin1, aspath1, nexthop1, med1, localpref1, + } + nlri1 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nlri1) + + // path 2 + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65100, 65000}) + nexthop2 := bgp.NewPathAttributeNextHop("192.168.100.1") + med2 := bgp.NewPathAttributeMultiExitDisc(200) + localpref2 := bgp.NewPathAttributeLocalPref(200) + + pathAttributes2 := []bgp.PathAttributeInterface{ + origin2, aspath2, nexthop2, med2, localpref2, + } + nlri2 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nlri2) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + peer2 := peerR2() + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv4_UC) + + // check PathAttribute + checkPattr := func(expected *bgp.BGPMessage, actual *Path) { + pathAttributes := expected.Body.(*bgp.BGPUpdate).PathAttributes + expectedOrigin := pathAttributes[0] + attr := actual.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[1] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedNexthopAttr := pathAttributes[2] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + pathNexthop := attr.(*bgp.PathAttributeNextHop) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedMed := pathAttributes[3] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, len(pathAttributes), len(path.GetPathAttrs())) + } + checkPattr(bgpMessage2, path) + // check destination + expectedPrefix := "10.10.10.0/24" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "192.168.100.1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + + //withdraw path + w1 := bgp.NewIPAddrPrefix(24, "10.10.10.0") + w := []*bgp.IPAddrPrefix{w1} + bgpMessage3 := bgp.NewBGPUpdateMessage(w, nil, nil) + + pList, err = tm.ProcessUpdate(peer2, bgpMessage3) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + path = pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv4_UC) + + checkPattr(bgpMessage1, path) + // check destination + expectedPrefix = "10.10.10.0/24" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop = "192.168.50.1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) +} + +// TODO MP_UNREACH +func TestProcessBGPUpdate_8_mpunreach_path_ipv6(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv6_UC}) + + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000}) + mp_reach1 := createMpReach("2001::192:168:50:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med1 := bgp.NewPathAttributeMultiExitDisc(200) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + mp_reach1, origin1, aspath1, med1, localpref1, + } + + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nil) + + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65100, 65000}) + mp_reach2 := createMpReach("2001::192:168:100:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med2 := bgp.NewPathAttributeMultiExitDisc(200) + localpref2 := bgp.NewPathAttributeLocalPref(200) + + pathAttributes2 := []bgp.PathAttributeInterface{ + mp_reach2, origin2, aspath2, med2, localpref2, + } + + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nil) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + peer2 := peerR2() + pList, err = tm.ProcessUpdate(peer2, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv6_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + + expectedNexthopAttr := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + pathNexthop := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute + checkPattr := func(expected *bgp.BGPMessage, actual *Path) { + pathAttributes := expected.Body.(*bgp.BGPUpdate).PathAttributes + + expectedNexthopAttr := pathAttributes[0] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + pathNexthop := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[1] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[2] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[3] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + // check PathAttribute length + assert.Equal(t, len(pathAttributes), len(path.GetPathAttrs())) + } + + checkPattr(bgpMessage2, path) + + // check destination + expectedPrefix := "2001:123:123:1::/64" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "2001::192:168:100:1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + + //mpunreach path + mp_unreach := createMpUNReach("2001:123:123:1::", 64) + bgpMessage3 := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{mp_unreach}, nil) + + pList, err = tm.ProcessUpdate(peer2, bgpMessage3) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + path = pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv6_UC) + + checkPattr(bgpMessage1, path) + // check destination + expectedPrefix = "2001:123:123:1::/64" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop = "2001::192:168:50:1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +// handle bestpath lost +func TestProcessBGPUpdate_bestpath_lost_ipv4(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv4_UC}) + + // path1 + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000}) + nexthop1 := bgp.NewPathAttributeNextHop("192.168.50.1") + med1 := bgp.NewPathAttributeMultiExitDisc(200) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + origin1, aspath1, nexthop1, med1, localpref1, + } + nlri1 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nlri1) + + // path 1 withdraw + w1 := bgp.NewIPAddrPrefix(24, "10.10.10.0") + w := []*bgp.IPAddrPrefix{w1} + bgpMessage1_w := bgp.NewBGPUpdateMessage(w, nil, nil) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + pList, err = tm.ProcessUpdate(peer1, bgpMessage1_w) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, true) + assert.NoError(t, err) + + // check old best path + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv4_UC) + + // check PathAttribute + checkPattr := func(expected *bgp.BGPMessage, actual *Path) { + pathAttributes := expected.Body.(*bgp.BGPUpdate).PathAttributes + expectedOrigin := pathAttributes[0] + attr := actual.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[1] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedNexthopAttr := pathAttributes[2] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + pathNexthop := attr.(*bgp.PathAttributeNextHop) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedMed := pathAttributes[3] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, len(pathAttributes), len(path.GetPathAttrs())) + } + + checkPattr(bgpMessage1, path) + // check destination + expectedPrefix := "10.10.10.0/24" + assert.Equal(t, expectedPrefix, path.getPrefix()) +} + +func TestProcessBGPUpdate_bestpath_lost_ipv6(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv6_UC}) + + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000}) + mp_reach1 := createMpReach("2001::192:168:50:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med1 := bgp.NewPathAttributeMultiExitDisc(200) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + mp_reach1, origin1, aspath1, med1, localpref1, + } + + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nil) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // path1 mpunreach + mp_unreach := createMpUNReach("2001:123:123:1::", 64) + bgpMessage1_w := bgp.NewBGPUpdateMessage(nil, []bgp.PathAttributeInterface{mp_unreach}, nil) + + pList, err = tm.ProcessUpdate(peer1, bgpMessage1_w) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, true) + assert.NoError(t, err) + + // check old best path + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv6_UC) + + // check PathAttribute + checkPattr := func(expected *bgp.BGPMessage, actual *Path) { + pathAttributes := expected.Body.(*bgp.BGPUpdate).PathAttributes + + expectedNexthopAttr := pathAttributes[0] + attr := actual.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + pathNexthop := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[1] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[2] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[3] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + // check PathAttribute length + assert.Equal(t, len(pathAttributes), len(path.GetPathAttrs())) + } + + checkPattr(bgpMessage1, path) + + // check destination + expectedPrefix := "2001:123:123:1::/64" + assert.Equal(t, expectedPrefix, path.getPrefix()) +} + +// test: implicit withdrawal case +func TestProcessBGPUpdate_implicit_withdrwal_ipv4(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv4_UC}) + + // path1 + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000, 65100, 65200}) + nexthop1 := bgp.NewPathAttributeNextHop("192.168.50.1") + med1 := bgp.NewPathAttributeMultiExitDisc(200) + localpref1 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes1 := []bgp.PathAttributeInterface{ + origin1, aspath1, nexthop1, med1, localpref1, + } + nlri1 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nlri1) + + // path 1 from same peer but short AS_PATH + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65000, 65100}) + nexthop2 := bgp.NewPathAttributeNextHop("192.168.50.1") + med2 := bgp.NewPathAttributeMultiExitDisc(200) + localpref2 := bgp.NewPathAttributeLocalPref(100) + + pathAttributes2 := []bgp.PathAttributeInterface{ + origin2, aspath2, nexthop2, med2, localpref2, + } + nlri2 := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nlri2) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + pList, err = tm.ProcessUpdate(peer1, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv4_UC) + + // check PathAttribute + checkPattr := func(expected *bgp.BGPMessage, actual *Path) { + pathAttributes := expected.Body.(*bgp.BGPUpdate).PathAttributes + expectedOrigin := pathAttributes[0] + attr := actual.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[1] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedNexthopAttr := pathAttributes[2] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + pathNexthop := attr.(*bgp.PathAttributeNextHop) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedMed := pathAttributes[3] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, len(pathAttributes), len(path.GetPathAttrs())) + } + checkPattr(bgpMessage2, path) + // check destination + expectedPrefix := "10.10.10.0/24" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "192.168.50.1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +func TestProcessBGPUpdate_implicit_withdrwal_ipv6(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv6_UC}) + + origin1 := bgp.NewPathAttributeOrigin(0) + aspath1 := createAsPathAttribute([]uint32{65000, 65100, 65200}) + mp_reach1 := createMpReach("2001::192:168:50:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med1 := bgp.NewPathAttributeMultiExitDisc(200) + localpref1 := bgp.NewPathAttributeLocalPref(200) + + pathAttributes1 := []bgp.PathAttributeInterface{ + mp_reach1, origin1, aspath1, med1, localpref1, + } + + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nil) + + origin2 := bgp.NewPathAttributeOrigin(0) + aspath2 := createAsPathAttribute([]uint32{65000, 65100}) + mp_reach2 := createMpReach("2001::192:168:50:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")}) + med2 := bgp.NewPathAttributeMultiExitDisc(200) + localpref2 := bgp.NewPathAttributeLocalPref(200) + + pathAttributes2 := []bgp.PathAttributeInterface{ + mp_reach2, origin2, aspath2, med2, localpref2, + } + + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nil) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + pList, err = tm.ProcessUpdate(peer1, bgpMessage2) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check type + path := pList[0] + assert.Equal(t, path.GetRouteFamily(), bgp.RF_IPv6_UC) + + // check PathAttribute + pathAttributes := bgpMessage2.Body.(*bgp.BGPUpdate).PathAttributes + + expectedNexthopAttr := pathAttributes[0] + attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + pathNexthop := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[1] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[2] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[3] + attr = path.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute + checkPattr := func(expected *bgp.BGPMessage, actual *Path) { + pathAttributes := expected.Body.(*bgp.BGPUpdate).PathAttributes + + expectedNexthopAttr := pathAttributes[0] + attr := actual.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + pathNexthop := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[1] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[2] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[3] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + // check PathAttribute length + assert.Equal(t, len(pathAttributes), len(path.GetPathAttrs())) + } + + checkPattr(bgpMessage2, path) + + // check destination + expectedPrefix := "2001:123:123:1::/64" + assert.Equal(t, expectedPrefix, path.getPrefix()) + // check nexthop + expectedNexthop := "2001::192:168:50:1" + assert.Equal(t, expectedNexthop, path.GetNexthop().String()) + +} + +// check multiple paths +func TestProcessBGPUpdate_multiple_nlri_ipv4(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv4_UC}) + + createPathAttr := func(aspaths []uint32, nh string) []bgp.PathAttributeInterface { + origin := bgp.NewPathAttributeOrigin(0) + aspath := createAsPathAttribute(aspaths) + nexthop := bgp.NewPathAttributeNextHop(nh) + med := bgp.NewPathAttributeMultiExitDisc(200) + localpref := bgp.NewPathAttributeLocalPref(100) + pathAttr := []bgp.PathAttributeInterface{ + origin, aspath, nexthop, med, localpref, + } + return pathAttr + } + + // check PathAttribute + checkPattr := func(expected *bgp.BGPMessage, actual *Path) { + pathAttributes := expected.Body.(*bgp.BGPUpdate).PathAttributes + expectedOrigin := pathAttributes[0] + attr := actual.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[1] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedNexthopAttr := pathAttributes[2] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_NEXT_HOP) + pathNexthop := attr.(*bgp.PathAttributeNextHop) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedMed := pathAttributes[3] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + // check PathAttribute length + assert.Equal(t, len(pathAttributes), len(actual.GetPathAttrs())) + } + + checkBestPathResult := func(rf bgp.RouteFamily, prefix, nexthop string, p *Path, m *bgp.BGPMessage) { + assert.Equal(t, p.GetRouteFamily(), rf) + checkPattr(m, p) + // check destination + assert.Equal(t, prefix, p.getPrefix()) + // check nexthop + assert.Equal(t, nexthop, p.GetNexthop().String()) + } + + // path1 + pathAttributes1 := createPathAttr([]uint32{65000, 65100, 65200}, "192.168.50.1") + nlri1 := []*bgp.IPAddrPrefix{ + bgp.NewIPAddrPrefix(24, "10.10.10.0"), + bgp.NewIPAddrPrefix(24, "20.20.20.0"), + bgp.NewIPAddrPrefix(24, "30.30.30.0"), + bgp.NewIPAddrPrefix(24, "40.40.40.0"), + bgp.NewIPAddrPrefix(24, "50.50.50.0")} + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nlri1) + + // path2 + pathAttributes2 := createPathAttr([]uint32{65000, 65100, 65300}, "192.168.50.1") + nlri2 := []*bgp.IPAddrPrefix{ + bgp.NewIPAddrPrefix(24, "11.11.11.0"), + bgp.NewIPAddrPrefix(24, "22.22.22.0"), + bgp.NewIPAddrPrefix(24, "33.33.33.0"), + bgp.NewIPAddrPrefix(24, "44.44.44.0"), + bgp.NewIPAddrPrefix(24, "55.55.55.0")} + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nlri2) + + // path3 + pathAttributes3 := createPathAttr([]uint32{65000, 65100, 65400}, "192.168.50.1") + nlri3 := []*bgp.IPAddrPrefix{ + bgp.NewIPAddrPrefix(24, "77.77.77.0"), + bgp.NewIPAddrPrefix(24, "88.88.88.0"), + } + bgpMessage3 := bgp.NewBGPUpdateMessage(nil, pathAttributes3, nlri3) + + // path4 + pathAttributes4 := createPathAttr([]uint32{65000, 65100, 65500}, "192.168.50.1") + nlri4 := []*bgp.IPAddrPrefix{ + bgp.NewIPAddrPrefix(24, "99.99.99.0"), + } + bgpMessage4 := bgp.NewBGPUpdateMessage(nil, pathAttributes4, nlri4) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 5, len(pList)) + for _, p := range pList { + assert.Equal(t, p.IsWithdraw, false) + } + assert.NoError(t, err) + + checkBestPathResult(bgp.RF_IPv4_UC, "10.10.10.0/24", "192.168.50.1", pList[0], bgpMessage1) + checkBestPathResult(bgp.RF_IPv4_UC, "20.20.20.0/24", "192.168.50.1", pList[1], bgpMessage1) + checkBestPathResult(bgp.RF_IPv4_UC, "30.30.30.0/24", "192.168.50.1", pList[2], bgpMessage1) + checkBestPathResult(bgp.RF_IPv4_UC, "40.40.40.0/24", "192.168.50.1", pList[3], bgpMessage1) + checkBestPathResult(bgp.RF_IPv4_UC, "50.50.50.0/24", "192.168.50.1", pList[4], bgpMessage1) + + pList, err = tm.ProcessUpdate(peer1, bgpMessage2) + assert.Equal(t, 5, len(pList)) + for _, p := range pList { + assert.Equal(t, p.IsWithdraw, false) + } + assert.NoError(t, err) + + checkBestPathResult(bgp.RF_IPv4_UC, "11.11.11.0/24", "192.168.50.1", pList[0], bgpMessage2) + checkBestPathResult(bgp.RF_IPv4_UC, "22.22.22.0/24", "192.168.50.1", pList[1], bgpMessage2) + checkBestPathResult(bgp.RF_IPv4_UC, "33.33.33.0/24", "192.168.50.1", pList[2], bgpMessage2) + checkBestPathResult(bgp.RF_IPv4_UC, "44.44.44.0/24", "192.168.50.1", pList[3], bgpMessage2) + checkBestPathResult(bgp.RF_IPv4_UC, "55.55.55.0/24", "192.168.50.1", pList[4], bgpMessage2) + + pList, err = tm.ProcessUpdate(peer1, bgpMessage3) + assert.Equal(t, 2, len(pList)) + for _, p := range pList { + assert.Equal(t, p.IsWithdraw, false) + } + assert.NoError(t, err) + + pList, err = tm.ProcessUpdate(peer1, bgpMessage4) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check table + table := tm.Tables[bgp.RF_IPv4_UC] + assert.Equal(t, 13, len(table.GetDestinations())) + +} + +// check multiple paths +func TestProcessBGPUpdate_multiple_nlri_ipv6(t *testing.T) { + + tm := NewTableManager([]bgp.RouteFamily{bgp.RF_IPv6_UC}) + + createPathAttr := func(aspaths []uint32) []bgp.PathAttributeInterface { + origin := bgp.NewPathAttributeOrigin(0) + aspath := createAsPathAttribute(aspaths) + med := bgp.NewPathAttributeMultiExitDisc(100) + localpref := bgp.NewPathAttributeLocalPref(100) + pathAttr := []bgp.PathAttributeInterface{ + origin, aspath, med, localpref, + } + return pathAttr + } + + // check PathAttribute + checkPattr := func(expected *bgp.BGPMessage, actual *Path) { + pathAttributes := expected.Body.(*bgp.BGPUpdate).PathAttributes + pathNexthop := pathAttributes[4] + attr := actual.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI) + expectedNexthopAttr := attr.(*bgp.PathAttributeMpReachNLRI) + assert.Equal(t, expectedNexthopAttr, pathNexthop) + + expectedOrigin := pathAttributes[0] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_ORIGIN) + pathOrigin := attr.(*bgp.PathAttributeOrigin) + assert.Equal(t, expectedOrigin, pathOrigin) + + expectedAsPath := pathAttributes[1] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_AS_PATH) + pathAspath := attr.(*bgp.PathAttributeAsPath) + assert.Equal(t, expectedAsPath, pathAspath) + + expectedMed := pathAttributes[2] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_MULTI_EXIT_DISC) + pathMed := attr.(*bgp.PathAttributeMultiExitDisc) + assert.Equal(t, expectedMed, pathMed) + + expectedLocalpref := pathAttributes[3] + attr = actual.getPathAttr(bgp.BGP_ATTR_TYPE_LOCAL_PREF) + localpref := attr.(*bgp.PathAttributeLocalPref) + assert.Equal(t, expectedLocalpref, localpref) + + // check PathAttribute length + assert.Equal(t, len(pathAttributes), len(actual.GetPathAttrs())) + + } + + checkBestPathResult := func(rf bgp.RouteFamily, prefix, nexthop string, p *Path, m *bgp.BGPMessage) { + assert.Equal(t, p.GetRouteFamily(), rf) + checkPattr(m, p) + // check destination + assert.Equal(t, prefix, p.getPrefix()) + // check nexthop + assert.Equal(t, nexthop, p.GetNexthop().String()) + } + + // path1 + pathAttributes1 := createPathAttr([]uint32{65000, 65100, 65200}) + mpreach1 := createMpReach("2001::192:168:50:1", []bgp.AddrPrefixInterface{ + bgp.NewIPv6AddrPrefix(64, "2001:123:1210:11::"), + bgp.NewIPv6AddrPrefix(64, "2001:123:1220:11::"), + bgp.NewIPv6AddrPrefix(64, "2001:123:1230:11::"), + bgp.NewIPv6AddrPrefix(64, "2001:123:1240:11::"), + bgp.NewIPv6AddrPrefix(64, "2001:123:1250:11::"), + }) + pathAttributes1 = append(pathAttributes1, mpreach1) + bgpMessage1 := bgp.NewBGPUpdateMessage(nil, pathAttributes1, nil) + + // path2 + pathAttributes2 := createPathAttr([]uint32{65000, 65100, 65300}) + mpreach2 := createMpReach("2001::192:168:50:1", []bgp.AddrPrefixInterface{ + bgp.NewIPv6AddrPrefix(64, "2001:123:1211:11::"), + bgp.NewIPv6AddrPrefix(64, "2001:123:1222:11::"), + bgp.NewIPv6AddrPrefix(64, "2001:123:1233:11::"), + bgp.NewIPv6AddrPrefix(64, "2001:123:1244:11::"), + bgp.NewIPv6AddrPrefix(64, "2001:123:1255:11::"), + }) + pathAttributes2 = append(pathAttributes2, mpreach2) + bgpMessage2 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nil) + + // path3 + pathAttributes3 := createPathAttr([]uint32{65000, 65100, 65400}) + mpreach3 := createMpReach("2001::192:168:50:1", []bgp.AddrPrefixInterface{ + bgp.NewIPv6AddrPrefix(64, "2001:123:1277:11::"), + bgp.NewIPv6AddrPrefix(64, "2001:123:1288:11::"), + }) + pathAttributes3 = append(pathAttributes3, mpreach3) + bgpMessage3 := bgp.NewBGPUpdateMessage(nil, pathAttributes3, nil) + + // path4 + pathAttributes4 := createPathAttr([]uint32{65000, 65100, 65500}) + mpreach4 := createMpReach("2001::192:168:50:1", []bgp.AddrPrefixInterface{ + bgp.NewIPv6AddrPrefix(64, "2001:123:1299:11::"), + }) + pathAttributes4 = append(pathAttributes4, mpreach4) + bgpMessage4 := bgp.NewBGPUpdateMessage(nil, pathAttributes4, nil) + + peer1 := peerR1() + pList, err := tm.ProcessUpdate(peer1, bgpMessage1) + assert.Equal(t, 5, len(pList)) + for _, p := range pList { + assert.Equal(t, p.IsWithdraw, false) + } + assert.NoError(t, err) + + checkBestPathResult(bgp.RF_IPv6_UC, "2001:123:1210:11::/64", "2001::192:168:50:1", pList[0], bgpMessage1) + checkBestPathResult(bgp.RF_IPv6_UC, "2001:123:1220:11::/64", "2001::192:168:50:1", pList[1], bgpMessage1) + checkBestPathResult(bgp.RF_IPv6_UC, "2001:123:1230:11::/64", "2001::192:168:50:1", pList[2], bgpMessage1) + checkBestPathResult(bgp.RF_IPv6_UC, "2001:123:1240:11::/64", "2001::192:168:50:1", pList[3], bgpMessage1) + checkBestPathResult(bgp.RF_IPv6_UC, "2001:123:1250:11::/64", "2001::192:168:50:1", pList[4], bgpMessage1) + + pList, err = tm.ProcessUpdate(peer1, bgpMessage2) + assert.Equal(t, 5, len(pList)) + for _, p := range pList { + assert.Equal(t, p.IsWithdraw, false) + } + assert.NoError(t, err) + + checkBestPathResult(bgp.RF_IPv6_UC, "2001:123:1211:11::/64", "2001::192:168:50:1", pList[0], bgpMessage2) + checkBestPathResult(bgp.RF_IPv6_UC, "2001:123:1222:11::/64", "2001::192:168:50:1", pList[1], bgpMessage2) + checkBestPathResult(bgp.RF_IPv6_UC, "2001:123:1233:11::/64", "2001::192:168:50:1", pList[2], bgpMessage2) + checkBestPathResult(bgp.RF_IPv6_UC, "2001:123:1244:11::/64", "2001::192:168:50:1", pList[3], bgpMessage2) + checkBestPathResult(bgp.RF_IPv6_UC, "2001:123:1255:11::/64", "2001::192:168:50:1", pList[4], bgpMessage2) + + pList, err = tm.ProcessUpdate(peer1, bgpMessage3) + assert.Equal(t, 2, len(pList)) + for _, p := range pList { + assert.Equal(t, p.IsWithdraw, false) + } + assert.NoError(t, err) + + pList, err = tm.ProcessUpdate(peer1, bgpMessage4) + assert.Equal(t, 1, len(pList)) + assert.Equal(t, pList[0].IsWithdraw, false) + assert.NoError(t, err) + + // check table + table := tm.Tables[bgp.RF_IPv6_UC] + assert.Equal(t, 13, len(table.GetDestinations())) + +} + +func TestProcessBGPUpdate_Timestamp(t *testing.T) { + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAs4PathParam(2, []uint32{65000})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.50.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + + adjRib := NewAdjRib([]bgp.RouteFamily{bgp.RF_IPv4_UC, bgp.RF_IPv6_UC}) + m1 := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + peer := peerR1() + pList1 := ProcessMessage(m1, peer, time.Now()) + path1 := pList1[0] + t1 := path1.GetTimestamp() + adjRib.Update(pList1) + + m2 := bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) + pList2 := ProcessMessage(m2, peer, time.Now()) + //path2 := pList2[0].(*IPv4Path) + //t2 = path2.timestamp + adjRib.Update(pList2) + + inList := adjRib.PathList([]bgp.RouteFamily{bgp.RF_IPv4_UC}, false) + assert.Equal(t, len(inList), 1) + assert.Equal(t, inList[0].GetTimestamp(), t1) + + med2 := bgp.NewPathAttributeMultiExitDisc(1) + pathAttributes2 := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med2, + } + + m3 := bgp.NewBGPUpdateMessage(nil, pathAttributes2, nlri) + pList3 := ProcessMessage(m3, peer, time.Now()) + t3 := pList3[0].GetTimestamp() + adjRib.Update(pList3) + + inList = adjRib.PathList([]bgp.RouteFamily{bgp.RF_IPv4_UC}, false) + assert.Equal(t, len(inList), 1) + assert.Equal(t, inList[0].GetTimestamp(), t3) +} + +func update_fromR1() *bgp.BGPMessage { + + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAs4PathParam(2, []uint32{65000})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.50.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + + return bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) +} + +func update_fromR1_ipv6() *bgp.BGPMessage { + + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAs4PathParam(2, []uint32{65000})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + + mp_nlri := []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2001:123:123:1::")} + mp_reach := bgp.NewPathAttributeMpReachNLRI("2001::192:168:50:1", mp_nlri) + med := bgp.NewPathAttributeMultiExitDisc(0) + + pathAttributes := []bgp.PathAttributeInterface{ + mp_reach, + origin, + aspath, + med, + } + return bgp.NewBGPUpdateMessage(nil, pathAttributes, nil) +} + +func update_fromR2() *bgp.BGPMessage { + + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAs4PathParam(2, []uint32{65100})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.100.1") + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "20.20.20.0")} + return bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) +} + +func update_fromR2_ipv6() *bgp.BGPMessage { + + origin := bgp.NewPathAttributeOrigin(0) + aspath := createAsPathAttribute([]uint32{65100}) + mp_reach := createMpReach("2001::192:168:100:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2002:223:123:1::")}) + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{ + mp_reach, + origin, + aspath, + med, + } + return bgp.NewBGPUpdateMessage(nil, pathAttributes, nil) +} + +func createAsPathAttribute(ases []uint32) *bgp.PathAttributeAsPath { + aspathParam := []bgp.AsPathParamInterface{bgp.NewAs4PathParam(2, ases)} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + return aspath +} + +func createMpReach(nexthop string, prefix []bgp.AddrPrefixInterface) *bgp.PathAttributeMpReachNLRI { + mp_reach := bgp.NewPathAttributeMpReachNLRI(nexthop, prefix) + return mp_reach +} + +func createMpUNReach(nlri string, len uint8) *bgp.PathAttributeMpUnreachNLRI { + mp_nlri := []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(len, nlri)} + mp_unreach := bgp.NewPathAttributeMpUnreachNLRI(mp_nlri) + return mp_unreach +} + +func update_fromR2viaR1() *bgp.BGPMessage { + + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAs4PathParam(2, []uint32{65000, 65100})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.50.1") + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "20.20.20.0")} + return bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) +} + +func update_fromR2viaR1_ipv6() *bgp.BGPMessage { + + origin := bgp.NewPathAttributeOrigin(0) + aspath := createAsPathAttribute([]uint32{65000, 65100}) + mp_reach := createMpReach("2001::192:168:50:1", + []bgp.AddrPrefixInterface{bgp.NewIPv6AddrPrefix(64, "2002:223:123:1::")}) + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{ + mp_reach, + origin, + aspath, + med, + } + return bgp.NewBGPUpdateMessage(nil, pathAttributes, nil) + +} diff --git a/internal/pkg/table/table_test.go b/internal/pkg/table/table_test.go new file mode 100644 index 00000000..1e91aa6b --- /dev/null +++ b/internal/pkg/table/table_test.go @@ -0,0 +1,180 @@ +// Copyright (C) 2014 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + "testing" + "time" + + "github.com/osrg/gobgp/pkg/packet/bgp" + + "github.com/stretchr/testify/assert" +) + +func TestTableDeleteDestByNlri(t *testing.T) { + peerT := TableCreatePeer() + pathT := TableCreatePath(peerT) + ipv4t := NewTable(bgp.RF_IPv4_UC) + for _, path := range pathT { + dest := NewDestination(path.GetNlri(), 0) + ipv4t.setDestination(dest) + } + gdest := ipv4t.GetDestination(pathT[0].GetNlri()) + rdest := ipv4t.deleteDestByNlri(pathT[0].GetNlri()) + assert.Equal(t, rdest, gdest) +} + +func TestTableDeleteDest(t *testing.T) { + peerT := TableCreatePeer() + pathT := TableCreatePath(peerT) + ipv4t := NewTable(bgp.RF_IPv4_UC) + for _, path := range pathT { + dest := NewDestination(path.GetNlri(), 0) + ipv4t.setDestination(dest) + } + dest := NewDestination(pathT[0].GetNlri(), 0) + ipv4t.setDestination(dest) + ipv4t.deleteDest(dest) + gdest := ipv4t.GetDestination(pathT[0].GetNlri()) + assert.Nil(t, gdest) +} + +func TestTableGetRouteFamily(t *testing.T) { + ipv4t := NewTable(bgp.RF_IPv4_UC) + rf := ipv4t.GetRoutefamily() + assert.Equal(t, rf, bgp.RF_IPv4_UC) +} + +func TestTableSetDestinations(t *testing.T) { + peerT := TableCreatePeer() + pathT := TableCreatePath(peerT) + ipv4t := NewTable(bgp.RF_IPv4_UC) + destinations := make(map[string]*Destination) + for _, path := range pathT { + tableKey := ipv4t.tableKey(path.GetNlri()) + dest := NewDestination(path.GetNlri(), 0) + destinations[tableKey] = dest + } + ipv4t.setDestinations(destinations) + ds := ipv4t.GetDestinations() + assert.Equal(t, ds, destinations) +} +func TestTableGetDestinations(t *testing.T) { + peerT := DestCreatePeer() + pathT := DestCreatePath(peerT) + ipv4t := NewTable(bgp.RF_IPv4_UC) + destinations := make(map[string]*Destination) + for _, path := range pathT { + tableKey := ipv4t.tableKey(path.GetNlri()) + dest := NewDestination(path.GetNlri(), 0) + destinations[tableKey] = dest + } + ipv4t.setDestinations(destinations) + ds := ipv4t.GetDestinations() + assert.Equal(t, ds, destinations) +} + +func TestTableKey(t *testing.T) { + tb := NewTable(bgp.RF_IPv4_UC) + n1, _ := bgp.NewPrefixFromRouteFamily(bgp.AFI_IP, bgp.SAFI_UNICAST, "0.0.0.0/0") + d1 := NewDestination(n1, 0) + n2, _ := bgp.NewPrefixFromRouteFamily(bgp.AFI_IP, bgp.SAFI_UNICAST, "0.0.0.0/1") + d2 := NewDestination(n2, 0) + assert.Equal(t, len(tb.tableKey(d1.GetNlri())), 5) + tb.setDestination(d1) + tb.setDestination(d2) + assert.Equal(t, len(tb.GetDestinations()), 2) +} + +func TableCreatePeer() []*PeerInfo { + peerT1 := &PeerInfo{AS: 65000} + peerT2 := &PeerInfo{AS: 65001} + peerT3 := &PeerInfo{AS: 65002} + peerT := []*PeerInfo{peerT1, peerT2, peerT3} + return peerT +} + +func TableCreatePath(peerT []*PeerInfo) []*Path { + bgpMsgT1 := updateMsgT1() + bgpMsgT2 := updateMsgT2() + bgpMsgT3 := updateMsgT3() + pathT := make([]*Path, 3) + for i, msg := range []*bgp.BGPMessage{bgpMsgT1, bgpMsgT2, bgpMsgT3} { + updateMsgT := msg.Body.(*bgp.BGPUpdate) + nlriList := updateMsgT.NLRI + pathAttributes := updateMsgT.PathAttributes + nlri_info := nlriList[0] + pathT[i] = NewPath(peerT[i], nlri_info, false, pathAttributes, time.Now(), false) + } + return pathT +} + +func updateMsgT1() *bgp.BGPMessage { + + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65000})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.50.1") + med := bgp.NewPathAttributeMultiExitDisc(0) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "10.10.10.0")} + return bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) +} + +func updateMsgT2() *bgp.BGPMessage { + + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65100})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.100.1") + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "20.20.20.0")} + return bgp.NewBGPUpdateMessage(nil, pathAttributes, nlri) +} +func updateMsgT3() *bgp.BGPMessage { + origin := bgp.NewPathAttributeOrigin(0) + aspathParam := []bgp.AsPathParamInterface{bgp.NewAsPathParam(2, []uint16{65100})} + aspath := bgp.NewPathAttributeAsPath(aspathParam) + nexthop := bgp.NewPathAttributeNextHop("192.168.150.1") + med := bgp.NewPathAttributeMultiExitDisc(100) + + pathAttributes := []bgp.PathAttributeInterface{ + origin, + aspath, + nexthop, + med, + } + + nlri := []*bgp.IPAddrPrefix{bgp.NewIPAddrPrefix(24, "30.30.30.0")} + w1 := bgp.NewIPAddrPrefix(23, "40.40.40.0") + withdrawnRoutes := []*bgp.IPAddrPrefix{w1} + return bgp.NewBGPUpdateMessage(withdrawnRoutes, pathAttributes, nlri) +} diff --git a/internal/pkg/table/vrf.go b/internal/pkg/table/vrf.go new file mode 100644 index 00000000..053f85ce --- /dev/null +++ b/internal/pkg/table/vrf.go @@ -0,0 +1,53 @@ +// Copyright (C) 2014-2016 Nippon Telegraph and Telephone Corporation. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package table + +import ( + "github.com/osrg/gobgp/pkg/packet/bgp" +) + +type Vrf struct { + Name string + Id uint32 + Rd bgp.RouteDistinguisherInterface + ImportRt []bgp.ExtendedCommunityInterface + ExportRt []bgp.ExtendedCommunityInterface +} + +func (v *Vrf) Clone() *Vrf { + f := func(rt []bgp.ExtendedCommunityInterface) []bgp.ExtendedCommunityInterface { + l := make([]bgp.ExtendedCommunityInterface, 0, len(rt)) + return append(l, rt...) + } + return &Vrf{ + Name: v.Name, + Id: v.Id, + Rd: v.Rd, + ImportRt: f(v.ImportRt), + ExportRt: f(v.ExportRt), + } +} + +func isLastTargetUser(vrfs map[string]*Vrf, target bgp.ExtendedCommunityInterface) bool { + for _, vrf := range vrfs { + for _, rt := range vrf.ImportRt { + if target.String() == rt.String() { + return false + } + } + } + return true +} |