diff --git a/README.md b/README.md index 5ea861a..d1b8e7f 100644 --- a/README.md +++ b/README.md @@ -6,23 +6,24 @@ Still a Work In Progress. API may change before v1.0.0. - PFCP Sessions handling (currently only PFCP Session establishment procedure is supported) ## Getting started -### Server (UPF) +### UPF ```golang -upNode := NewPFCPServerEntity(upAddress) +upNode := NewPFCPEntityUP(UPFADDR) upNode.Start() +// Access list of associations +associations := upNode.GetPFCPAssociations() // Access list of sessions sessions := upNode.GetPFCPSessions() ``` -### Client (SMF) +### SMF ```golang -cpNode := NewPFCPClientEntity(cpAddress) +cpNode := NewPFCPEntityCP(SMFADDR) cpNode.Start() -peer, _ := NewPFCPPeer(cpNode, pfcputils.CreateNodeID(nodeID) -a, _ := cpNode.NewPFCPAssociation(peer) -a.NewPFCPSession(cpNode.GetNextRemoteSessionID(), pdrs, fars) +association, _ := cpNode.NewEstablishedAssociation(pfcputils.CreateNodeID(UPFADDR)) +session, _ := a.CreateSession(pdrs, fars) ``` diff --git a/association.go b/association.go deleted file mode 100644 index 4e36c34..0000000 --- a/association.go +++ /dev/null @@ -1,167 +0,0 @@ -// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. -// Use of this source code is governed by a MIT-style license that can be -// found in the LICENSE file. -// SPDX-License-Identifier: MIT - -package pfcp_networking - -import ( - "fmt" - "log" - "net" - "sync" - "time" - - pfcprule "github.com/louisroyer/go-pfcp-networking/pfcprules" - "github.com/wmnsk/go-pfcp/ie" -) - -type PFCPAssociation struct { - *PFCPPeer - sessions PFCPSessionMapSEID - remoteSessions RemotePFCPSessionMapSEID - mu sync.Mutex -} - -func NewPFCPAssociation(peer *PFCPPeer) PFCPAssociation { - association := PFCPAssociation{ - PFCPPeer: peer, - sessions: make(PFCPSessionMapSEID), - remoteSessions: make(RemotePFCPSessionMapSEID), - } - go association.heartMonitoring() - return association -} - -// Start monitoring heart of a PFCP Association -func (association *PFCPAssociation) heartMonitoring() error { - defer association.Close() - checkInterval := 30 * time.Second - for { - select { - case <-time.After(checkInterval): - alive, err := association.IsAlive() - if !alive { - return fmt.Errorf("PFCP Peer is dead") - } - if err != nil { - return err - } - } - } -} - -func (association *PFCPAssociation) GetSessions() PFCPSessionMapSEID { - return association.sessions -} - -// Generate a local FSEID IE for the session (to be created) identified by a given SEID -func (association *PFCPAssociation) getFSEID(seid uint64) (*ie.IE, error) { - ieNodeID := association.Srv.NodeID() - nodeID, err := ieNodeID.NodeID() - if err != nil { - return nil, err - } - var localFseid *ie.IE - switch ieNodeID.Payload[0] { - case ie.NodeIDIPv4Address: - ip4, err := net.ResolveIPAddr("ip4", nodeID) - if err != nil { - return nil, err - } - localFseid, err = NewFSEID(seid, ip4, nil) - if err != nil { - return nil, err - } - case ie.NodeIDIPv6Address: - ip6, err := net.ResolveIPAddr("ip6", nodeID) - if err != nil { - return nil, err - } - localFseid, err = NewFSEID(seid, nil, ip6) - if err != nil { - return nil, err - } - case ie.NodeIDFQDN: - ip4, err4 := net.ResolveIPAddr("ip4", nodeID) - ip6, err6 := net.ResolveIPAddr("ip6", nodeID) - if err4 != nil && err6 != nil { - return nil, fmt.Errorf("Cannot resolve NodeID") - } - localFseid = ie.NewFSEID(seid, ip4.IP.To4(), ip6.IP.To16()) - } - return localFseid, nil -} - -func (association *PFCPAssociation) CreateSession(localSEID uint64, remoteFseid *ie.IE, pdrs []*pfcprule.PDR, fars []*pfcprule.FAR) (session *PFCPSession, err error) { - localFseid, err := association.getFSEID(localSEID) - if err != nil { - return nil, err - } - s := NewPFCPSession(localFseid, remoteFseid) - tmpPDR := make(map[uint16]*pfcprule.PDR) - if pdrs == nil { - return nil, fmt.Errorf("No PDR in session") - } - log.Println("Adding", len(pdrs), "PDRs to session") - for _, pdr := range pdrs { - if pdr == nil { - log.Println("A PDR is nil") - continue - } - id, err := pdr.ID() - if err != nil { - return nil, err - } - tmpPDR[id] = pdr - } - tmpFAR := make(map[uint32]*pfcprule.FAR) - if fars == nil { - return nil, fmt.Errorf("No FAR in session") - } - log.Println("Adding", len(fars), "FARs to session") - for _, far := range fars { - if far == nil { - log.Println("A FAR is nil") - continue - } - id, err := far.ID() - if err != nil { - return nil, err - } - tmpFAR[id] = far - } - s.AddFARs(tmpFAR) - s.AddPDRs(tmpPDR) - association.mu.Lock() - association.sessions[localSEID] = &s - association.mu.Unlock() - return &s, nil -} - -func (association *PFCPAssociation) NewPFCPSession(localSEID uint64, pdrs []*pfcprule.PDR, fars []*pfcprule.FAR) (session *RemotePFCPSession, err error) { - localFseid, err := association.getFSEID(localSEID) - if err != nil { - return nil, err - } - s := NewRemotePFCPSession(localFseid, association) - s.Start(pdrs, fars) - association.mu.Lock() - association.remoteSessions[localSEID] = &s - association.mu.Unlock() - return &s, nil -} - -func NewFSEID(seid uint64, v4, v6 *net.IPAddr) (*ie.IE, error) { - if v4 == nil && v6 == nil { - return nil, fmt.Errorf("Cannot create FSEID with no IP Address") - } - var ip4, ip6 net.IP - if v4 != nil { - ip4 = v4.IP.To4() - } - if v6 != nil { - ip6 = v6.IP.To16() - } - return ie.NewFSEID(seid, ip4, ip6), nil -} diff --git a/pfcp/api/association_interface.go b/pfcp/api/association_interface.go new file mode 100644 index 0000000..e5477f4 --- /dev/null +++ b/pfcp/api/association_interface.go @@ -0,0 +1,19 @@ +// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. +// Use of this source code is governed by a MIT-style license that can be +// found in the LICENSE file. +// SPDX-License-Identifier: MIT + +package api + +import ( + pfcprule "github.com/louisroyer/go-pfcp-networking/pfcprules" + "github.com/wmnsk/go-pfcp/ie" +) + +type SEID = uint64 +type PFCPAssociationInterface interface { + PFCPPeerInterface + SetupInitiatedByCP() error + GetNextSEID() SEID + CreateSession(remoteFseid *ie.IE, pdrs pfcprule.PDRs, fars pfcprule.FARs) (session PFCPSessionInterface, err error) +} diff --git a/pfcp/api/entity_interface.go b/pfcp/api/entity_interface.go new file mode 100644 index 0000000..f98cc29 --- /dev/null +++ b/pfcp/api/entity_interface.go @@ -0,0 +1,26 @@ +// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. +// Use of this source code is governed by a MIT-style license that can be +// found in the LICENSE file. +// SPDX-License-Identifier: MIT + +package api + +import ( + "net" + + "github.com/wmnsk/go-pfcp/ie" +) + +type PFCPEntityInterface interface { + IsUserPlane() bool + IsControlPlane() bool + NodeID() *ie.IE + RecoveryTimeStamp() *ie.IE + NewEstablishedPFCPAssociation(nodeID *ie.IE) (association PFCPAssociationInterface, err error) + RemovePFCPAssociation(association PFCPAssociationInterface) error + GetPFCPAssociation(nid string) (association PFCPAssociationInterface, err error) + //GetLocalSessions() PFCPSessionMapSEID + SendTo(msg []byte, dst net.Addr) error + GetPFCPSessions() []PFCPSessionInterface + AddEstablishedPFCPSession(session PFCPSessionInterface) error +} diff --git a/pfcp/api/peer_interface.go b/pfcp/api/peer_interface.go new file mode 100644 index 0000000..2fd426b --- /dev/null +++ b/pfcp/api/peer_interface.go @@ -0,0 +1,23 @@ +// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. +// Use of this source code is governed by a MIT-style license that can be +// found in the LICENSE file. +// SPDX-License-Identifier: MIT + +package api + +import ( + "github.com/wmnsk/go-pfcp/ie" + "github.com/wmnsk/go-pfcp/message" +) + +type PFCPPeerInterface interface { + IsRunning() bool + Close() error + Send(msg message.Message) (m message.Message, err error) + IsAlive() (res bool, err error) + NodeID() *ie.IE + IsUserPlane() bool + IsControlPlane() bool + LocalEntity() PFCPEntityInterface + NewEstablishedPFCPAssociation() (PFCPAssociationInterface, error) +} diff --git a/pfcp/api/session_interface.go b/pfcp/api/session_interface.go new file mode 100644 index 0000000..113052e --- /dev/null +++ b/pfcp/api/session_interface.go @@ -0,0 +1,27 @@ +// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. +// Use of this source code is governed by a MIT-style license that can be +// found in the LICENSE file. +// SPDX-License-Identifier: MIT + +package api + +import ( + "net" + + pfcprule "github.com/louisroyer/go-pfcp-networking/pfcprules" + "github.com/wmnsk/go-pfcp/ie" +) + +type PFCPSessionInterface interface { + LocalFSEID() *ie.IE + LocalSEID() (SEID, error) + LocalIPAddress() (net.IP, error) + RemoteFSEID() *ie.IE + RemoteSEID() (SEID, error) + RemoteIPAddress() (net.IP, error) + GetPDRs() pfcprule.PDRs + GetFAR(farid uint32) (*pfcprule.FAR, error) + AddPDRsFARs(pdrs pfcprule.PDRMap, fars pfcprule.FARMap) + // SetRemoteFSEID(FSEID *ie.IE) + Setup() error +} diff --git a/pfcp/association.go b/pfcp/association.go new file mode 100644 index 0000000..da3de0d --- /dev/null +++ b/pfcp/association.go @@ -0,0 +1,212 @@ +// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. +// Use of this source code is governed by a MIT-style license that can be +// found in the LICENSE file. +// SPDX-License-Identifier: MIT + +package pfcp_networking + +import ( + "fmt" + "log" + "net" + "time" + + "github.com/louisroyer/go-pfcp-networking/pfcp/api" + pfcprule "github.com/louisroyer/go-pfcp-networking/pfcprules" + "github.com/wmnsk/go-pfcp/ie" + "github.com/wmnsk/go-pfcp/message" +) + +type PFCPAssociation struct { + api.PFCPPeerInterface // connection to remote peer + isSetup bool // true when session is already set-up + sessionIDPool SessionIDPool // used to generate SEIDs for this association +} + +// Create a new PFCPAssociation, this association is already set-up +func newEstablishedPFCPAssociation(peer api.PFCPPeerInterface) (api.PFCPAssociationInterface, error) { + association := PFCPAssociation{ + PFCPPeerInterface: peer, + isSetup: false, + } + err := association.SetupInitiatedByCP() + if err != nil { + return nil, err + } + return association, nil +} + +// Get next available SEID for this PFCPAssociation. +// SEID are not globally unique, F-SEID are globally unique. +// F-SEID are constitued of IPv4 and/or IPv6 address(es) of the peer +// plus the SEID. So as long as SEID are unique per peer (i.e. per PFCPAssociation), +// everything should be okay. +func (association PFCPAssociation) GetNextSEID() api.SEID { + return association.sessionIDPool.GetNext() +} + +// Setup a PFCPAssociation with the PFCP Association Setup by the CP Function Procedure +// If the LocalEntity is a CP function, a PFCP Association Setup Request is sent, +// if the LocalEntity is a UP function, we assume this method is called +// because we received a Association Setup Request. +// +// PFCP Association Setup Initiated by the UP Function is NOT supported (yet). PR are welcome. +// +// See 129.244 v16.0.1, section 6.2.6.1: +// The setup of a PFCP association may be initiated by the CP function (see clause 6.2.6.2) or the UP function (see +// clause 6.2.6.3). +// The CP function and the UP function shall support the PFCP Association Setup initiated by the CP function. The CP +// function and the UP function may additionally support the PFCP Association Setup initiated by the UP function. +func (association PFCPAssociation) SetupInitiatedByCP() error { + if association.isSetup { + return fmt.Errorf("Association is already set up") + } + switch { + case association.LocalEntity().IsUserPlane(): + association.isSetup = true + go association.heartMonitoring() + return nil + case association.LocalEntity().IsControlPlane(): + sar := message.NewAssociationSetupRequest(0, association.LocalEntity().NodeID(), association.LocalEntity().RecoveryTimeStamp()) + resp, err := association.Send(sar) + if err != nil { + return err + } + asres, ok := resp.(*message.AssociationSetupResponse) + if !ok { + log.Printf("got unexpected message: %s\n", resp.MessageTypeName()) + } + cause, err := asres.Cause.Cause() + if err != nil { + // TODO: send missing ie message + return err + } + if cause == ie.CauseRequestAccepted { + association.isSetup = true + go association.heartMonitoring() + return nil + } + return fmt.Errorf("Associaton setup request rejected") + default: + return fmt.Errorf("Local PFCP entity is not a UP function, neither a CP function.") + } +} + +// Start monitoring heart of a PFCP Association +func (association *PFCPAssociation) heartMonitoring() error { + defer association.Close() + checkInterval := 30 * time.Second + for { + select { + case <-time.After(checkInterval): + alive, err := association.IsAlive() + if !alive { + return fmt.Errorf("PFCP Peer is dead") + } + if err != nil { + return err + } + } + } +} + +// Generate a local FSEID IE for the session (to be created) identified by a given SEID +func (association *PFCPAssociation) getFSEID(seid api.SEID) (*ie.IE, error) { + ieNodeID := association.LocalEntity().NodeID() + nodeID, err := ieNodeID.NodeID() + if err != nil { + return nil, err + } + var localFseid *ie.IE + switch ieNodeID.Payload[0] { + case ie.NodeIDIPv4Address: + ip4, err := net.ResolveIPAddr("ip4", nodeID) + if err != nil { + return nil, err + } + localFseid, err = NewFSEID(seid, ip4, nil) + if err != nil { + return nil, err + } + case ie.NodeIDIPv6Address: + ip6, err := net.ResolveIPAddr("ip6", nodeID) + if err != nil { + return nil, err + } + localFseid, err = NewFSEID(seid, nil, ip6) + if err != nil { + return nil, err + } + case ie.NodeIDFQDN: + ip4, err4 := net.ResolveIPAddr("ip4", nodeID) + ip6, err6 := net.ResolveIPAddr("ip6", nodeID) + if err4 != nil && err6 != nil { + return nil, fmt.Errorf("Cannot resolve NodeID") + } + localFseid = ie.NewFSEID(seid, ip4.IP.To4(), ip6.IP.To16()) + } + return localFseid, nil +} + +func (association PFCPAssociation) CreateSession(remoteFseid *ie.IE, pdrs pfcprule.PDRs, fars pfcprule.FARs) (session api.PFCPSessionInterface, err error) { + // Generation of the F-SEID + localSEID := association.GetNextSEID() + localFseid, err := association.getFSEID(localSEID) + if err != nil { + return nil, err + } + // Checking PDRs, and FARs + tmpPDR := make(pfcprule.PDRMap) + if pdrs == nil { + return nil, fmt.Errorf("No PDR in session") + } + log.Println("Adding", len(pdrs), "PDRs to session") + for _, pdr := range pdrs { + if pdr == nil { + log.Println("A PDR is nil") + continue + } + id, err := pdr.ID() + if err != nil { + return nil, err + } + tmpPDR[id] = pdr + } + tmpFAR := make(pfcprule.FARMap) + if fars == nil { + return nil, fmt.Errorf("No FAR in session") + } + log.Println("Adding", len(fars), "FARs to session") + for _, far := range fars { + if far == nil { + log.Println("A FAR is nil") + continue + } + id, err := far.ID() + if err != nil { + return nil, err + } + tmpFAR[id] = far + } + // Establishment of a PFCP Session if CP / Creation if UP + s, err := newEstablishedPFCPSession(association, localFseid, remoteFseid, tmpPDR, tmpFAR) + if err != nil { + return nil, err + } + return s, nil +} + +// Safe function to create FSEID +func NewFSEID(seid api.SEID, v4, v6 *net.IPAddr) (*ie.IE, error) { + if v4 == nil && v6 == nil { + return nil, fmt.Errorf("Cannot create FSEID with no IP Address") + } + var ip4, ip6 net.IP + if v4 != nil { + ip4 = v4.IP.To4() + } + if v6 != nil { + ip6 = v6.IP.To16() + } + return ie.NewFSEID(seid, ip4, ip6), nil +} diff --git a/pfcp/association_map.go b/pfcp/association_map.go new file mode 100644 index 0000000..cd089f7 --- /dev/null +++ b/pfcp/association_map.go @@ -0,0 +1,74 @@ +// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. +// Use of this source code is governed by a MIT-style license that can be +// found in the LICENSE file. +// SPDX-License-Identifier: MIT + +package pfcp_networking + +import ( + "fmt" + "sync" + + "github.com/louisroyer/go-pfcp-networking/pfcp/api" +) + +type associationsMap = map[string]api.PFCPAssociationInterface +type AssociationsMap struct { + associations associationsMap + muAssociations sync.RWMutex +} + +func NewAssociationsMap() AssociationsMap { + return AssociationsMap{ + associations: make(associationsMap), + muAssociations: sync.RWMutex{}, + } +} + +// Remove an association from the association table +func (a *AssociationsMap) Remove(association api.PFCPAssociationInterface) error { + nid, err := association.NodeID().NodeID() + if err != nil { + return err + } + a.muAssociations.Lock() + defer a.muAssociations.Unlock() + delete(a.associations, nid) + return nil +} + +// Add an association to the association table +func (a *AssociationsMap) Add(association api.PFCPAssociationInterface) error { + nid, err := association.NodeID().NodeID() + if err != nil { + return err + } + if _, exists := a.associations[nid]; exists { + // Only one association shall be setup between given pair of CP and UP functions. + return fmt.Errorf("Association already exist.") + } + a.muAssociations.Lock() + defer a.muAssociations.Unlock() + a.associations[nid] = association + return nil +} + +// Returns an existing PFCP Association +func (a *AssociationsMap) Get(nid string) (association api.PFCPAssociationInterface, err error) { + a.muAssociations.RLock() + defer a.muAssociations.RUnlock() + if asso, exists := a.associations[nid]; exists { + return asso, nil + } + return nil, fmt.Errorf("Association does not exist.") +} + +// Returns true if the association does not exist +func (a *AssociationsMap) CheckNonExist(nid string) bool { + a.muAssociations.RLock() + defer a.muAssociations.RUnlock() + if _, exists := a.associations[nid]; exists { + return false + } + return true +} diff --git a/pfcp/entity.go b/pfcp/entity.go new file mode 100644 index 0000000..a2c4780 --- /dev/null +++ b/pfcp/entity.go @@ -0,0 +1,207 @@ +// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. +// Use of this source code is governed by a MIT-style license that can be +// found in the LICENSE file. +// SPDX-License-Identifier: MIT + +package pfcp_networking + +import ( + "fmt" + "log" + "net" + "sync" + "time" + + "github.com/louisroyer/go-pfcp-networking/pfcp/api" + "github.com/louisroyer/go-pfcp-networking/pfcputil" + "github.com/wmnsk/go-pfcp/ie" + "github.com/wmnsk/go-pfcp/message" +) + +type handler = func(receivedMessage ReceivedMessage) error + +type PFCPEntity struct { + nodeID *ie.IE + recoveryTimeStamp *ie.IE + handlers map[pfcputil.MessageType]handler + conn *net.UDPConn + connMu sync.Mutex + associationsMap AssociationsMap + // each session is associated with a specific PFCPAssociation + // (can be changed with some requests) + // UP function receives them from CP functions + // CP function send them to UP functions + sessionsMap SessionsMap + kind string // "CP" or "UP" +} + +// Add an Established PFCP Session +func (e *PFCPEntity) AddEstablishedPFCPSession(session api.PFCPSessionInterface) error { + return e.sessionsMap.Add(session) +} + +func (e *PFCPEntity) GetPFCPSessions() []api.PFCPSessionInterface { + return e.sessionsMap.GetPFCPSessions() +} + +func (e *PFCPEntity) SendTo(msg []byte, dst net.Addr) error { + e.connMu.Lock() + defer e.connMu.Unlock() + if _, err := e.conn.WriteTo(msg, dst); err != nil { + return err + } + return nil +} + +func (e *PFCPEntity) NodeID() *ie.IE { + return e.nodeID +} +func (e *PFCPEntity) RecoveryTimeStamp() *ie.IE { + return e.recoveryTimeStamp +} + +func newDefaultPFCPEntityHandlers() map[pfcputil.MessageType]handler { + m := make(map[pfcputil.MessageType]handler) + m[message.MsgTypeHeartbeatRequest] = handleHeartbeatRequest + return m +} + +func NewPFCPEntity(nodeID string, kind string) PFCPEntity { + return PFCPEntity{ + nodeID: ie.NewNodeIDHeuristic(nodeID), + recoveryTimeStamp: nil, + handlers: newDefaultPFCPEntityHandlers(), + conn: nil, + connMu: sync.Mutex{}, + associationsMap: NewAssociationsMap(), + sessionsMap: NewSessionsMap(), + kind: kind, + } +} + +func (e *PFCPEntity) listen() error { + e.recoveryTimeStamp = ie.NewRecoveryTimeStamp(time.Now()) + // TODO: if NodeID is a FQDN, we can expose multiple ip addresses + ipAddr, err := e.NodeID().NodeID() + if err != nil { + return err + } + udpAddr := pfcputil.CreateUDPAddr(ipAddr, pfcputil.PFCP_PORT) + laddr, err := net.ResolveUDPAddr("udp", udpAddr) + if err != nil { + return err + } + e.conn, err = net.ListenUDP("udp", laddr) + if err != nil { + return err + } + + return nil +} + +func (e *PFCPEntity) GetHandler(t pfcputil.MessageType) (h handler, err error) { + if f, exists := e.handlers[t]; exists { + return f, nil + } + return nil, fmt.Errorf("Received unexpected PFCP message type") +} + +func (e *PFCPEntity) AddHandler(t pfcputil.MessageType, h handler) error { + if e.RecoveryTimeStamp() != nil { + return fmt.Errorf("Cannot add handler to already started PFCP Entity") + } + if !pfcputil.IsMessageTypeRequest(t) { + return fmt.Errorf("Only request messages can have a handler") + } + e.handlers[t] = h + return nil +} + +func (e *PFCPEntity) AddHandlers(funcs map[pfcputil.MessageType]handler) error { + if e.RecoveryTimeStamp() != nil { + return fmt.Errorf("Cannot add handler to already started PFCP Entity") + } + for t, _ := range funcs { + if !pfcputil.IsMessageTypeRequest(t) { + return fmt.Errorf("Only request messages can have a handler") + } + } + + for t, h := range funcs { + e.handlers[t] = h + } + return nil +} + +// Remove an association from the association table +func (e *PFCPEntity) RemovePFCPAssociation(association api.PFCPAssociationInterface) error { + return e.associationsMap.Remove(association) +} + +// Returns an existing PFCP Association +func (e *PFCPEntity) GetPFCPAssociation(nid string) (association api.PFCPAssociationInterface, err error) { + return e.associationsMap.Get(nid) +} + +func (e *PFCPEntity) NewEstablishedPFCPAssociation(nodeID *ie.IE) (association api.PFCPAssociationInterface, err error) { + peer, err := newPFCPPeerUP(e, nodeID) + if err != nil { + return nil, err + } + if e.RecoveryTimeStamp == nil { + return nil, fmt.Errorf("Local PFCP entity is not started") + } + nid, err := nodeID.NodeID() + if err != nil { + return nil, err + } + if !e.associationsMap.CheckNonExist(nid) { + return nil, fmt.Errorf("Association already exists") + } + a, err := peer.NewEstablishedPFCPAssociation() + if err != nil { + return nil, err + } + if err := e.associationsMap.Add(a); err != nil { + return nil, err + } + return a, nil + +} + +func (e *PFCPEntity) Start() error { + if err := e.listen(); err != nil { + return err + } + buf := make([]byte, pfcputil.DEFAULT_MTU) // TODO: get MTU of interface instead of using DEFAULT_MTU + go func() error { + for { + n, addr, err := e.conn.ReadFrom(buf) + if err != nil { + return err + } + msg, err := message.Parse(buf[:n]) + if err != nil { + // undecodable pfcp message + continue + } + f, err := e.GetHandler(msg.MessageType()) + if err != nil { + log.Println(err) + } + err = f(ReceivedMessage{Message: msg, SenderAddr: addr, Entity: e}) + if err != nil { + log.Println(err) + } + } + }() + return nil +} + +func (e *PFCPEntity) IsUserPlane() bool { + return e.kind == "CP" +} + +func (e *PFCPEntity) IsControlPlane() bool { + return e.kind == "UP" +} diff --git a/pfcp/entity_cp.go b/pfcp/entity_cp.go new file mode 100644 index 0000000..34e5340 --- /dev/null +++ b/pfcp/entity_cp.go @@ -0,0 +1,15 @@ +// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. +// Use of this source code is governed by a MIT-style license that can be +// found in the LICENSE file. +// SPDX-License-Identifier: MIT + +package pfcp_networking + +type PFCPEntityCP struct { + PFCPEntity +} + +func NewPFCPEntityCP(nodeID string) *PFCPEntityCP { + e := PFCPEntityCP{PFCPEntity: NewPFCPEntity(nodeID, "CP")} + return &e +} diff --git a/pfcp/entity_up.go b/pfcp/entity_up.go new file mode 100644 index 0000000..a750ba3 --- /dev/null +++ b/pfcp/entity_up.go @@ -0,0 +1,35 @@ +// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. +// Use of this source code is governed by a MIT-style license that can be +// found in the LICENSE file. +// SPDX-License-Identifier: MIT + +package pfcp_networking + +import ( + "log" + + "github.com/wmnsk/go-pfcp/message" +) + +type PFCPEntityUP struct { + PFCPEntity +} + +func NewPFCPSEntityUP(nodeID string) *PFCPEntityUP { + e := PFCPEntityUP{PFCPEntity: NewPFCPEntity(nodeID, "UP")} + err := e.initDefaultHandlers() + if err != nil { + log.Println(err) + } + return &e +} + +func (e *PFCPEntityUP) initDefaultHandlers() error { + if err := e.AddHandler(message.MsgTypeAssociationSetupRequest, handleAssociationSetupRequest); err != nil { + return err + } + if err := e.AddHandler(message.MsgTypeSessionEstablishmentRequest, handleSessionEstablishmentRequest); err != nil { + return err + } + return nil +} diff --git a/handlers.go b/pfcp/handlers.go similarity index 73% rename from handlers.go rename to pfcp/handlers.go index b987b68..c4c8b49 100644 --- a/handlers.go +++ b/pfcp/handlers.go @@ -11,6 +11,7 @@ import ( "log" "net" + "github.com/louisroyer/go-pfcp-networking/pfcp/api" pfcprule "github.com/louisroyer/go-pfcp-networking/pfcprules" "github.com/wmnsk/go-pfcp/ie" "github.com/wmnsk/go-pfcp/message" @@ -28,15 +29,6 @@ func handleAssociationSetupRequest(msg ReceivedMessage) error { if !ok { return fmt.Errorf("Issue with Association Setup Request") } - peer, err := NewPFCPPeer(msg.Entity, m.NodeID) - if err != nil { - return err - } - association := NewPFCPAssociation(peer) - err = msg.Entity.CreatePFCPAssociation(&association) - if err != nil { - return err - } switch { case msg.Message == nil: return fmt.Errorf("msg is nil") @@ -50,6 +42,11 @@ func handleAssociationSetupRequest(msg ReceivedMessage) error { return fmt.Errorf("entity.RecoveryTimeStamp() is nil") } + if _, err := msg.Entity.NewEstablishedPFCPAssociation(m.NodeID); err != nil { + res := message.NewAssociationSetupResponse(msg.Sequence(), msg.Entity.NodeID(), ie.NewCause(ie.CauseRequestRejected), msg.Entity.RecoveryTimeStamp()) + return msg.ReplyTo(res) + } + res := message.NewAssociationSetupResponse(msg.Sequence(), msg.Entity.NodeID(), ie.NewCause(ie.CauseRequestAccepted), msg.Entity.RecoveryTimeStamp()) return msg.ReplyTo(res) } @@ -142,7 +139,7 @@ func handleSessionEstablishmentRequest(msg ReceivedMessage) error { } // create session with PDRs and FARs - session, err := association.CreateSession(msg.Entity.GetNextRemoteSessionID(), m.CPFSEID, pdrs, fars) + session, err := association.CreateSession(m.CPFSEID, pdrs, fars) if err != nil { // Send cause(Rule creation/modification failure) res := message.NewSessionEstablishmentResponse(0, 0, rseid, msg.Sequence(), 0, msg.Entity.NodeID(), ie.NewCause(ie.CauseRuleCreationModificationFailure)) @@ -156,44 +153,47 @@ func handleSessionEstablishmentRequest(msg ReceivedMessage) error { return msg.ReplyTo(res) } -func handleSessionModificationRequest(msg ReceivedMessage) error { - log.Println("Received Session Modification Request") - _, ok := msg.Message.(*message.SessionModificationRequest) - if !ok { - return fmt.Errorf("Issue with Session Modification Request") - } - // Peer must have an association established or the message will be rejected - if _, err := checkSenderAssociation(msg.Entity, msg.SenderAddr); err != nil { - var rseid uint64 = 0 - log.Println(err) - res := message.NewSessionEstablishmentResponse(0, 0, rseid, msg.Sequence(), 0, msg.Entity.NodeID(), ie.NewCause(ie.CauseNoEstablishedPFCPAssociation)) - return msg.ReplyTo(res) - } - - // Find the Session by its F-SEID - localseid := msg.SEID() - sessions := msg.Entity.GetLocalSessions() - if _, ok := sessions[localseid]; !ok { - res := message.NewSessionModificationResponse(0, 0, 0, msg.Sequence(), 0, ie.NewCause(ie.CauseSessionContextNotFound)) - return msg.ReplyTo(res) - } - session := sessions[localseid] - rseid, err := session.RemoteSEID() - if err != nil { - return err - } - - // CP F-SEID - // This IE shall be present if the CP function decides to change its F-SEID for the - // PFCP session. The UP function shall use the new CP F-SEID for subsequent - // PFCP Session related messages for this PFCP Session - - // TODO: - res := message.NewSessionModificationResponse(0, 0, rseid, msg.Sequence(), 0, ie.NewCause(ie.CauseRequestRejected)) - return msg.ReplyTo(res) -} - -func checkSenderAssociation(entity PFCPEntityInterface, senderAddr net.Addr) (*PFCPAssociation, error) { +//func handleSessionModificationRequest(msg ReceivedMessage) error { +// log.Println("Received Session Modification Request") +// _, ok := msg.Message.(*message.SessionModificationRequest) +// if !ok { +// return fmt.Errorf("Issue with Session Modification Request") +// } +// // Peer must have an association established or the message will be rejected +// +// // PFCP session related messages for sessions that are already established are sent to the IP address received +// // in the F-SEID allocated by the peer function or to the IP address of an alternative SMF in the SMF set +// // (see clause 5.22). The former IP address needs not be configured in the look up information. +// +// // Thereforce, use of checkSenderAssociation is prohibed +// +// // Find the Session by its F-SEID +// localseid := msg.SEID() +// sessions := msg.Entity.GetLocalSessions() +// if _, ok := sessions[localseid]; !ok { +// res := message.NewSessionModificationResponse(0, 0, 0, msg.Sequence(), 0, ie.NewCause(ie.CauseSessionContextNotFound)) +// return msg.ReplyTo(res) +// } +// session := sessions[localseid] +// rseid, err := session.RemoteSEID() +// if err != nil { +// return err +// } +// +// // CP F-SEID +// // This IE shall be present if the CP function decides to change its F-SEID for the +// // PFCP session. The UP function shall use the new CP F-SEID for subsequent +// // PFCP Session related messages for this PFCP Session +// +// // TODO: +// res := message.NewSessionModificationResponse(0, 0, rseid, msg.Sequence(), 0, ie.NewCause(ie.CauseRequestRejected)) +// return msg.ReplyTo(res) +//} + +func checkSenderAssociation(entity api.PFCPEntityInterface, senderAddr net.Addr) (api.PFCPAssociationInterface, error) { + // Once the PFCP Association is established, any of the IP addresses of the peer + // function (found during the look-up) may then be used to send subsequent PFCP node related messages and PFCP + // session establishment requests for that PFCP Association. nid := senderAddr.(*net.UDPAddr).IP.String() association, err := entity.GetPFCPAssociation(nid) if err != nil { diff --git a/pfcp_peer.go b/pfcp/peer.go similarity index 57% rename from pfcp_peer.go rename to pfcp/peer.go index b91f308..d605cac 100644 --- a/pfcp_peer.go +++ b/pfcp/peer.go @@ -11,6 +11,7 @@ import ( "sync" "time" + "github.com/louisroyer/go-pfcp-networking/pfcp/api" "github.com/louisroyer/go-pfcp-networking/pfcputil" "github.com/wmnsk/go-pfcp/ie" "github.com/wmnsk/go-pfcp/message" @@ -18,18 +19,32 @@ import ( type messageChan chan []byte +// A PFCPPeer is a remote PFCPEntity type PFCPPeer struct { - NodeID *ie.IE - Srv PFCPEntityInterface + nodeID *ie.IE + srv api.PFCPEntityInterface conn *net.UDPConn udpAddr *net.UDPAddr seq uint32 + seqMu sync.Mutex queue map[uint32]messageChan - mu sync.Mutex + queueMu sync.Mutex stop bool + kind string } -func NewPFCPPeer(srv PFCPEntityInterface, nodeID *ie.IE) (peer *PFCPPeer, err error) { +func (peer *PFCPPeer) NewEstablishedPFCPAssociation() (api.PFCPAssociationInterface, error) { + return newEstablishedPFCPAssociation(peer) +} + +func (peer *PFCPPeer) LocalEntity() api.PFCPEntityInterface { + return peer.srv +} + +func (peer *PFCPPeer) NodeID() *ie.IE { + return peer.nodeID +} +func newPFCPPeer(srv api.PFCPEntityInterface, nodeID *ie.IE, kind string) (peer *PFCPPeer, err error) { ipAddr, err := nodeID.NodeID() if err != nil { return nil, err @@ -50,85 +65,117 @@ func NewPFCPPeer(srv PFCPEntityInterface, nodeID *ie.IE) (peer *PFCPPeer, err er return nil, err } p := PFCPPeer{ - Srv: srv, - NodeID: nodeID, + srv: srv, + nodeID: nodeID, conn: conn, udpAddr: raddr, seq: 1, + seqMu: sync.Mutex{}, queue: make(map[uint32]messageChan), - mu: sync.Mutex{}, + queueMu: sync.Mutex{}, stop: false, + kind: kind, } // Read incomming messages - go func(e PFCPPeer) { - var stop bool = false - for !stop { - b := make([]byte, pfcputil.DEFAULT_MTU) // TODO: detect MTU for interface instead of using DEFAULT_MTU - n, _, err := e.conn.ReadFromUDP(b) - if err != nil { - // socket has been closed - return - } - msg, err := message.ParseHeader(b[:n]) - if err != nil { - e.mu.Lock() - stop = e.stop - e.mu.Unlock() - if stop { - return - } else { - continue - } - } - sn := msg.SequenceNumber - e.mu.Lock() - ch, exists := e.queue[sn] - if exists { - ch <- b[:n] - } - stop = e.stop - e.mu.Unlock() - } - }(p) + p.start() return &p, nil } -// Close connection of PFCPPeer -func (peer *PFCPPeer) Close() error { - peer.mu.Lock() - if !peer.stop { - peer.stop = true - err := peer.conn.Close() +func newPFCPPeerUP(srv api.PFCPEntityInterface, nodeID *ie.IE) (peer *PFCPPeer, err error) { + return newPFCPPeer(srv, nodeID, "UP") +} +func newPFCPPeerCP(srv api.PFCPEntityInterface, nodeID *ie.IE) (peer *PFCPPeer, err error) { + return newPFCPPeer(srv, nodeID, "CP") +} + +func (peer *PFCPPeer) IsUserPlane() bool { + return peer.kind == "UP" +} + +func (peer *PFCPPeer) IsControlPlane() bool { + return peer.kind == "CP" +} + +func (peer *PFCPPeer) start() { + go func(e *PFCPPeer) { + e.startLoop() + }(peer) +} +func (peer *PFCPPeer) startLoop() { + for peer.IsRunning() { + peer.loopUnwrapped() + } +} + +func (peer *PFCPPeer) loopUnwrapped() { + b := make([]byte, pfcputil.DEFAULT_MTU) // TODO: detect MTU for interface instead of using DEFAULT_MTU + n, _, err := peer.conn.ReadFromUDP(b) + if err != nil { + // socket has been closed + return + } + // Processing of message in a new thread to avoid blocking + go func(msgArray []byte, size int, e *PFCPPeer) { + msg, err := message.ParseHeader(msgArray[:size]) if err != nil { - return err + return + } + sn := msg.SequenceNumber + + e.queueMu.Lock() + defer e.queueMu.Unlock() + ch, exists := e.queue[sn] + if exists { + ch <- msgArray[:size] } + }(b, n, peer) +} + +func (peer *PFCPPeer) IsRunning() bool { + return peer.stop +} + +// Close connection of PFCPPeer +func (peer *PFCPPeer) Close() error { + // if already stopped, for whatever reason, we exit + if peer.stop { + return nil + } + // setting stop state and closing connection + peer.stop = true + err := peer.conn.Close() + if err != nil { + return err } - peer.mu.Unlock() return nil } // Get next sequence number available for this PFCPPeer +// Sequence murber shall be unique for each oustanding +// message sourced from the same IP/UDP endpoint. +// Since we use exactly 1 IP/UDP endpoint per peer to send Requests, +// our sequence numbers are also unique per peer. func (peer *PFCPPeer) getNextSequenceNumber() uint32 { - peer.mu.Lock() + peer.seqMu.Lock() + defer peer.seqMu.Unlock() s := peer.seq peer.seq += 1 - peer.mu.Unlock() return s } // Add a message to queue. Response will be send to channel ch messageChan func (peer *PFCPPeer) addToQueue(sn uint32, ch messageChan) { - peer.mu.Lock() + peer.queueMu.Lock() + defer peer.queueMu.Unlock() peer.queue[sn] = ch - peer.mu.Unlock() } // Remove a message from queue (used when a response is received, or when timeout is reached) func (peer *PFCPPeer) deleteFromQueue(sn uint32) { - peer.mu.Lock() + peer.queueMu.Lock() + defer peer.queueMu.Unlock() close(peer.queue[sn]) delete(peer.queue, sn) - peer.mu.Unlock() } // Send a PFCP message @@ -188,12 +235,12 @@ func (peer *PFCPPeer) Send(msg message.Message) (m message.Message, err error) { // Send an Heartbeat request, return true if the PFCP peer is alive. func (peer *PFCPPeer) IsAlive() (res bool, err error) { - if peer.Srv.RecoveryTimeStamp() == nil { - return false, fmt.Errorf("PFCP server entity is not started.") + if peer.LocalEntity().RecoveryTimeStamp() == nil { + return false, fmt.Errorf("Local PFCP Entity is not yet started.") } hreq := message.NewHeartbeatRequest( 0, - peer.Srv.RecoveryTimeStamp(), + peer.LocalEntity().RecoveryTimeStamp(), nil) _, err = peer.Send(hreq) diff --git a/pfcp/received_message.go b/pfcp/received_message.go new file mode 100644 index 0000000..addf883 --- /dev/null +++ b/pfcp/received_message.go @@ -0,0 +1,42 @@ +// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. +// Use of this source code is governed by a MIT-style license that can be +// found in the LICENSE file. +// SPDX-License-Identifier: MIT + +package pfcp_networking + +import ( + "fmt" + "net" + + "github.com/louisroyer/go-pfcp-networking/pfcp/api" + "github.com/louisroyer/go-pfcp-networking/pfcputil" + "github.com/wmnsk/go-pfcp/message" +) + +type ReceivedMessage struct { + message.Message + SenderAddr net.Addr + Entity api.PFCPEntityInterface +} + +func (receivedMessage *ReceivedMessage) ReplyTo(responseMessage message.Message) error { + if !pfcputil.IsMessageTypeRequest(receivedMessage.MessageType()) { + return fmt.Errorf("receivedMessage shall be a Request Message") + } + if !pfcputil.IsMessageTypeResponse(responseMessage.MessageType()) { + return fmt.Errorf("responseMessage shall be a Response Message") + } + if receivedMessage.Sequence() != responseMessage.Sequence() { + return fmt.Errorf("responseMessage shall have the same Sequence Number than receivedMessage") + } + //XXX: message.Message interface does not implement Marshal() + b := make([]byte, responseMessage.MarshalLen()) + if err := responseMessage.MarshalTo(b); err != nil { + return err + } + if err := receivedMessage.Entity.SendTo(b, receivedMessage.SenderAddr); err != nil { + return err + } + return nil +} diff --git a/pfcp/session.go b/pfcp/session.go new file mode 100644 index 0000000..e03f00e --- /dev/null +++ b/pfcp/session.go @@ -0,0 +1,236 @@ +// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. +// Use of this source code is governed by a MIT-style license that can be +// found in the LICENSE file. +// SPDX-License-Identifier: MIT + +package pfcp_networking + +import ( + "fmt" + "log" + "net" + "sort" + "sync" + + "github.com/louisroyer/go-pfcp-networking/pfcp/api" + pfcprule "github.com/louisroyer/go-pfcp-networking/pfcprules" + "github.com/wmnsk/go-pfcp/ie" + "github.com/wmnsk/go-pfcp/message" +) + +type PFCPSession struct { + // isEstablished flag is used when PFCP Session Establishment Procedure has been completed + // (can be initiated from the Local Entity or the Remote Peer, depending on kind of peer (UP/CP) + isEstablished bool + // association is used to send Request type PFCP Messages + association api.PFCPAssociationInterface // XXX: use remoteFSEID to find the association from LocalEntity instead of storing an association + // When Peer A send a message (M) to Peer B + // M.PFCPHeader.SEID = B.LocalSEID() = A.RemoteSEID() + // M.IPHeader.IP_DST = B.LocalIPAddress = A.RemoteIPAddress() + localFseid *ie.IE // local F-SEID + remoteFseid *ie.IE // remote F-SEID, on UP function this is allocated at Setup() time + // PDR Map allow to retrieve a specific PDR by its ID + pdr pfcprule.PDRMap + // sortedPDR is used to perform PDR finding using PDI Matching + sortedPDR pfcprule.PDRs + // FAR Map allow to retrieve a specific FAR by its ID + far pfcprule.FARMap + // allows to perform atomic operations + // This RWMutex applies on sortedPDR, pdr, and far + atomicMu sync.RWMutex +} + +// Create an EstablishedPFCPSession +// Use this function when a PFCP Session Establishment Request is received (UP case), +// or when the Entity want to send a PFCP Session Establishment Request (CP case). +func newEstablishedPFCPSession(association api.PFCPAssociationInterface, fseid, rseid *ie.IE, pdrs pfcprule.PDRMap, fars pfcprule.FARMap) (api.PFCPSessionInterface, error) { + s := PFCPSession{ + isEstablished: false, + association: association, + localFseid: fseid, // local F-SEID + remoteFseid: rseid, // SEID present in FSEID ie send by remote peer + pdr: pdrs, + far: fars, + sortedPDR: make(pfcprule.PDRs, 0), + atomicMu: sync.RWMutex{}, + } + // sort PDRs + for _, p := range pdrs { + s.sortedPDR = append(s.sortedPDR, p) + } + sort.Sort(s.sortedPDR) + if err := s.Setup(); err != nil { + return nil, err + } + // Add to SessionFSEIDMap of LocalEntity + s.association.LocalEntity().AddEstablishedPFCPSession(s) + return s, nil +} + +// Get local F-SEID of this session +// This value should be used when a session related message is received. +func (s PFCPSession) LocalFSEID() *ie.IE { + return s.localFseid +} + +// Get SEID part of local F-SEID +// This value should be used when a session related message is received. +func (s PFCPSession) LocalSEID() (api.SEID, error) { + fseid, err := s.localFseid.FSEID() + if err != nil { + return 0, err + } + return fseid.SEID, nil +} + +// Get IP Address part of local F-SEID +// This value should be used when a session related message is received. +func (s PFCPSession) LocalIPAddress() (net.IP, error) { + fseid, err := s.localFseid.FSEID() + if err != nil { + return nil, err + } + switch { + case fseid.HasIPv6(): + return fseid.IPv6Address, nil + case fseid.HasIPv4(): + return fseid.IPv4Address, nil + default: + return nil, fmt.Errorf("Local IP Address not set") + } +} + +// Get remote F-SEID of this session +// This value should be used when a session related message is send. +func (s PFCPSession) RemoteFSEID() *ie.IE { + return s.remoteFseid +} + +// Get SEID part of remote F-SEID +// This value should be used when a session related message is send. +func (s PFCPSession) RemoteSEID() (api.SEID, error) { + fseid, err := s.remoteFseid.FSEID() + if err != nil { + return 0, err + } + return fseid.SEID, nil +} + +// Get IP Address part of remote F-SEID +// This value should be used when a session related message is send. +func (s PFCPSession) RemoteIPAddress() (net.IP, error) { + fseid, err := s.remoteFseid.FSEID() + if err != nil { + return nil, err + } + switch { + case fseid.HasIPv6(): + return fseid.IPv6Address, nil + case fseid.HasIPv4(): + return fseid.IPv4Address, nil + default: + return nil, fmt.Errorf("Remote IP Address not set") + } +} + +// Returns PDRs sorted by Precedence +// For PDI checking, the checking order is: +// look first at the first item of the array, +// look last at the last item of the array. +func (s PFCPSession) GetPDRs() pfcprule.PDRs { + s.atomicMu.RLock() + defer s.atomicMu.RUnlock() + return s.sortedPDR +} + +// Get FAR associated with this FARID +func (s PFCPSession) GetFAR(farid pfcprule.FARID) (*pfcprule.FAR, error) { + s.atomicMu.RLock() + defer s.atomicMu.RUnlock() + if far, ok := s.far[farid]; ok { + return far, nil + } + return nil, fmt.Errorf("No FAR with id", farid) +} + +// Add PDRs to the session +// This is an internal function, not thread safe +func (s *PFCPSession) addPDRsUnsafe(pdrs pfcprule.PDRMap) { + // Transactions must be atomic to avoid having a PDR referring to a deleted FAR / not yet created FAR + for id, pdr := range pdrs { + s.pdr[id] = pdr + s.sortedPDR = append(s.sortedPDR, pdr) + } + sort.Sort(s.sortedPDR) +} + +// Add FARs to the session +// This is an internal function, not thread safe +func (s *PFCPSession) addFARsUnsafe(fars pfcprule.FARMap) { + // Transactions must be atomic to avoid having a PDR referring to a deleted FAR / not yet created FAR + for id, far := range fars { + s.far[id] = far + } +} + +// Add PDRs and FARs to the session +func (s PFCPSession) AddPDRsFARs(pdrs pfcprule.PDRMap, fars pfcprule.FARMap) { + // Transactions must be atomic to avoid having a PDR referring to a deleted FAR / not yet created FAR + s.atomicMu.Lock() + defer s.atomicMu.Unlock() + s.addPDRsUnsafe(pdrs) + s.addFARsUnsafe(fars) + // TODO: if isUserPlane() -> send the Session Modification Request +} + +// Set the remote FSEID of a PFCPSession +// it must be used for next session related messages +//func (s PFCPSession) SetRemoteFSEID(FSEID *ie.IE) { +// s.remoteFseid = FSEID +//XXX: change association to the right-one (unless XXX line 26 is fixed) +// update sessionsmap in local entity +//} + +// Setup function, either by: +// performing the PFCP Session Establishment Procedure (if CP function), +// or by doing nothing particular (if UP function) since +// the PFCP Session Establishment Procedure is already performed +func (s PFCPSession) Setup() error { + if s.isEstablished { + return fmt.Errorf("Session is already establihed") + } + switch { + case s.association.LocalEntity().IsUserPlane(): + // Nothing more to do + s.isEstablished = true + return nil + case s.association.LocalEntity().IsControlPlane(): + // Send PFCP Session Setup Request + // first add to temporary map to avoid erroring after msg is send + ies := make([]*ie.IE, 0) + ies = append(ies, s.association.LocalEntity().NodeID()) + ies = append(ies, s.localFseid) + for _, pdr := range pfcprule.NewCreatePDRs(s.pdr) { + ies = append(ies, pdr) + } + for _, far := range pfcprule.NewCreateFARs(s.far) { + ies = append(ies, far) + } + + msg := message.NewSessionEstablishmentRequest(0, 0, 0, 0, 0, ies...) + resp, err := s.association.Send(msg) + if err != nil { + return err + } + ser, ok := resp.(*message.SessionEstablishmentResponse) + if !ok { + log.Printf("got unexpected message: %s\n", resp.MessageTypeName()) + } + s.remoteFseid = ser.UPFSEID + s.isEstablished = true + return nil + default: + return fmt.Errorf("Local PFCP entity is not a CP or a UP function") + } + return nil +} diff --git a/pfcp/session_id_pool.go b/pfcp/session_id_pool.go new file mode 100644 index 0000000..c738a9d --- /dev/null +++ b/pfcp/session_id_pool.go @@ -0,0 +1,35 @@ +// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. +// Use of this source code is governed by a MIT-style license that can be +// found in the LICENSE file. +// SPDX-License-Identifier: MIT + +package pfcp_networking + +import ( + "sync" + + "github.com/louisroyer/go-pfcp-networking/pfcp/api" +) + +// SessionIDPool is a generator of session IDs +type SessionIDPool struct { + currentSessionID api.SEID + muSessionID sync.Mutex +} + +// Create a SessionIDPool +func NewSessionIDPool() SessionIDPool { + return SessionIDPool{ + currentSessionID: 0, + muSessionID: sync.Mutex{}, + } +} + +// Get next id available in SessionIDPool +func (pool *SessionIDPool) GetNext() api.SEID { + pool.muSessionID.Lock() + defer pool.muSessionID.Unlock() + id := pool.currentSessionID + pool.currentSessionID = id + 1 + return id +} diff --git a/pfcp/session_map.go b/pfcp/session_map.go new file mode 100644 index 0000000..9af3d84 --- /dev/null +++ b/pfcp/session_map.go @@ -0,0 +1,63 @@ +// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. +// Use of this source code is governed by a MIT-style license that can be +// found in the LICENSE file. +// SPDX-License-Identifier: MIT + +package pfcp_networking + +import ( + "sync" + + "github.com/louisroyer/go-pfcp-networking/pfcp/api" +) + +type sessionsMapSEID = map[api.SEID]api.PFCPSessionInterface +type sessionsMapFSEID = map[string]sessionsMapSEID +type SessionsMap struct { + sessions sessionsMapFSEID + muSessions sync.RWMutex +} + +// Add a session to the map +func (sm *SessionsMap) Add(session api.PFCPSessionInterface) error { + sm.muSessions.Lock() + defer sm.muSessions.Unlock() + // Get splitted F-SEID + localIPAddr, err := session.LocalIPAddress() + if err != nil { + return err + } + localIP := localIPAddr.String() + localSEID, err := session.LocalSEID() + if err != nil { + return err + } + // Create submap if first session with this localIP + if _, exists := sm.sessions[localIP]; !exists { + sm.sessions[localIP] = make(sessionsMapSEID, 0) + } + // Add session + sm.sessions[localIP][localSEID] = session + return nil +} + +// Create a new SessionMap +func NewSessionsMap() SessionsMap { + return SessionsMap{ + sessions: make(sessionsMapFSEID, 0), + muSessions: sync.RWMutex{}, + } +} + +// Returns pfcpsessions in an array +func (sm *SessionsMap) GetPFCPSessions() []api.PFCPSessionInterface { + sm.muSessions.RLock() + defer sm.muSessions.RUnlock() + sessions := make([]api.PFCPSessionInterface, 0) + for _, byseid := range sm.sessions { + for _, session := range byseid { + sessions = append(sessions, session) + } + } + return sessions +} diff --git a/pfcp_client_entity.go b/pfcp_client_entity.go deleted file mode 100644 index ae5deaa..0000000 --- a/pfcp_client_entity.go +++ /dev/null @@ -1,136 +0,0 @@ -// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. -// Use of this source code is governed by a MIT-style license that can be -// found in the LICENSE file. -// SPDX-License-Identifier: MIT - -package pfcp_networking - -import ( - "fmt" - "log" - "sync" - - "github.com/louisroyer/go-pfcp-networking/pfcputil" - "github.com/wmnsk/go-pfcp/ie" - "github.com/wmnsk/go-pfcp/message" -) - -type PFCPClientEntity struct { - PFCPEntity - associations map[string]*PFCPAssociation - muAssociations sync.Mutex -} - -func NewPFCPClientEntity(nodeID string) *PFCPClientEntity { - e := PFCPClientEntity{PFCPEntity: NewPFCPEntity(nodeID), - associations: make(map[string]*PFCPAssociation), - muAssociations: sync.Mutex{}, - } - return &e -} - -func (e *PFCPClientEntity) GetLocalSessions() PFCPSessionMapSEID { - // TODO: Store Session global map directly in the entity and only store array of SEIDs in association - var s PFCPSessionMapSEID - for _, a := range e.associations { - for k, v := range a.GetSessions() { - s[k] = v - } - } - return s -} - -// Add an association to the association table -func (e *PFCPClientEntity) CreatePFCPAssociation(association *PFCPAssociation) error { - nid, err := association.NodeID.NodeID() - if err != nil { - return err - } - e.muAssociations.Lock() - e.associations[nid] = association - e.muAssociations.Unlock() - return nil -} - -// Remove an association from the association table -func (e *PFCPClientEntity) RemovePFCPAssociation(association *PFCPAssociation) error { - nid, err := association.NodeID.NodeID() - if err != nil { - return err - } - e.muAssociations.Lock() - delete(e.associations, nid) - e.muAssociations.Unlock() - return nil -} - -// Returns an existing PFCP Association -func (e *PFCPClientEntity) GetPFCPAssociation(nid string) (association *PFCPAssociation, err error) { - if a, exists := e.associations[nid]; exists { - return a, nil - } - return nil, fmt.Errorf("Association does not exist.") -} - -// Create a PFCP Association, by sending a PFCP Association Setup Request -func (e *PFCPClientEntity) NewPFCPAssociation(peer *PFCPPeer) (association *PFCPAssociation, err error) { - if e.RecoveryTimeStamp == nil { - return nil, fmt.Errorf("Local PFCP entity is not started") - } - nid, err := peer.NodeID.NodeID() - if err != nil { - return nil, err - } - if _, exists := e.associations[nid]; exists { - return nil, fmt.Errorf("Only one association shall be setup between given pair of CP and UP functions.") - } - sar := message.NewAssociationSetupRequest(0, e.NodeID(), e.RecoveryTimeStamp()) - resp, err := peer.Send(sar) - if err != nil { - return nil, err - } - asres, ok := resp.(*message.AssociationSetupResponse) - if !ok { - log.Printf("got unexpected message: %s\n", resp.MessageTypeName()) - } - cause, err := asres.Cause.Cause() - if err != nil { - // TODO: send missing ie message - return nil, err - } - if cause == ie.CauseRequestAccepted { - a := NewPFCPAssociation(peer) - e.CreatePFCPAssociation(&a) - return &a, nil - } - return nil, fmt.Errorf("Associaton setup request rejected") -} - -func (e *PFCPClientEntity) Start() error { - if err := e.listen(); err != nil { - return err - } - buf := make([]byte, pfcputil.DEFAULT_MTU) // TODO: get MTU of interface instead of using DEFAULT_MTU - go func() error { - for { - n, addr, err := e.conn.ReadFrom(buf) - if err != nil { - return err - } - msg, err := message.Parse(buf[:n]) - if err != nil { - // undecodable pfcp message - continue - } - f, err := e.GetHandler(msg.MessageType()) - if err != nil { - log.Println(err) - } - err = f(ReceivedMessage{Message: msg, SenderAddr: addr, Entity: e}) - if err != nil { - log.Println(err) - } - } - }() - return nil -} diff --git a/pfcp_entity.go b/pfcp_entity.go deleted file mode 100644 index 4b61e8f..0000000 --- a/pfcp_entity.go +++ /dev/null @@ -1,163 +0,0 @@ -// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. -// Use of this source code is governed by a MIT-style license that can be -// found in the LICENSE file. -// SPDX-License-Identifier: MIT - -package pfcp_networking - -import ( - "fmt" - "net" - "sync" - "time" - - "github.com/louisroyer/go-pfcp-networking/pfcputil" - "github.com/wmnsk/go-pfcp/ie" - "github.com/wmnsk/go-pfcp/message" -) - -type PFCPEntityInterface interface { - NodeID() *ie.IE - RecoveryTimeStamp() *ie.IE - CreatePFCPAssociation(association *PFCPAssociation) error - RemovePFCPAssociation(association *PFCPAssociation) error - GetPFCPAssociation(nid string) (association *PFCPAssociation, err error) - GetNextRemoteSessionID() uint64 - GetLocalSessions() PFCPSessionMapSEID - sendTo(msg []byte, dst net.Addr) error -} - -type ReceivedMessage struct { - message.Message - SenderAddr net.Addr - Entity PFCPEntityInterface -} - -func (receivedMessage *ReceivedMessage) ReplyTo(responseMessage message.Message) error { - if !pfcputil.IsMessageTypeRequest(receivedMessage.MessageType()) { - return fmt.Errorf("receivedMessage shall be a Request Message") - } - if !pfcputil.IsMessageTypeResponse(responseMessage.MessageType()) { - return fmt.Errorf("responseMessage shall be a Response Message") - } - if receivedMessage.Sequence() != responseMessage.Sequence() { - return fmt.Errorf("responseMessage shall have the same Sequence Number than receivedMessage") - } - //XXX: message.Message interface does not implement Marshal() - b := make([]byte, responseMessage.MarshalLen()) - if err := responseMessage.MarshalTo(b); err != nil { - return err - } - if err := receivedMessage.Entity.sendTo(b, receivedMessage.SenderAddr); err != nil { - return err - } - return nil -} - -type handler = func(receivedMessage ReceivedMessage) error - -type PFCPEntity struct { - nodeID *ie.IE - recoveryTimeStamp *ie.IE - handlers map[pfcputil.MessageType]handler - conn *net.UDPConn - mu sync.Mutex - nextRemoteSessionID uint64 - muSessionID sync.Mutex -} - -func (e *PFCPEntity) sendTo(msg []byte, dst net.Addr) error { - e.mu.Lock() - defer e.mu.Unlock() - if _, err := e.conn.WriteTo(msg, dst); err != nil { - return err - } - return nil -} - -func (e *PFCPEntity) GetNextRemoteSessionID() uint64 { - e.muSessionID.Lock() - id := e.nextRemoteSessionID - e.nextRemoteSessionID = id + 1 - e.muSessionID.Unlock() - return id -} - -func (e *PFCPEntity) NodeID() *ie.IE { - return e.nodeID -} -func (e *PFCPEntity) RecoveryTimeStamp() *ie.IE { - return e.recoveryTimeStamp -} - -func newDefaultPFCPEntityHandlers() map[pfcputil.MessageType]handler { - m := make(map[pfcputil.MessageType]handler) - m[message.MsgTypeHeartbeatRequest] = handleHeartbeatRequest - return m -} - -func NewPFCPEntity(nodeID string) PFCPEntity { - return PFCPEntity{ - nodeID: ie.NewNodeIDHeuristic(nodeID), - recoveryTimeStamp: nil, - handlers: newDefaultPFCPEntityHandlers(), - conn: nil, - mu: sync.Mutex{}, - nextRemoteSessionID: 1, - muSessionID: sync.Mutex{}, - } -} - -func (e *PFCPEntity) listen() error { - e.recoveryTimeStamp = ie.NewRecoveryTimeStamp(time.Now()) - // TODO: if NodeID is a FQDN, we can expose multiple ip addresses - ipAddr, err := e.NodeID().NodeID() - if err != nil { - return err - } - udpAddr := pfcputil.CreateUDPAddr(ipAddr, pfcputil.PFCP_PORT) - laddr, err := net.ResolveUDPAddr("udp", udpAddr) - if err != nil { - return err - } - e.conn, err = net.ListenUDP("udp", laddr) - if err != nil { - return err - } - - return nil -} - -func (e *PFCPEntity) GetHandler(t pfcputil.MessageType) (h handler, err error) { - if f, exists := e.handlers[t]; exists { - return f, nil - } - return nil, fmt.Errorf("Received unexpected PFCP message type") -} - -func (e *PFCPEntity) AddHandler(t pfcputil.MessageType, h handler) error { - if e.RecoveryTimeStamp() != nil { - return fmt.Errorf("Cannot add handler to already started PFCP Entity") - } - if !pfcputil.IsMessageTypeRequest(t) { - return fmt.Errorf("Only request messages can have a handler") - } - e.handlers[t] = h - return nil -} - -func (e *PFCPEntity) AddHandlers(funcs map[pfcputil.MessageType]handler) error { - if e.RecoveryTimeStamp() != nil { - return fmt.Errorf("Cannot add handler to already started PFCP Entity") - } - for t, _ := range funcs { - if !pfcputil.IsMessageTypeRequest(t) { - return fmt.Errorf("Only request messages can have a handler") - } - } - - for t, h := range funcs { - e.handlers[t] = h - } - return nil -} diff --git a/pfcp_server_entity.go b/pfcp_server_entity.go deleted file mode 100644 index 2c9dd9b..0000000 --- a/pfcp_server_entity.go +++ /dev/null @@ -1,132 +0,0 @@ -// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. -// Use of this source code is governed by a MIT-style license that can be -// found in the LICENSE file. -// SPDX-License-Identifier: MIT - -package pfcp_networking - -import ( - "fmt" - "log" - "sync" - - "github.com/louisroyer/go-pfcp-networking/pfcputil" - "github.com/wmnsk/go-pfcp/message" -) - -type PFCPServerEntity struct { - PFCPEntity - associations map[string]*PFCPAssociation - muAssociations sync.Mutex -} - -func NewPFCPServerEntity(nodeID string) *PFCPServerEntity { - e := PFCPServerEntity{PFCPEntity: NewPFCPEntity(nodeID), - associations: make(map[string]*PFCPAssociation), - muAssociations: sync.Mutex{}, - } - err := e.initDefaultHandlers() - if err != nil { - log.Println(err) - } - return &e -} - -func (e *PFCPServerEntity) initDefaultHandlers() error { - if err := e.AddHandler(message.MsgTypeAssociationSetupRequest, handleAssociationSetupRequest); err != nil { - return err - } - if err := e.AddHandler(message.MsgTypeSessionEstablishmentRequest, handleSessionEstablishmentRequest); err != nil { - return err - } - return nil -} - -func (e *PFCPServerEntity) GetLocalSessions() PFCPSessionMapSEID { - // TODO: Store Session global map directly in the entity and only store array of SEIDs in association - var s PFCPSessionMapSEID - for _, a := range e.associations { - for k, v := range a.GetSessions() { - s[k] = v - } - } - return s -} - -// Add an association to the association table -func (e *PFCPServerEntity) CreatePFCPAssociation(association *PFCPAssociation) error { - nid, err := association.NodeID.NodeID() - if err != nil { - return err - } - log.Println("Storing new association with nodeid: ", nid) - e.muAssociations.Lock() - // TODO: - // if the PFCP Association for this nid was already established: - // 1. if PFCP Session Retention Information was received in the request: retain existing sessions and set PSREI flag to 1 in response - // else: delete existing sessions - // 2. delete previous association - e.associations[nid] = association - e.muAssociations.Unlock() - return nil -} - -// Returns an existing PFCP Association -func (e *PFCPServerEntity) GetPFCPAssociation(nid string) (association *PFCPAssociation, err error) { - if a, exists := e.associations[nid]; exists { - return a, nil - } - return nil, fmt.Errorf("Association does not exist.") -} - -// Remove an association from the association table -func (e *PFCPServerEntity) RemovePFCPAssociation(association *PFCPAssociation) error { - nid, err := association.NodeID.NodeID() - if err != nil { - return err - } - e.muAssociations.Lock() - delete(e.associations, nid) - e.muAssociations.Unlock() - return nil -} - -func (e *PFCPServerEntity) Start() error { - if err := e.listen(); err != nil { - return err - } - buf := make([]byte, pfcputil.DEFAULT_MTU) // TODO: get MTU of interface instead of using DEFAULT_MTU - go func() error { - for { - n, addr, err := e.conn.ReadFrom(buf) - if err != nil { - return err - } - msg, err := message.Parse(buf[:n]) - if err != nil { - // undecodable pfcp message - continue - } - f, err := e.GetHandler(msg.MessageType()) - if err != nil { - log.Println(err) - } - err = f(ReceivedMessage{Message: msg, SenderAddr: addr, Entity: e}) - if err != nil { - log.Println(err) - } - } - }() - return nil -} - -func (e *PFCPServerEntity) GetPFCPSessions() []*PFCPSession { - sessions := make([]*PFCPSession, 0) - for _, a := range e.associations { - as := a.GetSessions() - for _, s := range as { - sessions = append(sessions, s) - } - } - return sessions -} diff --git a/pfcp_session.go b/pfcp_session.go deleted file mode 100644 index f715271..0000000 --- a/pfcp_session.go +++ /dev/null @@ -1,191 +0,0 @@ -// Copyright 2022 Louis Royer and the go-pfcp-networking contributors. All rights reserved. -// Use of this source code is governed by a MIT-style license that can be -// found in the LICENSE file. -// SPDX-License-Identifier: MIT - -package pfcp_networking - -import ( - "fmt" - "log" - "net" - "sort" - "sync" - - pfcprule "github.com/louisroyer/go-pfcp-networking/pfcprules" - "github.com/wmnsk/go-pfcp/ie" - "github.com/wmnsk/go-pfcp/message" -) - -type PFCPSessionMapSEID = map[uint64]*PFCPSession -type RemotePFCPSessionMapSEID = map[uint64]*RemotePFCPSession - -// A PFCPSession is controlled localy -type PFCPSession struct { - // When Peer A send a message (M) to Peer B - // M.PFCPHeader.SEID = B.LocalSEID() = A.RemoteSEID() - // M.IPHeader.IP_DST = B.LocalIPAddress = A.RemoteIPAddress() - localFseid *ie.IE // local F-SEID - remoteFseid *ie.IE // remote F-SEID - pdr map[uint16]*pfcprule.PDR - far map[uint32]*pfcprule.FAR - sortedPDR pfcprule.PDRs - mu sync.Mutex -} - -func NewPFCPSession(fseid, rseid *ie.IE) PFCPSession { - return PFCPSession{ - localFseid: fseid, // local F-SEID - remoteFseid: rseid, // SEID present in FSEID ie send by remote peer - pdr: make(map[uint16]*pfcprule.PDR), - far: make(map[uint32]*pfcprule.FAR), - sortedPDR: make(pfcprule.PDRs, 0), - mu: sync.Mutex{}, - } -} - -func (s *PFCPSession) LocalFSEID() *ie.IE { - return s.localFseid -} - -func (s *PFCPSession) LocalSEID() (uint64, error) { - fseid, err := s.localFseid.FSEID() - if err != nil { - return 0, err - } - return fseid.SEID, nil -} - -func (s *PFCPSession) LocalIPAddress() (net.IP, error) { - fseid, err := s.localFseid.FSEID() - if err != nil { - return nil, err - } - switch { - case fseid.HasIPv6(): - return fseid.IPv6Address, nil - case fseid.HasIPv4(): - return fseid.IPv4Address, nil - default: - return nil, fmt.Errorf("Local IP Address not set") - } -} - -func (s *PFCPSession) RemoteFSEID() *ie.IE { - return s.remoteFseid -} - -func (s *PFCPSession) RemoteSEID() (uint64, error) { - fseid, err := s.remoteFseid.FSEID() - if err != nil { - return 0, err - } - return fseid.SEID, nil -} - -func (s *PFCPSession) RemoteIPAddress() (net.IP, error) { - fseid, err := s.remoteFseid.FSEID() - if err != nil { - return nil, err - } - switch { - case fseid.HasIPv6(): - return fseid.IPv6Address, nil - case fseid.HasIPv4(): - return fseid.IPv4Address, nil - default: - return nil, fmt.Errorf("Remote IP Address not set") - } -} - -func (s *PFCPSession) GetPDRs() pfcprule.PDRs { - return s.sortedPDR -} - -func (s *PFCPSession) GetFAR(farid uint32) (*pfcprule.FAR, error) { - if far, ok := s.far[farid]; ok { - return far, nil - } - return nil, fmt.Errorf("No far with id", farid) -} - -func (s *PFCPSession) AddPDRs(pdrs map[uint16]*pfcprule.PDR) { - s.mu.Lock() - for id, pdr := range pdrs { - s.pdr[id] = pdr - s.sortedPDR = append(s.sortedPDR, pdr) - } - sort.Sort(s.sortedPDR) - s.mu.Unlock() - -} -func (s *PFCPSession) AddFARs(fars map[uint32]*pfcprule.FAR) { - s.mu.Lock() - for id, far := range fars { - s.far[id] = far - } - s.mu.Unlock() -} - -// Set the remote FSEID of a PFCPSession -func (s *PFCPSession) SetRemoteFSEID(FSEID *ie.IE) { -} - -// A RemotePFCPSession is a PFCPSession that will be pushed to a remote Associated Peer -// Peer A create a RemotePFCPSession and push it to Peer B: -// 1. A.NewRemotePFCPSessionp(…).Start() -// 2. Now on B side, a PFCPSession is created with the rules defined by A -type RemotePFCPSession struct { - PFCPSession - association *PFCPAssociation -} - -func NewRemotePFCPSession(localFseid *ie.IE, association *PFCPAssociation) RemotePFCPSession { - return RemotePFCPSession{ - PFCPSession: NewPFCPSession(localFseid, nil), // Remote SEID is initialized when a Session Establishment Response is received - association: association, - } -} - -func (s *RemotePFCPSession) Start(pdrs []*pfcprule.PDR, fars []*pfcprule.FAR) error { - // first add to temporary map to avoid erroring after msg is send - tmpPDR := make(map[uint16]*pfcprule.PDR) - for _, pdr := range pdrs { - id, err := pdr.ID() - if err != nil { - return err - } - tmpPDR[id] = pdr - } - tmpFAR := make(map[uint32]*pfcprule.FAR) - for _, far := range fars { - id, err := far.ID() - if err != nil { - return err - } - tmpFAR[id] = far - } - ies := make([]*ie.IE, 0) - ies = append(ies, s.association.Srv.NodeID()) - ies = append(ies, s.localFseid) - for _, pdr := range pfcprule.NewCreatePDRs(pdrs) { - ies = append(ies, pdr) - } - for _, far := range pfcprule.NewCreateFARs(fars) { - ies = append(ies, far) - } - - msg := message.NewSessionEstablishmentRequest(0, 0, 0, 0, 0, ies...) - resp, err := s.association.Send(msg) - if err != nil { - return err - } - ser, ok := resp.(*message.SessionEstablishmentResponse) - if !ok { - log.Printf("got unexpected message: %s\n", resp.MessageTypeName()) - } - s.remoteFseid = ser.UPFSEID - s.AddFARs(tmpFAR) - s.AddPDRs(tmpPDR) - return nil -} diff --git a/pfcprules/far.go b/pfcprules/far.go index 7f141ba..f0d6e74 100644 --- a/pfcprules/far.go +++ b/pfcprules/far.go @@ -17,7 +17,11 @@ type FAR struct { forwardingParameters *ie.IE } -func (far *FAR) ID() (uint32, error) { +type FARID = uint32 +type FARs []*FAR +type FARMap map[FARID]*FAR + +func (far *FAR) ID() (FARID, error) { return far.id.FARID() } @@ -47,7 +51,7 @@ func (far *FAR) NewCreateFAR() *ie.IE { return ie.NewCreateFAR(ies...) } -func NewCreateFARs(fars []*FAR) []*ie.IE { +func NewCreateFARs(fars FARMap) []*ie.IE { f := make([]*ie.IE, 0) for _, far := range fars { f = append(f, far.NewCreateFAR()) @@ -55,7 +59,7 @@ func NewCreateFARs(fars []*FAR) []*ie.IE { return f } -func NewFARs(fars []*ie.IE) (far []*FAR, err error, cause uint8, offendingIE uint16) { +func NewFARs(fars []*ie.IE) (far FARs, err error, cause uint8, offendingIE uint16) { f := make([]*FAR, 0) for _, far := range fars { id, err := far.FARID() diff --git a/pfcprules/pdr.go b/pfcprules/pdr.go index e76eb5a..1041f4f 100644 --- a/pfcprules/pdr.go +++ b/pfcprules/pdr.go @@ -29,7 +29,9 @@ func NewPDR(id *ie.IE, pdi *ie.IE, precedence *ie.IE, farid *ie.IE, outerHeaderR } } +type PDRID = uint16 type PDRs []*PDR +type PDRMap map[PDRID]*PDR func (pdrs PDRs) Less(i, j int) bool { // element with highest precedence (lowest value in Precedence IE) should be sorted first @@ -51,7 +53,7 @@ func (pdrs PDRs) Swap(i, j int) { pdrs[i], pdrs[j] = pdrs[j], pdrs[i] } -func (pdr *PDR) ID() (uint16, error) { +func (pdr *PDR) ID() (PDRID, error) { return pdr.id.PDRID() } @@ -62,7 +64,7 @@ func (pdr *PDR) Precedence() (uint32, error) { return pdr.precedence.Precedence() } -func (pdr *PDR) FARID() (uint32, error) { +func (pdr *PDR) FARID() (FARID, error) { return pdr.farid.FARID() } @@ -84,7 +86,7 @@ func (pdr *PDR) NewCreatePDR() *ie.IE { return ie.NewCreatePDR(ies...) } -func NewCreatePDRs(pdrs []*PDR) []*ie.IE { +func NewCreatePDRs(pdrs PDRMap) []*ie.IE { p := make([]*ie.IE, 0) for _, pdr := range pdrs { p = append(p, pdr.NewCreatePDR())