summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip/network/ipv4/ipv4.go
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/tcpip/network/ipv4/ipv4.go')
-rw-r--r--pkg/tcpip/network/ipv4/ipv4.go419
1 files changed, 271 insertions, 148 deletions
diff --git a/pkg/tcpip/network/ipv4/ipv4.go b/pkg/tcpip/network/ipv4/ipv4.go
index 90f4406e5..79872ec9a 100644
--- a/pkg/tcpip/network/ipv4/ipv4.go
+++ b/pkg/tcpip/network/ipv4/ipv4.go
@@ -44,31 +44,29 @@ const (
// buckets is the number of identifier buckets.
buckets = 2048
+
+ // The size of a fragment block, in bytes, as per RFC 791 section 3.1,
+ // page 14.
+ fragmentblockSize = 8
)
type endpoint struct {
- nicid tcpip.NICID
- id stack.NetworkEndpointID
- prefixLen int
- linkEP stack.LinkEndpoint
- dispatcher stack.TransportDispatcher
- fragmentation *fragmentation.Fragmentation
- protocol *protocol
+ nicID tcpip.NICID
+ linkEP stack.LinkEndpoint
+ dispatcher stack.TransportDispatcher
+ protocol *protocol
+ stack *stack.Stack
}
// NewEndpoint creates a new ipv4 endpoint.
-func (p *protocol) NewEndpoint(nicid tcpip.NICID, addrWithPrefix tcpip.AddressWithPrefix, linkAddrCache stack.LinkAddressCache, dispatcher stack.TransportDispatcher, linkEP stack.LinkEndpoint) (stack.NetworkEndpoint, *tcpip.Error) {
- e := &endpoint{
- nicid: nicid,
- id: stack.NetworkEndpointID{LocalAddress: addrWithPrefix.Address},
- prefixLen: addrWithPrefix.PrefixLen,
- linkEP: linkEP,
- dispatcher: dispatcher,
- fragmentation: fragmentation.NewFragmentation(fragmentation.HighFragThreshold, fragmentation.LowFragThreshold, fragmentation.DefaultReassembleTimeout),
- protocol: p,
- }
-
- return e, nil
+func (p *protocol) NewEndpoint(nicID tcpip.NICID, linkAddrCache stack.LinkAddressCache, dispatcher stack.TransportDispatcher, linkEP stack.LinkEndpoint, st *stack.Stack) stack.NetworkEndpoint {
+ return &endpoint{
+ nicID: nicID,
+ linkEP: linkEP,
+ dispatcher: dispatcher,
+ protocol: p,
+ stack: st,
+ }
}
// DefaultTTL is the default time-to-live value for this endpoint.
@@ -89,17 +87,7 @@ func (e *endpoint) Capabilities() stack.LinkEndpointCapabilities {
// NICID returns the ID of the NIC this endpoint belongs to.
func (e *endpoint) NICID() tcpip.NICID {
- return e.nicid
-}
-
-// ID returns the ipv4 endpoint ID.
-func (e *endpoint) ID() *stack.NetworkEndpointID {
- return &e.id
-}
-
-// PrefixLen returns the ipv4 endpoint subnet prefix length in bits.
-func (e *endpoint) PrefixLen() int {
- return e.prefixLen
+ return e.nicID
}
// MaxHeaderLength returns the maximum length needed by ipv4 headers (and
@@ -116,14 +104,18 @@ func (e *endpoint) GSOMaxSize() uint32 {
return 0
}
+// NetworkProtocolNumber implements stack.NetworkEndpoint.NetworkProtocolNumber.
+func (e *endpoint) NetworkProtocolNumber() tcpip.NetworkProtocolNumber {
+ return e.protocol.Number()
+}
+
// writePacketFragments calls e.linkEP.WritePacket with each packet fragment to
-// write. It assumes that the IP header is entirely in hdr but does not assume
-// that only the IP header is in hdr. It assumes that the input packet's stated
-// length matches the length of the hdr+payload. mtu includes the IP header and
-// options. This does not support the DontFragment IP flag.
-func (e *endpoint) writePacketFragments(r *stack.Route, gso *stack.GSO, hdr buffer.Prependable, payload buffer.VectorisedView, mtu int) *tcpip.Error {
+// write. It assumes that the IP header is already present in pkt.NetworkHeader.
+// pkt.TransportHeader may be set. mtu includes the IP header and options. This
+// does not support the DontFragment IP flag.
+func (e *endpoint) writePacketFragments(r *stack.Route, gso *stack.GSO, mtu int, pkt *stack.PacketBuffer) *tcpip.Error {
// This packet is too big, it needs to be fragmented.
- ip := header.IPv4(hdr.View())
+ ip := header.IPv4(pkt.NetworkHeader().View())
flags := ip.Flags()
// Update mtu to take into account the header, which will exist in all
@@ -137,76 +129,88 @@ func (e *endpoint) writePacketFragments(r *stack.Route, gso *stack.GSO, hdr buff
outerMTU := innerMTU + int(ip.HeaderLength())
offset := ip.FragmentOffset()
- originalAvailableLength := hdr.AvailableLength()
+
+ // Keep the length reserved for link-layer, we need to create fragments with
+ // the same reserved length.
+ reservedForLink := pkt.AvailableHeaderBytes()
+
+ // Destroy the packet, pull all payloads out for fragmentation.
+ transHeader, data := pkt.TransportHeader().View(), pkt.Data
+
+ // Where possible, the first fragment that is sent has the same
+ // number of bytes reserved for header as the input packet. The link-layer
+ // endpoint may depend on this for looking at, eg, L4 headers.
+ transFitsFirst := len(transHeader) <= innerMTU
+
for i := 0; i < n; i++ {
- // Where possible, the first fragment that is sent has the same
- // hdr.UsedLength() as the input packet. The link-layer endpoint may depends
- // on this for looking at, eg, L4 headers.
- h := ip
- if i > 0 {
- hdr = buffer.NewPrependable(int(ip.HeaderLength()) + originalAvailableLength)
- h = header.IPv4(hdr.Prepend(int(ip.HeaderLength())))
- copy(h, ip[:ip.HeaderLength()])
+ reserve := reservedForLink + int(ip.HeaderLength())
+ if i == 0 && transFitsFirst {
+ // Reserve for transport header if it's going to be put in the first
+ // fragment.
+ reserve += len(transHeader)
+ }
+ fragPkt := stack.NewPacketBuffer(stack.PacketBufferOptions{
+ ReserveHeaderBytes: reserve,
+ })
+ fragPkt.NetworkProtocolNumber = header.IPv4ProtocolNumber
+
+ // Copy data for the fragment.
+ avail := innerMTU
+
+ if n := len(transHeader); n > 0 {
+ if n > avail {
+ n = avail
+ }
+ if i == 0 && transFitsFirst {
+ copy(fragPkt.TransportHeader().Push(n), transHeader)
+ } else {
+ fragPkt.Data.AppendView(transHeader[:n:n])
+ }
+ transHeader = transHeader[n:]
+ avail -= n
+ }
+
+ if avail > 0 {
+ n := data.Size()
+ if n > avail {
+ n = avail
+ }
+ data.ReadToVV(&fragPkt.Data, n)
+ avail -= n
}
+
+ copied := uint16(innerMTU - avail)
+
+ // Set lengths in header and calculate checksum.
+ h := header.IPv4(fragPkt.NetworkHeader().Push(len(ip)))
+ copy(h, ip)
if i != n-1 {
h.SetTotalLength(uint16(outerMTU))
h.SetFlagsFragmentOffset(flags|header.IPv4FlagMoreFragments, offset)
} else {
- h.SetTotalLength(uint16(h.HeaderLength()) + uint16(payload.Size()))
+ h.SetTotalLength(uint16(h.HeaderLength()) + copied)
h.SetFlagsFragmentOffset(flags, offset)
}
h.SetChecksum(0)
h.SetChecksum(^h.CalculateChecksum())
- offset += uint16(innerMTU)
- if i > 0 {
- newPayload := payload.Clone([]buffer.View{})
- newPayload.CapLength(innerMTU)
- if err := e.linkEP.WritePacket(r, gso, hdr, newPayload, ProtocolNumber); err != nil {
- return err
- }
- r.Stats().IP.PacketsSent.Increment()
- payload.TrimFront(newPayload.Size())
- continue
- }
- // Special handling for the first fragment because it comes from the hdr.
- if outerMTU >= hdr.UsedLength() {
- // This fragment can fit all of hdr and possibly some of payload, too.
- newPayload := payload.Clone([]buffer.View{})
- newPayloadLength := outerMTU - hdr.UsedLength()
- newPayload.CapLength(newPayloadLength)
- if err := e.linkEP.WritePacket(r, gso, hdr, newPayload, ProtocolNumber); err != nil {
- return err
- }
- r.Stats().IP.PacketsSent.Increment()
- payload.TrimFront(newPayloadLength)
- } else {
- // The fragment is too small to fit all of hdr.
- startOfHdr := hdr
- startOfHdr.TrimBack(hdr.UsedLength() - outerMTU)
- emptyVV := buffer.NewVectorisedView(0, []buffer.View{})
- if err := e.linkEP.WritePacket(r, gso, startOfHdr, emptyVV, ProtocolNumber); err != nil {
- return err
- }
- r.Stats().IP.PacketsSent.Increment()
- // Add the unused bytes of hdr into the payload that remains to be sent.
- restOfHdr := hdr.View()[outerMTU:]
- tmp := buffer.NewVectorisedView(len(restOfHdr), []buffer.View{buffer.NewViewFromBytes(restOfHdr)})
- tmp.Append(payload)
- payload = tmp
+ offset += copied
+
+ // Send out the fragment.
+ if err := e.linkEP.WritePacket(r, gso, ProtocolNumber, fragPkt); err != nil {
+ return err
}
+ r.Stats().IP.PacketsSent.Increment()
}
return nil
}
-func (e *endpoint) addIPHeader(r *stack.Route, hdr *buffer.Prependable, payloadSize int, params stack.NetworkHeaderParams) {
- ip := header.IPv4(hdr.Prepend(header.IPv4MinimumSize))
- length := uint16(hdr.UsedLength() + payloadSize)
- id := uint32(0)
- if length > header.IPv4MaximumHeaderSize+8 {
- // Packets of 68 bytes or less are required by RFC 791 to not be
- // fragmented, so we only assign ids to larger packets.
- id = atomic.AddUint32(&e.protocol.ids[hashRoute(r, params.Protocol, e.protocol.hashIV)%buckets], 1)
- }
+func (e *endpoint) addIPHeader(r *stack.Route, pkt *stack.PacketBuffer, params stack.NetworkHeaderParams) {
+ ip := header.IPv4(pkt.NetworkHeader().Push(header.IPv4MinimumSize))
+ length := uint16(pkt.Size())
+ // RFC 6864 section 4.3 mandates uniqueness of ID values for non-atomic
+ // datagrams. Since the DF bit is never being set here, all datagrams
+ // are non-atomic and need an ID.
+ id := atomic.AddUint32(&e.protocol.ids[hashRoute(r, params.Protocol, e.protocol.hashIV)%buckets], 1)
ip.Encode(&header.IPv4Fields{
IHL: header.IPv4MinimumSize,
TotalLength: length,
@@ -218,28 +222,49 @@ func (e *endpoint) addIPHeader(r *stack.Route, hdr *buffer.Prependable, payloadS
DstAddr: r.RemoteAddress,
})
ip.SetChecksum(^ip.CalculateChecksum())
+ pkt.NetworkProtocolNumber = header.IPv4ProtocolNumber
}
// WritePacket writes a packet to the given destination address and protocol.
-func (e *endpoint) WritePacket(r *stack.Route, gso *stack.GSO, hdr buffer.Prependable, payload buffer.VectorisedView, params stack.NetworkHeaderParams, loop stack.PacketLooping) *tcpip.Error {
- e.addIPHeader(r, &hdr, payload.Size(), params)
-
- if loop&stack.PacketLoop != 0 {
- views := make([]buffer.View, 1, 1+len(payload.Views()))
- views[0] = hdr.View()
- views = append(views, payload.Views()...)
- vv := buffer.NewVectorisedView(len(views[0])+payload.Size(), views)
+func (e *endpoint) WritePacket(r *stack.Route, gso *stack.GSO, params stack.NetworkHeaderParams, pkt *stack.PacketBuffer) *tcpip.Error {
+ e.addIPHeader(r, pkt, params)
+
+ // iptables filtering. All packets that reach here are locally
+ // generated.
+ nicName := e.stack.FindNICNameFromID(e.NICID())
+ ipt := e.stack.IPTables()
+ if ok := ipt.Check(stack.Output, pkt, gso, r, "", nicName); !ok {
+ // iptables is telling us to drop the packet.
+ return nil
+ }
+
+ // If the packet is manipulated as per NAT Ouput rules, handle packet
+ // based on destination address and do not send the packet to link layer.
+ // TODO(gvisor.dev/issue/170): We should do this for every packet, rather than
+ // only NATted packets, but removing this check short circuits broadcasts
+ // before they are sent out to other hosts.
+ if pkt.NatDone {
+ netHeader := header.IPv4(pkt.NetworkHeader().View())
+ ep, err := e.stack.FindNetworkEndpoint(header.IPv4ProtocolNumber, netHeader.DestinationAddress())
+ if err == nil {
+ route := r.ReverseRoute(netHeader.SourceAddress(), netHeader.DestinationAddress())
+ ep.HandlePacket(&route, pkt)
+ return nil
+ }
+ }
+
+ if r.Loop&stack.PacketLoop != 0 {
loopedR := r.MakeLoopedRoute()
- e.HandlePacket(&loopedR, vv)
+ e.HandlePacket(&loopedR, pkt)
loopedR.Release()
}
- if loop&stack.PacketOut == 0 {
+ if r.Loop&stack.PacketOut == 0 {
return nil
}
- if hdr.UsedLength()+payload.Size() > int(e.linkEP.MTU()) && (gso == nil || gso.Type == stack.GSONone) {
- return e.writePacketFragments(r, gso, hdr, payload, int(e.linkEP.MTU()))
+ if pkt.Size() > int(e.linkEP.MTU()) && (gso == nil || gso.Type == stack.GSONone) {
+ return e.writePacketFragments(r, gso, int(e.linkEP.MTU()), pkt)
}
- if err := e.linkEP.WritePacket(r, gso, hdr, payload, ProtocolNumber); err != nil {
+ if err := e.linkEP.WritePacket(r, gso, ProtocolNumber, pkt); err != nil {
return err
}
r.Stats().IP.PacketsSent.Increment()
@@ -247,34 +272,76 @@ func (e *endpoint) WritePacket(r *stack.Route, gso *stack.GSO, hdr buffer.Prepen
}
// WritePackets implements stack.NetworkEndpoint.WritePackets.
-func (e *endpoint) WritePackets(r *stack.Route, gso *stack.GSO, hdrs []stack.PacketDescriptor, payload buffer.VectorisedView, params stack.NetworkHeaderParams, loop stack.PacketLooping) (int, *tcpip.Error) {
- if loop&stack.PacketLoop != 0 {
+func (e *endpoint) WritePackets(r *stack.Route, gso *stack.GSO, pkts stack.PacketBufferList, params stack.NetworkHeaderParams) (int, *tcpip.Error) {
+ if r.Loop&stack.PacketLoop != 0 {
panic("multiple packets in local loop")
}
- if loop&stack.PacketOut == 0 {
- return len(hdrs), nil
+ if r.Loop&stack.PacketOut == 0 {
+ return pkts.Len(), nil
+ }
+
+ for pkt := pkts.Front(); pkt != nil; {
+ e.addIPHeader(r, pkt, params)
+ pkt = pkt.Next()
+ }
+
+ nicName := e.stack.FindNICNameFromID(e.NICID())
+ // iptables filtering. All packets that reach here are locally
+ // generated.
+ ipt := e.stack.IPTables()
+ dropped, natPkts := ipt.CheckPackets(stack.Output, pkts, gso, r, nicName)
+ if len(dropped) == 0 && len(natPkts) == 0 {
+ // Fast path: If no packets are to be dropped then we can just invoke the
+ // faster WritePackets API directly.
+ n, err := e.linkEP.WritePackets(r, gso, pkts, ProtocolNumber)
+ r.Stats().IP.PacketsSent.IncrementBy(uint64(n))
+ return n, err
}
- for i := range hdrs {
- e.addIPHeader(r, &hdrs[i].Hdr, hdrs[i].Size, params)
+ // Slow Path as we are dropping some packets in the batch degrade to
+ // emitting one packet at a time.
+ n := 0
+ for pkt := pkts.Front(); pkt != nil; pkt = pkt.Next() {
+ if _, ok := dropped[pkt]; ok {
+ continue
+ }
+ if _, ok := natPkts[pkt]; ok {
+ netHeader := header.IPv4(pkt.NetworkHeader().View())
+ if ep, err := e.stack.FindNetworkEndpoint(header.IPv4ProtocolNumber, netHeader.DestinationAddress()); err == nil {
+ src := netHeader.SourceAddress()
+ dst := netHeader.DestinationAddress()
+ route := r.ReverseRoute(src, dst)
+ ep.HandlePacket(&route, pkt)
+ n++
+ continue
+ }
+ }
+ if err := e.linkEP.WritePacket(r, gso, ProtocolNumber, pkt); err != nil {
+ r.Stats().IP.PacketsSent.IncrementBy(uint64(n))
+ return n, err
+ }
+ n++
}
- n, err := e.linkEP.WritePackets(r, gso, hdrs, payload, ProtocolNumber)
r.Stats().IP.PacketsSent.IncrementBy(uint64(n))
- return n, err
+ return n, nil
}
// WriteHeaderIncludedPacket writes a packet already containing a network
// header through the given route.
-func (e *endpoint) WriteHeaderIncludedPacket(r *stack.Route, payload buffer.VectorisedView, loop stack.PacketLooping) *tcpip.Error {
+func (e *endpoint) WriteHeaderIncludedPacket(r *stack.Route, pkt *stack.PacketBuffer) *tcpip.Error {
// The packet already has an IP header, but there are a few required
// checks.
- ip := header.IPv4(payload.First())
- if !ip.IsValid(payload.Size()) {
+ h, ok := pkt.Data.PullUp(header.IPv4MinimumSize)
+ if !ok {
+ return tcpip.ErrInvalidOptionValue
+ }
+ ip := header.IPv4(h)
+ if !ip.IsValid(pkt.Data.Size()) {
return tcpip.ErrInvalidOptionValue
}
// Always set the total length.
- ip.SetTotalLength(uint16(payload.Size()))
+ ip.SetTotalLength(uint16(pkt.Data.Size()))
// Set the source address when zero.
if ip.SourceAddress() == tcpip.Address(([]byte{0, 0, 0, 0})) {
@@ -287,51 +354,49 @@ func (e *endpoint) WriteHeaderIncludedPacket(r *stack.Route, payload buffer.Vect
// Set the packet ID when zero.
if ip.ID() == 0 {
- id := uint32(0)
- if payload.Size() > header.IPv4MaximumHeaderSize+8 {
- // Packets of 68 bytes or less are required by RFC 791 to not be
- // fragmented, so we only assign ids to larger packets.
- id = atomic.AddUint32(&e.protocol.ids[hashRoute(r, 0 /* protocol */, e.protocol.hashIV)%buckets], 1)
+ // RFC 6864 section 4.3 mandates uniqueness of ID values for
+ // non-atomic datagrams, so assign an ID to all such datagrams
+ // according to the definition given in RFC 6864 section 4.
+ if ip.Flags()&header.IPv4FlagDontFragment == 0 || ip.Flags()&header.IPv4FlagMoreFragments != 0 || ip.FragmentOffset() > 0 {
+ ip.SetID(uint16(atomic.AddUint32(&e.protocol.ids[hashRoute(r, 0 /* protocol */, e.protocol.hashIV)%buckets], 1)))
}
- ip.SetID(uint16(id))
}
// Always set the checksum.
ip.SetChecksum(0)
ip.SetChecksum(^ip.CalculateChecksum())
- if loop&stack.PacketLoop != 0 {
- e.HandlePacket(r, payload)
+ if r.Loop&stack.PacketLoop != 0 {
+ e.HandlePacket(r, pkt.Clone())
}
- if loop&stack.PacketOut == 0 {
+ if r.Loop&stack.PacketOut == 0 {
return nil
}
- // If we want to send the packet to a link-layer,
- // we have to reserve space for an Ethernet header.
- hdr := buffer.NewPrependableFromView(payload.ToView(), int(e.linkEP.MaxHeaderLength()))
r.Stats().IP.PacketsSent.Increment()
- return e.linkEP.WritePacket(r, nil /* gso */, hdr, buffer.VectorisedView{}, ProtocolNumber)
+
+ return e.linkEP.WritePacket(r, nil /* gso */, ProtocolNumber, pkt)
}
// HandlePacket is called by the link layer when new ipv4 packets arrive for
// this endpoint.
-func (e *endpoint) HandlePacket(r *stack.Route, vv buffer.VectorisedView) {
- headerView := vv.First()
- h := header.IPv4(headerView)
- if !h.IsValid(vv.Size()) {
+func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) {
+ h := header.IPv4(pkt.NetworkHeader().View())
+ if !h.IsValid(pkt.Data.Size() + pkt.NetworkHeader().View().Size() + pkt.TransportHeader().View().Size()) {
r.Stats().IP.MalformedPacketsReceived.Increment()
return
}
- hlen := int(h.HeaderLength())
- tlen := int(h.TotalLength())
- vv.TrimFront(hlen)
- vv.CapLength(tlen - hlen)
+ // iptables filtering. All packets that reach here are intended for
+ // this machine and will not be forwarded.
+ ipt := e.stack.IPTables()
+ if ok := ipt.Check(stack.Input, pkt, nil, nil, "", ""); !ok {
+ // iptables is telling us to drop the packet.
+ return
+ }
- more := (h.Flags() & header.IPv4FlagMoreFragments) != 0
- if more || h.FragmentOffset() != 0 {
- if vv.Size() == 0 {
+ if h.More() || h.FragmentOffset() != 0 {
+ if pkt.Data.Size()+pkt.TransportHeader().View().Size() == 0 {
// Drop the packet as it's marked as a fragment but has
// no payload.
r.Stats().IP.MalformedPacketsReceived.Increment()
@@ -339,10 +404,10 @@ func (e *endpoint) HandlePacket(r *stack.Route, vv buffer.VectorisedView) {
return
}
// The packet is a fragment, let's try to reassemble it.
- last := h.FragmentOffset() + uint16(vv.Size()) - 1
+ last := h.FragmentOffset() + uint16(pkt.Data.Size()) - 1
// Drop the packet if the fragmentOffset is incorrect. i.e the
- // combination of fragmentOffset and vv.size() causes a wrap
- // around resulting in last being less than the offset.
+ // combination of fragmentOffset and pkt.Data.size() causes a
+ // wrap around resulting in last being less than the offset.
if last < h.FragmentOffset() {
r.Stats().IP.MalformedPacketsReceived.Increment()
r.Stats().IP.MalformedFragmentsReceived.Increment()
@@ -350,7 +415,20 @@ func (e *endpoint) HandlePacket(r *stack.Route, vv buffer.VectorisedView) {
}
var ready bool
var err error
- vv, ready, err = e.fragmentation.Process(hash.IPv4FragmentHash(h), h.FragmentOffset(), last, more, vv)
+ pkt.Data, ready, err = e.protocol.fragmentation.Process(
+ // As per RFC 791 section 2.3, the identification value is unique
+ // for a source-destination pair and protocol.
+ fragmentation.FragmentID{
+ Source: h.SourceAddress(),
+ Destination: h.DestinationAddress(),
+ ID: uint32(h.ID()),
+ Protocol: h.Protocol(),
+ },
+ h.FragmentOffset(),
+ last,
+ h.More(),
+ pkt.Data,
+ )
if err != nil {
r.Stats().IP.MalformedPacketsReceived.Increment()
r.Stats().IP.MalformedFragmentsReceived.Increment()
@@ -362,12 +440,11 @@ func (e *endpoint) HandlePacket(r *stack.Route, vv buffer.VectorisedView) {
}
p := h.TransportProtocol()
if p == header.ICMPv4ProtocolNumber {
- headerView.CapLength(hlen)
- e.handleICMP(r, headerView, vv)
+ e.handleICMP(r, pkt)
return
}
r.Stats().IP.PacketsDelivered.Increment()
- e.dispatcher.DeliverTransportPacket(r, p, headerView, vv)
+ e.dispatcher.DeliverTransportPacket(r, p, pkt)
}
// Close cleans up resources associated with the endpoint.
@@ -381,6 +458,8 @@ type protocol struct {
// uint8 portion of it is meaningful and it must be accessed
// atomically.
defaultTTL uint32
+
+ fragmentation *fragmentation.Fragmentation
}
// Number returns the ipv4 protocol number.
@@ -436,6 +515,45 @@ func (p *protocol) DefaultTTL() uint8 {
return uint8(atomic.LoadUint32(&p.defaultTTL))
}
+// Close implements stack.TransportProtocol.Close.
+func (*protocol) Close() {}
+
+// Wait implements stack.TransportProtocol.Wait.
+func (*protocol) Wait() {}
+
+// Parse implements stack.TransportProtocol.Parse.
+func (*protocol) Parse(pkt *stack.PacketBuffer) (proto tcpip.TransportProtocolNumber, hasTransportHdr bool, ok bool) {
+ hdr, ok := pkt.Data.PullUp(header.IPv4MinimumSize)
+ if !ok {
+ return 0, false, false
+ }
+ ipHdr := header.IPv4(hdr)
+
+ // Header may have options, determine the true header length.
+ headerLen := int(ipHdr.HeaderLength())
+ if headerLen < header.IPv4MinimumSize {
+ // TODO(gvisor.dev/issue/2404): Per RFC 791, IHL needs to be at least 5 in
+ // order for the packet to be valid. Figure out if we want to reject this
+ // case.
+ headerLen = header.IPv4MinimumSize
+ }
+ hdr, ok = pkt.NetworkHeader().Consume(headerLen)
+ if !ok {
+ return 0, false, false
+ }
+ ipHdr = header.IPv4(hdr)
+
+ // If this is a fragment, don't bother parsing the transport header.
+ parseTransportHeader := true
+ if ipHdr.More() || ipHdr.FragmentOffset() != 0 {
+ parseTransportHeader = false
+ }
+
+ pkt.NetworkProtocolNumber = header.IPv4ProtocolNumber
+ pkt.Data.CapLength(int(ipHdr.TotalLength()) - len(hdr))
+ return ipHdr.TransportProtocol(), parseTransportHeader, true
+}
+
// calculateMTU calculates the network-layer payload MTU based on the link-layer
// payload mtu.
func calculateMTU(mtu uint32) uint32 {
@@ -467,5 +585,10 @@ func NewProtocol() stack.NetworkProtocol {
}
hashIV := r[buckets]
- return &protocol{ids: ids, hashIV: hashIV, defaultTTL: DefaultTTL}
+ return &protocol{
+ ids: ids,
+ hashIV: hashIV,
+ defaultTTL: DefaultTTL,
+ fragmentation: fragmentation.NewFragmentation(fragmentblockSize, fragmentation.HighFragThreshold, fragmentation.LowFragThreshold, fragmentation.DefaultReassembleTimeout),
+ }
}