Skip to content

Commit

Permalink
More scrubbing.
Browse files Browse the repository at this point in the history
Merged pkt dst and src addr as they're never used in the same time (except
to make them identical). Moved src address check for transit packets to
underlay. Expose bfd session address via packet. Delete bfdSession interface
as it served no purpose and added some cost.
  • Loading branch information
jiceatscion committed Mar 3, 2025
1 parent 0c98715 commit a4be932
Show file tree
Hide file tree
Showing 5 changed files with 132 additions and 144 deletions.
119 changes: 43 additions & 76 deletions router/dataplane.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,12 +78,12 @@ const (
is32bit = 1 - (ptrSize-4)/4
)

type BFDSession interface {
Run(ctx context.Context) error
ReceiveMessage(*layers.BFD)
Close() error
IsUp() bool
}
// type BFDSession interface {
// Run(ctx context.Context) error
// ReceiveMessage(*layers.BFD)
// Close() error
// IsUp() bool
// }

// BatchConn is a connection that supports batch reads and writes.
type BatchConn interface {
Expand Down Expand Up @@ -128,13 +128,13 @@ type Packet struct {
RawPacket []byte
// The entire packet buffer. We don't need it as a slice; we know its size.
buffer *[bufSize]byte
// The source address. Will be set by the receiver from smsg.Addr. We could update it in-place,
// but the IP address bytes in it are allocated by readbatch, so if we copy into a recyclable
// location, it's the original we throw away. No gain (may be a tiny bit?).
SrcAddr *net.UDPAddr
// The address to where we are forwarding the packet.
// Will be set by the processing routine; it is updated in-place.
DstAddr *net.UDPAddr
// The source address during ingest and the destination during forwarding.
// We never need both src and dst at the same time and src is only ever used after
// ingest checks to be copied to dst. That's why we have only one. We keep the storage
// Loc and recycle it.
RemoteAddr *net.UDPAddr
// The ingest link has the bfdSesssion. It passes it to us here so we don't need to dig it up.
BfdSession *bfd.Session
// Additional metadata in case the packet is put on the slow path. Updated in-place.
slowPathRequest slowPathRequest
// The ingress on which this packet arrived. This is set by the receiver.
Expand All @@ -144,7 +144,7 @@ type Packet struct {
// The type of traffic. This is used for metrics at the forwarding stage, but is most
// economically determined at the processing stage. So store it here. It's 2 bytes long.
trafficType trafficType
// Pad to 64 bytes. For 64bit arch, add 4 bytes. For 32bit arch, add 28 bytes.
// Pad to 64 bytes. For 64bit arch, add 12 bytes. For 32bit arch, add 28 bytes.
_ [4 + is32bit*24]byte
}

Expand All @@ -165,18 +165,18 @@ const _ uintptr = unsafe.Sizeof(Packet{}) - 64 // assert sizeof(Packet) >= 64
func (p *Packet) init(buffer *[bufSize]byte) *Packet {
p.buffer = buffer
p.RawPacket = p.buffer[:]
p.DstAddr = &net.UDPAddr{IP: make(net.IP, net.IPv6len)}
p.RemoteAddr = &net.UDPAddr{IP: make(net.IP, net.IPv6len)}
return p
}

// reset() makes the packet ready to receive a new underlay message.
// A cleared dstAddr is represented with a zero-length IP so we keep reusing the IP storage bytes.
func (p *Packet) Reset() {
p.DstAddr.IP = p.DstAddr.IP[0:0] // We're keeping the object, just blank it.
p.RemoteAddr.IP = p.RemoteAddr.IP[0:0] // We're keeping the object, just blank it.
*p = Packet{
buffer: p.buffer, // keep the buffer
RawPacket: p.buffer[:], // restore the full packet capacity
DstAddr: p.DstAddr, // keep the dstAddr and so the IP slice and bytes
buffer: p.buffer, // keep the buffer
RawPacket: p.buffer[:], // restore the full packet capacity
RemoteAddr: p.RemoteAddr, // keep the dstAddr and so the IP slice and bytes
}
// Everything else is reset to zero value.
}
Expand Down Expand Up @@ -449,7 +449,7 @@ func (d *dataPlane) AddLinkType(ifID uint16, linkTo topology.LinkType) error {

// newExternalInterfaceBFD adds the inter AS connection BFD session.
func (d *dataPlane) newExternalInterfaceBFD(ifID uint16,
src, dst control.LinkEnd, cfg control.BFD) (BFDSession, error) {
src, dst control.LinkEnd, cfg control.BFD) (*bfd.Session, error) {

if *cfg.Disable {
return nil, nil
Expand Down Expand Up @@ -551,7 +551,7 @@ func (d *dataPlane) AddNextHop(ifID uint16, src, dst netip.AddrPort, cfg control
// If the remote ifID belongs to an existing address, the existing
// BFD session will be re-used.
func (d *dataPlane) newNextHopBFD(ifID uint16, src, dst netip.AddrPort, cfg control.BFD,
sibling string) (BFDSession, error) {
sibling string) (*bfd.Session, error) {

if *cfg.Disable {
return nil, nil
Expand Down Expand Up @@ -906,17 +906,17 @@ func (p *scionPacketProcessor) processPkt(pkt *Packet) disposition {
switch pathType {
case empty.PathType:
if p.lastLayer.NextLayerType() == layers.LayerTypeBFD {
return p.processIntraBFD(pld)
return p.processBFD(pld)
}
return errorDiscard("error", unsupportedPathTypeNextHeader)

case onehop.PathType:
if p.lastLayer.NextLayerType() == layers.LayerTypeBFD {
ohp, ok := p.scionLayer.Path.(*onehop.Path)
_, ok := p.scionLayer.Path.(*onehop.Path)
if !ok {
return errorDiscard("error", malformedPath)
}
return p.processInterBFD(ohp, pld)
return p.processBFD(pld)
}
return p.processOHP()
case scion.PathType:
Expand All @@ -928,44 +928,16 @@ func (p *scionPacketProcessor) processPkt(pkt *Packet) disposition {
}
}

func (p *scionPacketProcessor) processInterBFD(oh *onehop.Path, data []byte) disposition {
func (p *scionPacketProcessor) processBFD(data []byte) disposition {

// If this is an inter-AS BFD, it can via an interface we own. So the ifID matches one link
// and the ifID better be valid. In the future that will be checked upstream from here.
link, exists := p.d.interfaces[p.pkt.Ingress]
if !exists {
return errorDiscard("error", noBFDSessionFound)
}
session := link.BFDSession()
if session == nil {
if p.pkt.BfdSession == nil {
return errorDiscard("error", noBFDSessionFound)
}
bfd := &p.bfdLayer
if err := bfd.DecodeFromBytes(data, gopacket.NilDecodeFeedback); err != nil {
return errorDiscard("error", err)
}
session.ReceiveMessage(bfd)
return pDiscard // All's fine. That packet's journey ends here.
}

func (p *scionPacketProcessor) processIntraBFD(data []byte) disposition {

// This packet came over a link that doesn't have a define ifID. We have to find it
// by srcAddress. We always find one. The internal link matches anything that is not known.
// TODO(multi_underlay): The underlay should find the Link for all packets (bfd or not), either
// by src address or by connection.
src := p.pkt.SrcAddr.AddrPort() // POSSIBLY EXPENSIVE CONVERSION
session := p.d.underlay.Link(src).BFDSession()
if session == nil {
return errorDiscard("error", noBFDSessionFound)
}

bfd := &p.bfdLayer
if err := bfd.DecodeFromBytes(data, gopacket.NilDecodeFeedback); err != nil {
return errorDiscard("error", err)
}

session.ReceiveMessage(bfd)
p.pkt.BfdSession.ReceiveMessage(bfd)
return pDiscard // All's fine. That packet's journey ends here.
}

Expand Down Expand Up @@ -1101,9 +1073,9 @@ func (p *slowPathPacketProcessor) packSCMP(

// We're about to send a packet that has little to do with the one we received.
// The original traffic type, if one had been set, no-longer applies.
// Since the packet is going back whence it came from, we can just reuse pkt.RemoteAddr as-is.
p.pkt.trafficType = ttOther
p.pkt.egress = p.pkt.Ingress
updateNetAddrFromNetAddr(p.pkt.DstAddr, p.pkt.SrcAddr)
return nil
}

Expand Down Expand Up @@ -1253,26 +1225,20 @@ func (p *scionPacketProcessor) respInvalidDstIA() disposition {
return pSlowPath
}

// validateTransitUnderlaySrc checks that the source address of transit packets
// matches the expected sibling router.
// Provided that underlying network infrastructure prevents address spoofing,
// this check prevents malicious end hosts in the local AS from bypassing the
// SrcIA checks by disguising packets as transit traffic.
// validateTransitUnderlaySrc prevents malicious end hosts in the local AS from bypassing the SrcIA
// checks by disguising packets as transit traffic: sibling links do ensure the src address is that
// of the expected sibling router. But we must verify that a sibling link was used in the first
// place.
func (p *scionPacketProcessor) validateTransitUnderlaySrc() disposition {
if p.path.IsFirstHop() || p.pkt.Ingress != 0 {
// not a transit packet, nothing to check
// Locally originated traffic, or came in via an external link. Not our concern.
return pForward
}
pktIngressID := p.ingressInterface()
ingressLink := p.d.interfaces[pktIngressID]
if ingressLink.Scope() != Sibling {
// Drop
return errorDiscard("error", invalidSrcAddrForTransit)
}
pktIngressID := p.ingressInterface() // Where this was supposed to enter the AS
ingressLink := p.d.interfaces[pktIngressID] // Our own link to that sibling router

// TODO(multi_underlay): This check should move to the underlay, in siblingLink.
src, okS := netip.AddrFromSlice(p.pkt.SrcAddr.IP)
if !(okS && ingressLink.Remote().Addr() == src) {
// Is that the same link? The link can confirm by the pkt src.
if !ingressLink.CheckPktSrc(p.pkt) {
// Drop
return errorDiscard("error", invalidSrcAddrForTransit)
}
Expand Down Expand Up @@ -1412,7 +1378,8 @@ func (p *scionPacketProcessor) verifyCurrentMAC() disposition {
}

func (p *scionPacketProcessor) resolveInbound() disposition {
err := p.d.resolveLocalDst(p.pkt.DstAddr, p.scionLayer, p.lastLayer)
// The internal link is by definition unbound; we need to update the destination.
err := p.d.resolveLocalDst(p.pkt.RemoteAddr, p.scionLayer, p.lastLayer)

switch err {
case nil:
Expand Down Expand Up @@ -1812,7 +1779,7 @@ func (p *scionPacketProcessor) processOHP() disposition {
if err := updateSCIONLayer(p.pkt.RawPacket, s); err != nil {
return errorDiscard("error", err)
}
err := p.d.resolveLocalDst(p.pkt.DstAddr, s, p.lastLayer)
err := p.d.resolveLocalDst(p.pkt.RemoteAddr, s, p.lastLayer)
if err != nil {
return errorDiscard("error", err)
}
Expand Down Expand Up @@ -2148,7 +2115,7 @@ func (b *bfdSend) Send(bfd *layers.BFD) error {

if b.ifID == 0 {
// Using the internal interface: must specify the destination address
UpdateNetAddrFromAddrPort(p.DstAddr, b.dstAddr)
UpdateNetAddrFromAddrPort(p.RemoteAddr, b.dstAddr)
}
// No need to specify pkt.egress. It isn't used downstream from here.
if !fwLink.Send(p) {
Expand Down Expand Up @@ -2509,10 +2476,10 @@ func updateNetAddrFromAddrAndPort(netAddr *net.UDPAddr, addr netip.Addr, port ui
}
}

// updateNetAddrFromNetAddr() copies fromNetAddr into netAddr while re-using the IP slice
// UpdateNetAddrFromNetAddr() copies fromNetAddr into netAddr while re-using the IP slice
// embedded in netAddr. This is to avoid giving work to the GC. Nil IPs get
// converted into empty slices. The backing array isn't discarded.
func updateNetAddrFromNetAddr(netAddr *net.UDPAddr, fromNetAddr *net.UDPAddr) {
func UpdateNetAddrFromNetAddr(netAddr *net.UDPAddr, fromNetAddr *net.UDPAddr) {
netAddr.Port = fromNetAddr.Port
netAddr.Zone = fromNetAddr.Zone
netAddr.IP = netAddr.IP[0:len(fromNetAddr.IP)]
Expand Down
42 changes: 30 additions & 12 deletions router/dataplane_internal_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"net"
"net/netip"
"reflect"
"sync"
"testing"
"time"

Expand All @@ -31,12 +32,14 @@ import (
"github.com/stretchr/testify/require"

"github.com/scionproto/scion/pkg/addr"
"github.com/scionproto/scion/pkg/private/ptr"
"github.com/scionproto/scion/pkg/private/util"
"github.com/scionproto/scion/pkg/scrypto"
"github.com/scionproto/scion/pkg/slayers"
"github.com/scionproto/scion/pkg/slayers/path"
"github.com/scionproto/scion/pkg/slayers/path/scion"
underlayconn "github.com/scionproto/scion/private/underlay/conn"
"github.com/scionproto/scion/router/control"
"github.com/scionproto/scion/router/mock_router"
)

Expand Down Expand Up @@ -100,7 +103,7 @@ func TestReceiver(t *testing.T) {
// make sure that the packet has the right size
assert.Equal(t, 84+i%10*18, len(pkt.RawPacket))
// make sure that the source address was set correctly
assert.Equal(t, net.UDPAddr{IP: net.IP{10, 0, 200, 200}}, *pkt.SrcAddr)
assert.Equal(t, net.UDPAddr{IP: net.IP{10, 0, 200, 200}}, *pkt.RemoteAddr)
// make sure that the received pkt buffer has not been seen before
ptr := reflect.ValueOf(pkt.RawPacket).Pointer()
assert.NotContains(t, ptrMap, ptr)
Expand Down Expand Up @@ -131,21 +134,22 @@ func TestForwarder(t *testing.T) {
prepareDP := func(ctrl *gomock.Controller) *dataPlane {
ret := newDataPlane(
RunConfig{NumProcessors: 20, BatchSize: 64, NumSlowPathProcessors: 1}, false)
mInternal := mock_router.NewMockBatchConn(ctrl)
mFakeConn := mock_router.NewMockBatchConn(ctrl)
totalCount := 0
expectedPktId := byte(0)
closeChan := make(chan struct{})
mInternal.EXPECT().Close().DoAndReturn(
var once sync.Once
mFakeConn.EXPECT().Close().DoAndReturn(
func() error {
close(closeChan)
once.Do(func() { close(closeChan) })
return nil
}).AnyTimes()
mInternal.EXPECT().ReadBatch(gomock.Any()).DoAndReturn(
mFakeConn.EXPECT().ReadBatch(gomock.Any()).DoAndReturn(
func(ms underlayconn.Messages) (int, error) {
<-closeChan
return 0, nil
}).AnyTimes()
mInternal.EXPECT().WriteBatch(gomock.Any(), 0).DoAndReturn(
mFakeConn.EXPECT().WriteBatch(gomock.Any(), 0).DoAndReturn(
func(ms underlayconn.Messages, flags int) (int, error) {
if totalCount == 255 {
return 0, nil
Expand All @@ -164,9 +168,14 @@ func TestForwarder(t *testing.T) {
"expected", expectedPktId, "got", pktId, "ms", ms)
}
if totalCount <= 100 {
// The first 100 packets are sent through the internal link
// They carry an explicit destination address.
assert.NotNil(t, m.Addr)
} else {
// stronger check than assert.Nil
// The other packets are sent through th external link. The
// destination is implicit and must not be copied to the batch
// messages.
// Addr == nil is a stronger check than assert.Nil
assert.True(t, m.Addr == nil)
}
expectedPktId++
Expand All @@ -181,7 +190,18 @@ func TestForwarder(t *testing.T) {

return len(ms), nil
}).AnyTimes()
_ = ret.AddInternalInterface(mInternal, netip.Addr{})
_ = ret.AddInternalInterface(mFakeConn, netip.Addr{})
l := control.LinkEnd{
IA: addr.MustParseIA("1-ff00:0:1"),
Addr: netip.MustParseAddrPort("10.0.0.100:0"),
}
r := control.LinkEnd{
IA: addr.MustParseIA("1-ff00:0:3"),
Addr: netip.MustParseAddrPort("10.0.0.200:0"),
}
nobfd := control.BFD{Disable: ptr.To(true)}

_ = ret.AddExternalInterface(42, mFakeConn, l, r, nobfd)
return ret
}
dp := prepareDP(ctrl)
Expand All @@ -198,10 +218,9 @@ func TestForwarder(t *testing.T) {
pkt.RawPacket = pkt.RawPacket[:1]
pkt.RawPacket[0] = byte(i)
if i < 100 {
pkt.DstAddr.IP = pkt.DstAddr.IP[:4]
copy(pkt.DstAddr.IP, dstAddr.IP)
pkt.RemoteAddr.IP = pkt.RemoteAddr.IP[:4]
copy(pkt.RemoteAddr.IP, dstAddr.IP)
}
pkt.SrcAddr = &net.UDPAddr{} // Receiver always sets this.
pkt.Ingress = 0

assert.NotEqual(t, initialPoolSize, len(dp.packetPool))
Expand Down Expand Up @@ -422,7 +441,6 @@ func TestSlowPathProcessing(t *testing.T) {
pkt.init(&[bufSize]byte{})
pkt.Reset()
pkt.Ingress = tc.srcInterface
pkt.SrcAddr = &net.UDPAddr{} // The receiver always sets this.
pkt.RawPacket = pkt.RawPacket[:len(rp)]
copy(pkt.RawPacket, rp)

Expand Down
14 changes: 6 additions & 8 deletions router/export_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,18 +41,16 @@ const PDiscard = Disposition(pDiscard)

func NewPacket(raw []byte, src, dst *net.UDPAddr, ingress, egress uint16) *Packet {
p := Packet{
DstAddr: &net.UDPAddr{IP: make(net.IP, 0, net.IPv6len)},
SrcAddr: &net.UDPAddr{IP: make(net.IP, 0, net.IPv6len)},
RawPacket: make([]byte, len(raw)),
Ingress: ingress,
egress: egress,
RemoteAddr: &net.UDPAddr{IP: make(net.IP, 0, net.IPv6len)},
RawPacket: make([]byte, len(raw)),
Ingress: ingress,
egress: egress,
}

if src != nil {
p.SrcAddr = src
p.RemoteAddr = src
}
if dst != nil {
p.DstAddr = dst
p.RemoteAddr = dst
}
copy(p.RawPacket, raw)
return &p
Expand Down
Loading

0 comments on commit a4be932

Please sign in to comment.