summaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
authorISHIDA Wataru <ishida.wataru@lab.ntt.co.jp>2015-07-22 13:24:31 +0900
committerFUJITA Tomonori <fujita.tomonori@lab.ntt.co.jp>2015-07-22 14:49:46 +0900
commita4501ccc3f680de0a9f9cc8621256d71d3679478 (patch)
tree1ab8e04fb4c6c39f512be1e9f76c26c78a127ac7
parent3409f6a5443cf0d6cf0d318feba845f237cd14ba (diff)
mrt: support mrt dump
$ gobgp mrt dump 10 -o ./dump Signed-off-by: ISHIDA Wataru <ishida.wataru@lab.ntt.co.jp>
-rw-r--r--api/gobgp.pb.go87
-rw-r--r--api/gobgp.proto11
-rw-r--r--gobgp/common.go7
-rw-r--r--gobgp/main.go6
-rw-r--r--gobgp/mrt.go141
-rw-r--r--packet/mrt.go41
-rw-r--r--server/grpc_server.go30
-rw-r--r--server/server.go137
-rw-r--r--table/destination.go32
-rw-r--r--table/destination_test.go4
-rw-r--r--table/message.go8
-rw-r--r--table/path.go8
-rw-r--r--table/path_test.go4
-rw-r--r--table/table.go6
-rw-r--r--table/table_manager.go20
-rw-r--r--table/table_manager_test.go54
16 files changed, 521 insertions, 75 deletions
diff --git a/api/gobgp.pb.go b/api/gobgp.pb.go
index fd8882e7..218f1629 100644
--- a/api/gobgp.pb.go
+++ b/api/gobgp.pb.go
@@ -13,6 +13,7 @@ It has these top-level messages:
Arguments
ModPathArguments
PolicyArguments
+ MrtArguments
AddressFamily
RouteDistinguisher
GracefulRestartTuple
@@ -51,6 +52,7 @@ It has these top-level messages:
Statement
PolicyDefinition
ApplyPolicy
+ MrtMessage
*/
package api
@@ -629,6 +631,23 @@ func (m *PolicyArguments) GetApplyPolicy() *ApplyPolicy {
return nil
}
+type MrtArguments struct {
+ Resource Resource `protobuf:"varint,1,opt,name=resource,enum=api.Resource" json:"resource,omitempty"`
+ Af *AddressFamily `protobuf:"bytes,2,opt,name=af" json:"af,omitempty"`
+ Interval uint64 `protobuf:"varint,3,opt,name=interval" json:"interval,omitempty"`
+}
+
+func (m *MrtArguments) Reset() { *m = MrtArguments{} }
+func (m *MrtArguments) String() string { return proto.CompactTextString(m) }
+func (*MrtArguments) ProtoMessage() {}
+
+func (m *MrtArguments) GetAf() *AddressFamily {
+ if m != nil {
+ return m.Af
+ }
+ return nil
+}
+
type AddressFamily struct {
Afi AFI `protobuf:"varint,1,opt,enum=api.AFI" json:"Afi,omitempty"`
Safi SAFI `protobuf:"varint,2,opt,enum=api.SAFI" json:"Safi,omitempty"`
@@ -1350,6 +1369,14 @@ func (m *ApplyPolicy) GetDistributePolicies() []*PolicyDefinition {
return nil
}
+type MrtMessage struct {
+ Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"`
+}
+
+func (m *MrtMessage) Reset() { *m = MrtMessage{} }
+func (m *MrtMessage) String() string { return proto.CompactTextString(m) }
+func (*MrtMessage) ProtoMessage() {}
+
func init() {
proto.RegisterEnum("api.Resource", Resource_name, Resource_value)
proto.RegisterEnum("api.Operation", Operation_name, Operation_value)
@@ -1390,6 +1417,7 @@ type GrpcClient interface {
ModPolicyRoutePolicy(ctx context.Context, opts ...grpc.CallOption) (Grpc_ModPolicyRoutePolicyClient, error)
MonitorBestChanged(ctx context.Context, in *Arguments, opts ...grpc.CallOption) (Grpc_MonitorBestChangedClient, error)
MonitorPeerState(ctx context.Context, in *Arguments, opts ...grpc.CallOption) (Grpc_MonitorPeerStateClient, error)
+ GetMrt(ctx context.Context, in *MrtArguments, opts ...grpc.CallOption) (Grpc_GetMrtClient, error)
}
type grpcClient struct {
@@ -1778,6 +1806,38 @@ func (x *grpcMonitorPeerStateClient) Recv() (*Peer, error) {
return m, nil
}
+func (c *grpcClient) GetMrt(ctx context.Context, in *MrtArguments, opts ...grpc.CallOption) (Grpc_GetMrtClient, error) {
+ stream, err := grpc.NewClientStream(ctx, &_Grpc_serviceDesc.Streams[9], c.cc, "/api.Grpc/GetMrt", opts...)
+ if err != nil {
+ return nil, err
+ }
+ x := &grpcGetMrtClient{stream}
+ if err := x.ClientStream.SendMsg(in); err != nil {
+ return nil, err
+ }
+ if err := x.ClientStream.CloseSend(); err != nil {
+ return nil, err
+ }
+ return x, nil
+}
+
+type Grpc_GetMrtClient interface {
+ Recv() (*MrtMessage, error)
+ grpc.ClientStream
+}
+
+type grpcGetMrtClient struct {
+ grpc.ClientStream
+}
+
+func (x *grpcGetMrtClient) Recv() (*MrtMessage, error) {
+ m := new(MrtMessage)
+ if err := x.ClientStream.RecvMsg(m); err != nil {
+ return nil, err
+ }
+ return m, nil
+}
+
// Server API for Grpc service
type GrpcServer interface {
@@ -1800,6 +1860,7 @@ type GrpcServer interface {
ModPolicyRoutePolicy(Grpc_ModPolicyRoutePolicyServer) error
MonitorBestChanged(*Arguments, Grpc_MonitorBestChangedServer) error
MonitorPeerState(*Arguments, Grpc_MonitorPeerStateServer) error
+ GetMrt(*MrtArguments, Grpc_GetMrtServer) error
}
func RegisterGrpcServer(s *grpc.Server, srv GrpcServer) {
@@ -2130,6 +2191,27 @@ func (x *grpcMonitorPeerStateServer) Send(m *Peer) error {
return x.ServerStream.SendMsg(m)
}
+func _Grpc_GetMrt_Handler(srv interface{}, stream grpc.ServerStream) error {
+ m := new(MrtArguments)
+ if err := stream.RecvMsg(m); err != nil {
+ return err
+ }
+ return srv.(GrpcServer).GetMrt(m, &grpcGetMrtServer{stream})
+}
+
+type Grpc_GetMrtServer interface {
+ Send(*MrtMessage) error
+ grpc.ServerStream
+}
+
+type grpcGetMrtServer struct {
+ grpc.ServerStream
+}
+
+func (x *grpcGetMrtServer) Send(m *MrtMessage) error {
+ return x.ServerStream.SendMsg(m)
+}
+
var _Grpc_serviceDesc = grpc.ServiceDesc{
ServiceName: "api.Grpc",
HandlerType: (*GrpcServer)(nil),
@@ -2223,5 +2305,10 @@ var _Grpc_serviceDesc = grpc.ServiceDesc{
Handler: _Grpc_MonitorPeerState_Handler,
ServerStreams: true,
},
+ {
+ StreamName: "GetMrt",
+ Handler: _Grpc_GetMrt_Handler,
+ ServerStreams: true,
+ },
},
}
diff --git a/api/gobgp.proto b/api/gobgp.proto
index c97e446a..5441647c 100644
--- a/api/gobgp.proto
+++ b/api/gobgp.proto
@@ -39,6 +39,7 @@ service Grpc {
rpc ModPolicyRoutePolicy(stream PolicyArguments) returns (stream Error) {}
rpc MonitorBestChanged(Arguments) returns (stream Path) {}
rpc MonitorPeerState(Arguments) returns (stream Peer) {}
+ rpc GetMrt(MrtArguments) returns (stream MrtMessage) {}
}
message Error {
@@ -70,6 +71,12 @@ message PolicyArguments {
ApplyPolicy apply_policy = 7;
}
+message MrtArguments {
+ Resource resource = 1;
+ AddressFamily af = 2;
+ uint64 interval = 3;
+}
+
enum Resource {
GLOBAL = 0;
LOCAL = 1;
@@ -486,3 +493,7 @@ message ApplyPolicy {
repeated PolicyDefinition distribute_policies = 5;
string default_distribute_policy = 6;
}
+
+message MrtMessage {
+ bytes data = 1;
+}
diff --git a/gobgp/common.go b/gobgp/common.go
index ce03c8f2..b65a7662 100644
--- a/gobgp/common.go
+++ b/gobgp/common.go
@@ -55,6 +55,9 @@ const (
CMD_EXPORT = "export"
CMD_DISTRIBUTE = "distribute"
CMD_MONITOR = "monitor"
+ CMD_MRT = "mrt"
+ CMD_DUMP = "dump"
+ CMD_INJECT = "inject"
)
var subOpts struct {
@@ -82,6 +85,10 @@ var actionOpts struct {
AsPathPrependAction string `long:"as-prepend" description:"specifying a as-prepend action of policy"`
}
+var mrtOpts struct {
+ OutputDir string
+}
+
func formatTimedelta(d int64) string {
u := uint64(d)
neg := d < 0
diff --git a/gobgp/main.go b/gobgp/main.go
index 79629380..0878c593 100644
--- a/gobgp/main.go
+++ b/gobgp/main.go
@@ -59,9 +59,7 @@ func main() {
neighborCmd := NewNeighborCmd()
policyCmd := NewPolicyCmd()
monitorCmd := NewMonitorCmd()
- rootCmd.AddCommand(globalCmd)
- rootCmd.AddCommand(neighborCmd)
- rootCmd.AddCommand(policyCmd)
- rootCmd.AddCommand(monitorCmd)
+ mrtCmd := NewMrtCmd()
+ rootCmd.AddCommand(globalCmd, neighborCmd, policyCmd, monitorCmd, mrtCmd)
rootCmd.Execute()
}
diff --git a/gobgp/mrt.go b/gobgp/mrt.go
new file mode 100644
index 00000000..3dc2d540
--- /dev/null
+++ b/gobgp/mrt.go
@@ -0,0 +1,141 @@
+// 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 main
+
+import (
+ "bytes"
+ "fmt"
+ "github.com/osrg/gobgp/api"
+ "github.com/osrg/gobgp/packet"
+ "github.com/spf13/cobra"
+ "golang.org/x/net/context"
+ "io"
+ "io/ioutil"
+ "net"
+ "os"
+ "path/filepath"
+ "strconv"
+ "time"
+)
+
+func printMrtMsgs(data []byte) {
+ buffer := bytes.NewBuffer(data)
+
+ for buffer.Len() > bgp.MRT_COMMON_HEADER_LEN {
+ buf := make([]byte, bgp.MRT_COMMON_HEADER_LEN)
+ _, err := buffer.Read(buf)
+ if err == io.EOF {
+ break
+ } else if err != nil {
+ fmt.Println("failed to read:", err)
+ os.Exit(1)
+ }
+
+ h := &bgp.MRTHeader{}
+ err = h.DecodeFromBytes(buf)
+ if err != nil {
+ fmt.Println("failed to parse")
+ os.Exit(1)
+ }
+
+ buf = make([]byte, h.Len)
+ _, err = buffer.Read(buf)
+ if err != nil {
+ fmt.Println("failed to read")
+ os.Exit(1)
+ }
+
+ msg, err := bgp.ParseMRTBody(h, buf)
+ if err != nil {
+ fmt.Println("failed to parse:", err)
+ os.Exit(1)
+ }
+
+ fmt.Println(msg)
+ }
+
+}
+
+func NewMrtCmd() *cobra.Command {
+ mrtCmd := &cobra.Command{
+ Use: CMD_MRT,
+ }
+ mrtCmd.PersistentFlags().StringVarP(&subOpts.AddressFamily, "address-family", "a", "", "address family")
+
+ dumpCmd := &cobra.Command{
+ Use: CMD_DUMP,
+ Run: func(cmd *cobra.Command, args []string) {
+ var interval uint64
+ if len(args) > 0 {
+ i, err := strconv.Atoi(args[0])
+ if err != nil {
+ fmt.Println(err)
+ os.Exit(1)
+ }
+ interval = uint64(i)
+ }
+ af, err := checkAddressFamily(net.IP{})
+ if err != nil {
+ fmt.Println(err)
+ os.Exit(1)
+ }
+
+ arg := &api.MrtArguments{
+ Resource: api.Resource_GLOBAL,
+ Af: af,
+ Interval: interval,
+ }
+
+ stream, err := client.GetMrt(context.Background(), arg)
+ if err != nil {
+ fmt.Println(err)
+ os.Exit(1)
+ }
+
+ for {
+ s, err := stream.Recv()
+ if err == io.EOF {
+ break
+ } else if err != nil {
+ fmt.Println(err)
+ os.Exit(1)
+ }
+
+ if globalOpts.Debug {
+ printMrtMsgs(s.Data)
+ }
+
+ now := time.Now()
+ y, m, d := now.Date()
+ h, min, sec := now.Clock()
+ filename := fmt.Sprintf("%s/rib.%04d%02d%02d.%02d%02d%02d", mrtOpts.OutputDir, y, m, d, h, min, sec)
+
+ err = ioutil.WriteFile(filename, s.Data, 0600)
+ if err != nil {
+ fmt.Println(err)
+ os.Exit(1)
+ }
+
+ fmt.Println("mrt dump:", filepath.Clean(filename))
+ }
+ },
+ }
+ dumpCmd.Flags().StringVarP(&mrtOpts.OutputDir, "outdir", "o", ".", "output directory")
+
+ mrtCmd.AddCommand(dumpCmd)
+
+ return mrtCmd
+}
diff --git a/packet/mrt.go b/packet/mrt.go
index f31013ec..5e4f2597 100644
--- a/packet/mrt.go
+++ b/packet/mrt.go
@@ -144,6 +144,30 @@ type MRTMessage struct {
Body Body
}
+func (m *MRTMessage) Serialize() ([]byte, error) {
+ buf, err := m.Body.Serialize()
+ if err != nil {
+ return nil, err
+ }
+ m.Header.Len = uint32(len(buf))
+ bbuf, err := m.Header.Serialize()
+ if err != nil {
+ return nil, err
+ }
+ return append(bbuf, buf...), nil
+}
+
+func NewMRTMessage(timestamp uint32, t MRTType, subtype MRTSubTyper, body Body) (*MRTMessage, error) {
+ header, err := NewMRTHeader(timestamp, t, subtype, 0)
+ if err != nil {
+ return nil, err
+ }
+ return &MRTMessage{
+ Header: *header,
+ Body: body,
+ }, nil
+}
+
type Body interface {
DecodeFromBytes([]byte) error
Serialize() ([]byte, error)
@@ -201,9 +225,6 @@ func (p *Peer) Serialize() ([]byte, error) {
var bbuf []byte
buf := make([]byte, 5)
buf[0] = uint8(p.Type)
- if p.BgpId.To4() == nil {
- return nil, fmt.Errorf("invalid BGP ID")
- }
copy(buf[1:], p.BgpId.To4())
if p.Type&1 > 0 {
buf = append(buf, p.IpAddress.To16()...)
@@ -354,6 +375,16 @@ func (e *RibEntry) Serialize() ([]byte, error) {
totalLen := 0
binary.BigEndian.PutUint16(buf[6:], uint16(totalLen))
for _, pattr := range e.PathAttributes {
+ // TODO special modification is needed for MP_REACH_NLRI
+ // but also Quagga doesn't implement this.
+ //
+ // RFC 6396 4.3.4
+ // There is one exception to the encoding of BGP attributes for the BGP
+ // MP_REACH_NLRI attribute (BGP Type Code 14).
+ // Since the AFI, SAFI, and NLRI information is already encoded
+ // in the RIB Entry Header or RIB_GENERIC Entry Header,
+ // only the Next Hop Address Length and Next Hop Address fields are included.
+
bbuf, err := pattr.Serialize()
if err != nil {
return nil, err
@@ -373,6 +404,10 @@ func NewRibEntry(index uint16, time uint32, pathattrs []PathAttributeInterface)
}
}
+func (e *RibEntry) String() string {
+ return fmt.Sprintf("RIB_ENTRY: PeerIndex [%d] OriginatedTime [%d] PathAttrs [%v]", e.PeerIndex, e.OriginatedTime, e.PathAttributes)
+}
+
type Rib struct {
SequenceNumber uint32
Prefix AddrPrefixInterface
diff --git a/server/grpc_server.go b/server/grpc_server.go
index 160e9cfc..4a652c4f 100644
--- a/server/grpc_server.go
+++ b/server/grpc_server.go
@@ -82,6 +82,7 @@ const (
REQ_POLICY_EXTCOMMUNITIES_DELETE
REQ_MONITOR_GLOBAL_BEST_CHANGED
REQ_MONITOR_NEIGHBOR_PEER_STATE
+ REQ_MRT_GLOBAL_RIB
)
const GRPC_PORT = 8080
@@ -591,6 +592,35 @@ func (s *Server) ModPolicyRoutePolicy(stream api.Grpc_ModPolicyRoutePolicyServer
}
}
+func (s *Server) GetMrt(arg *api.MrtArguments, stream api.Grpc_GetMrtServer) error {
+ var reqType int
+ switch arg.Resource {
+ case api.Resource_GLOBAL:
+ reqType = REQ_MRT_GLOBAL_RIB
+ default:
+ return fmt.Errorf("unsupported resource type: %v", arg.Resource)
+ }
+ rf, err := convertAf2Rf(arg.Af)
+ if err != nil {
+ return err
+ }
+
+ req := NewGrpcRequest(reqType, "", rf, arg.Interval)
+ s.bgpServerCh <- req
+ for res := range req.ResponseCh {
+ if err = res.Err(); err != nil {
+ log.Debug(err.Error())
+ goto END
+ }
+ if err = stream.Send(res.Data.(*api.MrtMessage)); err != nil {
+ goto END
+ }
+ }
+END:
+ req.EndCh <- struct{}{}
+ return err
+}
+
type GrpcRequest struct {
RequestType int
RemoteAddr string
diff --git a/server/server.go b/server/server.go
index 78dd3b80..859abba2 100644
--- a/server/server.go
+++ b/server/server.go
@@ -1235,6 +1235,8 @@ func (server *BgpServer) handleGrpc(grpcReq *GrpcRequest) []*SenderMsg {
server.handleGrpcDelPolicies(grpcReq)
case REQ_MONITOR_GLOBAL_BEST_CHANGED, REQ_MONITOR_NEIGHBOR_PEER_STATE:
server.broadcastReqs = append(server.broadcastReqs, grpcReq)
+ case REQ_MRT_GLOBAL_RIB:
+ server.handleMrt(grpcReq)
default:
errmsg := fmt.Errorf("Unknown request type: %v", grpcReq.RequestType)
result := &GrpcResponse{
@@ -1853,3 +1855,138 @@ func (server *BgpServer) handleGrpcDelPolicies(grpcReq *GrpcRequest) {
grpcReq.ResponseCh <- result
close(grpcReq.ResponseCh)
}
+
+func (server *BgpServer) handleMrt(grpcReq *GrpcRequest) {
+ now := uint32(time.Now().Unix())
+ msg, err := server.mkMrtPeerIndexTableMsg(now)
+ result := &GrpcResponse{}
+ if err != nil {
+ result.ResponseErr = fmt.Errorf("failed to make new mrt peer index table message: %s", err)
+ grpcReq.ResponseCh <- result
+ close(grpcReq.ResponseCh)
+ return
+ }
+ data, err := msg.Serialize()
+ if err != nil {
+ result.ResponseErr = fmt.Errorf("failed to serialize table: %s", err)
+ grpcReq.ResponseCh <- result
+ close(grpcReq.ResponseCh)
+ return
+ }
+
+ msgs, err := server.mkMrtRibMsgs(grpcReq.RouteFamily, now)
+ if err != nil {
+ result.ResponseErr = fmt.Errorf("failed to make new mrt rib message: %s", err)
+ grpcReq.ResponseCh <- result
+ close(grpcReq.ResponseCh)
+ return
+ }
+ for _, msg := range msgs {
+ d, err := msg.Serialize()
+ if err != nil {
+ result.ResponseErr = fmt.Errorf("failed to serialize rib msg: %s", err)
+ grpcReq.ResponseCh <- result
+ close(grpcReq.ResponseCh)
+ return
+ }
+ data = append(data, d...)
+ }
+
+ result.Data = &api.MrtMessage{
+ Data: data,
+ }
+
+ select {
+ case <-grpcReq.EndCh:
+ return
+ case grpcReq.ResponseCh <- result:
+ default:
+ }
+
+ interval := int64(grpcReq.Data.(uint64))
+ if interval > 0 {
+ go func() {
+ t := time.NewTimer(time.Second * time.Duration(interval))
+ <-t.C
+ server.GrpcReqCh <- grpcReq
+ }()
+ } else {
+ close(grpcReq.ResponseCh)
+ }
+
+ return
+}
+
+func (server *BgpServer) mkMrtPeerIndexTableMsg(t uint32) (*bgp.MRTMessage, error) {
+ peers := make([]*bgp.Peer, 0, len(server.neighborMap))
+ for _, peer := range server.neighborMap {
+ id := peer.peerInfo.ID.To4().String()
+ ipaddr := peer.config.NeighborAddress.String()
+ asn := peer.config.PeerAs
+ peers = append(peers, bgp.NewPeer(id, ipaddr, asn, true))
+ }
+ bgpid := server.bgpConfig.Global.RouterId.To4().String()
+ table := bgp.NewPeerIndexTable(bgpid, "", peers)
+ return bgp.NewMRTMessage(t, bgp.TABLE_DUMPv2, bgp.PEER_INDEX_TABLE, table)
+}
+
+func (server *BgpServer) mkMrtRibMsgs(rf bgp.RouteFamily, t uint32) ([]*bgp.MRTMessage, error) {
+ tbl, ok := server.localRibMap[GLOBAL_RIB_NAME].rib.Tables[rf]
+ if !ok {
+ return nil, fmt.Errorf("unsupported route family: %s", rf)
+ }
+
+ getPeerIndex := func(info *table.PeerInfo) uint16 {
+ var idx uint16
+ for _, peer := range server.neighborMap {
+ if peer.peerInfo.Equal(info) {
+ return idx
+ }
+ idx++
+ }
+ return idx
+ }
+
+ var subtype bgp.MRTSubTypeTableDumpv2
+
+ switch rf {
+ case bgp.RF_IPv4_UC:
+ subtype = bgp.RIB_IPV4_UNICAST
+ case bgp.RF_IPv4_MC:
+ subtype = bgp.RIB_IPV4_MULTICAST
+ case bgp.RF_IPv6_UC:
+ subtype = bgp.RIB_IPV6_UNICAST
+ case bgp.RF_IPv6_MC:
+ subtype = bgp.RIB_IPV6_MULTICAST
+ default:
+ subtype = bgp.RIB_GENERIC
+ }
+
+ var seq uint32
+ msgs := make([]*bgp.MRTMessage, 0, len(tbl.GetDestinations()))
+ for _, dst := range tbl.GetDestinations() {
+ l := dst.GetKnownPathList()
+ entries := make([]*bgp.RibEntry, 0, len(l))
+ for _, p := range l {
+ // mrt doesn't assume to dump locally generated routes
+ if p.IsLocal() {
+ continue
+ }
+ idx := getPeerIndex(p.GetSource())
+ e := bgp.NewRibEntry(idx, uint32(p.GetTimestamp().Unix()), p.GetPathAttrs())
+ entries = append(entries, e)
+ }
+ // if dst only contains locally generated routes, ignore it
+ if len(entries) == 0 {
+ continue
+ }
+ rib := bgp.NewRib(seq, dst.GetNlri(), entries)
+ seq++
+ msg, err := bgp.NewMRTMessage(t, bgp.TABLE_DUMPv2, subtype, rib)
+ if err != nil {
+ return nil, err
+ }
+ msgs = append(msgs, msg)
+ }
+ return msgs, nil
+}
diff --git a/table/destination.go b/table/destination.go
index aa30fc2f..65697570 100644
--- a/table/destination.go
+++ b/table/destination.go
@@ -87,7 +87,7 @@ func (dd *Destination) MarshalJSON() ([]byte, error) {
}
func (dd *Destination) ToApiStruct() *api.Destination {
- prefix := dd.getNlri().String()
+ prefix := dd.GetNlri().String()
idx := func() int {
for i, p := range dd.knownPathList {
@@ -125,7 +125,7 @@ func (dd *Destination) setRouteFamily(routeFamily bgp.RouteFamily) {
dd.routeFamily = routeFamily
}
-func (dd *Destination) getNlri() bgp.AddrPrefixInterface {
+func (dd *Destination) GetNlri() bgp.AddrPrefixInterface {
return dd.nlri
}
@@ -149,7 +149,7 @@ func (dd *Destination) setBestPath(path *Path) {
dd.bestPath = path
}
-func (dd *Destination) getKnownPathList() []*Path {
+func (dd *Destination) GetKnownPathList() []*Path {
return dd.knownPathList
}
@@ -172,7 +172,7 @@ func (dd *Destination) validatePath(path *Path) {
log.WithFields(log.Fields{
"Topic": "Table",
- "Key": dd.getNlri().String(),
+ "Key": dd.GetNlri().String(),
"Path": path,
"ExpectedRF": dd.routeFamily,
}).Error("path is nil or invalid route family")
@@ -201,7 +201,7 @@ func (dest *Destination) Calculate(localAsn uint32) (*Path, string, error) {
dest.newPathList, _ = deleteAt(dest.newPathList, 0)
log.WithFields(log.Fields{
"Topic": "Table",
- "Key": dest.getNlri().String(),
+ "Key": dest.GetNlri().String(),
"Path": dest.knownPathList[0],
"Reason": BPR_ONLY_PATH,
}).Debug("best path")
@@ -246,7 +246,7 @@ func (dest *Destination) removeWithdrawals() {
log.WithFields(log.Fields{
"Topic": "Table",
- "Key": dest.getNlri().String(),
+ "Key": dest.GetNlri().String(),
"Length": len(dest.withdrawList),
}).Debug("Removing withdrawals")
// If we have no withdrawals, we have nothing to do.
@@ -259,7 +259,7 @@ func (dest *Destination) removeWithdrawals() {
if len(dest.knownPathList) == 0 {
log.WithFields(log.Fields{
"Topic": "Table",
- "Key": dest.getNlri().String(),
+ "Key": dest.GetNlri().String(),
"Length": len(dest.withdrawList),
}).Debug("Found withdrawals for path(s) that did not get installed")
@@ -289,7 +289,7 @@ func (dest *Destination) removeWithdrawals() {
if !isFound {
log.WithFields(log.Fields{
"Topic": "Table",
- "Key": dest.getNlri().String(),
+ "Key": dest.GetNlri().String(),
"Path": withdraw,
}).Debug("No matching path for withdraw found, may be path was not installed into table")
}
@@ -299,7 +299,7 @@ func (dest *Destination) removeWithdrawals() {
if len(matches) != len(dest.withdrawList) {
log.WithFields(log.Fields{
"Topic": "Table",
- "Key": dest.getNlri().String(),
+ "Key": dest.GetNlri().String(),
"MatchLength": len(matches),
"WithdrawLength": len(dest.withdrawList),
}).Debug("Did not find match for some withdrawals.")
@@ -312,7 +312,7 @@ func (dest *Destination) removeWithdrawals() {
if !result {
log.WithFields(log.Fields{
"Topic": "Table",
- "Key": dest.getNlri().String(),
+ "Key": dest.GetNlri().String(),
"Path": path,
}).Debug("could not remove path from knownPathList")
}
@@ -323,7 +323,7 @@ func (dest *Destination) removeWithdrawals() {
if !result {
log.WithFields(log.Fields{
"Topic": "Table",
- "Key": dest.getNlri().String(),
+ "Key": dest.GetNlri().String(),
"Path": path,
}).Debug("could not remove path from withdrawList")
}
@@ -386,14 +386,14 @@ func (dest *Destination) removeOldPaths() {
if !match {
log.WithFields(log.Fields{
"Topic": "Table",
- "Key": dest.getNlri().String(),
+ "Key": dest.GetNlri().String(),
"Path": oldPath,
}).Debug("not matched")
}
log.WithFields(log.Fields{
"Topic": "Table",
- "Key": dest.getNlri().String(),
+ "Key": dest.GetNlri().String(),
"Path": oldPath,
}).Debug("Implicit withdrawal of old path, since we have learned new path from the same peer")
}
@@ -577,11 +577,11 @@ func compareByLocalOrigin(path1, path2 *Path) *Path {
// Here we consider prefix from NC as locally originating static route.
// Hence it is preferred.
- if path1.isLocal() {
+ if path1.IsLocal() {
return path1
}
- if path2.isLocal() {
+ if path2.IsLocal() {
return path2
}
return nil
@@ -729,7 +729,7 @@ func compareByRouterID(localAsn uint32, path1, path2 *Path) (*Path, error) {
source2 := path2.GetSource()
// If both paths are from NC we have same router Id, hence cannot compare.
- if path1.isLocal() && path2.isLocal() {
+ if path1.IsLocal() && path2.IsLocal() {
return nil, nil
}
diff --git a/table/destination_test.go b/table/destination_test.go
index dbdf19a2..0b1b8e3b 100644
--- a/table/destination_test.go
+++ b/table/destination_test.go
@@ -53,14 +53,14 @@ func TestDestinationSetNlri(t *testing.T) {
dd := &Destination{}
nlri := bgp.NewNLRInfo(24, "13.2.3.1")
dd.setNlri(nlri)
- r_nlri := dd.getNlri()
+ r_nlri := dd.GetNlri()
assert.Equal(t, r_nlri, nlri)
}
func TestDestinationGetNlri(t *testing.T) {
dd := &Destination{}
nlri := bgp.NewNLRInfo(24, "10.110.123.1")
dd.setNlri(nlri)
- r_nlri := dd.getNlri()
+ r_nlri := dd.GetNlri()
assert.Equal(t, r_nlri, nlri)
}
func TestDestinationSetBestPathReason(t *testing.T) {
diff --git a/table/message.go b/table/message.go
index d8bb3cff..ffcbfa08 100644
--- a/table/message.go
+++ b/table/message.go
@@ -146,7 +146,7 @@ func createUpdateMsgFromPath(path *Path, msg *bgp.BGPMessage) *bgp.BGPMessage {
u := msg.Body.(*bgp.BGPUpdate)
u.NLRI = append(u.NLRI, *nlri)
} else {
- pathAttrs := path.getPathAttrs()
+ pathAttrs := path.GetPathAttrs()
return bgp.NewBGPUpdateMessage([]bgp.WithdrawnRoute{}, pathAttrs, []bgp.NLRInfo{*nlri})
}
}
@@ -158,7 +158,7 @@ func createUpdateMsgFromPath(path *Path, msg *bgp.BGPMessage) *bgp.BGPMessage {
unreach := u.PathAttributes[idx].(*bgp.PathAttributeMpUnreachNLRI)
unreach.Value = append(unreach.Value, path.GetNlri())
} else {
- clonedAttrs := cloneAttrSlice(path.getPathAttrs())
+ clonedAttrs := cloneAttrSlice(path.GetPathAttrs())
idx, attr := path.getPathAttr(bgp.BGP_ATTR_TYPE_MP_REACH_NLRI)
reach := attr.(*bgp.PathAttributeMpReachNLRI)
clonedAttrs[idx] = bgp.NewPathAttributeMpUnreachNLRI(reach.Value)
@@ -175,7 +175,7 @@ func createUpdateMsgFromPath(path *Path, msg *bgp.BGPMessage) *bgp.BGPMessage {
// we don't need to clone here but we
// might merge path to this message in
// the future so let's clone anyway.
- clonedAttrs := cloneAttrSlice(path.getPathAttrs())
+ clonedAttrs := cloneAttrSlice(path.GetPathAttrs())
return bgp.NewBGPUpdateMessage([]bgp.WithdrawnRoute{}, clonedAttrs, []bgp.NLRInfo{})
}
}
@@ -210,7 +210,7 @@ func isMergeable(p1, p2 *Path) bool {
if p1.GetRouteFamily() != bgp.RF_IPv4_UC {
return false
}
- if p1.GetSource().Equal(p2.GetSource()) && isSamePathAttrs(p1.getPathAttrs(), p2.getPathAttrs()) {
+ if p1.GetSource().Equal(p2.GetSource()) && isSamePathAttrs(p1.GetPathAttrs(), p2.GetPathAttrs()) {
return true
}
return false
diff --git a/table/path.go b/table/path.go
index 8734c513..12f7d498 100644
--- a/table/path.go
+++ b/table/path.go
@@ -122,7 +122,7 @@ func (path *Path) UpdatePathAttrs(global *config.Global, peer *config.Neighbor)
}
}
-func (path *Path) getTimestamp() time.Time {
+func (path *Path) GetTimestamp() time.Time {
return path.timestamp
}
@@ -130,7 +130,7 @@ func (path *Path) setTimestamp(t time.Time) {
path.timestamp = t
}
-func (path *Path) isLocal() bool {
+func (path *Path) IsLocal() bool {
var ret bool
if path.source.Address == nil {
ret = true
@@ -145,7 +145,7 @@ func (path *Path) ToApiStruct() *api.Path {
ret = append(ret, a.ToApiStruct())
}
return ret
- }(path.getPathAttrs())
+ }(path.GetPathAttrs())
return &api.Path{
Nlri: path.GetNlri().ToApiStruct(),
Nexthop: path.GetNexthop().String(),
@@ -239,7 +239,7 @@ func (path *Path) getMedSetByTargetNeighbor() bool {
return path.medSetByTargetNeighbor
}
-func (path *Path) getPathAttrs() []bgp.PathAttributeInterface {
+func (path *Path) GetPathAttrs() []bgp.PathAttributeInterface {
return path.pathAttrs
}
diff --git a/table/path_test.go b/table/path_test.go
index 976a5620..d973294a 100644
--- a/table/path_test.go
+++ b/table/path_test.go
@@ -14,13 +14,13 @@ import (
func TestPathNewIPv4(t *testing.T) {
peerP := PathCreatePeer()
pathP := PathCreatePath(peerP)
- ipv4p := NewPath(pathP[0].GetSource(), pathP[0].GetNlri(), true, pathP[0].getPathAttrs(), pathP[0].getMedSetByTargetNeighbor(), time.Now())
+ ipv4p := NewPath(pathP[0].GetSource(), pathP[0].GetNlri(), true, pathP[0].GetPathAttrs(), pathP[0].getMedSetByTargetNeighbor(), time.Now())
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(), pathP[0].getMedSetByTargetNeighbor(), time.Now())
+ ipv6p := NewPath(pathP[0].GetSource(), pathP[0].GetNlri(), true, pathP[0].GetPathAttrs(), pathP[0].getMedSetByTargetNeighbor(), time.Now())
assert.NotNil(t, ipv6p)
}
diff --git a/table/table.go b/table/table.go
index b3d21840..0c7bb653 100644
--- a/table/table.go
+++ b/table/table.go
@@ -58,12 +58,12 @@ func (t *Table) DeleteDestByPeer(peerInfo *PeerInfo) []*Destination {
changedDests := make([]*Destination, 0)
for _, dest := range t.destinations {
newKnownPathList := make([]*Path, 0)
- for _, p := range dest.getKnownPathList() {
+ for _, p := range dest.GetKnownPathList() {
if !p.GetSource().Equal(peerInfo) {
newKnownPathList = append(newKnownPathList, p)
}
}
- if len(newKnownPathList) != len(dest.getKnownPathList()) {
+ if len(newKnownPathList) != len(dest.GetKnownPathList()) {
changedDests = append(changedDests, dest)
dest.setKnownPathList(newKnownPathList)
}
@@ -83,7 +83,7 @@ func (t *Table) deleteDestByNlri(nlri bgp.AddrPrefixInterface) *Destination {
func (t *Table) deleteDest(dest *Destination) {
destinations := t.GetDestinations()
- delete(destinations, t.tableKey(dest.getNlri()))
+ delete(destinations, t.tableKey(dest.GetNlri()))
}
func (t *Table) validatePath(path *Path) {
diff --git a/table/table_manager.go b/table/table_manager.go
index 4a692d23..65ead593 100644
--- a/table/table_manager.go
+++ b/table/table_manager.go
@@ -138,7 +138,7 @@ func (manager *TableManager) calculate(destinationList []*Destination) ([]*Path,
log.WithFields(log.Fields{
"Topic": "table",
"Owner": manager.owner,
- "Key": destination.getNlri().String(),
+ "Key": destination.GetNlri().String(),
}).Debug("Processing destination")
newBestPath, reason, err := destination.Calculate(manager.localAsn)
@@ -156,7 +156,7 @@ func (manager *TableManager) calculate(destinationList []*Destination) ([]*Path,
log.WithFields(log.Fields{
"Topic": "table",
"Owner": manager.owner,
- "Key": destination.getNlri().String(),
+ "Key": destination.GetNlri().String(),
"peer": newBestPath.GetSource().Address,
"next_hop": newBestPath.GetNexthop().String(),
"reason": reason,
@@ -168,16 +168,16 @@ func (manager *TableManager) calculate(destinationList []*Destination) ([]*Path,
log.WithFields(log.Fields{
"Topic": "table",
"Owner": manager.owner,
- "Key": destination.getNlri().String(),
+ "Key": destination.GetNlri().String(),
}).Debug("best path is nil")
- if len(destination.getKnownPathList()) == 0 {
+ if len(destination.GetKnownPathList()) == 0 {
// create withdraw path
if currentBestPath != nil {
log.WithFields(log.Fields{
"Topic": "table",
"Owner": manager.owner,
- "Key": destination.getNlri().String(),
+ "Key": destination.GetNlri().String(),
"peer": currentBestPath.GetSource().Address,
"next_hop": currentBestPath.GetNexthop().String(),
}).Debug("best path is lost")
@@ -190,7 +190,7 @@ func (manager *TableManager) calculate(destinationList []*Destination) ([]*Path,
log.WithFields(log.Fields{
"Topic": "table",
"Owner": manager.owner,
- "Key": destination.getNlri().String(),
+ "Key": destination.GetNlri().String(),
}).Error("known path list is not empty")
}
} else {
@@ -207,14 +207,14 @@ func (manager *TableManager) calculate(destinationList []*Destination) ([]*Path,
destination.setBestPath(newBestPath)
}
- if len(destination.getKnownPathList()) == 0 && destination.GetBestPath() == nil {
+ if len(destination.GetKnownPathList()) == 0 && destination.GetBestPath() == nil {
rf := destination.getRouteFamily()
t := manager.Tables[rf]
t.deleteDest(destination)
log.WithFields(log.Fields{
"Topic": "table",
"Owner": manager.owner,
- "Key": destination.getNlri().String(),
+ "Key": destination.GetNlri().String(),
"route_family": rf,
}).Debug("destination removed")
}
@@ -297,8 +297,8 @@ func (adj *AdjRib) update(rib map[bgp.RouteFamily]map[string]*ReceivedRoute, pat
delete(rib[rf], key)
}
} else {
- if found && reflect.DeepEqual(old.path.getPathAttrs(), path.getPathAttrs()) {
- path.setTimestamp(old.path.getTimestamp())
+ if found && reflect.DeepEqual(old.path.GetPathAttrs(), path.GetPathAttrs()) {
+ path.setTimestamp(old.path.GetTimestamp())
}
rib[rf][key] = NewReceivedRoute(path, false)
}
diff --git a/table/table_manager_test.go b/table/table_manager_test.go
index 206b322e..5a875c9b 100644
--- a/table/table_manager_test.go
+++ b/table/table_manager_test.go
@@ -102,7 +102,7 @@ func TestProcessBGPUpdate_0_select_onlypath_ipv4(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, 4, len(path.getPathAttrs()))
+ assert.Equal(t, 4, len(path.GetPathAttrs()))
// check destination
expectedPrefix := "10.10.10.0/24"
@@ -153,7 +153,7 @@ func TestProcessBGPUpdate_0_select_onlypath_ipv6(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, 4, len(path.getPathAttrs()))
+ assert.Equal(t, 4, len(path.GetPathAttrs()))
// check destination
expectedPrefix := "2001:123:123:1::/64"
@@ -237,7 +237,7 @@ func TestProcessBGPUpdate_1_select_high_localpref_ipv4(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes2), len(path.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes2), len(path.GetPathAttrs()))
// check destination
expectedPrefix := "10.10.10.0/24"
@@ -323,7 +323,7 @@ func TestProcessBGPUpdate_1_select_high_localpref_ipv6(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, 5, len(path.getPathAttrs()))
+ assert.Equal(t, 5, len(path.GetPathAttrs()))
// check destination
expectedPrefix := "2001:123:123:1::/64"
@@ -409,7 +409,7 @@ func TestProcessBGPUpdate_2_select_local_origin_ipv4(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes2), len(path.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes2), len(path.GetPathAttrs()))
// check destination
expectedPrefix := "10.10.10.0/24"
@@ -498,7 +498,7 @@ func TestProcessBGPUpdate_2_select_local_origin_ipv6(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, 5, len(path.getPathAttrs()))
+ assert.Equal(t, 5, len(path.GetPathAttrs()))
// check destination
expectedPrefix := "2001:123:123:1::/64"
@@ -555,7 +555,7 @@ func TestProcessBGPUpdate_3_select_aspath_ipv4(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, 4, len(path.getPathAttrs()))
+ assert.Equal(t, 4, len(path.GetPathAttrs()))
// check destination
expectedPrefix := "20.20.20.0/24"
@@ -612,7 +612,7 @@ func TestProcessBGPUpdate_3_select_aspath_ipv6(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, 4, len(path.getPathAttrs()))
+ assert.Equal(t, 4, len(path.GetPathAttrs()))
// check destination
expectedPrefix := "2002:223:123:1::/64"
@@ -696,7 +696,7 @@ func TestProcessBGPUpdate_4_select_low_origin_ipv4(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes2), len(path.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes2), len(path.GetPathAttrs()))
// check destination
expectedPrefix := "10.10.10.0/24"
@@ -782,7 +782,7 @@ func TestProcessBGPUpdate_4_select_low_origin_ipv6(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, 5, len(path.getPathAttrs()))
+ assert.Equal(t, 5, len(path.GetPathAttrs()))
// check destination
expectedPrefix := "2001:123:123:1::/64"
@@ -866,7 +866,7 @@ func TestProcessBGPUpdate_5_select_low_med_ipv4(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes2), len(path.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes2), len(path.GetPathAttrs()))
// check destination
expectedPrefix := "10.10.10.0/24"
@@ -952,7 +952,7 @@ func TestProcessBGPUpdate_5_select_low_med_ipv6(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, 5, len(path.getPathAttrs()))
+ assert.Equal(t, 5, len(path.GetPathAttrs()))
// check destination
expectedPrefix := "2001:123:123:1::/64"
@@ -1038,7 +1038,7 @@ func TestProcessBGPUpdate_6_select_ebgp_path_ipv4(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes2), len(path.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes2), len(path.GetPathAttrs()))
// check destination
expectedPrefix := "10.10.10.0/24"
@@ -1125,7 +1125,7 @@ func TestProcessBGPUpdate_6_select_ebgp_path_ipv6(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, 5, len(path.getPathAttrs()))
+ assert.Equal(t, 5, len(path.GetPathAttrs()))
// check destination
expectedPrefix := "2001:123:123:1::/64"
@@ -1213,7 +1213,7 @@ func TestProcessBGPUpdate_7_select_low_routerid_path_ipv4(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes2), len(path.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes2), len(path.GetPathAttrs()))
// check destination
expectedPrefix := "10.10.10.0/24"
@@ -1300,7 +1300,7 @@ func TestProcessBGPUpdate_7_select_low_routerid_path_ipv6(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, 5, len(path.getPathAttrs()))
+ assert.Equal(t, 5, len(path.GetPathAttrs()))
// check destination
expectedPrefix := "2001:123:123:1::/64"
@@ -1386,7 +1386,7 @@ func TestProcessBGPUpdate_8_withdraw_path_ipv4(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes), len(path.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes), len(path.GetPathAttrs()))
}
checkPattr(bgpMessage2, path)
// check destination
@@ -1517,7 +1517,7 @@ func TestProcessBGPUpdate_8_mpunreach_path_ipv6(t *testing.T) {
pathMed := attr.(*bgp.PathAttributeMultiExitDisc)
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes), len(path.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes), len(path.GetPathAttrs()))
}
checkPattr(bgpMessage2, path)
@@ -1616,7 +1616,7 @@ func TestProcessBGPUpdate_bestpath_lost_ipv4(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes), len(path.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes), len(path.GetPathAttrs()))
}
checkPattr(bgpMessage1, path)
@@ -1689,7 +1689,7 @@ func TestProcessBGPUpdate_bestpath_lost_ipv6(t *testing.T) {
pathMed := attr.(*bgp.PathAttributeMultiExitDisc)
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes), len(path.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes), len(path.GetPathAttrs()))
}
checkPattr(bgpMessage1, path)
@@ -1773,7 +1773,7 @@ func TestProcessBGPUpdate_implicit_withdrwal_ipv4(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes), len(path.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes), len(path.GetPathAttrs()))
}
checkPattr(bgpMessage2, path)
// check destination
@@ -1882,7 +1882,7 @@ func TestProcessBGPUpdate_implicit_withdrwal_ipv6(t *testing.T) {
pathMed := attr.(*bgp.PathAttributeMultiExitDisc)
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes), len(path.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes), len(path.GetPathAttrs()))
}
checkPattr(bgpMessage2, path)
@@ -1939,7 +1939,7 @@ func TestProcessBGPUpdate_multiple_nlri_ipv4(t *testing.T) {
assert.Equal(t, expectedMed, pathMed)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes), len(actual.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes), len(actual.GetPathAttrs()))
}
checkBestPathResult := func(rf bgp.RouteFamily, prefix, nexthop string, p *Path, m *bgp.BGPMessage) {
@@ -2078,7 +2078,7 @@ func TestProcessBGPUpdate_multiple_nlri_ipv6(t *testing.T) {
assert.Equal(t, expectedLocalpref, localpref)
// check PathAttribute length
- assert.Equal(t, len(pathAttributes), len(actual.getPathAttrs()))
+ assert.Equal(t, len(pathAttributes), len(actual.GetPathAttrs()))
}
@@ -2210,7 +2210,7 @@ func TestProcessBGPUpdate_Timestamp(t *testing.T) {
inList := adjRib.GetInPathList(bgp.RF_IPv4_UC)
assert.Equal(t, len(inList), 1)
- assert.Equal(t, inList[0].getTimestamp(), t1)
+ assert.Equal(t, inList[0].GetTimestamp(), t1)
med2 := bgp.NewPathAttributeMultiExitDisc(1)
pathAttributes2 := []bgp.PathAttributeInterface{
@@ -2222,12 +2222,12 @@ func TestProcessBGPUpdate_Timestamp(t *testing.T) {
m3 := bgp.NewBGPUpdateMessage(withdrawnRoutes, pathAttributes2, nlri)
pList3 := ProcessMessage(m3, peer)
- t3 := pList3[0].getTimestamp()
+ t3 := pList3[0].GetTimestamp()
adjRib.UpdateIn(pList3)
inList = adjRib.GetInPathList(bgp.RF_IPv4_UC)
assert.Equal(t, len(inList), 1)
- assert.Equal(t, inList[0].getTimestamp(), t3)
+ assert.Equal(t, inList[0].GetTimestamp(), t3)
}
func update_fromR1() *bgp.BGPMessage {