summaryrefslogtreecommitdiffhomepage
path: root/pkg
diff options
context:
space:
mode:
authorgVisor bot <gvisor-bot@google.com>2020-03-26 21:08:58 +0000
committergVisor bot <gvisor-bot@google.com>2020-03-26 21:08:58 +0000
commit98c58a207c4b3dfc9dd9b0d0ccab4e85fe82be75 (patch)
tree290b0559d6e42abb33d944e407b59996cc940e74 /pkg
parentabd55c233ce4ceca722a6b5dcf52727db48987b0 (diff)
parentfbe80460a7eb34147b928fa1023b28a3c094c070 (diff)
Merge release-20200219.0-247-gfbe8046 (automated)
Diffstat (limited to 'pkg')
-rw-r--r--pkg/tcpip/buffer/view.go20
-rwxr-xr-xpkg/tcpip/header/ipv6_extension_headers.go344
-rw-r--r--pkg/tcpip/network/hash/hash.go4
-rw-r--r--pkg/tcpip/network/ipv6/ipv6.go122
4 files changed, 483 insertions, 7 deletions
diff --git a/pkg/tcpip/buffer/view.go b/pkg/tcpip/buffer/view.go
index 17e94c562..8d42cd066 100644
--- a/pkg/tcpip/buffer/view.go
+++ b/pkg/tcpip/buffer/view.go
@@ -15,6 +15,10 @@
// Package buffer provides the implementation of a buffer view.
package buffer
+import (
+ "bytes"
+)
+
// View is a slice of a buffer, with convenience methods.
type View []byte
@@ -45,6 +49,13 @@ func (v *View) CapLength(length int) {
*v = (*v)[:length:length]
}
+// Reader returns a bytes.Reader for v.
+func (v *View) Reader() bytes.Reader {
+ var r bytes.Reader
+ r.Reset(*v)
+ return r
+}
+
// ToVectorisedView returns a VectorisedView containing the receiver.
func (v View) ToVectorisedView() VectorisedView {
return NewVectorisedView(len(v), []View{v})
@@ -162,3 +173,12 @@ func (vv *VectorisedView) AppendView(v View) {
vv.views = append(vv.views, v)
vv.size += len(v)
}
+
+// Readers returns a bytes.Reader for each of vv's views.
+func (vv *VectorisedView) Readers() []bytes.Reader {
+ readers := make([]bytes.Reader, 0, len(vv.views))
+ for _, v := range vv.views {
+ readers = append(readers, v.Reader())
+ }
+ return readers
+}
diff --git a/pkg/tcpip/header/ipv6_extension_headers.go b/pkg/tcpip/header/ipv6_extension_headers.go
new file mode 100755
index 000000000..b8866d4d2
--- /dev/null
+++ b/pkg/tcpip/header/ipv6_extension_headers.go
@@ -0,0 +1,344 @@
+// Copyright 2020 The gVisor Authors.
+//
+// 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 header
+
+import (
+ "bufio"
+ "encoding/binary"
+ "fmt"
+ "io"
+
+ "gvisor.dev/gvisor/pkg/tcpip/buffer"
+)
+
+// IPv6ExtensionHeaderIdentifier is an IPv6 extension header identifier.
+type IPv6ExtensionHeaderIdentifier uint8
+
+const (
+ // IPv6RoutingExtHdrIdentifier is the header identifier of a Routing extension
+ // header, as per RFC 8200 section 4.4.
+ IPv6RoutingExtHdrIdentifier IPv6ExtensionHeaderIdentifier = 43
+
+ // IPv6FragmentExtHdrIdentifier is the header identifier of a Fragment
+ // extension header, as per RFC 8200 section 4.5.
+ IPv6FragmentExtHdrIdentifier IPv6ExtensionHeaderIdentifier = 44
+
+ // IPv6NoNextHeaderIdentifier is the header identifier used to signify the end
+ // of an IPv6 payload, as per RFC 8200 section 4.7.
+ IPv6NoNextHeaderIdentifier IPv6ExtensionHeaderIdentifier = 59
+)
+
+const (
+ // ipv6RoutingExtHdrSegmentsLeftIdx is the index to the Segments Left field
+ // within an IPv6RoutingExtHdr.
+ ipv6RoutingExtHdrSegmentsLeftIdx = 1
+
+ // ipv6FragmentExtHdrFragmentOffsetOffset is the offset to the start of the
+ // Fragment Offset field within an IPv6FragmentExtHdr.
+ ipv6FragmentExtHdrFragmentOffsetOffset = 0
+
+ // ipv6FragmentExtHdrFragmentOffsetShift is the least significant bits to
+ // discard from the Fragment Offset.
+ ipv6FragmentExtHdrFragmentOffsetShift = 3
+
+ // ipv6FragmentExtHdrFlagsIdx is the index to the flags field within an
+ // IPv6FragmentExtHdr.
+ ipv6FragmentExtHdrFlagsIdx = 1
+
+ // ipv6FragmentExtHdrMFlagMask is the mask of the More (M) flag within the
+ // flags field of an IPv6FragmentExtHdr.
+ ipv6FragmentExtHdrMFlagMask = 1
+
+ // ipv6FragmentExtHdrIdentificationOffset is the offset to the Identification
+ // field within an IPv6FragmentExtHdr.
+ ipv6FragmentExtHdrIdentificationOffset = 2
+
+ // ipv6ExtHdrLenBytesPerUnit is the unit size of an extension header's length
+ // field. That is, given a Length field of 2, the extension header expects
+ // 16 bytes following the first 8 bytes (see ipv6ExtHdrLenBytesExcluded for
+ // details about the first 8 bytes' exclusion from the Length field).
+ ipv6ExtHdrLenBytesPerUnit = 8
+
+ // ipv6ExtHdrLenBytesExcluded is the number of bytes excluded from an
+ // extension header's Length field following the Length field.
+ //
+ // The Length field excludes the first 8 bytes, but the Next Header and Length
+ // field take up the first 2 of the 8 bytes so we expect (at minimum) 6 bytes
+ // after the Length field.
+ //
+ // This ensures that every extension header is at least 8 bytes.
+ ipv6ExtHdrLenBytesExcluded = 6
+
+ // IPv6FragmentExtHdrFragmentOffsetBytesPerUnit is the unit size of a Fragment
+ // extension header's Fragment Offset field. That is, given a Fragment Offset
+ // of 2, the extension header is indiciating that the fragment's payload
+ // starts at the 16th byte in the reassembled packet.
+ IPv6FragmentExtHdrFragmentOffsetBytesPerUnit = 8
+)
+
+// IPv6PayloadHeader is implemented by the various headers that can be found
+// in an IPv6 payload.
+//
+// These headers include IPv6 extension headers or upper layer data.
+type IPv6PayloadHeader interface {
+ isIPv6PayloadHeader()
+}
+
+// IPv6RawPayloadHeader the remainder of an IPv6 payload after an iterator
+// encounters a Next Header field it does not recognize as an IPv6 extension
+// header.
+type IPv6RawPayloadHeader struct {
+ Identifier IPv6ExtensionHeaderIdentifier
+ Buf buffer.VectorisedView
+}
+
+// isIPv6PayloadHeader implements IPv6PayloadHeader.isIPv6PayloadHeader.
+func (IPv6RawPayloadHeader) isIPv6PayloadHeader() {}
+
+// IPv6RoutingExtHdr is a buffer holding the Routing extension header specific
+// data as outlined in RFC 8200 section 4.4.
+type IPv6RoutingExtHdr []byte
+
+// isIPv6PayloadHeader implements IPv6PayloadHeader.isIPv6PayloadHeader.
+func (IPv6RoutingExtHdr) isIPv6PayloadHeader() {}
+
+// SegmentsLeft returns the Segments Left field.
+func (b IPv6RoutingExtHdr) SegmentsLeft() uint8 {
+ return b[ipv6RoutingExtHdrSegmentsLeftIdx]
+}
+
+// IPv6FragmentExtHdr is a buffer holding the Fragment extension header specific
+// data as outlined in RFC 8200 section 4.5.
+//
+// Note, the buffer does not include the Next Header and Reserved fields.
+type IPv6FragmentExtHdr [6]byte
+
+// isIPv6PayloadHeader implements IPv6PayloadHeader.isIPv6PayloadHeader.
+func (IPv6FragmentExtHdr) isIPv6PayloadHeader() {}
+
+// FragmentOffset returns the Fragment Offset field.
+//
+// This value indicates where the buffer following the Fragment extension header
+// starts in the target (reassembled) packet.
+func (b IPv6FragmentExtHdr) FragmentOffset() uint16 {
+ return binary.BigEndian.Uint16(b[ipv6FragmentExtHdrFragmentOffsetOffset:]) >> ipv6FragmentExtHdrFragmentOffsetShift
+}
+
+// More returns the More (M) flag.
+//
+// This indicates whether any fragments are expected to succeed b.
+func (b IPv6FragmentExtHdr) More() bool {
+ return b[ipv6FragmentExtHdrFlagsIdx]&ipv6FragmentExtHdrMFlagMask != 0
+}
+
+// ID returns the Identification field.
+//
+// This value is used to uniquely identify the packet, between a
+// souce and destination.
+func (b IPv6FragmentExtHdr) ID() uint32 {
+ return binary.BigEndian.Uint32(b[ipv6FragmentExtHdrIdentificationOffset:])
+}
+
+// IPv6PayloadIterator is an iterator over the contents of an IPv6 payload.
+//
+// The IPv6 payload may contain IPv6 extension headers before any upper layer
+// data.
+//
+// Note, between when an IPv6PayloadIterator is obtained and last used, no
+// changes to the payload may happen. Doing so may cause undefined and
+// unexpected behaviour. It is fine to obtain an IPv6PayloadIterator, iterate
+// over the first few headers then modify the backing payload so long as the
+// IPv6PayloadIterator obtained before modification is no longer used.
+type IPv6PayloadIterator struct {
+ // The identifier of the next header to parse.
+ nextHdrIdentifier IPv6ExtensionHeaderIdentifier
+
+ // reader is an io.Reader over payload.
+ reader bufio.Reader
+ payload buffer.VectorisedView
+
+ // Indicates to the iterator that it should return the remaining payload as a
+ // raw payload on the next call to Next.
+ forceRaw bool
+}
+
+// MakeIPv6PayloadIterator returns an iterator over the IPv6 payload containing
+// extension headers, or a raw payload if the payload cannot be parsed.
+func MakeIPv6PayloadIterator(nextHdrIdentifier IPv6ExtensionHeaderIdentifier, payload buffer.VectorisedView, check bool) (IPv6PayloadIterator, error) {
+ readers := payload.Readers()
+ readerPs := make([]io.Reader, 0, len(readers))
+ for i := range readers {
+ readerPs = append(readerPs, &readers[i])
+ }
+
+ // We need a buffer of size 1 for calls to bufio.Reader.ReadByte.
+ reader := *bufio.NewReaderSize(io.MultiReader(readerPs...), 1)
+
+ it := IPv6PayloadIterator{
+ nextHdrIdentifier: nextHdrIdentifier,
+ payload: payload.Clone(nil),
+ reader: reader,
+ }
+
+ var err error
+
+ if check {
+ for {
+ var done bool
+ if _, done, err = it.Next(); err != nil || done {
+ break
+ }
+ }
+
+ // Reset it (and its underlying readers) before returning it.
+ for i := range readers {
+ readers[i].Seek(0, io.SeekStart)
+ }
+ reader.Reset(io.MultiReader(readerPs...))
+ it = IPv6PayloadIterator{
+ nextHdrIdentifier: nextHdrIdentifier,
+ payload: payload.Clone(nil),
+ reader: reader,
+ }
+ }
+
+ return it, err
+}
+
+// AsRawHeader returns the remaining payload of i as a raw header and
+// completes the iterator.
+//
+// Calls to Next after calling AsRawHeader on i will indicate that the
+// iterator is done.
+func (i *IPv6PayloadIterator) AsRawHeader() IPv6RawPayloadHeader {
+ buf := i.payload
+ identifier := i.nextHdrIdentifier
+
+ // Mark i as done.
+ *i = IPv6PayloadIterator{
+ nextHdrIdentifier: IPv6NoNextHeaderIdentifier,
+ }
+
+ return IPv6RawPayloadHeader{Identifier: identifier, Buf: buf}
+}
+
+// Next returns the next item in the payload.
+//
+// If the next item is not a known IPv6 extension header, IPv6RawPayloadHeader
+// will be returned with the remaining bytes and next header identifier.
+//
+// The return is of the format (header, done, error). done will be true when
+// Next is unable to return anything because the iterator has reached the end of
+// the payload, or an error occured.
+func (i *IPv6PayloadIterator) Next() (IPv6PayloadHeader, bool, error) {
+ // We could be forced to return i as a raw header when the previous header was
+ // a fragment extension header as the data following the fragment extension
+ // header may not be complete.
+ if i.forceRaw {
+ return i.AsRawHeader(), false, nil
+ }
+
+ // Is the header we are parsing a known extension header?
+ switch i.nextHdrIdentifier {
+ case IPv6RoutingExtHdrIdentifier:
+ nextHdrIdentifier, bytes, err := i.nextHeaderData(false /* fragmentHdr */, nil)
+ if err != nil {
+ return nil, true, err
+ }
+
+ i.nextHdrIdentifier = nextHdrIdentifier
+ return IPv6RoutingExtHdr(bytes), false, nil
+ case IPv6FragmentExtHdrIdentifier:
+ var data [6]byte
+ // We ignore the returned bytes becauase we know the fragment extension
+ // header specific data will fit in data.
+ nextHdrIdentifier, _, err := i.nextHeaderData(true /* fragmentHdr */, data[:])
+ if err != nil {
+ return nil, true, err
+ }
+
+ fragmentExtHdr := IPv6FragmentExtHdr(data)
+
+ // If the packet is a fragmented packet, do not attempt to parse
+ // anything after the fragment extension header as the data following
+ // the extension header may not be complete.
+ if fragmentExtHdr.More() || fragmentExtHdr.FragmentOffset() != 0 {
+ i.forceRaw = true
+ }
+
+ i.nextHdrIdentifier = nextHdrIdentifier
+ return fragmentExtHdr, false, nil
+ case IPv6NoNextHeaderIdentifier:
+ // This indicates the end of the IPv6 payload.
+ return nil, true, nil
+
+ default:
+ // The header we are parsing is not a known extension header. Return the
+ // raw payload.
+ return i.AsRawHeader(), false, nil
+ }
+}
+
+// nextHeaderData returns the extension header's Next Header field and raw data.
+//
+// fragmentHdr indicates that the extension header being parsed is the Fragment
+// extension header so the Length field should be ignored as it is Reserved
+// for the Fragment extension header.
+//
+// If bytes is not nil, extension header specific data will be read into bytes
+// if it has enough capacity. If bytes is provided but does not have enough
+// capacity for the data, nextHeaderData will panic.
+func (i *IPv6PayloadIterator) nextHeaderData(fragmentHdr bool, bytes []byte) (IPv6ExtensionHeaderIdentifier, []byte, error) {
+ // We ignore the number of bytes read because we know we will only ever read
+ // at max 1 bytes since rune has a length of 1. If we read 0 bytes, the Read
+ // would return io.EOF to indicate that io.Reader has reached the end of the
+ // payload.
+ nextHdrIdentifier, err := i.reader.ReadByte()
+ i.payload.TrimFront(1)
+ if err != nil {
+ return 0, nil, fmt.Errorf("error when reading the Next Header field for extension header with id = %d: %w", i.nextHdrIdentifier, err)
+ }
+
+ var length uint8
+ length, err = i.reader.ReadByte()
+ i.payload.TrimFront(1)
+ if err != nil {
+ var ret error
+ if fragmentHdr {
+ ret = fmt.Errorf("error when reading the Length field for extension header with id = %d: %w", i.nextHdrIdentifier, err)
+ } else {
+ ret = fmt.Errorf("error when reading the Reserved field for extension header with id = %d: %w", i.nextHdrIdentifier, err)
+ }
+ return 0, nil, ret
+ }
+ if fragmentHdr {
+ length = 0
+ }
+
+ bytesLen := int(length)*ipv6ExtHdrLenBytesPerUnit + ipv6ExtHdrLenBytesExcluded
+ if bytes == nil {
+ bytes = make([]byte, bytesLen)
+ } else if n := len(bytes); n < bytesLen {
+ panic(fmt.Sprintf("bytes only has space for %d bytes but need space for %d bytes (length = %d) for extension header with id = %d", n, bytesLen, length, i.nextHdrIdentifier))
+ }
+
+ n, err := io.ReadFull(&i.reader, bytes)
+ i.payload.TrimFront(n)
+ if err != nil {
+ return 0, nil, fmt.Errorf("read %d out of %d extension header data bytes (length = %d) for header with id = %d: %w", n, bytesLen, length, i.nextHdrIdentifier, err)
+ }
+
+ return IPv6ExtensionHeaderIdentifier(nextHdrIdentifier), bytes, nil
+}
diff --git a/pkg/tcpip/network/hash/hash.go b/pkg/tcpip/network/hash/hash.go
index 6a215938b..8f65713c5 100644
--- a/pkg/tcpip/network/hash/hash.go
+++ b/pkg/tcpip/network/hash/hash.go
@@ -80,12 +80,12 @@ func IPv4FragmentHash(h header.IPv4) uint32 {
// RFC 2640 (sec 4.5) is not very sharp on this aspect.
// As a reference, also Linux ignores the protocol to compute
// the hash (inet6_hash_frag).
-func IPv6FragmentHash(h header.IPv6, f header.IPv6Fragment) uint32 {
+func IPv6FragmentHash(h header.IPv6, id uint32) uint32 {
t := h.SourceAddress()
y := uint32(t[0]) | uint32(t[1])<<8 | uint32(t[2])<<16 | uint32(t[3])<<24
t = h.DestinationAddress()
z := uint32(t[0]) | uint32(t[1])<<8 | uint32(t[2])<<16 | uint32(t[3])<<24
- return Hash3Words(f.ID(), y, z, hashIV)
+ return Hash3Words(id, y, z, hashIV)
}
func rol32(v, shift uint32) uint32 {
diff --git a/pkg/tcpip/network/ipv6/ipv6.go b/pkg/tcpip/network/ipv6/ipv6.go
index 29e597002..a703a768c 100644
--- a/pkg/tcpip/network/ipv6/ipv6.go
+++ b/pkg/tcpip/network/ipv6/ipv6.go
@@ -21,11 +21,14 @@
package ipv6
import (
+ "fmt"
"sync/atomic"
"gvisor.dev/gvisor/pkg/tcpip"
"gvisor.dev/gvisor/pkg/tcpip/buffer"
"gvisor.dev/gvisor/pkg/tcpip/header"
+ "gvisor.dev/gvisor/pkg/tcpip/network/fragmentation"
+ "gvisor.dev/gvisor/pkg/tcpip/network/hash"
"gvisor.dev/gvisor/pkg/tcpip/stack"
)
@@ -49,6 +52,7 @@ type endpoint struct {
linkEP stack.LinkEndpoint
linkAddrCache stack.LinkAddressCache
dispatcher stack.TransportDispatcher
+ fragmentation *fragmentation.Fragmentation
protocol *protocol
}
@@ -172,6 +176,7 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt stack.PacketBuffer) {
headerView := pkt.Data.First()
h := header.IPv6(headerView)
if !h.IsValid(pkt.Data.Size()) {
+ r.Stats().IP.MalformedPacketsReceived.Increment()
return
}
@@ -179,14 +184,120 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt stack.PacketBuffer) {
pkt.Data.TrimFront(header.IPv6MinimumSize)
pkt.Data.CapLength(int(h.PayloadLength()))
- p := h.TransportProtocol()
- if p == header.ICMPv6ProtocolNumber {
- e.handleICMP(r, headerView, pkt)
+ it, err := header.MakeIPv6PayloadIterator(header.IPv6ExtensionHeaderIdentifier(h.NextHeader()), pkt.Data, true)
+ if err != nil {
+ r.Stats().IP.MalformedPacketsReceived.Increment()
return
}
- r.Stats().IP.PacketsDelivered.Increment()
- e.dispatcher.DeliverTransportPacket(r, p, pkt)
+ for {
+ extHdr, done, err := it.Next()
+ if err != nil {
+ // This should never happen as MakeIPv6PayloadIterator above did not
+ // return an error.
+ panic(fmt.Sprintf("unexpected error when iterating over IPv6 payload: %s", err))
+ }
+ if done {
+ break
+ }
+
+ switch extHdr := extHdr.(type) {
+ case header.IPv6RoutingExtHdr:
+ // As per RFC 8200 section 4.4, if a node encounters a routing header with
+ // an unrecognized routing type value, with a non-zero Segments Left
+ // value, the node must discard the packet and send an ICMP Parameter
+ // Problem, Code 0. If the Segments Left is 0, the node must ignore the
+ // Routing extension header and process the next header in the packet.
+ //
+ // Note, the stack does not yet handle any type of routing extension
+ // header, so we just make sure Segments Left is zero before processing
+ // the next extension header.
+ //
+ // TODO(b/152019344): Send an ICMPv6 Parameter Problem Code 0 for
+ // unrecognized routing types with a non-zero Segments Left value.
+ if extHdr.SegmentsLeft() != 0 {
+ return
+ }
+
+ case header.IPv6FragmentExtHdr:
+ fragmentOffset := extHdr.FragmentOffset()
+ more := extHdr.More()
+ if !more && fragmentOffset == 0 {
+ // This fragment extension header indicates that this packet is an
+ // atomic fragment. An atomic fragment is a fragment that contains
+ // all the data required to reassemble a full packet. As per RFC 6946,
+ // atomic fragments must not interfere with "normal" fragmented traffic
+ // so we skip processing the fragment instead of feeding it through the
+ // reassembly process below.
+ continue
+ }
+
+ rawPayload := it.AsRawHeader()
+ fragmentPayloadLen := rawPayload.Buf.Size()
+ if fragmentPayloadLen == 0 {
+ // Drop the packet as it's marked as a fragment but has no payload.
+ r.Stats().IP.MalformedPacketsReceived.Increment()
+ r.Stats().IP.MalformedFragmentsReceived.Increment()
+ return
+ }
+
+ // The packet is a fragment, let's try to reassemble it.
+ start := fragmentOffset * header.IPv6FragmentExtHdrFragmentOffsetBytesPerUnit
+ last := start + uint16(fragmentPayloadLen) - 1
+
+ // Drop the packet if the fragmentOffset is incorrect. i.e the
+ // combination of fragmentOffset and pkt.Data.size() causes a
+ // wrap around resulting in last being less than the offset.
+ if last < start {
+ r.Stats().IP.MalformedPacketsReceived.Increment()
+ r.Stats().IP.MalformedFragmentsReceived.Increment()
+ return
+ }
+
+ var ready bool
+ pkt.Data, ready, err = e.fragmentation.Process(hash.IPv6FragmentHash(h, extHdr.ID()), start, last, more, rawPayload.Buf)
+ if err != nil {
+ r.Stats().IP.MalformedPacketsReceived.Increment()
+ r.Stats().IP.MalformedFragmentsReceived.Increment()
+ return
+ }
+
+ if ready {
+ // We create a new iterator with the reassembled packet because we could
+ // have more extension headers in the reassembled payload, as per RFC
+ // 8200 section 4.5.
+ it, err = header.MakeIPv6PayloadIterator(rawPayload.Identifier, pkt.Data, true)
+ if err != nil {
+ r.Stats().IP.MalformedPacketsReceived.Increment()
+ r.Stats().IP.MalformedFragmentsReceived.Increment()
+ return
+ }
+ }
+
+ case header.IPv6RawPayloadHeader:
+ // If the last header in the payload isn't a known IPv6 extension header,
+ // handle it as if it is transport layer data.
+ pkt.Data = extHdr.Buf
+
+ if p := tcpip.TransportProtocolNumber(extHdr.Identifier); p == header.ICMPv6ProtocolNumber {
+ e.handleICMP(r, headerView, pkt)
+ } else {
+ r.Stats().IP.PacketsDelivered.Increment()
+ // TODO(b/152019344): Send an ICMPv6 Parameter Problem, Code 1 error
+ // in response to unrecognized next header values.
+ e.dispatcher.DeliverTransportPacket(r, p, pkt)
+ }
+
+ default:
+ // If we receive a packet for an extension header we do not yet handle,
+ // drop the packet for now.
+ //
+ // TODO(b/152019344): Send an ICMPv6 Parameter Problem, Code 1 error
+ // in response to unrecognized next header values.
+ r.Stats().UnknownProtocolRcvdPackets.Increment()
+ return
+ }
+ }
}
// Close cleans up resources associated with the endpoint.
@@ -229,6 +340,7 @@ func (p *protocol) NewEndpoint(nicID tcpip.NICID, addrWithPrefix tcpip.AddressWi
linkEP: linkEP,
linkAddrCache: linkAddrCache,
dispatcher: dispatcher,
+ fragmentation: fragmentation.NewFragmentation(fragmentation.HighFragThreshold, fragmentation.LowFragThreshold, fragmentation.DefaultReassembleTimeout),
protocol: p,
}, nil
}