summaryrefslogtreecommitdiffhomepage
path: root/test
diff options
context:
space:
mode:
authorEyal Soha <eyalsoha@google.com>2020-04-24 15:55:11 -0700
committergVisor bot <gvisor-bot@google.com>2020-04-24 15:56:27 -0700
commitdfff265fe422499af3bbe7d58e8db35ba32304f5 (patch)
tree4ea81063b2f322166d1d6919d33c8a75e99e0f2a /test
parent3d860530a904004aea5bc95e6331b3b11cec1877 (diff)
Add ICMP6 param problem test
Tested: When run on Linux, a correct ICMPv6 response is received. On netstack, no ICMPv6 response is received. PiperOrigin-RevId: 308343113
Diffstat (limited to 'test')
-rw-r--r--test/packetimpact/testbench/connections.go176
-rw-r--r--test/packetimpact/testbench/layers.go212
-rw-r--r--test/packetimpact/tests/BUILD13
-rw-r--r--test/packetimpact/tests/icmpv6_param_problem_test.go73
-rwxr-xr-xtest/packetimpact/tests/test_runner.sh41
5 files changed, 478 insertions, 37 deletions
diff --git a/test/packetimpact/testbench/connections.go b/test/packetimpact/testbench/connections.go
index 42a90a859..2280bd4ee 100644
--- a/test/packetimpact/testbench/connections.go
+++ b/test/packetimpact/testbench/connections.go
@@ -34,33 +34,60 @@ import (
var localIPv4 = flag.String("local_ipv4", "", "local IPv4 address for test packets")
var remoteIPv4 = flag.String("remote_ipv4", "", "remote IPv4 address for test packets")
+var localIPv6 = flag.String("local_ipv6", "", "local IPv6 address for test packets")
+var remoteIPv6 = flag.String("remote_ipv6", "", "remote IPv6 address for test packets")
var localMAC = flag.String("local_mac", "", "local mac address for test packets")
var remoteMAC = flag.String("remote_mac", "", "remote mac address for test packets")
-// pickPort makes a new socket and returns the socket FD and port. The caller
-// must close the FD when done with the port if there is no error.
-func pickPort() (int, uint16, error) {
- fd, err := unix.Socket(unix.AF_INET, unix.SOCK_STREAM, 0)
+// pickPort makes a new socket and returns the socket FD and port. The domain
+// should be AF_INET or AF_INET6. The caller must close the FD when done with
+// the port if there is no error.
+func pickPort(domain, typ int) (fd int, port uint16, err error) {
+ fd, err = unix.Socket(domain, typ, 0)
if err != nil {
return -1, 0, err
}
- var sa unix.SockaddrInet4
- copy(sa.Addr[0:4], net.ParseIP(*localIPv4).To4())
- if err := unix.Bind(fd, &sa); err != nil {
- unix.Close(fd)
+ defer func() {
+ if err != nil {
+ err = multierr.Append(err, unix.Close(fd))
+ }
+ }()
+ var sa unix.Sockaddr
+ switch domain {
+ case unix.AF_INET:
+ var sa4 unix.SockaddrInet4
+ copy(sa4.Addr[:], net.ParseIP(*localIPv4).To4())
+ sa = &sa4
+ case unix.AF_INET6:
+ var sa6 unix.SockaddrInet6
+ copy(sa6.Addr[:], net.ParseIP(*localIPv6).To16())
+ sa = &sa6
+ default:
+ return -1, 0, fmt.Errorf("invalid domain %d, it should be one of unix.AF_INET or unix.AF_INET6", domain)
+ }
+ if err = unix.Bind(fd, sa); err != nil {
return -1, 0, err
}
newSockAddr, err := unix.Getsockname(fd)
if err != nil {
- unix.Close(fd)
return -1, 0, err
}
- newSockAddrInet4, ok := newSockAddr.(*unix.SockaddrInet4)
- if !ok {
- unix.Close(fd)
- return -1, 0, fmt.Errorf("can't cast Getsockname result to SockaddrInet4")
+ switch domain {
+ case unix.AF_INET:
+ newSockAddrInet4, ok := newSockAddr.(*unix.SockaddrInet4)
+ if !ok {
+ return -1, 0, fmt.Errorf("can't cast Getsockname result %T to SockaddrInet4", newSockAddr)
+ }
+ return fd, uint16(newSockAddrInet4.Port), nil
+ case unix.AF_INET6:
+ newSockAddrInet6, ok := newSockAddr.(*unix.SockaddrInet6)
+ if !ok {
+ return -1, 0, fmt.Errorf("can't cast Getsockname result %T to SockaddrInet6", newSockAddr)
+ }
+ return fd, uint16(newSockAddrInet6.Port), nil
+ default:
+ return -1, 0, fmt.Errorf("invalid domain %d, it should be one of unix.AF_INET or unix.AF_INET6", domain)
}
- return fd, uint16(newSockAddrInet4.Port), nil
}
// layerState stores the state of a layer of a connection.
@@ -123,7 +150,7 @@ func newEtherState(out, in Ether) (*etherState, error) {
}
func (s *etherState) outgoing() Layer {
- return &s.out
+ return deepcopy.Copy(&s.out).(Layer)
}
// incoming implements layerState.incoming.
@@ -168,7 +195,7 @@ func newIPv4State(out, in IPv4) (*ipv4State, error) {
}
func (s *ipv4State) outgoing() Layer {
- return &s.out
+ return deepcopy.Copy(&s.out).(Layer)
}
// incoming implements layerState.incoming.
@@ -188,6 +215,54 @@ func (*ipv4State) close() error {
return nil
}
+// ipv6State maintains state about an IPv6 connection.
+type ipv6State struct {
+ out, in IPv6
+}
+
+var _ layerState = (*ipv6State)(nil)
+
+// newIPv6State creates a new ipv6State.
+func newIPv6State(out, in IPv6) (*ipv6State, error) {
+ lIP := tcpip.Address(net.ParseIP(*localIPv6).To16())
+ rIP := tcpip.Address(net.ParseIP(*remoteIPv6).To16())
+ s := ipv6State{
+ out: IPv6{SrcAddr: &lIP, DstAddr: &rIP},
+ in: IPv6{SrcAddr: &rIP, DstAddr: &lIP},
+ }
+ if err := s.out.merge(&out); err != nil {
+ return nil, err
+ }
+ if err := s.in.merge(&in); err != nil {
+ return nil, err
+ }
+ return &s, nil
+}
+
+// outgoing returns an outgoing layer to be sent in a frame.
+func (s *ipv6State) outgoing() Layer {
+ return deepcopy.Copy(&s.out).(Layer)
+}
+
+func (s *ipv6State) incoming(Layer) Layer {
+ return deepcopy.Copy(&s.in).(Layer)
+}
+
+func (s *ipv6State) sent(Layer) error {
+ // Nothing to do.
+ return nil
+}
+
+func (s *ipv6State) received(Layer) error {
+ // Nothing to do.
+ return nil
+}
+
+// close cleans up any resources held.
+func (s *ipv6State) close() error {
+ return nil
+}
+
// tcpState maintains state about a TCP connection.
type tcpState struct {
out, in TCP
@@ -206,8 +281,8 @@ func SeqNumValue(v seqnum.Value) *seqnum.Value {
}
// newTCPState creates a new TCPState.
-func newTCPState(out, in TCP) (*tcpState, error) {
- portPickerFD, localPort, err := pickPort()
+func newTCPState(domain int, out, in TCP) (*tcpState, error) {
+ portPickerFD, localPort, err := pickPort(domain, unix.SOCK_STREAM)
if err != nil {
return nil, err
}
@@ -310,8 +385,8 @@ type udpState struct {
var _ layerState = (*udpState)(nil)
// newUDPState creates a new udpState.
-func newUDPState(out, in UDP) (*udpState, error) {
- portPickerFD, localPort, err := pickPort()
+func newUDPState(domain int, out, in UDP) (*udpState, error) {
+ portPickerFD, localPort, err := pickPort(domain, unix.SOCK_DGRAM)
if err != nil {
return nil, err
}
@@ -330,7 +405,7 @@ func newUDPState(out, in UDP) (*udpState, error) {
}
func (s *udpState) outgoing() Layer {
- return &s.out
+ return deepcopy.Copy(&s.out).(Layer)
}
// incoming implements layerState.incoming.
@@ -422,7 +497,7 @@ func (conn *Connection) CreateFrame(layer Layer, additionalLayers ...Layer) Laye
// SendFrame sends a frame on the wire and updates the state of all layers.
func (conn *Connection) SendFrame(frame Layers) {
- outBytes, err := frame.toBytes()
+ outBytes, err := frame.ToBytes()
if err != nil {
conn.t.Fatalf("can't build outgoing TCP packet: %s", err)
}
@@ -536,7 +611,7 @@ func NewTCPIPv4(t *testing.T, outgoingTCP, incomingTCP TCP) TCPIPv4 {
if err != nil {
t.Fatalf("can't make ipv4State: %s", err)
}
- tcpState, err := newTCPState(outgoingTCP, incomingTCP)
+ tcpState, err := newTCPState(unix.AF_INET, outgoingTCP, incomingTCP)
if err != nil {
t.Fatalf("can't make tcpState: %s", err)
}
@@ -633,6 +708,59 @@ func (conn *TCPIPv4) SynAck() *TCP {
return conn.state().synAck
}
+// IPv6Conn maintains the state for all the layers in a IPv6 connection.
+type IPv6Conn Connection
+
+// NewIPv6Conn creates a new IPv6Conn connection with reasonable defaults.
+func NewIPv6Conn(t *testing.T, outgoingIPv6, incomingIPv6 IPv6) IPv6Conn {
+ etherState, err := newEtherState(Ether{}, Ether{})
+ if err != nil {
+ t.Fatalf("can't make EtherState: %s", err)
+ }
+ ipv6State, err := newIPv6State(outgoingIPv6, incomingIPv6)
+ if err != nil {
+ t.Fatalf("can't make IPv6State: %s", err)
+ }
+
+ injector, err := NewInjector(t)
+ if err != nil {
+ t.Fatalf("can't make injector: %s", err)
+ }
+ sniffer, err := NewSniffer(t)
+ if err != nil {
+ t.Fatalf("can't make sniffer: %s", err)
+ }
+
+ return IPv6Conn{
+ layerStates: []layerState{etherState, ipv6State},
+ injector: injector,
+ sniffer: sniffer,
+ t: t,
+ }
+}
+
+// SendFrame sends a frame on the wire and updates the state of all layers.
+func (conn *IPv6Conn) SendFrame(frame Layers) {
+ (*Connection)(conn).SendFrame(frame)
+}
+
+// CreateFrame builds a frame for the connection with ipv6 overriding the ipv6
+// layer defaults and additionalLayers added after it.
+func (conn *IPv6Conn) CreateFrame(ipv6 IPv6, additionalLayers ...Layer) Layers {
+ return (*Connection)(conn).CreateFrame(&ipv6, additionalLayers...)
+}
+
+// Close to clean up any resources held.
+func (conn *IPv6Conn) Close() {
+ (*Connection)(conn).Close()
+}
+
+// ExpectFrame expects a frame that matches the provided Layers within the
+// timeout specified. If it doesn't arrive in time, it returns nil.
+func (conn *IPv6Conn) ExpectFrame(frame Layers, timeout time.Duration) (Layers, error) {
+ return (*Connection)(conn).ExpectFrame(frame, timeout)
+}
+
// Drain drains the sniffer's receive buffer by receiving packets until there's
// nothing else to receive.
func (conn *TCPIPv4) Drain() {
@@ -652,7 +780,7 @@ func NewUDPIPv4(t *testing.T, outgoingUDP, incomingUDP UDP) UDPIPv4 {
if err != nil {
t.Fatalf("can't make ipv4State: %s", err)
}
- tcpState, err := newUDPState(outgoingUDP, incomingUDP)
+ tcpState, err := newUDPState(unix.AF_INET, outgoingUDP, incomingUDP)
if err != nil {
t.Fatalf("can't make udpState: %s", err)
}
diff --git a/test/packetimpact/testbench/layers.go b/test/packetimpact/testbench/layers.go
index 2cbbbb318..817f5c261 100644
--- a/test/packetimpact/testbench/layers.go
+++ b/test/packetimpact/testbench/layers.go
@@ -36,14 +36,14 @@ import (
type Layer interface {
fmt.Stringer
- // toBytes converts the Layer into bytes. In places where the Layer's field
+ // ToBytes converts the Layer into bytes. In places where the Layer's field
// isn't nil, the value that is pointed to is used. When the field is nil, a
// reasonable default for the Layer is used. For example, "64" for IPv4 TTL
// and a calculated checksum for TCP or IP. Some layers require information
// from the previous or next layers in order to compute a default, such as
// TCP's checksum or Ethernet's type, so each Layer has a doubly-linked list
// to the layer's neighbors.
- toBytes() ([]byte, error)
+ ToBytes() ([]byte, error)
// match checks if the current Layer matches the provided Layer. If either
// Layer has a nil in a given field, that field is considered matching.
@@ -174,7 +174,8 @@ func (l *Ether) String() string {
return stringLayer(l)
}
-func (l *Ether) toBytes() ([]byte, error) {
+// ToBytes implements Layer.ToBytes.
+func (l *Ether) ToBytes() ([]byte, error) {
b := make([]byte, header.EthernetMinimumSize)
h := header.Ethernet(b)
fields := &header.EthernetFields{}
@@ -190,8 +191,9 @@ func (l *Ether) toBytes() ([]byte, error) {
switch n := l.next().(type) {
case *IPv4:
fields.Type = header.IPv4ProtocolNumber
+ case *IPv6:
+ fields.Type = header.IPv6ProtocolNumber
default:
- // TODO(b/150301488): Support more protocols, like IPv6.
return nil, fmt.Errorf("ethernet header's next layer is unrecognized: %#v", n)
}
}
@@ -246,6 +248,8 @@ func parseEther(b []byte) (Layer, layerParser) {
switch h.Type() {
case header.IPv4ProtocolNumber:
nextParser = parseIPv4
+ case header.IPv6ProtocolNumber:
+ nextParser = parseIPv6
default:
// Assume that the rest is a payload.
nextParser = parsePayload
@@ -286,7 +290,8 @@ func (l *IPv4) String() string {
return stringLayer(l)
}
-func (l *IPv4) toBytes() ([]byte, error) {
+// ToBytes implements Layer.ToBytes.
+func (l *IPv4) ToBytes() ([]byte, error) {
b := make([]byte, header.IPv4MinimumSize)
h := header.IPv4(b)
fields := &header.IPv4Fields{
@@ -421,6 +426,186 @@ func (l *IPv4) merge(other Layer) error {
return mergeLayer(l, other)
}
+// IPv6 can construct and match an IPv6 encapsulation.
+type IPv6 struct {
+ LayerBase
+ TrafficClass *uint8
+ FlowLabel *uint32
+ PayloadLength *uint16
+ NextHeader *uint8
+ HopLimit *uint8
+ SrcAddr *tcpip.Address
+ DstAddr *tcpip.Address
+}
+
+func (l *IPv6) String() string {
+ return stringLayer(l)
+}
+
+// ToBytes implements Layer.ToBytes.
+func (l *IPv6) ToBytes() ([]byte, error) {
+ b := make([]byte, header.IPv6MinimumSize)
+ h := header.IPv6(b)
+ fields := &header.IPv6Fields{
+ HopLimit: 64,
+ }
+ if l.TrafficClass != nil {
+ fields.TrafficClass = *l.TrafficClass
+ }
+ if l.FlowLabel != nil {
+ fields.FlowLabel = *l.FlowLabel
+ }
+ if l.PayloadLength != nil {
+ fields.PayloadLength = *l.PayloadLength
+ } else {
+ for current := l.next(); current != nil; current = current.next() {
+ fields.PayloadLength += uint16(current.length())
+ }
+ }
+ if l.NextHeader != nil {
+ fields.NextHeader = *l.NextHeader
+ } else {
+ switch n := l.next().(type) {
+ case *TCP:
+ fields.NextHeader = uint8(header.TCPProtocolNumber)
+ case *UDP:
+ fields.NextHeader = uint8(header.UDPProtocolNumber)
+ case *ICMPv6:
+ fields.NextHeader = uint8(header.ICMPv6ProtocolNumber)
+ default:
+ // TODO(b/150301488): Support more protocols as needed.
+ return nil, fmt.Errorf("ToBytes can't deduce the IPv6 header's next protocol: %#v", n)
+ }
+ }
+ if l.HopLimit != nil {
+ fields.HopLimit = *l.HopLimit
+ }
+ if l.SrcAddr != nil {
+ fields.SrcAddr = *l.SrcAddr
+ }
+ if l.DstAddr != nil {
+ fields.DstAddr = *l.DstAddr
+ }
+ h.Encode(fields)
+ return h, nil
+}
+
+// parseIPv6 parses the bytes assuming that they start with an ipv6 header and
+// continues parsing further encapsulations.
+func parseIPv6(b []byte) (Layer, layerParser) {
+ h := header.IPv6(b)
+ tos, flowLabel := h.TOS()
+ ipv6 := IPv6{
+ TrafficClass: &tos,
+ FlowLabel: &flowLabel,
+ PayloadLength: Uint16(h.PayloadLength()),
+ NextHeader: Uint8(h.NextHeader()),
+ HopLimit: Uint8(h.HopLimit()),
+ SrcAddr: Address(h.SourceAddress()),
+ DstAddr: Address(h.DestinationAddress()),
+ }
+ var nextParser layerParser
+ switch h.TransportProtocol() {
+ case header.TCPProtocolNumber:
+ nextParser = parseTCP
+ case header.UDPProtocolNumber:
+ nextParser = parseUDP
+ case header.ICMPv6ProtocolNumber:
+ nextParser = parseICMPv6
+ default:
+ // Assume that the rest is a payload.
+ nextParser = parsePayload
+ }
+ return &ipv6, nextParser
+}
+
+func (l *IPv6) match(other Layer) bool {
+ return equalLayer(l, other)
+}
+
+func (l *IPv6) length() int {
+ return header.IPv6MinimumSize
+}
+
+// merge overrides the values in l with the values from other but only in fields
+// where the value is not nil.
+func (l *IPv6) merge(other Layer) error {
+ return mergeLayer(l, other)
+}
+
+// ICMPv6 can construct and match an ICMPv6 encapsulation.
+type ICMPv6 struct {
+ LayerBase
+ Type *header.ICMPv6Type
+ Code *byte
+ Checksum *uint16
+ NDPPayload []byte
+}
+
+func (l *ICMPv6) String() string {
+ // TODO(eyalsoha): Do something smarter here when *l.Type is ParameterProblem?
+ // We could parse the contents of the Payload as if it were an IPv6 packet.
+ return stringLayer(l)
+}
+
+// ToBytes implements Layer.ToBytes.
+func (l *ICMPv6) ToBytes() ([]byte, error) {
+ b := make([]byte, header.ICMPv6HeaderSize+len(l.NDPPayload))
+ h := header.ICMPv6(b)
+ if l.Type != nil {
+ h.SetType(*l.Type)
+ }
+ if l.Code != nil {
+ h.SetCode(*l.Code)
+ }
+ copy(h.NDPPayload(), l.NDPPayload)
+ if l.Checksum != nil {
+ h.SetChecksum(*l.Checksum)
+ } else {
+ ipv6 := l.prev().(*IPv6)
+ h.SetChecksum(header.ICMPv6Checksum(h, *ipv6.SrcAddr, *ipv6.DstAddr, buffer.VectorisedView{}))
+ }
+ return h, nil
+}
+
+// ICMPv6Type is a helper routine that allocates a new ICMPv6Type value to store
+// v and returns a pointer to it.
+func ICMPv6Type(v header.ICMPv6Type) *header.ICMPv6Type {
+ return &v
+}
+
+// Byte is a helper routine that allocates a new byte value to store
+// v and returns a pointer to it.
+func Byte(v byte) *byte {
+ return &v
+}
+
+// parseICMPv6 parses the bytes assuming that they start with an ICMPv6 header.
+func parseICMPv6(b []byte) (Layer, layerParser) {
+ h := header.ICMPv6(b)
+ icmpv6 := ICMPv6{
+ Type: ICMPv6Type(h.Type()),
+ Code: Byte(h.Code()),
+ Checksum: Uint16(h.Checksum()),
+ NDPPayload: h.NDPPayload(),
+ }
+ return &icmpv6, nil
+}
+
+func (l *ICMPv6) match(other Layer) bool {
+ return equalLayer(l, other)
+}
+
+func (l *ICMPv6) length() int {
+ return header.ICMPv6HeaderSize + len(l.NDPPayload)
+}
+
+// merge overrides the values in l with the values from other but only in fields
+// where the value is not nil.
+func (l *ICMPv6) merge(other Layer) error {
+ return mergeLayer(l, other)
+}
+
// TCP can construct and match a TCP encapsulation.
type TCP struct {
LayerBase
@@ -439,7 +624,8 @@ func (l *TCP) String() string {
return stringLayer(l)
}
-func (l *TCP) toBytes() ([]byte, error) {
+// ToBytes implements Layer.ToBytes.
+func (l *TCP) ToBytes() ([]byte, error) {
b := make([]byte, header.TCPMinimumSize)
h := header.TCP(b)
if l.SrcPort != nil {
@@ -504,7 +690,7 @@ func layerChecksum(l Layer, protoNumber tcpip.TransportProtocolNumber) (uint16,
}
var payloadBytes buffer.VectorisedView
for current := l.next(); current != nil; current = current.next() {
- payload, err := current.toBytes()
+ payload, err := current.ToBytes()
if err != nil {
return 0, fmt.Errorf("can't get bytes for next header: %s", payload)
}
@@ -578,7 +764,8 @@ func (l *UDP) String() string {
return stringLayer(l)
}
-func (l *UDP) toBytes() ([]byte, error) {
+// ToBytes implements Layer.ToBytes.
+func (l *UDP) ToBytes() ([]byte, error) {
b := make([]byte, header.UDPMinimumSize)
h := header.UDP(b)
if l.SrcPort != nil {
@@ -661,7 +848,8 @@ func parsePayload(b []byte) (Layer, layerParser) {
return &payload, nil
}
-func (l *Payload) toBytes() ([]byte, error) {
+// ToBytes implements Layer.ToBytes.
+func (l *Payload) ToBytes() ([]byte, error) {
return l.Bytes, nil
}
@@ -697,11 +885,13 @@ func (ls *Layers) linkLayers() {
}
}
-func (ls *Layers) toBytes() ([]byte, error) {
+// ToBytes converts the Layers into bytes. It creates a linked list of the Layer
+// structs and then concatentates the output of ToBytes on each Layer.
+func (ls *Layers) ToBytes() ([]byte, error) {
ls.linkLayers()
outBytes := []byte{}
for _, l := range *ls {
- layerBytes, err := l.toBytes()
+ layerBytes, err := l.ToBytes()
if err != nil {
return nil, err
}
diff --git a/test/packetimpact/tests/BUILD b/test/packetimpact/tests/BUILD
index 47c722ccd..42f87e3f3 100644
--- a/test/packetimpact/tests/BUILD
+++ b/test/packetimpact/tests/BUILD
@@ -96,6 +96,19 @@ packetimpact_go_test(
],
)
+packetimpact_go_test(
+ name = "icmpv6_param_problem",
+ srcs = ["icmpv6_param_problem_test.go"],
+ # TODO(b/153485026): Fix netstack then remove the line below.
+ netstack = False,
+ deps = [
+ "//pkg/tcpip",
+ "//pkg/tcpip/header",
+ "//test/packetimpact/testbench",
+ "@org_golang_x_sys//unix:go_default_library",
+ ],
+)
+
sh_binary(
name = "test_runner",
srcs = ["test_runner.sh"],
diff --git a/test/packetimpact/tests/icmpv6_param_problem_test.go b/test/packetimpact/tests/icmpv6_param_problem_test.go
new file mode 100644
index 000000000..b48e55df4
--- /dev/null
+++ b/test/packetimpact/tests/icmpv6_param_problem_test.go
@@ -0,0 +1,73 @@
+// 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 icmpv6_param_problem_test
+
+import (
+ "encoding/binary"
+ "testing"
+ "time"
+
+ "gvisor.dev/gvisor/pkg/tcpip/header"
+ tb "gvisor.dev/gvisor/test/packetimpact/testbench"
+)
+
+// TestICMPv6ParamProblemTest sends a packet with a bad next header. The DUT
+// should respond with an ICMPv6 Parameter Problem message.
+func TestICMPv6ParamProblemTest(t *testing.T) {
+ dut := tb.NewDUT(t)
+ defer dut.TearDown()
+ conn := tb.NewIPv6Conn(t, tb.IPv6{}, tb.IPv6{})
+ defer conn.Close()
+ ipv6 := tb.IPv6{
+ // 254 is reserved and used for experimentation and testing. This should
+ // cause an error.
+ NextHeader: tb.Uint8(254),
+ }
+ icmpv6 := tb.ICMPv6{
+ Type: tb.ICMPv6Type(header.ICMPv6EchoRequest),
+ NDPPayload: []byte("hello world"),
+ }
+
+ toSend := conn.CreateFrame(ipv6, &icmpv6)
+ conn.SendFrame(toSend)
+
+ // Build the expected ICMPv6 payload, which includes an index to the
+ // problematic byte and also the problematic packet as described in
+ // https://tools.ietf.org/html/rfc4443#page-12 .
+ ipv6Sent := toSend[1:]
+ expectedPayload, err := ipv6Sent.ToBytes()
+ if err != nil {
+ t.Fatalf("can't convert %s to bytes: %s", ipv6Sent, err)
+ }
+
+ // The problematic field is the NextHeader.
+ b := make([]byte, 4)
+ binary.BigEndian.PutUint32(b, header.IPv6NextHeaderOffset)
+ expectedPayload = append(b, expectedPayload...)
+ expectedICMPv6 := tb.ICMPv6{
+ Type: tb.ICMPv6Type(header.ICMPv6ParamProblem),
+ NDPPayload: expectedPayload,
+ }
+
+ paramProblem := tb.Layers{
+ &tb.Ether{},
+ &tb.IPv6{},
+ &expectedICMPv6,
+ }
+ timeout := time.Second
+ if _, err := conn.ExpectFrame(paramProblem, timeout); err != nil {
+ t.Errorf("expected %s within %s but got none: %s", paramProblem, timeout, err)
+ }
+}
diff --git a/test/packetimpact/tests/test_runner.sh b/test/packetimpact/tests/test_runner.sh
index e938de782..706441cce 100755
--- a/test/packetimpact/tests/test_runner.sh
+++ b/test/packetimpact/tests/test_runner.sh
@@ -192,6 +192,8 @@ docker pull "${IMAGE_TAG}"
# Create the DUT container and connect to network.
DUT=$(docker create ${RUNTIME_ARG} --privileged --rm \
+ --cap-add NET_ADMIN \
+ --sysctl net.ipv6.conf.all.disable_ipv6=0 \
--stop-timeout ${TIMEOUT} -it ${IMAGE_TAG})
docker network connect "${CTRL_NET}" \
--ip "${CTRL_NET_PREFIX}${DUT_NET_SUFFIX}" "${DUT}" \
@@ -203,6 +205,8 @@ docker start "${DUT}"
# Create the test bench container and connect to network.
TESTBENCH=$(docker create --privileged --rm \
+ --cap-add NET_ADMIN \
+ --sysctl net.ipv6.conf.all.disable_ipv6=0 \
--stop-timeout ${TIMEOUT} -it ${IMAGE_TAG})
docker network connect "${CTRL_NET}" \
--ip "${CTRL_NET_PREFIX}${TESTBENCH_NET_SUFFIX}" "${TESTBENCH}" \
@@ -237,6 +241,32 @@ declare -r REMOTE_MAC=$(docker exec -t "${DUT}" ip link show \
"${TEST_DEVICE}" | tail -1 | cut -d' ' -f6)
declare -r LOCAL_MAC=$(docker exec -t "${TESTBENCH}" ip link show \
"${TEST_DEVICE}" | tail -1 | cut -d' ' -f6)
+declare REMOTE_IPV6=$(docker exec -t "${DUT}" ip addr show scope link \
+ "${TEST_DEVICE}" | grep inet6 | cut -d' ' -f6 | cut -d'/' -f1)
+declare -r LOCAL_IPV6=$(docker exec -t "${TESTBENCH}" ip addr show scope link \
+ "${TEST_DEVICE}" | grep inet6 | cut -d' ' -f6 | cut -d'/' -f1)
+
+# Netstack as DUT doesn't assign IPv6 addresses automatically so do it if
+# needed. Convert the MAC address to an IPv6 link local address as described in
+# RFC 4291 page 20: https://tools.ietf.org/html/rfc4291#page-20
+if [[ -z "${REMOTE_IPV6}" ]]; then
+ # Split the octets of the MAC into an array of strings.
+ IFS=":" read -a REMOTE_OCTETS <<< "${REMOTE_MAC}"
+ # Flip the global bit.
+ REMOTE_OCTETS[0]=$(printf '%x' "$((0x${REMOTE_OCTETS[0]} ^ 2))")
+ # Add the IPv6 address.
+ docker exec "${DUT}" \
+ ip addr add $(printf 'fe80::%02x%02x:%02xff:fe%02x:%02x%02x/64' \
+ "0x${REMOTE_OCTETS[0]}" "0x${REMOTE_OCTETS[1]}" "0x${REMOTE_OCTETS[2]}" \
+ "0x${REMOTE_OCTETS[3]}" "0x${REMOTE_OCTETS[4]}" "0x${REMOTE_OCTETS[5]}") \
+ scope link \
+ dev "${TEST_DEVICE}"
+ # Re-extract the IPv6 address.
+ # TODO(eyalsoha): Add "scope link" below when netstack supports correctly
+ # creating link-local IPv6 addresses.
+ REMOTE_IPV6=$(docker exec -t "${DUT}" ip addr show \
+ "${TEST_DEVICE}" | grep inet6 | cut -d' ' -f6 | cut -d'/' -f1)
+fi
declare -r DOCKER_TESTBENCH_BINARY="/$(basename ${TESTBENCH_BINARY})"
docker cp -L "${TESTBENCH_BINARY}" "${TESTBENCH}:${DOCKER_TESTBENCH_BINARY}"
@@ -245,7 +275,10 @@ if [[ -z "${TSHARK-}" ]]; then
# Run tcpdump in the test bench unbuffered, without dns resolution, just on
# the interface with the test packets.
docker exec -t "${TESTBENCH}" \
- tcpdump -S -vvv -U -n -i "${TEST_DEVICE}" net "${TEST_NET_PREFIX}/24" &
+ tcpdump -S -vvv -U -n -i "${TEST_DEVICE}" \
+ net "${TEST_NET_PREFIX}/24" or \
+ host "${REMOTE_IPV6}" or \
+ host "${LOCAL_IPV6}" &
else
# Run tshark in the test bench unbuffered, without dns resolution, just on the
# interface with the test packets.
@@ -253,7 +286,9 @@ else
tshark -V -l -n -i "${TEST_DEVICE}" \
-o tcp.check_checksum:TRUE \
-o udp.check_checksum:TRUE \
- host "${TEST_NET_PREFIX}${TESTBENCH_NET_SUFFIX}" &
+ net "${TEST_NET_PREFIX}/24" or \
+ host "${REMOTE_IPV6}" or \
+ host "${LOCAL_IPV6}" &
fi
# tcpdump and tshark take time to startup
@@ -272,6 +307,8 @@ docker exec \
--posix_server_port=${CTRL_PORT} \
--remote_ipv4=${TEST_NET_PREFIX}${DUT_NET_SUFFIX} \
--local_ipv4=${TEST_NET_PREFIX}${TESTBENCH_NET_SUFFIX} \
+ --remote_ipv6=${REMOTE_IPV6} \
+ --local_ipv6=${LOCAL_IPV6} \
--remote_mac=${REMOTE_MAC} \
--local_mac=${LOCAL_MAC} \
--device=${TEST_DEVICE}" && true