From a38c0e37b8754f9ece1984b87bec9b6d815e2d5d Mon Sep 17 00:00:00 2001 From: Jean-Christophe Hugly Date: Mon, 18 Nov 2024 18:09:14 +0100 Subject: [PATCH 1/6] avoid copy required by gopacket.serializeBuffer To that end: * Introduce an implementation of SerializeBuffer that writes directly into the raw buffer of the packet to be sent. * Use this in BfdSend, for now. * Other uses coming in subsequent commits. We cannot necessarily use this everywhere: The implementation starts filling buffers in the middle to ensure that there is space for headers. That also means that the payload is less than 1/2 the buffer size. --- router/BUILD.bazel | 1 + router/dataplane.go | 46 +++++++++++++++----------- router/serializeProxy.go | 71 ++++++++++++++++++++++++++++++++++++++++ 3 files changed, 99 insertions(+), 19 deletions(-) create mode 100644 router/serializeProxy.go diff --git a/router/BUILD.bazel b/router/BUILD.bazel index caf311a7f1..e0c562ce8e 100644 --- a/router/BUILD.bazel +++ b/router/BUILD.bazel @@ -5,6 +5,7 @@ go_library( srcs = [ "connector.go", "dataplane.go", + "serializeProxy.go", "fnv1aCheap.go", "metrics.go", "svc.go", diff --git a/router/dataplane.go b/router/dataplane.go index c1322a5f4a..f4f98e85c4 100644 --- a/router/dataplane.go +++ b/router/dataplane.go @@ -1,5 +1,6 @@ // Copyright 2020 Anapaya Systems // Copyright 2023 ETH Zurich +// Copyright 2024 SCION Association // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -111,8 +112,10 @@ const ( // required to occupy exactly 64 bytes depends on the architecture. type packet struct { // The useful part of the raw packet at a point in time (i.e. a slice of the full buffer). - // TODO(jiceatscion): would it be beneficial to store the length instead, like readBatch does? + // It can be any portion of the full buffer; not necessarily the start. 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?). @@ -130,9 +133,7 @@ type packet struct { // 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 12 bytes. For 32bit arch, add 32 bytes. - // TODO(jiceatscion): see if packing two packets per cache line instead is good or bad for 32bit - // machines. - _ [12 + is32bit*20]byte + _ [4 + is32bit*24]byte } // Keep this 6 bytes long. See comment for packet. @@ -150,7 +151,8 @@ const _ uintptr = unsafe.Sizeof(packet{}) - 64 // assert sizeof(packet) >= 64 // initPacket configures the given blank packet (and returns it, for convenience). func (p *packet) init(buffer *[bufSize]byte) *packet { - p.rawPacket = buffer[:] + p.buffer = buffer + p.rawPacket = p.buffer[:] p.dstAddr = &net.UDPAddr{IP: make(net.IP, net.IPv6len)} return p } @@ -160,11 +162,11 @@ func (p *packet) init(buffer *[bufSize]byte) *packet { func (p *packet) reset() { p.dstAddr.IP = p.dstAddr.IP[0:0] // We're keeping the object, just blank it. *p = packet{ - rawPacket: p.rawPacket[:cap(p.rawPacket)], // keep the slice and so the backing array. - 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 + dstAddr: p.dstAddr, // keep the dstAddr and so the IP slice and bytes } // Everything else is reset to zero value. - } // DataPlane contains a SCION Border Router's forwarding logic. It reads packets @@ -2328,7 +2330,6 @@ type bfdSend struct { ohp *onehop.Path mac hash.Hash macBuffer []byte - buffer gopacket.SerializeBuffer } // newBFDSend creates and initializes a BFD Sender @@ -2392,7 +2393,6 @@ func newBFDSend(d *DataPlane, srcIA, dstIA addr.IA, srcAddr, dstAddr netip.AddrP ohp: ohp, mac: mac, macBuffer: make([]byte, path.MACBufferSize), - buffer: gopacket.NewSerializeBuffer(), }, nil } @@ -2400,6 +2400,8 @@ func (b *bfdSend) String() string { return b.srcAddr.String() } +var count = 0 + // Send sends out a BFD message. // Due to the internal state of the MAC computation, this is not goroutine // safe. @@ -2411,24 +2413,30 @@ func (b *bfdSend) Send(bfd *layers.BFD) error { ohp.FirstHop.Mac = path.MAC(b.mac, ohp.Info, ohp.FirstHop, b.macBuffer) } - err := gopacket.SerializeLayers(b.buffer, gopacket.SerializeOptions{FixLengths: true}, + p := b.dataPlane.getPacketFromPool() + p.reset() + + serBuf := newSerializeProxy(p.rawPacket) + + // serialized bytes lend directly into p.rawPacket, but somewhere in the middle. + // Bytes() will give us the correct slice, which we then set in the packet. + err := gopacket.SerializeLayers(&serBuf, gopacket.SerializeOptions{FixLengths: true}, b.scn, bfd) if err != nil { return err } + // The usefull part of the buffer is given by Bytes. We don't copy the bytes; just the slice's + // metadata. + p.rawPacket = serBuf.Bytes() + if count < 10 { + count++ + } + // BfdControllers and fwQs are initialized from the same set of ifIDs. So not finding // the forwarding queue is an serious internal error. Let that panic. fwChan := b.dataPlane.fwQs[b.ifID] - p := b.dataPlane.getPacketFromPool() - p.reset() - - // TODO: it would be best to serialize directly into the packet buffer. This would require - // a custom SerializeBuffer implementation and some changes to the packet structure. To be - // considered in a future refactoring. - sz := copy(p.rawPacket, b.buffer.Bytes()) - p.rawPacket = p.rawPacket[:sz] if b.ifID == 0 { // Using the internal interface: must specify the destination address updateNetAddrFromAddrPort(p.dstAddr, b.dstAddr) diff --git a/router/serializeProxy.go b/router/serializeProxy.go new file mode 100644 index 0000000000..1b3087b65d --- /dev/null +++ b/router/serializeProxy.go @@ -0,0 +1,71 @@ +// Copyright 2024 SCION Association +// +// 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 router + +import ( + "github.com/google/gopacket" +) + +// serializeProxy implements gopacket.SerializeBuffer. It is a very simple implementation that +// writes to a separately allocated buffer (such as a packet's raw buffer). It is designed with the +// assumption that the buffer is large enough for all the prepends and appends that will be thrown +// at it, so there never is a need for realocation. Should that be false, it would be a severe +// internal error; i.e. Panic is fine. The starting point of appends and prepends is the middle of +// the buffer. It is designed to be a local variable, so New() returns a value. The entire buffer +// underpinning the given slice may be used; that is, from the start up to the remaining capacity. +type serializeProxy struct { + data []byte + start int // data[0] == buf[0] (bc changing it is one way), so we keep track of the real start. + layers []gopacket.LayerType +} + +func newSerializeProxy(buf []byte) serializeProxy { + serBuf := serializeProxy{ + data: buf, + } + serBuf.Clear() + return serBuf +} + +func (s *serializeProxy) Clear() error { + s.start = cap(s.data) / 2 + s.data = s.data[:s.start] + s.layers = s.layers[:0] + return nil +} + +func (s *serializeProxy) PrependBytes(num int) ([]byte, error) { + s.start -= num + return s.data[s.start : s.start+num], nil +} + +func (s *serializeProxy) AppendBytes(num int) ([]byte, error) { + ol := len(s.data) + nl := ol + num + s.data = s.data[:nl] + return s.data[ol:nl], nil +} + +func (s *serializeProxy) Bytes() []byte { + return s.data[s.start:] +} + +func (s *serializeProxy) Layers() []gopacket.LayerType { + return s.layers +} + +func (s *serializeProxy) PushLayer(l gopacket.LayerType) { + s.layers = append(s.layers, l) +} From 2f37106615353ee1b3360cae3626dc6c9a5991fc Mon Sep 17 00:00:00 2001 From: Jean-Christophe Hugly Date: Tue, 19 Nov 2024 18:05:33 +0100 Subject: [PATCH 2/6] Use serializeProxy everywhere instead of serializeBuffer. To support some use cases, added the feature that the insertion point can be changed when resetting a serializeProxy. The default is now the end of the buffer, which is by far the most common case. --- router/dataplane.go | 118 ++++++++++++++++----------------------- router/serializeProxy.go | 40 +++++++++---- 2 files changed, 78 insertions(+), 80 deletions(-) diff --git a/router/dataplane.go b/router/dataplane.go index f4f98e85c4..2ce5a18740 100644 --- a/router/dataplane.go +++ b/router/dataplane.go @@ -909,7 +909,6 @@ func (d *DataPlane) runSlowPathProcessor(id int, q <-chan *packet, func newSlowPathProcessor(d *DataPlane) *slowPathPacketProcessor { p := &slowPathPacketProcessor{ d: d, - buffer: gopacket.NewSerializeBuffer(), macInputBuffer: make([]byte, spao.MACBufferSize), drkeyProvider: &drkeyutil.FakeProvider{ EpochDuration: drkeyutil.LoadEpochDuration(), @@ -923,9 +922,8 @@ func newSlowPathProcessor(d *DataPlane) *slowPathPacketProcessor { } type slowPathPacketProcessor struct { - d *DataPlane - pkt *packet - buffer gopacket.SerializeBuffer + d *DataPlane + pkt *packet scionLayer slayers.SCION hbhLayer slayers.HopByHopExtnSkipper @@ -947,11 +945,6 @@ type slowPathPacketProcessor struct { } func (p *slowPathPacketProcessor) reset() { - if err := p.buffer.Clear(); err != nil { - // The serializeBuffer returned by NewSerializeBuffer isn't actually capable of failing to - // clear, so planning on doing something about it is pointless (and what might that be?). - panic(fmt.Sprintf("Error while clearing buffer: %v", err)) - } p.path = nil p.hbhLayer = slayers.HopByHopExtnSkipper{} p.e2eLayer = slayers.EndToEndExtnSkipper{} @@ -1125,7 +1118,6 @@ func readUpTo(c <-chan *packet, n int, needsBlocking bool, pkts []*packet) int { func newPacketProcessor(d *DataPlane) *scionPacketProcessor { p := &scionPacketProcessor{ d: d, - buffer: gopacket.NewSerializeBuffer(), mac: d.macFactory(), macInputBuffer: make([]byte, max(path.MACBufferSize, libepic.MACBufferSize)), } @@ -1141,11 +1133,6 @@ func (p *scionPacketProcessor) reset() error { p.infoField = path.InfoField{} p.effectiveXover = false p.peering = false - if err := p.buffer.Clear(); err != nil { - // The serializeBuffer returned by NewSerializeBuffer isn't actually capable of failing to - // clear, so planning on doing something about it is pointless (and what might that be?). - panic(fmt.Sprintf("Error while clearing buffer: %v", err)) - } p.mac.Reset() p.cachedMac = nil // Reset hbh layer @@ -1315,8 +1302,6 @@ type scionPacketProcessor struct { d *DataPlane // pkt is the packet currently being processed by this processor. pkt *packet - // buffer is the buffer that can be used to serialize gopacket layers. - buffer gopacket.SerializeBuffer // mac is the hasher for the MAC computation. mac hash.Hash @@ -1376,10 +1361,8 @@ func (p *slowPathPacketProcessor) packSCMP( } } - rawSCMP, err := p.prepareSCMP(typ, code, scmpP, isError) - if rawSCMP != nil { - p.pkt.rawPacket = p.pkt.rawPacket[:len(rawSCMP)] - copy(p.pkt.rawPacket, rawSCMP) + if err := p.prepareSCMP(typ, code, scmpP, isError); err != nil { + return err } // We're about to send a packet that has little to do with the one we received. @@ -1387,7 +1370,7 @@ func (p *slowPathPacketProcessor) packSCMP( p.pkt.trafficType = ttOther p.pkt.egress = p.pkt.ingress updateNetAddrFromNetAddr(p.pkt.dstAddr, p.pkt.srcAddr) - return err + return nil } func (p *scionPacketProcessor) parsePath() disposition { @@ -2074,7 +2057,7 @@ func (p *scionPacketProcessor) processOHP() disposition { } ohp.Info.UpdateSegID(ohp.FirstHop.Mac) - if err := updateSCIONLayer(p.pkt.rawPacket, s, p.buffer); err != nil { + if err := updateSCIONLayer(p.pkt.rawPacket, s); err != nil { return errorDiscard("error", err) } p.pkt.egress = ohp.FirstHop.ConsEgress @@ -2100,7 +2083,7 @@ func (p *scionPacketProcessor) processOHP() disposition { ohp.SecondHop.Mac = path.MAC(p.mac, ohp.Info, ohp.SecondHop, p.macInputBuffer[:path.MACBufferSize]) - if err := updateSCIONLayer(p.pkt.rawPacket, s, p.buffer); err != nil { + if err := updateSCIONLayer(p.pkt.rawPacket, s); err != nil { return errorDiscard("error", err) } err := p.d.resolveLocalDst(p.pkt.dstAddr, s, p.lastLayer) @@ -2304,22 +2287,17 @@ func decodeSCMP(scmp *slayers.SCMP) ([]gopacket.SerializableLayer, error) { return ret, nil } -// TODO(matzf) this function is now only used to update the OneHop-path. -// This should be changed so that the OneHop-path can be updated in-place, like -// the scion.Raw path. -func updateSCIONLayer(rawPkt []byte, s slayers.SCION, buffer gopacket.SerializeBuffer) error { - if err := buffer.Clear(); err != nil { - return err - } - if err := s.SerializeTo(buffer, gopacket.SerializeOptions{}); err != nil { - return err - } - // TODO(lukedirtwalker): We should add a method to the scion layers - // which can write into the existing buffer, see also the discussion in - // https://fsnets.slack.com/archives/C8ADBBG0J/p1592805884250700 - rawContents := buffer.Bytes() - copy(rawPkt[:len(rawContents)], rawContents) - return nil +// updateSCIONLayer rewrites the SCION header at the start of the given raw packet buffer; replacing +// it with the serialization of the given new SCION header. This works only if the new header +// is of the same size as the old one. This function has no knowledge of the actual size of the +// headers; it only ensures that the new one ends exactly where the old one did. It is possible to +// use this function to replace a header with a smaller one; but the preceding headers, and the +// start of the slice must be corrected afterwards. +func updateSCIONLayer(rawPkt []byte, s slayers.SCION) error { + payloadOffset := len(rawPkt) - len(s.LayerPayload()) + serBuf := newSerializeProxy(rawPkt) + serBuf.clear(payloadOffset) // So, prepending just in front of the payload. Better not append! + return s.SerializeTo(&serBuf, gopacket.SerializeOptions{}) } type bfdSend struct { @@ -2416,10 +2394,9 @@ func (b *bfdSend) Send(bfd *layers.BFD) error { p := b.dataPlane.getPacketFromPool() p.reset() - serBuf := newSerializeProxy(p.rawPacket) + serBuf := newSerializeProxy(p.rawPacket) // set for prepend-only by default. Perfect here. - // serialized bytes lend directly into p.rawPacket, but somewhere in the middle. - // Bytes() will give us the correct slice, which we then set in the packet. + // serialized bytes lend directly into p.rawPacket (alignedd at the end). err := gopacket.SerializeLayers(&serBuf, gopacket.SerializeOptions{FixLengths: true}, b.scn, bfd) if err != nil { @@ -2457,7 +2434,7 @@ func (p *slowPathPacketProcessor) prepareSCMP( code slayers.SCMPCode, scmpP gopacket.SerializableLayer, isError bool, -) ([]byte, error) { +) error { // *copy* and reverse path -- the original path should not be modified as this writes directly // back to rawPkt (quote). @@ -2468,36 +2445,36 @@ func (p *slowPathPacketProcessor) prepareSCMP( var ok bool path, ok = p.scionLayer.Path.(*scion.Raw) if !ok { - return nil, serrors.JoinNoStack(cannotRoute, nil, "details", "unsupported path type", + return serrors.JoinNoStack(cannotRoute, nil, "details", "unsupported path type", "path type", pathType) } case epic.PathType: epicPath, ok := p.scionLayer.Path.(*epic.Path) if !ok { - return nil, serrors.JoinNoStack(cannotRoute, nil, "details", "unsupported path type", + return serrors.JoinNoStack(cannotRoute, nil, "details", "unsupported path type", "path type", pathType) } path = epicPath.ScionPath default: - return nil, serrors.JoinNoStack(cannotRoute, nil, "details", "unsupported path type", + return serrors.JoinNoStack(cannotRoute, nil, "details", "unsupported path type", "path type", pathType) } decPath, err := path.ToDecoded() if err != nil { - return nil, serrors.JoinNoStack(cannotRoute, err, "details", "decoding raw path") + return serrors.JoinNoStack(cannotRoute, err, "details", "decoding raw path") } revPathTmp, err := decPath.Reverse() if err != nil { - return nil, serrors.JoinNoStack(cannotRoute, err, "details", "reversing path for SCMP") + return serrors.JoinNoStack(cannotRoute, err, "details", "reversing path for SCMP") } revPath := revPathTmp.(*scion.Decoded) peering, err := determinePeer(revPath.PathMeta, revPath.InfoFields[revPath.PathMeta.CurrINF]) if err != nil { - return nil, serrors.JoinNoStack(cannotRoute, err, "details", "peering cannot be determined") + return serrors.JoinNoStack(cannotRoute, err, "details", "peering cannot be determined") } // Revert potential path segment switches that were done during processing. @@ -2505,7 +2482,7 @@ func (p *slowPathPacketProcessor) prepareSCMP( // An effective cross-over is a change of segment other than at // a peering hop. if err := revPath.IncPath(); err != nil { - return nil, serrors.JoinNoStack(cannotRoute, err, + return serrors.JoinNoStack(cannotRoute, err, "details", "reverting cross over for SCMP") } } @@ -2519,7 +2496,7 @@ func (p *slowPathPacketProcessor) prepareSCMP( infoField.UpdateSegID(hopField.Mac) } if err := revPath.IncPath(); err != nil { - return nil, serrors.JoinNoStack(cannotRoute, err, + return serrors.JoinNoStack(cannotRoute, err, "details", "incrementing path for SCMP") } } @@ -2537,7 +2514,7 @@ func (p *slowPathPacketProcessor) prepareSCMP( scionL.NextHdr = slayers.L4SCMP if err := scionL.SetSrcAddr(addr.HostIP(p.d.internalIP)); err != nil { - return nil, serrors.JoinNoStack(cannotRoute, err, "details", "setting src addr") + return serrors.JoinNoStack(cannotRoute, err, "details", "setting src addr") } typeCode := slayers.CreateSCMPTypeCode(typ, code) scmpH := slayers.SCMP{TypeCode: typeCode} @@ -2577,19 +2554,19 @@ func (p *slowPathPacketProcessor) prepareSCMP( } } - if err := p.buffer.Clear(); err != nil { - return nil, err - } + serBuf := newSerializeProxy(p.pkt.rawPacket) // Prepend-only by default. It's all we need. sopts := gopacket.SerializeOptions{ ComputeChecksums: true, FixLengths: true, } // First write the SCMP message only without the SCION header(s) to get a buffer that we - // can (re-)use as input in the MAC computation. - // XXX(matzf) could we use iovec gather to avoid copying quote? - err = gopacket.SerializeLayers(p.buffer, sopts, &scmpH, scmpP, gopacket.Payload(quote)) + // can (re-)use as input in the MAC computation. Note that we move the quoted part of the packet + // to the end of the buffer (go supports overlaps properly). + // TODO(jiceatscion): in the future we may be able to leave room at the head of the + // buffer on ingest, so we won't need to move the quote at all. + err = gopacket.SerializeLayers(&serBuf, sopts, &scmpH, scmpP, gopacket.Payload(quote)) if err != nil { - return nil, serrors.JoinNoStack(cannotRoute, err, "details", "serializing SCMP message") + return serrors.JoinNoStack(cannotRoute, err, "details", "serializing SCMP message") } if needsAuth { @@ -2599,15 +2576,15 @@ func (p *slowPathPacketProcessor) prepareSCMP( now := time.Now() dstA, err := scionL.DstAddr() if err != nil { - return nil, serrors.JoinNoStack(cannotRoute, err, + return serrors.JoinNoStack(cannotRoute, err, "details", "parsing destination address") } key, err := p.drkeyProvider.GetASHostKey(now, scionL.DstIA, dstA) if err != nil { - return nil, serrors.JoinNoStack(cannotRoute, err, "details", "retrieving DRKey") + return serrors.JoinNoStack(cannotRoute, err, "details", "retrieving DRKey") } if err := p.resetSPAOMetadata(key, now); err != nil { - return nil, serrors.JoinNoStack(cannotRoute, err, "details", "resetting SPAO header") + return serrors.JoinNoStack(cannotRoute, err, "details", "resetting SPAO header") } e2e.Options = []*slayers.EndToEndOption{p.optAuth.EndToEndOption} @@ -2618,27 +2595,28 @@ func (p *slowPathPacketProcessor) prepareSCMP( Header: p.optAuth, ScionLayer: &scionL, PldType: slayers.L4SCMP, - Pld: p.buffer.Bytes(), + Pld: serBuf.Bytes(), }, p.macInputBuffer, p.optAuth.Authenticator(), ) if err != nil { - return nil, serrors.JoinNoStack(cannotRoute, err, "details", "computing CMAC") + return serrors.JoinNoStack(cannotRoute, err, "details", "computing CMAC") } - if err := e2e.SerializeTo(p.buffer, sopts); err != nil { - return nil, serrors.JoinNoStack(cannotRoute, err, + if err := e2e.SerializeTo(&serBuf, sopts); err != nil { + return serrors.JoinNoStack(cannotRoute, err, "details", "serializing SCION E2E headers") } } else { scionL.NextHdr = slayers.L4SCMP } - if err := scionL.SerializeTo(p.buffer, sopts); err != nil { - return nil, serrors.JoinNoStack(cannotRoute, err, "details", "serializing SCION header") + if err := scionL.SerializeTo(&serBuf, sopts); err != nil { + return serrors.JoinNoStack(cannotRoute, err, "details", "serializing SCION header") } + p.pkt.rawPacket = serBuf.Bytes() log.Debug("scmp", "typecode", scmpH.TypeCode) - return p.buffer.Bytes(), nil + return nil } func (p *slowPathPacketProcessor) resetSPAOMetadata(key drkey.ASHostKey, now time.Time) error { diff --git a/router/serializeProxy.go b/router/serializeProxy.go index 1b3087b65d..e146d03298 100644 --- a/router/serializeProxy.go +++ b/router/serializeProxy.go @@ -19,38 +19,53 @@ import ( ) // serializeProxy implements gopacket.SerializeBuffer. It is a very simple implementation that -// writes to a separately allocated buffer (such as a packet's raw buffer). It is designed with the -// assumption that the buffer is large enough for all the prepends and appends that will be thrown -// at it, so there never is a need for realocation. Should that be false, it would be a severe -// internal error; i.e. Panic is fine. The starting point of appends and prepends is the middle of -// the buffer. It is designed to be a local variable, so New() returns a value. The entire buffer +// writes to a separately allocated buffer (such as a packet's raw buffer). Space is added to the +// buffer via PrependBytes and AppendBytes simply by changing the starting point and length of the +// data slice. No reallocation is ever performed. Running out of append or prepend space triggers a +// panic. It is designed to be a local variable, so New() returns a value. The entire buffer // underpinning the given slice may be used; that is, from the start up to the remaining capacity. type serializeProxy struct { - data []byte - start int // data[0] == buf[0] (bc changing it is one way), so we keep track of the real start. - layers []gopacket.LayerType + initStart int // Initial starting point (where the first prepend or append occurs) + data []byte + start int // data[0] == buf[0] (bc changing it is one way), so we keep track of the real start. + layers []gopacket.LayerType } +// Initializes a new serializeProxy. The initial prepend/append point is set to the end of the +// buffer in anticipation of AppendBytes never being used. This can be changed by calling clear() +// if AppendBytes is to be used. func newSerializeProxy(buf []byte) serializeProxy { serBuf := serializeProxy{ data: buf, } - serBuf.Clear() + serBuf.clear(cap(buf)) return serBuf } +// Resets the buffer to empty and sets the initial prepend/append point to the given position. +func (s *serializeProxy) clear(newStart int) { + s.initStart = newStart + s.Clear() +} + +// Implements serializeBuffer.Clear(). Never returns an error. +// The initial prepend/append point is reset to that which was set by the last call to clear(). func (s *serializeProxy) Clear() error { - s.start = cap(s.data) / 2 + s.start = s.initStart s.data = s.data[:s.start] s.layers = s.layers[:0] return nil } +// PrependBytes implements serializeBuffer.PrependBytes(). It never returns an error. +// It can panic if attenpting to prepend before the start of the buffer. func (s *serializeProxy) PrependBytes(num int) ([]byte, error) { s.start -= num return s.data[s.start : s.start+num], nil } +// AppendBytes implements serializeBuffer.AppendBytes(). It never returns an error. +// It can panic if attempting to append past the end of the buffer. func (s *serializeProxy) AppendBytes(num int) ([]byte, error) { ol := len(s.data) nl := ol + num @@ -58,14 +73,19 @@ func (s *serializeProxy) AppendBytes(num int) ([]byte, error) { return s.data[ol:nl], nil } +// Bytes implements serializeBuffer.Bytes(). It returns a slice that represents the useful portion +// of the buffer. That is the portion that contains all the prepended and appended bytes since the +// last call to Clear(). func (s *serializeProxy) Bytes() []byte { return s.data[s.start:] } +// Bytes implements serializeBuffer.Layers. func (s *serializeProxy) Layers() []gopacket.LayerType { return s.layers } +// Bytes implements serializeBuffer.PushLayer. func (s *serializeProxy) PushLayer(l gopacket.LayerType) { s.layers = append(s.layers, l) } From 3990f9664e77d8828a0e04b498b8e603e5c95332 Mon Sep 17 00:00:00 2001 From: Jean-Christophe Hugly Date: Wed, 20 Nov 2024 11:32:33 +0100 Subject: [PATCH 3/6] Buildify, lintify, etc. --- router/BUILD.bazel | 2 +- router/dataplane.go | 14 +++++++------- router/serializeProxy.go | 12 ++++++------ 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/router/BUILD.bazel b/router/BUILD.bazel index e0c562ce8e..f7fee76cd7 100644 --- a/router/BUILD.bazel +++ b/router/BUILD.bazel @@ -5,9 +5,9 @@ go_library( srcs = [ "connector.go", "dataplane.go", - "serializeProxy.go", "fnv1aCheap.go", "metrics.go", + "serializeProxy.go", "svc.go", ], importpath = "github.com/scionproto/scion/router", diff --git a/router/dataplane.go b/router/dataplane.go index 2ce5a18740..57cb9ee108 100644 --- a/router/dataplane.go +++ b/router/dataplane.go @@ -2288,15 +2288,15 @@ func decodeSCMP(scmp *slayers.SCMP) ([]gopacket.SerializableLayer, error) { } // updateSCIONLayer rewrites the SCION header at the start of the given raw packet buffer; replacing -// it with the serialization of the given new SCION header. This works only if the new header -// is of the same size as the old one. This function has no knowledge of the actual size of the -// headers; it only ensures that the new one ends exactly where the old one did. It is possible to -// use this function to replace a header with a smaller one; but the preceding headers, and the -// start of the slice must be corrected afterwards. +// it with the serialization of the given new SCION header. This works only if the new header is of +// the same size as the old one. This function has no knowledge of the actual size of the headers; +// it only ensures that the new one ends exactly where the old one did. It is possible to use this +// function to replace a header with a smaller one; but the rawPacket's slice must be fixed +// afterwards (and the preceding headers, if any). func updateSCIONLayer(rawPkt []byte, s slayers.SCION) error { payloadOffset := len(rawPkt) - len(s.LayerPayload()) serBuf := newSerializeProxy(rawPkt) - serBuf.clear(payloadOffset) // So, prepending just in front of the payload. Better not append! + serBuf.clear(payloadOffset) // Prepends will go just before payload. (Appends will wreck it) return s.SerializeTo(&serBuf, gopacket.SerializeOptions{}) } @@ -2403,7 +2403,7 @@ func (b *bfdSend) Send(bfd *layers.BFD) error { return err } - // The usefull part of the buffer is given by Bytes. We don't copy the bytes; just the slice's + // The useful part of the buffer is given by Bytes. We don't copy the bytes; just the slice's // metadata. p.rawPacket = serBuf.Bytes() if count < 10 { diff --git a/router/serializeProxy.go b/router/serializeProxy.go index e146d03298..96a355301d 100644 --- a/router/serializeProxy.go +++ b/router/serializeProxy.go @@ -27,7 +27,7 @@ import ( type serializeProxy struct { initStart int // Initial starting point (where the first prepend or append occurs) data []byte - start int // data[0] == buf[0] (bc changing it is one way), so we keep track of the real start. + start int // The slice's offset can't change (irreversible). So keep track separately. layers []gopacket.LayerType } @@ -45,15 +45,15 @@ func newSerializeProxy(buf []byte) serializeProxy { // Resets the buffer to empty and sets the initial prepend/append point to the given position. func (s *serializeProxy) clear(newStart int) { s.initStart = newStart - s.Clear() + s.start = newStart + s.data = s.data[:newStart] + s.layers = s.layers[:0] } -// Implements serializeBuffer.Clear(). Never returns an error. +// Implements serializeBuffer.Clear(). This implementation never returns an error. // The initial prepend/append point is reset to that which was set by the last call to clear(). func (s *serializeProxy) Clear() error { - s.start = s.initStart - s.data = s.data[:s.start] - s.layers = s.layers[:0] + s.clear(s.initStart) return nil } From 6e5b782db0a826a81f0960d9add4f7e6a9e65c34 Mon Sep 17 00:00:00 2001 From: Jean-Christophe Hugly Date: Wed, 20 Nov 2024 12:29:03 +0100 Subject: [PATCH 4/6] Remove debugging leftover. --- router/dataplane.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/router/dataplane.go b/router/dataplane.go index 57cb9ee108..4eb62b0e76 100644 --- a/router/dataplane.go +++ b/router/dataplane.go @@ -2378,8 +2378,6 @@ func (b *bfdSend) String() string { return b.srcAddr.String() } -var count = 0 - // Send sends out a BFD message. // Due to the internal state of the MAC computation, this is not goroutine // safe. @@ -2406,9 +2404,6 @@ func (b *bfdSend) Send(bfd *layers.BFD) error { // The useful part of the buffer is given by Bytes. We don't copy the bytes; just the slice's // metadata. p.rawPacket = serBuf.Bytes() - if count < 10 { - count++ - } // BfdControllers and fwQs are initialized from the same set of ifIDs. So not finding // the forwarding queue is an serious internal error. Let that panic. From dfe7a6a67ec229af84131ec688a3bd3bd2fc81de Mon Sep 17 00:00:00 2001 From: Jean-Christophe Hugly Date: Thu, 28 Nov 2024 17:39:42 +0100 Subject: [PATCH 5/6] implement reviewer's suggestions. --- router/#connector.go# | 250 ++++++++++++++++++ router/BUILD.bazel | 2 +- router/dataplane.go | 5 +- .../{serializeProxy.go => serialize_proxy.go} | 32 ++- 4 files changed, 276 insertions(+), 13 deletions(-) create mode 100644 router/#connector.go# rename router/{serializeProxy.go => serialize_proxy.go} (74%) diff --git a/router/#connector.go# b/router/#connector.go# new file mode 100644 index 0000000000..113c68eb28 --- /dev/null +++ b/router/#connector.go# @@ -0,0 +1,250 @@ +// Copyright 2020 Anapaya Systems +// +// 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 router + +import ( + "net/netip" + "sync" + + "github.com/scionproto/scion/pkg/addr" + "github.com/scionproto/scion/pkg/log" + "github.com/scionproto/scion/pkg/private/serrors" + "github.com/scionproto/scion/pkg/segment/iface" + "github.com/scionproto/scion/private/underlay/conn" + "github.com/scionproto/scion/router/config" + "github.com/scionproto/scion/router/control" +) + +// Connector implements the Dataplane API of the router control process. It sets +// up connections for the DataPlane. +type Connector struct { + DataPlane DataPlane + + ia addr.IA + mtx sync.Mutex + internalInterfaces []control.InternalInterface + externalInterfaces map[uint16]control.ExternalInterface + siblingInterfaces map[uint16]control.SiblingInterface + internalConnection conn.Conn + + ReceiveBufferSize int + SendBufferSize int + BFD config.BFD + DispatchedPortStart *int + DispatchedPortEnd *int +} + +var errMultiIA = serrors.New("different IA not allowed") + +// CreateIACtx creates the context for ISD-AS. +func (c *Connector) CreateIACtx(ia addr.IA) error { + c.mtx.Lock() + defer c.mtx.Unlock() + log.Debug("CreateIACtx", "isd_as", ia) + if !c.ia.IsZero() { + return serrors.JoinNoStack(errMultiIA, nil, "current", c.ia, "new", ia) + } + c.ia = ia + return c.DataPlane.SetIA(ia) +} + +// AddInternalInterface adds the internal interface. +func (c *Connector) AddInternalInterface(ia addr.IA, local netip.AddrPort) error { + c.mtx.Lock() + defer c.mtx.Unlock() + log.Debug("Adding internal interface", "isd_as", ia, "local", local) + if !c.ia.Equal(ia) { + return serrors.JoinNoStack(errMultiIA, nil, "current", c.ia, "new", ia) + } + connection, err := conn.New(local, netip.AddrPort{}, + &conn.Config{ReceiveBufferSize: c.ReceiveBufferSize, SendBufferSize: c.SendBufferSize}) + if err != nil { + return err + } + c.internalConnection = connection + c.internalInterfaces = append(c.internalInterfaces, control.InternalInterface{ + IA: ia, + Addr: local, + }) + return c.DataPlane.AddInternalInterface(connection, local.Addr()) +} + +// AddExternalInterface adds a link between the local and remote address. +// TODO(jiceatscion): do not depend on AddInternalInterface having been called first. +func (c *Connector) AddExternalInterface(localIfID iface.ID, link control.LinkInfo, + owned bool) error { + + c.mtx.Lock() + defer c.mtx.Unlock() + intf := uint16(localIfID) + log.Debug("Adding external interface", "interface", localIfID, + "local_isd_as", link.Local.IA, "local_addr", link.Local.Addr, + "remote_isd_as", link.Remote.IA, "remote_addr", link.Remote.Addr, + "owned", owned, + "link_bfd_configured", link.BFD.Disable != nil, + "link_bfd_enabled", link.BFD.Disable == nil || !*link.BFD.Disable, + "dataplane_bfd_enabled", !c.BFD.Disable) + + if !c.ia.Equal(link.Local.IA) { + return serrors.JoinNoStack(errMultiIA, nil, "current", c.ia, "new", link.Local.IA) + } + if err := c.DataPlane.AddLinkType(intf, link.LinkTo); err != nil { + return serrors.Wrap("adding link type", err, "if_id", localIfID) + } + if err := c.DataPlane.AddNeighborIA(intf, link.Remote.IA); err != nil { + return serrors.Wrap("adding neighboring IA", err, "if_id", localIfID) + } + + link.BFD = c.applyBFDDefaults(link.BFD) + if owned { + if len(c.externalInterfaces) == 0 { + c.externalInterfaces = make(map[uint16]control.ExternalInterface) + } + c.externalInterfaces[intf] = control.ExternalInterface{ + IfID: intf, + Link: link, + State: control.InterfaceDown, + } + connection, err := conn.New(link.Local.Addr, link.Remote.Addr, + &conn.Config{ReceiveBufferSize: c.ReceiveBufferSize, SendBufferSize: c.SendBufferSize}) + if err != nil { + return err + } + + return c.DataPlane.AddExternalInterface(intf, connection, link.Local, link.Remote, link.BFD, + owned) + } + + if len(c.siblingInterfaces) == 0 { + c.siblingInterfaces = make(map[uint16]control.SiblingInterface) + } + c.siblingInterfaces[intf] = control.SiblingInterface{ + IfID: intf, + InternalInterface: link.Remote.Addr, + Relationship: link.LinkTo, + MTU: link.MTU, + NeighborIA: link.Remote.IA, + State: control.InterfaceDown, + } + return c.DataPlane.AddNextHop(intf, link.Local.Addr, link.Remote.Addr, + link.BFD, link.Instance) +} + +// AddSvc adds the service address for the given ISD-AS. +func (c *Connector) AddSvc(ia addr.IA, svc addr.SVC, a netip.AddrPort) error { + + c.mtx.Lock() + defer c.mtx.Unlock() + log.Debug("Adding service", "isd_as", ia, "svc", svc, "address", a) + if !c.ia.Equal(ia) { + return serrors.JoinNoStack(errMultiIA, nil, "current", c.ia, "new", a) + } + return c.DataPlane.AddSvc(svc, a) +} + +// DelSvc deletes the service entry for the given ISD-AS and IP pair. +func (c *Connector) DelSvc(ia addr.IA, svc addr.SVC, a netip.AddrPort) error { + c.mtx.Lock() + defer c.mtx.Unlock() + log.Debug("Deleting service", "isd_as", ia, "svc", svc, "address", a) + if !c.ia.Equal(ia) { + return serrors.JoinNoStack(errMultiIA, nil, "current", c.ia, "new", a) + } + return c.DataPlane.DelSvc(svc, a) +} + +// SetKey sets the key for the given ISD-AS at the given index. +func (c *Connector) SetKey(ia addr.IA, index int, key []byte) error { + c.mtx.Lock() + defer c.mtx.Unlock() + log.Debug("Setting key", "isd_as", ia, "index", index) + if !c.ia.Equal(ia) { + return serrors.JoinNoStack(errMultiIA, nil, "current", c.ia, "new", ia) + } + if index != 0 { + return serrors.New("currently only index 0 key is supported") + } + return c.DataPlane.SetKey(key) +} + +func (c *Connector) ListInternalInterfaces() ([]control.InternalInterface, error) { + c.mtx.Lock() + defer c.mtx.Unlock() + + if len(c.internalInterfaces) == 0 { + return nil, serrors.New("internal interface is not set") + } + return c.internalInterfaces, nil +} + +func (c *Connector) ListExternalInterfaces() ([]control.ExternalInterface, error) { + c.mtx.Lock() + defer c.mtx.Unlock() + + externalInterfaceList := make([]control.ExternalInterface, 0, len(c.externalInterfaces)) + for _, externalInterface := range c.externalInterfaces { + externalInterface.State = c.DataPlane.getInterfaceState(externalInterface.IfID) + externalInterfaceList = append(externalInterfaceList, externalInterface) + } + return externalInterfaceList, nil +} + +func (c *Connector) ListSiblingInterfaces() ([]control.SiblingInterface, error) { + c.mtx.Lock() + defer c.mtx.Unlock() + + siblingInterfaceList := make([]control.SiblingInterface, 0, len(c.siblingInterfaces)) + for _, siblingInterface := range c.siblingInterfaces { + siblingInterface.State = c.DataPlane.getInterfaceState(siblingInterface.IfID) + siblingInterfaceList = append(siblingInterfaceList, siblingInterface) + } + return siblingInterfaceList, nil +} + +// applyBFDDefaults updates the given cfg object with the global default BFD settings. +// Link-specific settings, if configured, remain unchanged. IMPORTANT: cfg.Disable isn't a boolean +// but a pointer to boolean, allowing a simple representation of the unconfigured state: nil. This +// means that using a cfg object that hasn't been processed by this function may lead to a NPE. +// In particular, "control.BFD{}" is invalid. +func (c *Connector) applyBFDDefaults(cfg control.BFD) control.BFD { + + if cfg.Disable == nil { + disable := c.BFD.Disable + cfg.Disable = &disable + } + if cfg.DetectMult == 0 { + cfg.DetectMult = c.BFD.DetectMult + } + if cfg.DesiredMinTxInterval == 0 { + cfg.DesiredMinTxInterval = c.BFD.DesiredMinTxInterval.Duration + } + if cfg.RequiredMinRxInterval == 0 { + cfg.RequiredMinRxInterval = c.BFD.RequiredMinRxInterval.Duration + } + return cfg +} + +func (c *Connector) SetPortRange(start, end uint16) { + c.mtx.Lock() + defer c.mtx.Unlock() + if c.DispatchedPortStart != nil { + start = uint16(*c.DispatchedPortStart) + } + if c.DispatchedPortEnd != nil { + end = uint16(*c.DispatchedPortEnd) + } + log.Debug("Endhost port range configuration", "startPort", start, "endPort", end) + c.DataPlane.SetPortRange(start, end) +} diff --git a/router/BUILD.bazel b/router/BUILD.bazel index f7fee76cd7..ef54cc4e5e 100644 --- a/router/BUILD.bazel +++ b/router/BUILD.bazel @@ -7,7 +7,7 @@ go_library( "dataplane.go", "fnv1aCheap.go", "metrics.go", - "serializeProxy.go", + "serialize_proxy.go", "svc.go", ], importpath = "github.com/scionproto/scion/router", diff --git a/router/dataplane.go b/router/dataplane.go index 4eb62b0e76..92faf22ded 100644 --- a/router/dataplane.go +++ b/router/dataplane.go @@ -2295,8 +2295,9 @@ func decodeSCMP(scmp *slayers.SCMP) ([]gopacket.SerializableLayer, error) { // afterwards (and the preceding headers, if any). func updateSCIONLayer(rawPkt []byte, s slayers.SCION) error { payloadOffset := len(rawPkt) - len(s.LayerPayload()) - serBuf := newSerializeProxy(rawPkt) - serBuf.clear(payloadOffset) // Prepends will go just before payload. (Appends will wreck it) + + // Prepends must go just before payload. (and any Append will wreck it) + serBuf := newSerializeProxyOffset(rawPkt, payloadOffset) return s.SerializeTo(&serBuf, gopacket.SerializeOptions{}) } diff --git a/router/serializeProxy.go b/router/serialize_proxy.go similarity index 74% rename from router/serializeProxy.go rename to router/serialize_proxy.go index 96a355301d..1abbeec199 100644 --- a/router/serializeProxy.go +++ b/router/serialize_proxy.go @@ -25,26 +25,38 @@ import ( // panic. It is designed to be a local variable, so New() returns a value. The entire buffer // underpinning the given slice may be used; that is, from the start up to the remaining capacity. type serializeProxy struct { - initStart int // Initial starting point (where the first prepend or append occurs) - data []byte - start int // The slice's offset can't change (irreversible). So keep track separately. - layers []gopacket.LayerType + + // The slice's offset can't be changed as that is irreversible. + // So we keep track separately from the slice. + + restart int // the value to reset start to during Clear(). + start int // current start of the useful data in the buffer. + data []byte + layers []gopacket.LayerType } -// Initializes a new serializeProxy. The initial prepend/append point is set to the end of the -// buffer in anticipation of AppendBytes never being used. This can be changed by calling clear() -// if AppendBytes is to be used. +// newSerializeProxy returns a new serializeProxy. The initial prepend/append point is set to the +// end of the buffer in anticipation of AppendBytes never being used. The prepend/append point can +// be changed when calling clear(). func newSerializeProxy(buf []byte) serializeProxy { + return newSerializeProxyOffset(buf, cap(buf)) +} + +// newSerializeProxyOffset returns a new serializeProxy. The initial prepend/append point is set to +// the given start value. This has the same effect as calling clear(statr). +func newSerializeProxyOffset(buf []byte, start int) serializeProxy { serBuf := serializeProxy{ data: buf, } - serBuf.clear(cap(buf)) + serBuf.clear(start) return serBuf } // Resets the buffer to empty and sets the initial prepend/append point to the given position. +// The next prepend will claim an area ending with index newStart - 1. The next append will claim an +// area starting with index newStart. func (s *serializeProxy) clear(newStart int) { - s.initStart = newStart + s.restart = newStart s.start = newStart s.data = s.data[:newStart] s.layers = s.layers[:0] @@ -53,7 +65,7 @@ func (s *serializeProxy) clear(newStart int) { // Implements serializeBuffer.Clear(). This implementation never returns an error. // The initial prepend/append point is reset to that which was set by the last call to clear(). func (s *serializeProxy) Clear() error { - s.clear(s.initStart) + s.clear(s.restart) return nil } From b12d2eef54add2e0d93d043468d97ade83f11a53 Mon Sep 17 00:00:00 2001 From: Jean-Christophe Hugly Date: Fri, 29 Nov 2024 13:46:19 +0100 Subject: [PATCH 6/6] Apply reviewers suggestions --- router/#connector.go# | 250 -------------------------------------- router/dataplane.go | 2 +- router/serialize_proxy.go | 8 +- 3 files changed, 5 insertions(+), 255 deletions(-) delete mode 100644 router/#connector.go# diff --git a/router/#connector.go# b/router/#connector.go# deleted file mode 100644 index 113c68eb28..0000000000 --- a/router/#connector.go# +++ /dev/null @@ -1,250 +0,0 @@ -// Copyright 2020 Anapaya Systems -// -// 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 router - -import ( - "net/netip" - "sync" - - "github.com/scionproto/scion/pkg/addr" - "github.com/scionproto/scion/pkg/log" - "github.com/scionproto/scion/pkg/private/serrors" - "github.com/scionproto/scion/pkg/segment/iface" - "github.com/scionproto/scion/private/underlay/conn" - "github.com/scionproto/scion/router/config" - "github.com/scionproto/scion/router/control" -) - -// Connector implements the Dataplane API of the router control process. It sets -// up connections for the DataPlane. -type Connector struct { - DataPlane DataPlane - - ia addr.IA - mtx sync.Mutex - internalInterfaces []control.InternalInterface - externalInterfaces map[uint16]control.ExternalInterface - siblingInterfaces map[uint16]control.SiblingInterface - internalConnection conn.Conn - - ReceiveBufferSize int - SendBufferSize int - BFD config.BFD - DispatchedPortStart *int - DispatchedPortEnd *int -} - -var errMultiIA = serrors.New("different IA not allowed") - -// CreateIACtx creates the context for ISD-AS. -func (c *Connector) CreateIACtx(ia addr.IA) error { - c.mtx.Lock() - defer c.mtx.Unlock() - log.Debug("CreateIACtx", "isd_as", ia) - if !c.ia.IsZero() { - return serrors.JoinNoStack(errMultiIA, nil, "current", c.ia, "new", ia) - } - c.ia = ia - return c.DataPlane.SetIA(ia) -} - -// AddInternalInterface adds the internal interface. -func (c *Connector) AddInternalInterface(ia addr.IA, local netip.AddrPort) error { - c.mtx.Lock() - defer c.mtx.Unlock() - log.Debug("Adding internal interface", "isd_as", ia, "local", local) - if !c.ia.Equal(ia) { - return serrors.JoinNoStack(errMultiIA, nil, "current", c.ia, "new", ia) - } - connection, err := conn.New(local, netip.AddrPort{}, - &conn.Config{ReceiveBufferSize: c.ReceiveBufferSize, SendBufferSize: c.SendBufferSize}) - if err != nil { - return err - } - c.internalConnection = connection - c.internalInterfaces = append(c.internalInterfaces, control.InternalInterface{ - IA: ia, - Addr: local, - }) - return c.DataPlane.AddInternalInterface(connection, local.Addr()) -} - -// AddExternalInterface adds a link between the local and remote address. -// TODO(jiceatscion): do not depend on AddInternalInterface having been called first. -func (c *Connector) AddExternalInterface(localIfID iface.ID, link control.LinkInfo, - owned bool) error { - - c.mtx.Lock() - defer c.mtx.Unlock() - intf := uint16(localIfID) - log.Debug("Adding external interface", "interface", localIfID, - "local_isd_as", link.Local.IA, "local_addr", link.Local.Addr, - "remote_isd_as", link.Remote.IA, "remote_addr", link.Remote.Addr, - "owned", owned, - "link_bfd_configured", link.BFD.Disable != nil, - "link_bfd_enabled", link.BFD.Disable == nil || !*link.BFD.Disable, - "dataplane_bfd_enabled", !c.BFD.Disable) - - if !c.ia.Equal(link.Local.IA) { - return serrors.JoinNoStack(errMultiIA, nil, "current", c.ia, "new", link.Local.IA) - } - if err := c.DataPlane.AddLinkType(intf, link.LinkTo); err != nil { - return serrors.Wrap("adding link type", err, "if_id", localIfID) - } - if err := c.DataPlane.AddNeighborIA(intf, link.Remote.IA); err != nil { - return serrors.Wrap("adding neighboring IA", err, "if_id", localIfID) - } - - link.BFD = c.applyBFDDefaults(link.BFD) - if owned { - if len(c.externalInterfaces) == 0 { - c.externalInterfaces = make(map[uint16]control.ExternalInterface) - } - c.externalInterfaces[intf] = control.ExternalInterface{ - IfID: intf, - Link: link, - State: control.InterfaceDown, - } - connection, err := conn.New(link.Local.Addr, link.Remote.Addr, - &conn.Config{ReceiveBufferSize: c.ReceiveBufferSize, SendBufferSize: c.SendBufferSize}) - if err != nil { - return err - } - - return c.DataPlane.AddExternalInterface(intf, connection, link.Local, link.Remote, link.BFD, - owned) - } - - if len(c.siblingInterfaces) == 0 { - c.siblingInterfaces = make(map[uint16]control.SiblingInterface) - } - c.siblingInterfaces[intf] = control.SiblingInterface{ - IfID: intf, - InternalInterface: link.Remote.Addr, - Relationship: link.LinkTo, - MTU: link.MTU, - NeighborIA: link.Remote.IA, - State: control.InterfaceDown, - } - return c.DataPlane.AddNextHop(intf, link.Local.Addr, link.Remote.Addr, - link.BFD, link.Instance) -} - -// AddSvc adds the service address for the given ISD-AS. -func (c *Connector) AddSvc(ia addr.IA, svc addr.SVC, a netip.AddrPort) error { - - c.mtx.Lock() - defer c.mtx.Unlock() - log.Debug("Adding service", "isd_as", ia, "svc", svc, "address", a) - if !c.ia.Equal(ia) { - return serrors.JoinNoStack(errMultiIA, nil, "current", c.ia, "new", a) - } - return c.DataPlane.AddSvc(svc, a) -} - -// DelSvc deletes the service entry for the given ISD-AS and IP pair. -func (c *Connector) DelSvc(ia addr.IA, svc addr.SVC, a netip.AddrPort) error { - c.mtx.Lock() - defer c.mtx.Unlock() - log.Debug("Deleting service", "isd_as", ia, "svc", svc, "address", a) - if !c.ia.Equal(ia) { - return serrors.JoinNoStack(errMultiIA, nil, "current", c.ia, "new", a) - } - return c.DataPlane.DelSvc(svc, a) -} - -// SetKey sets the key for the given ISD-AS at the given index. -func (c *Connector) SetKey(ia addr.IA, index int, key []byte) error { - c.mtx.Lock() - defer c.mtx.Unlock() - log.Debug("Setting key", "isd_as", ia, "index", index) - if !c.ia.Equal(ia) { - return serrors.JoinNoStack(errMultiIA, nil, "current", c.ia, "new", ia) - } - if index != 0 { - return serrors.New("currently only index 0 key is supported") - } - return c.DataPlane.SetKey(key) -} - -func (c *Connector) ListInternalInterfaces() ([]control.InternalInterface, error) { - c.mtx.Lock() - defer c.mtx.Unlock() - - if len(c.internalInterfaces) == 0 { - return nil, serrors.New("internal interface is not set") - } - return c.internalInterfaces, nil -} - -func (c *Connector) ListExternalInterfaces() ([]control.ExternalInterface, error) { - c.mtx.Lock() - defer c.mtx.Unlock() - - externalInterfaceList := make([]control.ExternalInterface, 0, len(c.externalInterfaces)) - for _, externalInterface := range c.externalInterfaces { - externalInterface.State = c.DataPlane.getInterfaceState(externalInterface.IfID) - externalInterfaceList = append(externalInterfaceList, externalInterface) - } - return externalInterfaceList, nil -} - -func (c *Connector) ListSiblingInterfaces() ([]control.SiblingInterface, error) { - c.mtx.Lock() - defer c.mtx.Unlock() - - siblingInterfaceList := make([]control.SiblingInterface, 0, len(c.siblingInterfaces)) - for _, siblingInterface := range c.siblingInterfaces { - siblingInterface.State = c.DataPlane.getInterfaceState(siblingInterface.IfID) - siblingInterfaceList = append(siblingInterfaceList, siblingInterface) - } - return siblingInterfaceList, nil -} - -// applyBFDDefaults updates the given cfg object with the global default BFD settings. -// Link-specific settings, if configured, remain unchanged. IMPORTANT: cfg.Disable isn't a boolean -// but a pointer to boolean, allowing a simple representation of the unconfigured state: nil. This -// means that using a cfg object that hasn't been processed by this function may lead to a NPE. -// In particular, "control.BFD{}" is invalid. -func (c *Connector) applyBFDDefaults(cfg control.BFD) control.BFD { - - if cfg.Disable == nil { - disable := c.BFD.Disable - cfg.Disable = &disable - } - if cfg.DetectMult == 0 { - cfg.DetectMult = c.BFD.DetectMult - } - if cfg.DesiredMinTxInterval == 0 { - cfg.DesiredMinTxInterval = c.BFD.DesiredMinTxInterval.Duration - } - if cfg.RequiredMinRxInterval == 0 { - cfg.RequiredMinRxInterval = c.BFD.RequiredMinRxInterval.Duration - } - return cfg -} - -func (c *Connector) SetPortRange(start, end uint16) { - c.mtx.Lock() - defer c.mtx.Unlock() - if c.DispatchedPortStart != nil { - start = uint16(*c.DispatchedPortStart) - } - if c.DispatchedPortEnd != nil { - end = uint16(*c.DispatchedPortEnd) - } - log.Debug("Endhost port range configuration", "startPort", start, "endPort", end) - c.DataPlane.SetPortRange(start, end) -} diff --git a/router/dataplane.go b/router/dataplane.go index 92faf22ded..6c407ab1c6 100644 --- a/router/dataplane.go +++ b/router/dataplane.go @@ -2297,7 +2297,7 @@ func updateSCIONLayer(rawPkt []byte, s slayers.SCION) error { payloadOffset := len(rawPkt) - len(s.LayerPayload()) // Prepends must go just before payload. (and any Append will wreck it) - serBuf := newSerializeProxyOffset(rawPkt, payloadOffset) + serBuf := newSerializeProxyStart(rawPkt, payloadOffset) return s.SerializeTo(&serBuf, gopacket.SerializeOptions{}) } diff --git a/router/serialize_proxy.go b/router/serialize_proxy.go index 1abbeec199..e3b6b1cfbc 100644 --- a/router/serialize_proxy.go +++ b/router/serialize_proxy.go @@ -27,7 +27,7 @@ import ( type serializeProxy struct { // The slice's offset can't be changed as that is irreversible. - // So we keep track separately from the slice. + // So we keep track of the prepend point separately from the slice. restart int // the value to reset start to during Clear(). start int // current start of the useful data in the buffer. @@ -39,12 +39,12 @@ type serializeProxy struct { // end of the buffer in anticipation of AppendBytes never being used. The prepend/append point can // be changed when calling clear(). func newSerializeProxy(buf []byte) serializeProxy { - return newSerializeProxyOffset(buf, cap(buf)) + return newSerializeProxyStart(buf, cap(buf)) } -// newSerializeProxyOffset returns a new serializeProxy. The initial prepend/append point is set to +// newSerializeProxyStart returns a new serializeProxy. The initial prepend/append point is set to // the given start value. This has the same effect as calling clear(statr). -func newSerializeProxyOffset(buf []byte, start int) serializeProxy { +func newSerializeProxyStart(buf []byte, start int) serializeProxy { serBuf := serializeProxy{ data: buf, }