summaryrefslogtreecommitdiffhomepage
path: root/internal/pkg/table
diff options
context:
space:
mode:
Diffstat (limited to 'internal/pkg/table')
-rw-r--r--internal/pkg/table/adj.go186
-rw-r--r--internal/pkg/table/adj_test.go52
-rw-r--r--internal/pkg/table/destination.go1041
-rw-r--r--internal/pkg/table/destination_test.go442
-rw-r--r--internal/pkg/table/message.go502
-rw-r--r--internal/pkg/table/message_test.go663
-rw-r--r--internal/pkg/table/path.go1179
-rw-r--r--internal/pkg/table/path_test.go365
-rw-r--r--internal/pkg/table/policy.go3994
-rw-r--r--internal/pkg/table/policy_test.go3140
-rw-r--r--internal/pkg/table/roa.go60
-rw-r--r--internal/pkg/table/table.go451
-rw-r--r--internal/pkg/table/table_manager.go356
-rw-r--r--internal/pkg/table/table_manager_test.go2282
-rw-r--r--internal/pkg/table/table_test.go180
-rw-r--r--internal/pkg/table/vrf.go53
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
+}