summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip/stack
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/tcpip/stack')
-rw-r--r--pkg/tcpip/stack/BUILD1
-rw-r--r--pkg/tcpip/stack/packet_buffer.go367
-rw-r--r--pkg/tcpip/stack/packet_buffer_test.go29
3 files changed, 177 insertions, 220 deletions
diff --git a/pkg/tcpip/stack/BUILD b/pkg/tcpip/stack/BUILD
index 63ab31083..84aa6a9e4 100644
--- a/pkg/tcpip/stack/BUILD
+++ b/pkg/tcpip/stack/BUILD
@@ -74,6 +74,7 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/atomicbitops",
+ "//pkg/buffer",
"//pkg/ilist",
"//pkg/log",
"//pkg/rand",
diff --git a/pkg/tcpip/stack/packet_buffer.go b/pkg/tcpip/stack/packet_buffer.go
index fc3c54e34..e2e073091 100644
--- a/pkg/tcpip/stack/packet_buffer.go
+++ b/pkg/tcpip/stack/packet_buffer.go
@@ -16,9 +16,10 @@ package stack
import (
"fmt"
+ "gvisor.dev/gvisor/pkg/buffer"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/tcpip"
- "gvisor.dev/gvisor/pkg/tcpip/buffer"
+ tcpipbuffer "gvisor.dev/gvisor/pkg/tcpip/buffer"
"gvisor.dev/gvisor/pkg/tcpip/header"
)
@@ -39,7 +40,7 @@ type PacketBufferOptions struct {
// Data is the initial unparsed data for the new packet. If set, it will be
// owned by the new packet.
- Data buffer.VectorisedView
+ Data tcpipbuffer.VectorisedView
// IsForwardedPacket identifies that the PacketBuffer being created is for a
// forwarded packet.
@@ -56,6 +57,34 @@ type PacketBufferOptions struct {
// empty. Use of PacketBuffer in any other order is unsupported.
//
// PacketBuffer must be created with NewPacketBuffer.
+//
+// Internal structure: A PacketBuffer holds a pointer to buffer.Buffer, which
+// exposes a logically-contiguous byte storage. The underlying storage structure
+// is abstracted out, and should not be a concern here for most of the time.
+//
+// |- reserved ->|
+// |--->| consumed (incoming)
+// 0 V V
+// +--------+----+----+--------------------+
+// | | | | current data ... | (buf)
+// +--------+----+----+--------------------+
+// ^ |
+// |<---| pushed (outgoing)
+//
+// When a PacketBuffer is created, a `reserved` header region can be specified,
+// which stack pushes headers in this region for an outgoing packet. There could
+// be no such region for an incoming packet, and `reserved` is 0. The value of
+// `reserved` never changes in the entire lifetime of the packet.
+//
+// Outgoing Packet: When a header is pushed, `pushed` gets incremented by the
+// pushed length, and the current value is stored for each header. PacketBuffer
+// substracts this value from `reserved` to compute the starting offset of each
+// header in `buf`.
+//
+// Incoming Packet: When a header is consumed (a.k.a. parsed), the current
+// `consumed` value is stored for each header, and it gets incremented by the
+// consumed length. PacketBuffer adds this value to `reserved` to compute the
+// starting offset of each header in `buf`.
type PacketBuffer struct {
_ sync.NoCopy
@@ -63,28 +92,16 @@ type PacketBuffer struct {
// PacketBuffers.
PacketBufferEntry
- // data holds the payload of the packet.
- //
- // For inbound packets, Data is initially the whole packet. Then gets moved to
- // headers via PacketHeader.Consume, when the packet is being parsed.
- //
- // For outbound packets, Data is the innermost layer, defined by the protocol.
- // Headers are pushed in front of it via PacketHeader.Push.
- //
- // The bytes backing Data are immutable, a.k.a. users shouldn't write to its
- // backing storage.
- data buffer.VectorisedView
+ // buf is the underlying buffer for the packet. See struct level docs for
+ // details.
+ buf *buffer.Buffer
+ reserved int
+ pushed int
+ consumed int
// headers stores metadata about each header.
headers [numHeaderType]headerInfo
- // header is the internal storage for outbound packets. Headers will be pushed
- // (prepended) on this storage as the packet is being constructed.
- //
- // TODO(gvisor.dev/issue/2404): Switch to an implementation that header and
- // data are held in the same underlying buffer storage.
- header buffer.Prependable
-
// NetworkProtocolNumber is only valid when NetworkHeader().View().IsEmpty()
// returns false.
// TODO(gvisor.dev/issue/3574): Remove the separately passed protocol
@@ -131,10 +148,14 @@ type PacketBuffer struct {
// NewPacketBuffer creates a new PacketBuffer with opts.
func NewPacketBuffer(opts PacketBufferOptions) *PacketBuffer {
pk := &PacketBuffer{
- data: opts.Data,
+ buf: &buffer.Buffer{},
}
if opts.ReserveHeaderBytes != 0 {
- pk.header = buffer.NewPrependable(opts.ReserveHeaderBytes)
+ pk.buf.AppendOwned(make([]byte, opts.ReserveHeaderBytes))
+ pk.reserved = opts.ReserveHeaderBytes
+ }
+ for _, v := range opts.Data.Views() {
+ pk.buf.AppendOwned(v)
}
if opts.IsForwardedPacket {
pk.NetworkPacketInfo.IsForwardedPacket = opts.IsForwardedPacket
@@ -145,13 +166,13 @@ func NewPacketBuffer(opts PacketBufferOptions) *PacketBuffer {
// ReservedHeaderBytes returns the number of bytes initially reserved for
// headers.
func (pk *PacketBuffer) ReservedHeaderBytes() int {
- return pk.header.UsedLength() + pk.header.AvailableLength()
+ return pk.reserved
}
// AvailableHeaderBytes returns the number of bytes currently available for
// headers. This is relevant to PacketHeader.Push method only.
func (pk *PacketBuffer) AvailableHeaderBytes() int {
- return pk.header.AvailableLength()
+ return pk.reserved - pk.pushed
}
// LinkHeader returns the handle to link-layer header.
@@ -180,24 +201,18 @@ func (pk *PacketBuffer) TransportHeader() PacketHeader {
// HeaderSize returns the total size of all headers in bytes.
func (pk *PacketBuffer) HeaderSize() int {
- // Note for inbound packets (Consume called), headers are not stored in
- // pk.header. Thus, calculation of size of each header is needed.
- var size int
- for i := range pk.headers {
- size += len(pk.headers[i].buf)
- }
- return size
+ return pk.pushed + pk.consumed
}
// Size returns the size of packet in bytes.
func (pk *PacketBuffer) Size() int {
- return pk.HeaderSize() + pk.data.Size()
+ return int(pk.buf.Size()) - pk.headerOffset()
}
// MemSize returns the estimation size of the pk in memory, including backing
// buffer data.
func (pk *PacketBuffer) MemSize() int {
- return pk.HeaderSize() + pk.data.MemSize() + packetBufferStructSize
+ return int(pk.buf.Size()) + packetBufferStructSize
}
// Data returns the handle to data portion of pk.
@@ -206,61 +221,65 @@ func (pk *PacketBuffer) Data() PacketData {
}
// Views returns the underlying storage of the whole packet.
-func (pk *PacketBuffer) Views() []buffer.View {
- // Optimization for outbound packets that headers are in pk.header.
- useHeader := true
- for i := range pk.headers {
- if !canUseHeader(&pk.headers[i]) {
- useHeader = false
- break
- }
- }
+func (pk *PacketBuffer) Views() []tcpipbuffer.View {
+ var views []tcpipbuffer.View
+ offset := pk.headerOffset()
+ pk.buf.SubApply(offset, int(pk.buf.Size())-offset, func(v []byte) {
+ views = append(views, v)
+ })
+ return views
+}
- dataViews := pk.data.Views()
-
- var vs []buffer.View
- if useHeader {
- vs = make([]buffer.View, 0, 1+len(dataViews))
- vs = append(vs, pk.header.View())
- } else {
- vs = make([]buffer.View, 0, len(pk.headers)+len(dataViews))
- for i := range pk.headers {
- if v := pk.headers[i].buf; len(v) > 0 {
- vs = append(vs, v)
- }
- }
- }
- return append(vs, dataViews...)
+func (pk *PacketBuffer) headerOffset() int {
+ return pk.reserved - pk.pushed
+}
+
+func (pk *PacketBuffer) headerOffsetOf(typ headerType) int {
+ return pk.reserved + pk.headers[typ].offset
}
-func canUseHeader(h *headerInfo) bool {
- // h.offset will be negative if the header was pushed in to prependable
- // portion, or doesn't matter when it's empty.
- return len(h.buf) == 0 || h.offset < 0
+func (pk *PacketBuffer) dataOffset() int {
+ return pk.reserved + pk.consumed
}
-func (pk *PacketBuffer) push(typ headerType, size int) buffer.View {
+func (pk *PacketBuffer) push(typ headerType, size int) tcpipbuffer.View {
h := &pk.headers[typ]
- if h.buf != nil {
+ if h.length > 0 {
panic(fmt.Sprintf("push must not be called twice: type %s", typ))
}
- h.buf = buffer.View(pk.header.Prepend(size))
- h.offset = -pk.header.UsedLength()
- return h.buf
+ if pk.pushed+size > pk.reserved {
+ panic("not enough headroom reserved")
+ }
+ pk.pushed += size
+ h.offset = -pk.pushed
+ h.length = size
+ return pk.headerView(typ)
}
-func (pk *PacketBuffer) consume(typ headerType, size int) (v buffer.View, consumed bool) {
+func (pk *PacketBuffer) consume(typ headerType, size int) (v tcpipbuffer.View, consumed bool) {
h := &pk.headers[typ]
- if h.buf != nil {
+ if h.length > 0 {
panic(fmt.Sprintf("consume must not be called twice: type %s", typ))
}
- v, ok := pk.data.PullUp(size)
+ if pk.headerOffset()+pk.consumed+size > int(pk.buf.Size()) {
+ return nil, false
+ }
+ h.offset = pk.consumed
+ h.length = size
+ pk.consumed += size
+ return pk.headerView(typ), true
+}
+
+func (pk *PacketBuffer) headerView(typ headerType) tcpipbuffer.View {
+ h := &pk.headers[typ]
+ if h.length == 0 {
+ return nil
+ }
+ v, ok := pk.buf.PullUp(pk.headerOffsetOf(typ), h.length)
if !ok {
- return
+ panic("PullUp failed")
}
- pk.data.TrimFront(size)
- h.buf = v
- return h.buf, true
+ return v
}
// Clone makes a shallow copy of pk.
@@ -270,9 +289,11 @@ func (pk *PacketBuffer) consume(typ headerType, size int) (v buffer.View, consum
func (pk *PacketBuffer) Clone() *PacketBuffer {
return &PacketBuffer{
PacketBufferEntry: pk.PacketBufferEntry,
- data: pk.data.Clone(nil),
+ buf: pk.buf,
+ reserved: pk.reserved,
+ pushed: pk.pushed,
+ consumed: pk.consumed,
headers: pk.headers,
- header: pk.header,
Hash: pk.Hash,
Owner: pk.Owner,
GSOOptions: pk.GSOOptions,
@@ -306,9 +327,11 @@ func (pk *PacketBuffer) Network() header.Network {
// See PacketBuffer.Data for details about how a packet buffer holds an inbound
// packet.
func (pk *PacketBuffer) CloneToInbound() *PacketBuffer {
- newPk := NewPacketBuffer(PacketBufferOptions{
- Data: buffer.NewVectorisedView(pk.Size(), pk.Views()),
- })
+ newPk := &PacketBuffer{
+ buf: pk.buf,
+ // Treat unfilled header portion as reserved.
+ reserved: pk.AvailableHeaderBytes(),
+ }
// TODO(gvisor.dev/issue/5696): reimplement conntrack so that no need to
// maintain this flag in the packet. Currently conntrack needs this flag to
// tell if a noop connection should be inserted at Input hook. Once conntrack
@@ -322,15 +345,12 @@ func (pk *PacketBuffer) CloneToInbound() *PacketBuffer {
// headerInfo stores metadata about a header in a packet.
type headerInfo struct {
- // buf is the memorized slice for both prepended and consumed header.
- // When header is prepended, buf serves as memorized value, which is a slice
- // of pk.header. When header is consumed, buf is the slice pulled out from
- // pk.Data, which is the only place to hold this header.
- buf buffer.View
-
- // offset will be a negative number denoting the offset where this header is
- // from the end of pk.header, if it is prepended. Otherwise, zero.
+ // offset is the offset of the header in pk.buf relative to
+ // pk.buf[pk.reserved]. See the PacketBuffer struct for details.
offset int
+
+ // length is the length of this header.
+ length int
}
// PacketHeader is a handle object to a header in the underlying packet.
@@ -340,14 +360,14 @@ type PacketHeader struct {
}
// View returns the underlying storage of h.
-func (h PacketHeader) View() buffer.View {
- return h.pk.headers[h.typ].buf
+func (h PacketHeader) View() tcpipbuffer.View {
+ return h.pk.headerView(h.typ)
}
// Push pushes size bytes in the front of its residing packet, and returns the
// backing storage. Callers may only call one of Push or Consume once on each
// header in the lifetime of the underlying packet.
-func (h PacketHeader) Push(size int) buffer.View {
+func (h PacketHeader) Push(size int) tcpipbuffer.View {
return h.pk.push(h.typ, size)
}
@@ -356,7 +376,7 @@ func (h PacketHeader) Push(size int) buffer.View {
// size, consumed will be false, and the state of h will not be affected.
// Callers may only call one of Push or Consume once on each header in the
// lifetime of the underlying packet.
-func (h PacketHeader) Consume(size int) (v buffer.View, consumed bool) {
+func (h PacketHeader) Consume(size int) (v tcpipbuffer.View, consumed bool) {
return h.pk.consume(h.typ, size)
}
@@ -367,55 +387,84 @@ type PacketData struct {
// PullUp returns a contiguous view of size bytes from the beginning of d.
// Callers should not write to or keep the view for later use.
-func (d PacketData) PullUp(size int) (buffer.View, bool) {
- return d.pk.data.PullUp(size)
+func (d PacketData) PullUp(size int) (tcpipbuffer.View, bool) {
+ return d.pk.buf.PullUp(d.pk.dataOffset(), size)
}
// DeleteFront removes count from the beginning of d. It panics if count >
// d.Size(). All backing storage references after the front of the d are
// invalidated.
func (d PacketData) DeleteFront(count int) {
- d.pk.data.TrimFront(count)
+ if !d.pk.buf.Remove(d.pk.dataOffset(), count) {
+ panic("count > d.Size()")
+ }
}
// CapLength reduces d to at most length bytes.
func (d PacketData) CapLength(length int) {
- d.pk.data.CapLength(length)
+ if length < 0 {
+ panic("length < 0")
+ }
+ if currLength := d.Size(); currLength > length {
+ trim := currLength - length
+ d.pk.buf.Remove(int(d.pk.buf.Size())-trim, trim)
+ }
}
// Views returns the underlying storage of d in a slice of Views. Caller should
// not modify the returned slice.
-func (d PacketData) Views() []buffer.View {
- return d.pk.data.Views()
+func (d PacketData) Views() []tcpipbuffer.View {
+ var views []tcpipbuffer.View
+ offset := d.pk.dataOffset()
+ d.pk.buf.SubApply(offset, int(d.pk.buf.Size())-offset, func(v []byte) {
+ views = append(views, v)
+ })
+ return views
}
// AppendView appends v into d, taking the ownership of v.
-func (d PacketData) AppendView(v buffer.View) {
- d.pk.data.AppendView(v)
+func (d PacketData) AppendView(v tcpipbuffer.View) {
+ d.pk.buf.AppendOwned(v)
}
-// ReadFromData moves at most count bytes from the beginning of srcData to the
-// end of d and returns the number of bytes moved.
-func (d PacketData) ReadFromData(srcData PacketData, count int) int {
- return srcData.pk.data.ReadToVV(&d.pk.data, count)
+// MergeFragment appends the data portion of frag to dst. It takes ownership of
+// frag and frag should not be used again.
+func MergeFragment(dst, frag *PacketBuffer) {
+ frag.buf.TrimFront(int64(frag.dataOffset()))
+ dst.buf.Merge(frag.buf)
}
// ReadFromVV moves at most count bytes from the beginning of srcVV to the end
// of d and returns the number of bytes moved.
-func (d PacketData) ReadFromVV(srcVV *buffer.VectorisedView, count int) int {
- return srcVV.ReadToVV(&d.pk.data, count)
+func (d PacketData) ReadFromVV(srcVV *tcpipbuffer.VectorisedView, count int) int {
+ done := 0
+ for _, v := range srcVV.Views() {
+ if len(v) < count {
+ count -= len(v)
+ done += len(v)
+ d.pk.buf.AppendOwned(v)
+ } else {
+ v = v[:count]
+ count -= len(v)
+ done += len(v)
+ d.pk.buf.Append(v)
+ break
+ }
+ }
+ srcVV.TrimFront(done)
+ return done
}
// Size returns the number of bytes in the data payload of the packet.
func (d PacketData) Size() int {
- return d.pk.data.Size()
+ return int(d.pk.buf.Size()) - d.pk.dataOffset()
}
// AsRange returns a Range representing the current data payload of the packet.
func (d PacketData) AsRange() Range {
return Range{
pk: d.pk,
- offset: d.pk.HeaderSize(),
+ offset: d.pk.dataOffset(),
length: d.Size(),
}
}
@@ -425,17 +474,12 @@ func (d PacketData) AsRange() Range {
//
// This method exists for compatibility between PacketBuffer and VectorisedView.
// It may be removed later and should be used with care.
-func (d PacketData) ExtractVV() buffer.VectorisedView {
- return d.pk.data
-}
-
-// Replace replaces the data portion of the packet with vv, taking the ownership
-// of vv.
-//
-// This method exists for compatibility between PacketBuffer and VectorisedView.
-// It may be removed later and should be used with care.
-func (d PacketData) Replace(vv buffer.VectorisedView) {
- d.pk.data = vv
+func (d PacketData) ExtractVV() tcpipbuffer.VectorisedView {
+ var vv tcpipbuffer.VectorisedView
+ d.pk.buf.SubApply(d.pk.dataOffset(), d.pk.Size(), func(v []byte) {
+ vv.AppendView(v)
+ })
+ return vv
}
// Range represents a contiguous subportion of a PacketBuffer.
@@ -479,9 +523,9 @@ func (r Range) Capped(max int) Range {
// AsView returns the backing storage of r if possible. It will allocate a new
// View if r spans multiple pieces internally. Caller should not write to the
// returned View in any way.
-func (r Range) AsView() buffer.View {
+func (r Range) AsView() tcpipbuffer.View {
var allocated bool
- var v buffer.View
+ var v tcpipbuffer.View
r.iterate(func(b []byte) {
if v == nil {
// v has not been assigned, allowing first view to be returned.
@@ -502,7 +546,7 @@ func (r Range) AsView() buffer.View {
}
// ToOwnedView returns a owned copy of data in r.
-func (r Range) ToOwnedView() buffer.View {
+func (r Range) ToOwnedView() tcpipbuffer.View {
if r.length == 0 {
return nil
}
@@ -523,63 +567,7 @@ func (r Range) Checksum() uint16 {
// iterate calls fn for each piece in r. fn is always called with a non-empty
// slice.
func (r Range) iterate(fn func([]byte)) {
- w := window{
- offset: r.offset,
- length: r.length,
- }
- // Header portion.
- for i := range r.pk.headers {
- if b := w.process(r.pk.headers[i].buf); len(b) > 0 {
- fn(b)
- }
- if w.isDone() {
- break
- }
- }
- // Data portion.
- if !w.isDone() {
- for _, v := range r.pk.data.Views() {
- if b := w.process(v); len(b) > 0 {
- fn(b)
- }
- if w.isDone() {
- break
- }
- }
- }
-}
-
-// window represents contiguous region of byte stream. User would call process()
-// to input bytes, and obtain a subslice that is inside the window.
-type window struct {
- offset int
- length int
-}
-
-// isDone returns true if the window has passed and further process() calls will
-// always return an empty slice. This can be used to end processing early.
-func (w *window) isDone() bool {
- return w.length == 0
-}
-
-// process feeds b in and returns a subslice that is inside the window. The
-// returned slice will be a subslice of b, and it does not keep b after method
-// returns. This method may return an empty slice if nothing in b is inside the
-// window.
-func (w *window) process(b []byte) (inWindow []byte) {
- if w.offset >= len(b) {
- w.offset -= len(b)
- return nil
- }
- if w.offset > 0 {
- b = b[w.offset:]
- w.offset = 0
- }
- if w.length < len(b) {
- b = b[:w.length]
- }
- w.length -= len(b)
- return b
+ r.pk.buf.SubApply(r.offset, r.length, fn)
}
// PayloadSince returns packet payload starting from and including a particular
@@ -587,21 +575,14 @@ func (w *window) process(b []byte) (inWindow []byte) {
//
// The returned View is owned by the caller - its backing buffer is separate
// from the packet header's underlying packet buffer.
-func PayloadSince(h PacketHeader) buffer.View {
- size := h.pk.data.Size()
- for _, hinfo := range h.pk.headers[h.typ:] {
- size += len(hinfo.buf)
+func PayloadSince(h PacketHeader) tcpipbuffer.View {
+ offset := h.pk.headerOffset()
+ for i := headerType(0); i < h.typ; i++ {
+ offset += h.pk.headers[i].length
}
-
- v := make(buffer.View, 0, size)
-
- for _, hinfo := range h.pk.headers[h.typ:] {
- v = append(v, hinfo.buf...)
- }
-
- for _, view := range h.pk.data.Views() {
- v = append(v, view...)
- }
-
- return v
+ return Range{
+ pk: h.pk,
+ offset: offset,
+ length: int(h.pk.buf.Size()) - offset,
+ }.ToOwnedView()
}
diff --git a/pkg/tcpip/stack/packet_buffer_test.go b/pkg/tcpip/stack/packet_buffer_test.go
index bd4eb4fed..1c1aeb950 100644
--- a/pkg/tcpip/stack/packet_buffer_test.go
+++ b/pkg/tcpip/stack/packet_buffer_test.go
@@ -444,23 +444,8 @@ func TestPacketBufferData(t *testing.T) {
checkData(t, pkt, []byte(tc.data+s))
})
- // ReadFromData/VV
+ // ReadFromVV
for _, n := range []int{0, 1, 2, 7, 10, 14, 20} {
- t.Run(fmt.Sprintf("ReadFromData%d", n), func(t *testing.T) {
- s := "TO READ"
- otherPkt := NewPacketBuffer(PacketBufferOptions{
- Data: vv(s, s),
- })
- s += s
-
- pkt := tc.makePkt(t)
- pkt.Data().ReadFromData(otherPkt.Data(), n)
-
- if n < len(s) {
- s = s[:n]
- }
- checkData(t, pkt, []byte(tc.data+s))
- })
t.Run(fmt.Sprintf("ReadFromVV%d", n), func(t *testing.T) {
s := "TO READ"
srcVV := vv(s, s)
@@ -487,16 +472,6 @@ func TestPacketBufferData(t *testing.T) {
t.Errorf("pkt.Data().ExtractVV().ToOwnedView() = %q, want %q", got, want)
}
})
-
- // Replace
- t.Run("Replace", func(t *testing.T) {
- s := "REPLACED"
-
- pkt := tc.makePkt(t)
- pkt.Data().Replace(vv(s))
-
- checkData(t, pkt, []byte(s))
- })
})
}
}
@@ -568,7 +543,7 @@ func checkViewEqual(t *testing.T, what string, got, want buffer.View) {
func checkData(t *testing.T, pkt *PacketBuffer, want []byte) {
t.Helper()
if got := concatViews(pkt.Data().Views()...); !bytes.Equal(got, want) {
- t.Errorf("pkt.Data().Views() = %x, want %x", got, want)
+ t.Errorf("pkt.Data().Views() = 0x%x, want 0x%x", got, want)
}
if got := pkt.Data().Size(); got != len(want) {
t.Errorf("pkt.Data().Size() = %d, want %d", got, len(want))