summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip/network
diff options
context:
space:
mode:
authorBhasker Hariharan <bhaskerh@google.com>2018-09-04 14:30:15 -0700
committerShentubot <shentubot@google.com>2018-09-04 14:31:52 -0700
commit2cff07381a911ad52cf9df70d702f39217e9539e (patch)
tree6ae3c3dd38e3c14302e0d053faf2b0106d66c467 /pkg/tcpip/network
parent9ae4e28f75979905a6396962a232e217323499f9 (diff)
Automated rollback of changelist 211156845
PiperOrigin-RevId: 211525182 Change-Id: I462c20328955c77ecc7bfd8ee803ac91f15858e6
Diffstat (limited to 'pkg/tcpip/network')
-rw-r--r--pkg/tcpip/network/ipv4/ipv4.go8
-rw-r--r--pkg/tcpip/network/ipv6/BUILD19
-rw-r--r--pkg/tcpip/network/ipv6/icmp.go135
-rw-r--r--pkg/tcpip/network/ipv6/icmp_test.go236
-rw-r--r--pkg/tcpip/network/ipv6/ipv6.go34
5 files changed, 415 insertions, 17 deletions
diff --git a/pkg/tcpip/network/ipv4/ipv4.go b/pkg/tcpip/network/ipv4/ipv4.go
index e6b1f5128..143d8e73e 100644
--- a/pkg/tcpip/network/ipv4/ipv4.go
+++ b/pkg/tcpip/network/ipv4/ipv4.go
@@ -44,6 +44,10 @@ const (
// buckets is the number of identifier buckets.
buckets = 2048
+
+ // defaultIPv4TTL is the defautl TTL for IPv4 Packets egressed by
+ // Netstack.
+ defaultIPv4TTL = 255
)
type address [header.IPv4AddressSize]byte
@@ -67,7 +71,7 @@ func newEndpoint(nicid tcpip.NICID, addr tcpip.Address, dispatcher stack.Transpo
fragmentation: fragmentation.NewFragmentation(fragmentation.HighFragThreshold, fragmentation.LowFragThreshold, fragmentation.DefaultReassembleTimeout),
}
copy(e.address[:], addr)
- e.id = stack.NetworkEndpointID{tcpip.Address(e.address[:])}
+ e.id = stack.NetworkEndpointID{LocalAddress: tcpip.Address(e.address[:])}
go e.echoReplier()
@@ -115,7 +119,7 @@ func (e *endpoint) WritePacket(r *stack.Route, hdr *buffer.Prependable, payload
IHL: header.IPv4MinimumSize,
TotalLength: length,
ID: uint16(id),
- TTL: 65,
+ TTL: defaultIPv4TTL,
Protocol: uint8(protocol),
SrcAddr: tcpip.Address(e.address[:]),
DstAddr: r.RemoteAddress,
diff --git a/pkg/tcpip/network/ipv6/BUILD b/pkg/tcpip/network/ipv6/BUILD
index 1c3eccae0..2f19a659e 100644
--- a/pkg/tcpip/network/ipv6/BUILD
+++ b/pkg/tcpip/network/ipv6/BUILD
@@ -1,6 +1,6 @@
package(licenses = ["notice"]) # Apache 2.0
-load("//tools/go_stateify:defs.bzl", "go_library")
+load("//tools/go_stateify:defs.bzl", "go_library", "go_test")
go_library(
name = "ipv6",
@@ -19,3 +19,20 @@ go_library(
"//pkg/tcpip/stack",
],
)
+
+go_test(
+ name = "ipv6_test",
+ size = "small",
+ srcs = ["icmp_test.go"],
+ embed = [":ipv6"],
+ deps = [
+ "//pkg/tcpip",
+ "//pkg/tcpip/buffer",
+ "//pkg/tcpip/header",
+ "//pkg/tcpip/link/channel",
+ "//pkg/tcpip/link/sniffer",
+ "//pkg/tcpip/stack",
+ "//pkg/tcpip/transport/ping",
+ "//pkg/waiter",
+ ],
+)
diff --git a/pkg/tcpip/network/ipv6/icmp.go b/pkg/tcpip/network/ipv6/icmp.go
index 8b8539def..e3ef89d26 100644
--- a/pkg/tcpip/network/ipv6/icmp.go
+++ b/pkg/tcpip/network/ipv6/icmp.go
@@ -17,6 +17,7 @@ package ipv6
import (
"encoding/binary"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip"
"gvisor.googlesource.com/gvisor/pkg/tcpip/buffer"
"gvisor.googlesource.com/gvisor/pkg/tcpip/header"
"gvisor.googlesource.com/gvisor/pkg/tcpip/stack"
@@ -86,5 +87,139 @@ func (e *endpoint) handleICMP(r *stack.Route, vv *buffer.VectorisedView) {
case header.ICMPv6PortUnreachable:
e.handleControl(stack.ControlPortUnreachable, 0, vv)
}
+
+ case header.ICMPv6NeighborSolicit:
+ if len(v) < header.ICMPv6NeighborSolicitMinimumSize {
+ return
+ }
+ targetAddr := tcpip.Address(v[8 : 8+16])
+ if e.linkAddrCache.CheckLocalAddress(e.nicid, ProtocolNumber, targetAddr) == 0 {
+ // We don't have a useful answer; the best we can do is ignore the request.
+ return
+ }
+ hdr := buffer.NewPrependable(int(r.MaxHeaderLength()) + header.IPv6MinimumSize + header.ICMPv6NeighborAdvertSize)
+ pkt := header.ICMPv6(hdr.Prepend(header.ICMPv6NeighborAdvertSize))
+ pkt.SetType(header.ICMPv6NeighborAdvert)
+ pkt[icmpV6FlagOffset] = ndpSolicitedFlag | ndpOverrideFlag
+ copy(pkt[icmpV6OptOffset-len(targetAddr):], targetAddr)
+ pkt[icmpV6OptOffset] = ndpOptDstLinkAddr
+ pkt[icmpV6LengthOffset] = 1
+ copy(pkt[icmpV6LengthOffset+1:], r.LocalLinkAddress[:])
+ pkt.SetChecksum(icmpChecksum(pkt, r.LocalAddress, r.RemoteAddress, nil))
+ r.WritePacket(&hdr, nil, header.ICMPv6ProtocolNumber)
+
+ e.linkAddrCache.AddLinkAddress(e.nicid, r.RemoteAddress, r.RemoteLinkAddress)
+
+ case header.ICMPv6NeighborAdvert:
+ if len(v) < header.ICMPv6NeighborAdvertSize {
+ return
+ }
+ targetAddr := tcpip.Address(v[8 : 8+16])
+ e.linkAddrCache.AddLinkAddress(e.nicid, targetAddr, r.RemoteLinkAddress)
+ if targetAddr != r.RemoteAddress {
+ e.linkAddrCache.AddLinkAddress(e.nicid, r.RemoteAddress, r.RemoteLinkAddress)
+ }
+
+ case header.ICMPv6EchoRequest:
+ if len(v) < header.ICMPv6EchoMinimumSize {
+ return
+ }
+ vv.TrimFront(header.ICMPv6EchoMinimumSize)
+ data := vv.ToView()
+ hdr := buffer.NewPrependable(int(r.MaxHeaderLength()) + header.IPv6MinimumSize + header.ICMPv6EchoMinimumSize)
+ pkt := header.ICMPv6(hdr.Prepend(header.ICMPv6EchoMinimumSize))
+ copy(pkt, h)
+ pkt.SetType(header.ICMPv6EchoReply)
+ pkt.SetChecksum(icmpChecksum(pkt, r.LocalAddress, r.RemoteAddress, data))
+ r.WritePacket(&hdr, data, header.ICMPv6ProtocolNumber)
+
+ case header.ICMPv6EchoReply:
+ if len(v) < header.ICMPv6EchoMinimumSize {
+ return
+ }
+ e.dispatcher.DeliverTransportPacket(r, header.ICMPv6ProtocolNumber, vv)
+
+ }
+}
+
+const (
+ ndpSolicitedFlag = 1 << 6
+ ndpOverrideFlag = 1 << 5
+
+ ndpOptSrcLinkAddr = 1
+ ndpOptDstLinkAddr = 2
+
+ icmpV6FlagOffset = 4
+ icmpV6OptOffset = 24
+ icmpV6LengthOffset = 25
+)
+
+// solicitedNodeAddr computes the solicited-node multicast address.
+// This is used for NDP. Described in RFC 4291.
+func solicitedNodeAddr(addr tcpip.Address) tcpip.Address {
+ const solicitedNodeMulticastPrefix = "\xff\x02\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\xff"
+ return solicitedNodeMulticastPrefix + addr[len(addr)-3:]
+}
+
+var broadcastMAC = tcpip.LinkAddress([]byte{0xff, 0xff, 0xff, 0xff, 0xff, 0xff})
+
+var _ stack.LinkAddressResolver = (*protocol)(nil)
+
+// LinkAddressProtocol implements stack.LinkAddressResolver.
+func (*protocol) LinkAddressProtocol() tcpip.NetworkProtocolNumber {
+ return header.IPv6ProtocolNumber
+}
+
+// LinkAddressRequest implements stack.LinkAddressResolver.
+func (*protocol) LinkAddressRequest(addr, localAddr tcpip.Address, linkEP stack.LinkEndpoint) *tcpip.Error {
+ snaddr := solicitedNodeAddr(addr)
+ r := &stack.Route{
+ LocalAddress: localAddr,
+ RemoteAddress: snaddr,
+ RemoteLinkAddress: broadcastMAC,
}
+ hdr := buffer.NewPrependable(int(linkEP.MaxHeaderLength()) + header.IPv6MinimumSize + header.ICMPv6NeighborAdvertSize)
+ pkt := header.ICMPv6(hdr.Prepend(header.ICMPv6NeighborAdvertSize))
+ pkt.SetType(header.ICMPv6NeighborSolicit)
+ copy(pkt[icmpV6OptOffset-len(addr):], addr)
+ pkt[icmpV6OptOffset] = ndpOptSrcLinkAddr
+ pkt[icmpV6LengthOffset] = 1
+ copy(pkt[icmpV6LengthOffset+1:], linkEP.LinkAddress())
+ pkt.SetChecksum(icmpChecksum(pkt, r.LocalAddress, r.RemoteAddress, nil))
+
+ length := uint16(hdr.UsedLength())
+ ip := header.IPv6(hdr.Prepend(header.IPv6MinimumSize))
+ ip.Encode(&header.IPv6Fields{
+ PayloadLength: length,
+ NextHeader: uint8(header.ICMPv6ProtocolNumber),
+ HopLimit: defaultIPv6HopLimit,
+ SrcAddr: r.LocalAddress,
+ DstAddr: r.RemoteAddress,
+ })
+
+ return linkEP.WritePacket(r, &hdr, nil, ProtocolNumber)
+}
+
+// ResolveStaticAddress implements stack.LinkAddressResolver.
+func (*protocol) ResolveStaticAddress(addr tcpip.Address) (tcpip.LinkAddress, bool) {
+ return "", false
+}
+
+func icmpChecksum(h header.ICMPv6, src, dst tcpip.Address, data []byte) uint16 {
+ // Calculate the IPv6 pseudo-header upper-layer checksum.
+ xsum := header.Checksum([]byte(src), 0)
+ xsum = header.Checksum([]byte(dst), xsum)
+ var upperLayerLength [4]byte
+ binary.BigEndian.PutUint32(upperLayerLength[:], uint32(len(h)+len(data)))
+ xsum = header.Checksum(upperLayerLength[:], xsum)
+ xsum = header.Checksum([]byte{0, 0, 0, uint8(header.ICMPv6ProtocolNumber)}, xsum)
+ xsum = header.Checksum(data, xsum)
+
+ // h[2:4] is the checksum itself, set it aside to avoid checksumming the checksum.
+ h2, h3 := h[2], h[3]
+ h[2], h[3] = 0, 0
+ xsum = ^header.Checksum(h, xsum)
+ h[2], h[3] = h2, h3
+
+ return xsum
}
diff --git a/pkg/tcpip/network/ipv6/icmp_test.go b/pkg/tcpip/network/ipv6/icmp_test.go
new file mode 100644
index 000000000..582bbb40e
--- /dev/null
+++ b/pkg/tcpip/network/ipv6/icmp_test.go
@@ -0,0 +1,236 @@
+// Copyright 2018 Google Inc.
+//
+// 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 ipv6
+
+import (
+ "context"
+ "runtime"
+ "strings"
+ "testing"
+ "time"
+
+ "gvisor.googlesource.com/gvisor/pkg/tcpip"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/buffer"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/header"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/link/channel"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/link/sniffer"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/stack"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/ping"
+ "gvisor.googlesource.com/gvisor/pkg/waiter"
+)
+
+const (
+ linkAddr0 = tcpip.LinkAddress("\x01\x02\x03\x04\x05\x06")
+ linkAddr1 = tcpip.LinkAddress("\x0a\x0b\x0c\x0d\x0e\x0f")
+)
+
+// linkLocalAddr computes the default IPv6 link-local address from
+// a link-layer (MAC) address.
+func linkLocalAddr(linkAddr tcpip.LinkAddress) tcpip.Address {
+ // Convert a 48-bit MAC to an EUI-64 and then prepend the
+ // link-local header, FE80::.
+ //
+ // The conversion is very nearly:
+ // aa:bb:cc:dd:ee:ff => FE80::Aabb:ccFF:FEdd:eeff
+ // Note the capital A. The conversion aa->Aa involves a bit flip.
+ lladdrb := [16]byte{
+ 0: 0xFE,
+ 1: 0x80,
+ 8: linkAddr[0] ^ 2,
+ 9: linkAddr[1],
+ 10: linkAddr[2],
+ 11: 0xFF,
+ 12: 0xFE,
+ 13: linkAddr[3],
+ 14: linkAddr[4],
+ 15: linkAddr[5],
+ }
+ return tcpip.Address(lladdrb[:])
+}
+
+var (
+ lladdr0 = linkLocalAddr(linkAddr0)
+ lladdr1 = linkLocalAddr(linkAddr1)
+)
+
+type testContext struct {
+ t *testing.T
+ s0 *stack.Stack
+ s1 *stack.Stack
+
+ linkEP0 *channel.Endpoint
+ linkEP1 *channel.Endpoint
+
+ icmpCh chan header.ICMPv6Type
+}
+
+type endpointWithResolutionCapability struct {
+ stack.LinkEndpoint
+}
+
+func (e endpointWithResolutionCapability) Capabilities() stack.LinkEndpointCapabilities {
+ return e.LinkEndpoint.Capabilities() | stack.CapabilityResolutionRequired
+}
+
+func newTestContext(t *testing.T) *testContext {
+ c := &testContext{
+ t: t,
+ s0: stack.New([]string{ProtocolName}, []string{ping.ProtocolName6}, stack.Options{}),
+ s1: stack.New([]string{ProtocolName}, []string{ping.ProtocolName6}, stack.Options{}),
+ icmpCh: make(chan header.ICMPv6Type, 10),
+ }
+
+ const defaultMTU = 65536
+ _, linkEP0 := channel.New(256, defaultMTU, linkAddr0)
+ c.linkEP0 = linkEP0
+ wrappedEP0 := endpointWithResolutionCapability{LinkEndpoint: linkEP0}
+ id0 := stack.RegisterLinkEndpoint(wrappedEP0)
+ if testing.Verbose() {
+ id0 = sniffer.New(id0)
+ }
+ if err := c.s0.CreateNIC(1, id0); err != nil {
+ t.Fatalf("CreateNIC s0: %v", err)
+ }
+ if err := c.s0.AddAddress(1, ProtocolNumber, lladdr0); err != nil {
+ t.Fatalf("AddAddress lladdr0: %v", err)
+ }
+ if err := c.s0.AddAddress(1, ProtocolNumber, solicitedNodeAddr(lladdr0)); err != nil {
+ t.Fatalf("AddAddress sn lladdr0: %v", err)
+ }
+
+ _, linkEP1 := channel.New(256, defaultMTU, linkAddr1)
+ c.linkEP1 = linkEP1
+ wrappedEP1 := endpointWithResolutionCapability{LinkEndpoint: linkEP1}
+ id1 := stack.RegisterLinkEndpoint(wrappedEP1)
+ if err := c.s1.CreateNIC(1, id1); err != nil {
+ t.Fatalf("CreateNIC failed: %v", err)
+ }
+ if err := c.s1.AddAddress(1, ProtocolNumber, lladdr1); err != nil {
+ t.Fatalf("AddAddress lladdr1: %v", err)
+ }
+ if err := c.s1.AddAddress(1, ProtocolNumber, solicitedNodeAddr(lladdr1)); err != nil {
+ t.Fatalf("AddAddress sn lladdr1: %v", err)
+ }
+
+ c.s0.SetRouteTable(
+ []tcpip.Route{{
+ Destination: lladdr1,
+ Mask: tcpip.Address(strings.Repeat("\xff", 16)),
+ NIC: 1,
+ }},
+ )
+ c.s1.SetRouteTable(
+ []tcpip.Route{{
+ Destination: lladdr0,
+ Mask: tcpip.Address(strings.Repeat("\xff", 16)),
+ NIC: 1,
+ }},
+ )
+
+ go c.routePackets(linkEP0.C, linkEP1)
+ go c.routePackets(linkEP1.C, linkEP0)
+
+ return c
+}
+
+func (c *testContext) countPacket(pkt channel.PacketInfo) {
+ if pkt.Proto != ProtocolNumber {
+ return
+ }
+ ipv6 := header.IPv6(pkt.Header)
+ transProto := tcpip.TransportProtocolNumber(ipv6.NextHeader())
+ if transProto != header.ICMPv6ProtocolNumber {
+ return
+ }
+ b := pkt.Header[header.IPv6MinimumSize:]
+ icmp := header.ICMPv6(b)
+ c.icmpCh <- icmp.Type()
+}
+
+func (c *testContext) routePackets(ch <-chan channel.PacketInfo, ep *channel.Endpoint) {
+ for pkt := range ch {
+ c.countPacket(pkt)
+ views := []buffer.View{pkt.Header, pkt.Payload}
+ size := len(pkt.Header) + len(pkt.Payload)
+ vv := buffer.NewVectorisedView(size, views)
+ ep.InjectLinkAddr(pkt.Proto, ep.LinkAddress(), &vv)
+ }
+}
+
+func (c *testContext) cleanup() {
+ close(c.linkEP0.C)
+ close(c.linkEP1.C)
+}
+
+func TestLinkResolution(t *testing.T) {
+ c := newTestContext(t)
+ defer c.cleanup()
+ r, err := c.s0.FindRoute(1, lladdr0, lladdr1, ProtocolNumber)
+ if err != nil {
+ t.Fatal(err)
+ }
+ defer r.Release()
+
+ hdr := buffer.NewPrependable(int(r.MaxHeaderLength()) + header.IPv6MinimumSize + header.ICMPv6EchoMinimumSize)
+ pkt := header.ICMPv6(hdr.Prepend(header.ICMPv6EchoMinimumSize))
+ pkt.SetType(header.ICMPv6EchoRequest)
+ pkt.SetChecksum(icmpChecksum(pkt, r.LocalAddress, r.RemoteAddress, nil))
+ payload := tcpip.SlicePayload(hdr.UsedBytes())
+
+ // We can't send our payload directly over the route because that
+ // doesn't provoke NDP discovery.
+ var wq waiter.Queue
+ ep, err := c.s0.NewEndpoint(header.ICMPv6ProtocolNumber, ProtocolNumber, &wq)
+ if err != nil {
+ t.Fatal(err)
+ }
+
+ // This actually takes about 10 milliseconds, so no need to wait for
+ // a multi-minute go test timeout if something is broken.
+ ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
+ defer cancel()
+
+ for {
+ if ctx.Err() != nil {
+ break
+ }
+ if _, err := ep.Write(payload, tcpip.WriteOptions{To: &tcpip.FullAddress{NIC: 1, Addr: lladdr1}}); err == tcpip.ErrNoLinkAddress {
+ // There's something asynchronous going on; yield to let it do its thing.
+ runtime.Gosched()
+ } else if err == nil {
+ break
+ } else {
+ t.Fatal(err)
+ }
+ }
+
+ stats := make(map[header.ICMPv6Type]int)
+ for {
+ select {
+ case <-ctx.Done():
+ t.Errorf("timeout waiting for ICMP, got: %#+v", stats)
+ return
+ case typ := <-c.icmpCh:
+ stats[typ]++
+
+ if stats[header.ICMPv6NeighborSolicit] > 0 &&
+ stats[header.ICMPv6NeighborAdvert] > 0 &&
+ stats[header.ICMPv6EchoRequest] > 0 &&
+ stats[header.ICMPv6EchoReply] > 0 {
+ return
+ }
+ }
+ }
+}
diff --git a/pkg/tcpip/network/ipv6/ipv6.go b/pkg/tcpip/network/ipv6/ipv6.go
index f48d120c5..cdb8284a2 100644
--- a/pkg/tcpip/network/ipv6/ipv6.go
+++ b/pkg/tcpip/network/ipv6/ipv6.go
@@ -37,23 +37,21 @@ const (
// maxTotalSize is maximum size that can be encoded in the 16-bit
// PayloadLength field of the ipv6 header.
maxPayloadSize = 0xffff
+
+ // defaultIPv6HopLimit is the default hop limit for IPv6 Packets
+ // egressed by Netstack.
+ defaultIPv6HopLimit = 255
)
type address [header.IPv6AddressSize]byte
type endpoint struct {
- nicid tcpip.NICID
- id stack.NetworkEndpointID
- address address
- linkEP stack.LinkEndpoint
- dispatcher stack.TransportDispatcher
-}
-
-func newEndpoint(nicid tcpip.NICID, addr tcpip.Address, dispatcher stack.TransportDispatcher, linkEP stack.LinkEndpoint) *endpoint {
- e := &endpoint{nicid: nicid, linkEP: linkEP, dispatcher: dispatcher}
- copy(e.address[:], addr)
- e.id = stack.NetworkEndpointID{tcpip.Address(e.address[:])}
- return e
+ nicid tcpip.NICID
+ id stack.NetworkEndpointID
+ address address
+ linkEP stack.LinkEndpoint
+ linkAddrCache stack.LinkAddressCache
+ dispatcher stack.TransportDispatcher
}
// MTU implements stack.NetworkEndpoint.MTU. It returns the link-layer MTU minus
@@ -93,7 +91,7 @@ func (e *endpoint) WritePacket(r *stack.Route, hdr *buffer.Prependable, payload
ip.Encode(&header.IPv6Fields{
PayloadLength: length,
NextHeader: uint8(protocol),
- HopLimit: 65,
+ HopLimit: defaultIPv6HopLimit,
SrcAddr: tcpip.Address(e.address[:]),
DstAddr: r.RemoteAddress,
})
@@ -154,7 +152,15 @@ func (*protocol) ParseAddresses(v buffer.View) (src, dst tcpip.Address) {
// NewEndpoint creates a new ipv6 endpoint.
func (p *protocol) NewEndpoint(nicid tcpip.NICID, addr tcpip.Address, linkAddrCache stack.LinkAddressCache, dispatcher stack.TransportDispatcher, linkEP stack.LinkEndpoint) (stack.NetworkEndpoint, *tcpip.Error) {
- return newEndpoint(nicid, addr, dispatcher, linkEP), nil
+ e := &endpoint{
+ nicid: nicid,
+ linkEP: linkEP,
+ linkAddrCache: linkAddrCache,
+ dispatcher: dispatcher,
+ }
+ copy(e.address[:], addr)
+ e.id = stack.NetworkEndpointID{LocalAddress: tcpip.Address(e.address[:])}
+ return e, nil
}
// SetOption implements NetworkProtocol.SetOption.