From 3c8e30fbdb64f27f8d166a3a116c12ca353aa601 Mon Sep 17 00:00:00 2001 From: jiceatscion <139873336+jiceatscion@users.noreply.github.com> Date: Fri, 29 Nov 2024 15:33:32 +0100 Subject: [PATCH] router: serialize directly into packet buffers (#4654) In every case where the router modified packets it would serialize updated headers to a temporary buffer and then copy that to the packet buffer. To avoid this extra copy, replaced gopacket.serializeBuffer with a custom implementation that writes to a given buffer. In this case, the packet's raw buffer. There is one remaining copy for some SCMP messages because we have to move the existing packet to the payload. This too could be avoided but that's for another PR; it would require to leave headroom in received packets. The performance impact is very small since, on the critical path, it just avoids copying a scion hdr per packet, but it is a simplification. It also pays back the copy added by a previous simplification of the BFD code. As such... Contributes to #4593 --- router/BUILD.bazel | 1 + router/dataplane.go | 154 +++++++++++++++++--------------------- router/serialize_proxy.go | 103 +++++++++++++++++++++++++ 3 files changed, 172 insertions(+), 86 deletions(-) create mode 100644 router/serialize_proxy.go diff --git a/router/BUILD.bazel b/router/BUILD.bazel index caf311a7f1..ef54cc4e5e 100644 --- a/router/BUILD.bazel +++ b/router/BUILD.bazel @@ -7,6 +7,7 @@ go_library( "dataplane.go", "fnv1aCheap.go", "metrics.go", + "serialize_proxy.go", "svc.go", ], importpath = "github.com/scionproto/scion/router", diff --git a/router/dataplane.go b/router/dataplane.go index c1322a5f4a..6c407ab1c6 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 @@ -907,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(), @@ -921,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 @@ -945,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{} @@ -1123,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)), } @@ -1139,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 @@ -1313,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 @@ -1374,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. @@ -1385,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 { @@ -2072,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 @@ -2098,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) @@ -2302,22 +2287,18 @@ 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 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()) + + // Prepends must go just before payload. (and any Append will wreck it) + serBuf := newSerializeProxyStart(rawPkt, payloadOffset) + return s.SerializeTo(&serBuf, gopacket.SerializeOptions{}) } type bfdSend struct { @@ -2328,7 +2309,6 @@ type bfdSend struct { ohp *onehop.Path mac hash.Hash macBuffer []byte - buffer gopacket.SerializeBuffer } // newBFDSend creates and initializes a BFD Sender @@ -2392,7 +2372,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 } @@ -2411,24 +2390,26 @@ 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) // set for prepend-only by default. Perfect here. + + // 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 { return err } + // 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() + // 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) @@ -2449,7 +2430,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). @@ -2460,36 +2441,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. @@ -2497,7 +2478,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") } } @@ -2511,7 +2492,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") } } @@ -2529,7 +2510,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} @@ -2569,19 +2550,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 { @@ -2591,15 +2572,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} @@ -2610,27 +2591,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/serialize_proxy.go b/router/serialize_proxy.go new file mode 100644 index 0000000000..e3b6b1cfbc --- /dev/null +++ b/router/serialize_proxy.go @@ -0,0 +1,103 @@ +// 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). 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 { + + // The slice's offset can't be changed as that is irreversible. + // 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. + data []byte + layers []gopacket.LayerType +} + +// 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 newSerializeProxyStart(buf, cap(buf)) +} + +// 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 newSerializeProxyStart(buf []byte, start int) serializeProxy { + serBuf := serializeProxy{ + data: 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.restart = newStart + s.start = newStart + s.data = s.data[:newStart] + s.layers = s.layers[:0] +} + +// 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.restart) + 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 + s.data = s.data[:nl] + 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) +}