diff options
Diffstat (limited to 'pkg/tcpip')
89 files changed, 8543 insertions, 4264 deletions
diff --git a/pkg/tcpip/adapters/gonet/gonet_test.go b/pkg/tcpip/adapters/gonet/gonet_test.go index c975ad9cf..12b061def 100644 --- a/pkg/tcpip/adapters/gonet/gonet_test.go +++ b/pkg/tcpip/adapters/gonet/gonet_test.go @@ -61,8 +61,8 @@ func TestTimeouts(t *testing.T) { func newLoopbackStack() (*stack.Stack, *tcpip.Error) { // Create the stack and add a NIC. s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{tcp.NewProtocol(), udp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{tcp.NewProtocol, udp.NewProtocol}, }) if err := s.CreateNIC(NICID, loopback.New()); err != nil { diff --git a/pkg/tcpip/buffer/BUILD b/pkg/tcpip/buffer/BUILD index 563bc78ea..c326fab54 100644 --- a/pkg/tcpip/buffer/BUILD +++ b/pkg/tcpip/buffer/BUILD @@ -14,6 +14,8 @@ go_library( go_test( name = "buffer_test", size = "small", - srcs = ["view_test.go"], + srcs = [ + "view_test.go", + ], library = ":buffer", ) diff --git a/pkg/tcpip/checker/checker.go b/pkg/tcpip/checker/checker.go index b769094dc..71b2f1bda 100644 --- a/pkg/tcpip/checker/checker.go +++ b/pkg/tcpip/checker/checker.go @@ -118,18 +118,62 @@ func TTL(ttl uint8) NetworkChecker { v = ip.HopLimit() } if v != ttl { - t.Fatalf("Bad TTL, got %v, want %v", v, ttl) + t.Fatalf("Bad TTL, got = %d, want = %d", v, ttl) + } + } +} + +// IPFullLength creates a checker for the full IP packet length. The +// expected size is checked against both the Total Length in the +// header and the number of bytes received. +func IPFullLength(packetLength uint16) NetworkChecker { + return func(t *testing.T, h []header.Network) { + t.Helper() + + var v uint16 + var l uint16 + switch ip := h[0].(type) { + case header.IPv4: + v = ip.TotalLength() + l = uint16(len(ip)) + case header.IPv6: + v = ip.PayloadLength() + header.IPv6FixedHeaderSize + l = uint16(len(ip)) + default: + t.Fatalf("unexpected network header passed to checker, got = %T, want = header.IPv4 or header.IPv6", ip) + } + if l != packetLength { + t.Errorf("bad packet length, got = %d, want = %d", l, packetLength) + } + if v != packetLength { + t.Errorf("unexpected packet length in header, got = %d, want = %d", v, packetLength) + } + } +} + +// IPv4HeaderLength creates a checker that checks the IPv4 Header length. +func IPv4HeaderLength(headerLength int) NetworkChecker { + return func(t *testing.T, h []header.Network) { + t.Helper() + + switch ip := h[0].(type) { + case header.IPv4: + if hl := ip.HeaderLength(); hl != uint8(headerLength) { + t.Errorf("Bad header length, got = %d, want = %d", hl, headerLength) + } + default: + t.Fatalf("unexpected network header passed to checker, got = %T, want = header.IPv4", ip) } } } // PayloadLen creates a checker that checks the payload length. -func PayloadLen(plen int) NetworkChecker { +func PayloadLen(payloadLength int) NetworkChecker { return func(t *testing.T, h []header.Network) { t.Helper() - if l := len(h[0].Payload()); l != plen { - t.Errorf("Bad payload length, got %v, want %v", l, plen) + if l := len(h[0].Payload()); l != payloadLength { + t.Errorf("Bad payload length, got = %d, want = %d", l, payloadLength) } } } @@ -143,7 +187,7 @@ func FragmentOffset(offset uint16) NetworkChecker { switch ip := h[0].(type) { case header.IPv4: if v := ip.FragmentOffset(); v != offset { - t.Errorf("Bad fragment offset, got %v, want %v", v, offset) + t.Errorf("Bad fragment offset, got = %d, want = %d", v, offset) } } } @@ -158,7 +202,7 @@ func FragmentFlags(flags uint8) NetworkChecker { switch ip := h[0].(type) { case header.IPv4: if v := ip.Flags(); v != flags { - t.Errorf("Bad fragment offset, got %v, want %v", v, flags) + t.Errorf("Bad fragment offset, got = %d, want = %d", v, flags) } } } @@ -208,7 +252,7 @@ func TOS(tos uint8, label uint32) NetworkChecker { t.Helper() if v, l := h[0].TOS(); v != tos || l != label { - t.Errorf("Bad TOS, got (%v, %v), want (%v,%v)", v, l, tos, label) + t.Errorf("Bad TOS, got = (%d, %d), want = (%d,%d)", v, l, tos, label) } } } @@ -234,7 +278,7 @@ func IPv6Fragment(checkers ...NetworkChecker) NetworkChecker { t.Helper() if p := h[0].TransportProtocol(); p != header.IPv6FragmentHeader { - t.Errorf("Bad protocol, got %v, want %v", p, header.UDPProtocolNumber) + t.Errorf("Bad protocol, got = %d, want = %d", p, header.UDPProtocolNumber) } ipv6Frag := header.IPv6Fragment(h[0].Payload()) @@ -261,7 +305,7 @@ func TCP(checkers ...TransportChecker) NetworkChecker { last := h[len(h)-1] if p := last.TransportProtocol(); p != header.TCPProtocolNumber { - t.Errorf("Bad protocol, got %v, want %v", p, header.TCPProtocolNumber) + t.Errorf("Bad protocol, got = %d, want = %d", p, header.TCPProtocolNumber) } // Verify the checksum. @@ -297,7 +341,7 @@ func UDP(checkers ...TransportChecker) NetworkChecker { last := h[len(h)-1] if p := last.TransportProtocol(); p != header.UDPProtocolNumber { - t.Errorf("Bad protocol, got %v, want %v", p, header.UDPProtocolNumber) + t.Errorf("Bad protocol, got = %d, want = %d", p, header.UDPProtocolNumber) } udp := header.UDP(last.Payload()) @@ -316,7 +360,7 @@ func SrcPort(port uint16) TransportChecker { t.Helper() if p := h.SourcePort(); p != port { - t.Errorf("Bad source port, got %v, want %v", p, port) + t.Errorf("Bad source port, got = %d, want = %d", p, port) } } } @@ -327,7 +371,7 @@ func DstPort(port uint16) TransportChecker { t.Helper() if p := h.DestinationPort(); p != port { - t.Errorf("Bad destination port, got %v, want %v", p, port) + t.Errorf("Bad destination port, got = %d, want = %d", p, port) } } } @@ -339,7 +383,7 @@ func NoChecksum(noChecksum bool) TransportChecker { udp, ok := h.(header.UDP) if !ok { - return + t.Fatalf("UDP header not found in h: %T", h) } if b := udp.Checksum() == 0; b != noChecksum { @@ -348,50 +392,84 @@ func NoChecksum(noChecksum bool) TransportChecker { } } -// SeqNum creates a checker that checks the sequence number. -func SeqNum(seq uint32) TransportChecker { +// TCPSeqNum creates a checker that checks the sequence number. +func TCPSeqNum(seq uint32) TransportChecker { return func(t *testing.T, h header.Transport) { t.Helper() tcp, ok := h.(header.TCP) if !ok { - return + t.Fatalf("TCP header not found in h: %T", h) } if s := tcp.SequenceNumber(); s != seq { - t.Errorf("Bad sequence number, got %v, want %v", s, seq) + t.Errorf("Bad sequence number, got = %d, want = %d", s, seq) } } } -// AckNum creates a checker that checks the ack number. -func AckNum(seq uint32) TransportChecker { +// TCPAckNum creates a checker that checks the ack number. +func TCPAckNum(seq uint32) TransportChecker { return func(t *testing.T, h header.Transport) { t.Helper() tcp, ok := h.(header.TCP) if !ok { - return + t.Fatalf("TCP header not found in h: %T", h) } if s := tcp.AckNumber(); s != seq { - t.Errorf("Bad ack number, got %v, want %v", s, seq) + t.Errorf("Bad ack number, got = %d, want = %d", s, seq) } } } -// Window creates a checker that checks the tcp window. -func Window(window uint16) TransportChecker { +// TCPWindow creates a checker that checks the tcp window. +func TCPWindow(window uint16) TransportChecker { return func(t *testing.T, h header.Transport) { t.Helper() tcp, ok := h.(header.TCP) if !ok { - return + t.Fatalf("TCP header not found in hdr : %T", h) } if w := tcp.WindowSize(); w != window { - t.Errorf("Bad window, got 0x%x, want 0x%x", w, window) + t.Errorf("Bad window, got %d, want %d", w, window) + } + } +} + +// TCPWindowGreaterThanEq creates a checker that checks that the TCP window +// is greater than or equal to the provided value. +func TCPWindowGreaterThanEq(window uint16) TransportChecker { + return func(t *testing.T, h header.Transport) { + t.Helper() + + tcp, ok := h.(header.TCP) + if !ok { + t.Fatalf("TCP header not found in h: %T", h) + } + + if w := tcp.WindowSize(); w < window { + t.Errorf("Bad window, got %d, want > %d", w, window) + } + } +} + +// TCPWindowLessThanEq creates a checker that checks that the tcp window +// is less than or equal to the provided value. +func TCPWindowLessThanEq(window uint16) TransportChecker { + return func(t *testing.T, h header.Transport) { + t.Helper() + + tcp, ok := h.(header.TCP) + if !ok { + t.Fatalf("TCP header not found in h: %T", h) + } + + if w := tcp.WindowSize(); w > window { + t.Errorf("Bad window, got %d, want < %d", w, window) } } } @@ -403,7 +481,7 @@ func TCPFlags(flags uint8) TransportChecker { tcp, ok := h.(header.TCP) if !ok { - return + t.Fatalf("TCP header not found in h: %T", h) } if f := tcp.Flags(); f != flags { @@ -420,7 +498,7 @@ func TCPFlagsMatch(flags, mask uint8) TransportChecker { tcp, ok := h.(header.TCP) if !ok { - return + t.Fatalf("TCP header not found in h: %T", h) } if f := tcp.Flags(); (f & mask) != (flags & mask) { @@ -458,7 +536,7 @@ func TCPSynOptions(wantOpts header.TCPSynOptions) TransportChecker { case header.TCPOptionMSS: v := uint16(opts[i+2])<<8 | uint16(opts[i+3]) if wantOpts.MSS != v { - t.Errorf("Bad MSS: got %v, want %v", v, wantOpts.MSS) + t.Errorf("Bad MSS, got = %d, want = %d", v, wantOpts.MSS) } foundMSS = true i += 4 @@ -468,7 +546,7 @@ func TCPSynOptions(wantOpts header.TCPSynOptions) TransportChecker { } v := int(opts[i+2]) if v != wantOpts.WS { - t.Errorf("Bad WS: got %v, want %v", v, wantOpts.WS) + t.Errorf("Bad WS, got = %d, want = %d", v, wantOpts.WS) } foundWS = true i += 3 @@ -517,7 +595,7 @@ func TCPSynOptions(wantOpts header.TCPSynOptions) TransportChecker { t.Error("TS option specified but the timestamp value is zero") } if foundTS && tsEcr == 0 && wantOpts.TSEcr != 0 { - t.Errorf("TS option specified but TSEcr is incorrect: got %d, want: %d", tsEcr, wantOpts.TSEcr) + t.Errorf("TS option specified but TSEcr is incorrect, got = %d, want = %d", tsEcr, wantOpts.TSEcr) } if wantOpts.SACKPermitted && !foundSACKPermitted { t.Errorf("SACKPermitted option not found. Options: %x", opts) @@ -555,7 +633,7 @@ func TCPTimestampChecker(wantTS bool, wantTSVal uint32, wantTSEcr uint32) Transp t.Errorf("TS option found, but option is truncated, option length: %d, want 10 bytes", limit-i) } if opts[i+1] != 10 { - t.Errorf("TS option found, but bad length specified: %d, want: 10", opts[i+1]) + t.Errorf("TS option found, but bad length specified: got = %d, want = 10", opts[i+1]) } tsVal = binary.BigEndian.Uint32(opts[i+2:]) tsEcr = binary.BigEndian.Uint32(opts[i+6:]) @@ -575,13 +653,13 @@ func TCPTimestampChecker(wantTS bool, wantTSVal uint32, wantTSEcr uint32) Transp } if wantTS != foundTS { - t.Errorf("TS Option mismatch: got TS= %v, want TS= %v", foundTS, wantTS) + t.Errorf("TS Option mismatch, got TS= %t, want TS= %t", foundTS, wantTS) } if wantTS && wantTSVal != 0 && wantTSVal != tsVal { - t.Errorf("Timestamp value is incorrect: got: %d, want: %d", tsVal, wantTSVal) + t.Errorf("Timestamp value is incorrect, got = %d, want = %d", tsVal, wantTSVal) } if wantTS && wantTSEcr != 0 && tsEcr != wantTSEcr { - t.Errorf("Timestamp Echo Reply is incorrect: got: %d, want: %d", tsEcr, wantTSEcr) + t.Errorf("Timestamp Echo Reply is incorrect, got = %d, want = %d", tsEcr, wantTSEcr) } } } @@ -645,7 +723,7 @@ func TCPSACKBlockChecker(sackBlocks []header.SACKBlock) TransportChecker { } if !reflect.DeepEqual(gotSACKBlocks, sackBlocks) { - t.Errorf("SACKBlocks are not equal, got: %v, want: %v", gotSACKBlocks, sackBlocks) + t.Errorf("SACKBlocks are not equal, got = %v, want = %v", gotSACKBlocks, sackBlocks) } } } @@ -690,10 +768,10 @@ func ICMPv4Type(want header.ICMPv4Type) TransportChecker { icmpv4, ok := h.(header.ICMPv4) if !ok { - t.Fatalf("unexpected transport header passed to checker got: %+v, want: header.ICMPv4", h) + t.Fatalf("unexpected transport header passed to checker, got = %T, want = header.ICMPv4", h) } if got := icmpv4.Type(); got != want { - t.Fatalf("unexpected icmp type got: %d, want: %d", got, want) + t.Fatalf("unexpected icmp type, got = %d, want = %d", got, want) } } } @@ -705,10 +783,76 @@ func ICMPv4Code(want header.ICMPv4Code) TransportChecker { icmpv4, ok := h.(header.ICMPv4) if !ok { - t.Fatalf("unexpected transport header passed to checker got: %+v, want: header.ICMPv4", h) + t.Fatalf("unexpected transport header passed to checker, got = %T, want = header.ICMPv4", h) } if got := icmpv4.Code(); got != want { - t.Fatalf("unexpected ICMP code got: %d, want: %d", got, want) + t.Fatalf("unexpected ICMP code, got = %d, want = %d", got, want) + } + } +} + +// ICMPv4Ident creates a checker that checks the ICMPv4 echo Ident. +func ICMPv4Ident(want uint16) TransportChecker { + return func(t *testing.T, h header.Transport) { + t.Helper() + + icmpv4, ok := h.(header.ICMPv4) + if !ok { + t.Fatalf("unexpected transport header passed to checker, got = %T, want = header.ICMPv4", h) + } + if got := icmpv4.Ident(); got != want { + t.Fatalf("unexpected ICMP ident, got = %d, want = %d", got, want) + } + } +} + +// ICMPv4Seq creates a checker that checks the ICMPv4 echo Sequence. +func ICMPv4Seq(want uint16) TransportChecker { + return func(t *testing.T, h header.Transport) { + t.Helper() + + icmpv4, ok := h.(header.ICMPv4) + if !ok { + t.Fatalf("unexpected transport header passed to checker, got = %T, want = header.ICMPv4", h) + } + if got := icmpv4.Sequence(); got != want { + t.Fatalf("unexpected ICMP sequence, got = %d, want = %d", got, want) + } + } +} + +// ICMPv4Checksum creates a checker that checks the ICMPv4 Checksum. +// This assumes that the payload exactly makes up the rest of the slice. +func ICMPv4Checksum() TransportChecker { + return func(t *testing.T, h header.Transport) { + t.Helper() + + icmpv4, ok := h.(header.ICMPv4) + if !ok { + t.Fatalf("unexpected transport header passed to checker, got = %T, want = header.ICMPv4", h) + } + heldChecksum := icmpv4.Checksum() + icmpv4.SetChecksum(0) + newChecksum := ^header.Checksum(icmpv4, 0) + icmpv4.SetChecksum(heldChecksum) + if heldChecksum != newChecksum { + t.Errorf("unexpected ICMP checksum, got = %d, want = %d", heldChecksum, newChecksum) + } + } +} + +// ICMPv4Payload creates a checker that checks the payload in an ICMPv4 packet. +func ICMPv4Payload(want []byte) TransportChecker { + return func(t *testing.T, h header.Transport) { + t.Helper() + + icmpv4, ok := h.(header.ICMPv4) + if !ok { + t.Fatalf("unexpected transport header passed to checker, got = %T, want = header.ICMPv4", h) + } + payload := icmpv4.Payload() + if diff := cmp.Diff(payload, want); diff != "" { + t.Errorf("got ICMP payload mismatch (-want +got):\n%s", diff) } } } @@ -748,10 +892,10 @@ func ICMPv6Type(want header.ICMPv6Type) TransportChecker { icmpv6, ok := h.(header.ICMPv6) if !ok { - t.Fatalf("unexpected transport header passed to checker got: %+v, want: header.ICMPv6", h) + t.Fatalf("unexpected transport header passed to checker, got = %T, want = header.ICMPv6", h) } if got := icmpv6.Type(); got != want { - t.Fatalf("unexpected icmp type got: %d, want: %d", got, want) + t.Fatalf("unexpected icmp type, got = %d, want = %d", got, want) } } } @@ -763,10 +907,10 @@ func ICMPv6Code(want header.ICMPv6Code) TransportChecker { icmpv6, ok := h.(header.ICMPv6) if !ok { - t.Fatalf("unexpected transport header passed to checker got: %+v, want: header.ICMPv6", h) + t.Fatalf("unexpected transport header passed to checker, got = %T, want = header.ICMPv6", h) } if got := icmpv6.Code(); got != want { - t.Fatalf("unexpected ICMP code got: %d, want: %d", got, want) + t.Fatalf("unexpected ICMP code, got = %d, want = %d", got, want) } } } diff --git a/pkg/tcpip/faketime/BUILD b/pkg/tcpip/faketime/BUILD new file mode 100644 index 000000000..114d43df3 --- /dev/null +++ b/pkg/tcpip/faketime/BUILD @@ -0,0 +1,24 @@ +load("//tools:defs.bzl", "go_library", "go_test") + +package(licenses = ["notice"]) + +go_library( + name = "faketime", + srcs = ["faketime.go"], + visibility = ["//visibility:public"], + deps = [ + "//pkg/tcpip", + "@com_github_dpjacques_clockwork//:go_default_library", + ], +) + +go_test( + name = "faketime_test", + size = "small", + srcs = [ + "faketime_test.go", + ], + deps = [ + "//pkg/tcpip/faketime", + ], +) diff --git a/pkg/tcpip/stack/fake_time_test.go b/pkg/tcpip/faketime/faketime.go index 92c8cb534..f7a4fbde1 100644 --- a/pkg/tcpip/stack/fake_time_test.go +++ b/pkg/tcpip/faketime/faketime.go @@ -12,7 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -package stack +// Package faketime provides a fake clock that implements tcpip.Clock interface. +package faketime import ( "container/heap" @@ -23,7 +24,29 @@ import ( "gvisor.dev/gvisor/pkg/tcpip" ) -type fakeClock struct { +// NullClock implements a clock that never advances. +type NullClock struct{} + +var _ tcpip.Clock = (*NullClock)(nil) + +// NowNanoseconds implements tcpip.Clock.NowNanoseconds. +func (*NullClock) NowNanoseconds() int64 { + return 0 +} + +// NowMonotonic implements tcpip.Clock.NowMonotonic. +func (*NullClock) NowMonotonic() int64 { + return 0 +} + +// AfterFunc implements tcpip.Clock.AfterFunc. +func (*NullClock) AfterFunc(time.Duration, func()) tcpip.Timer { + return nil +} + +// ManualClock implements tcpip.Clock and only advances manually with Advance +// method. +type ManualClock struct { clock clockwork.FakeClock // mu protects the fields below. @@ -39,34 +62,35 @@ type fakeClock struct { waitGroups map[time.Time]*sync.WaitGroup } -func newFakeClock() *fakeClock { - return &fakeClock{ +// NewManualClock creates a new ManualClock instance. +func NewManualClock() *ManualClock { + return &ManualClock{ clock: clockwork.NewFakeClock(), times: &timeHeap{}, waitGroups: make(map[time.Time]*sync.WaitGroup), } } -var _ tcpip.Clock = (*fakeClock)(nil) +var _ tcpip.Clock = (*ManualClock)(nil) // NowNanoseconds implements tcpip.Clock.NowNanoseconds. -func (fc *fakeClock) NowNanoseconds() int64 { - return fc.clock.Now().UnixNano() +func (mc *ManualClock) NowNanoseconds() int64 { + return mc.clock.Now().UnixNano() } // NowMonotonic implements tcpip.Clock.NowMonotonic. -func (fc *fakeClock) NowMonotonic() int64 { - return fc.NowNanoseconds() +func (mc *ManualClock) NowMonotonic() int64 { + return mc.NowNanoseconds() } // AfterFunc implements tcpip.Clock.AfterFunc. -func (fc *fakeClock) AfterFunc(d time.Duration, f func()) tcpip.Timer { - until := fc.clock.Now().Add(d) - wg := fc.addWait(until) - return &fakeTimer{ - clock: fc, +func (mc *ManualClock) AfterFunc(d time.Duration, f func()) tcpip.Timer { + until := mc.clock.Now().Add(d) + wg := mc.addWait(until) + return &manualTimer{ + clock: mc, until: until, - timer: fc.clock.AfterFunc(d, func() { + timer: mc.clock.AfterFunc(d, func() { defer wg.Done() f() }), @@ -75,110 +99,113 @@ func (fc *fakeClock) AfterFunc(d time.Duration, f func()) tcpip.Timer { // addWait adds an additional wait to the WaitGroup for parallel execution of // all work scheduled for t. Returns a reference to the WaitGroup modified. -func (fc *fakeClock) addWait(t time.Time) *sync.WaitGroup { - fc.mu.RLock() - wg, ok := fc.waitGroups[t] - fc.mu.RUnlock() +func (mc *ManualClock) addWait(t time.Time) *sync.WaitGroup { + mc.mu.RLock() + wg, ok := mc.waitGroups[t] + mc.mu.RUnlock() if ok { wg.Add(1) return wg } - fc.mu.Lock() - heap.Push(fc.times, t) - fc.mu.Unlock() + mc.mu.Lock() + heap.Push(mc.times, t) + mc.mu.Unlock() wg = &sync.WaitGroup{} wg.Add(1) - fc.mu.Lock() - fc.waitGroups[t] = wg - fc.mu.Unlock() + mc.mu.Lock() + mc.waitGroups[t] = wg + mc.mu.Unlock() return wg } // removeWait removes a wait from the WaitGroup for parallel execution of all // work scheduled for t. -func (fc *fakeClock) removeWait(t time.Time) { - fc.mu.RLock() - defer fc.mu.RUnlock() +func (mc *ManualClock) removeWait(t time.Time) { + mc.mu.RLock() + defer mc.mu.RUnlock() - wg := fc.waitGroups[t] + wg := mc.waitGroups[t] wg.Done() } -// advance executes all work that have been scheduled to execute within d from -// the current fake time. Blocks until all work has completed execution. -func (fc *fakeClock) advance(d time.Duration) { +// Advance executes all work that have been scheduled to execute within d from +// the current time. Blocks until all work has completed execution. +func (mc *ManualClock) Advance(d time.Duration) { // Block until all the work is done - until := fc.clock.Now().Add(d) + until := mc.clock.Now().Add(d) for { - fc.mu.Lock() - if fc.times.Len() == 0 { - fc.mu.Unlock() - return + mc.mu.Lock() + if mc.times.Len() == 0 { + mc.mu.Unlock() + break } - t := heap.Pop(fc.times).(time.Time) + t := heap.Pop(mc.times).(time.Time) if t.After(until) { // No work to do - heap.Push(fc.times, t) - fc.mu.Unlock() - return + heap.Push(mc.times, t) + mc.mu.Unlock() + break } - fc.mu.Unlock() + mc.mu.Unlock() - diff := t.Sub(fc.clock.Now()) - fc.clock.Advance(diff) + diff := t.Sub(mc.clock.Now()) + mc.clock.Advance(diff) - fc.mu.RLock() - wg := fc.waitGroups[t] - fc.mu.RUnlock() + mc.mu.RLock() + wg := mc.waitGroups[t] + mc.mu.RUnlock() wg.Wait() - fc.mu.Lock() - delete(fc.waitGroups, t) - fc.mu.Unlock() + mc.mu.Lock() + delete(mc.waitGroups, t) + mc.mu.Unlock() + } + if now := mc.clock.Now(); until.After(now) { + mc.clock.Advance(until.Sub(now)) } } -type fakeTimer struct { - clock *fakeClock +type manualTimer struct { + clock *ManualClock timer clockwork.Timer mu sync.RWMutex until time.Time } -var _ tcpip.Timer = (*fakeTimer)(nil) +var _ tcpip.Timer = (*manualTimer)(nil) // Reset implements tcpip.Timer.Reset. -func (ft *fakeTimer) Reset(d time.Duration) { - if !ft.timer.Reset(d) { +func (t *manualTimer) Reset(d time.Duration) { + if !t.timer.Reset(d) { return } - ft.mu.Lock() - defer ft.mu.Unlock() + t.mu.Lock() + defer t.mu.Unlock() - ft.clock.removeWait(ft.until) - ft.until = ft.clock.clock.Now().Add(d) - ft.clock.addWait(ft.until) + t.clock.removeWait(t.until) + t.until = t.clock.clock.Now().Add(d) + t.clock.addWait(t.until) } // Stop implements tcpip.Timer.Stop. -func (ft *fakeTimer) Stop() bool { - if !ft.timer.Stop() { +func (t *manualTimer) Stop() bool { + if !t.timer.Stop() { return false } - ft.mu.RLock() - defer ft.mu.RUnlock() + t.mu.RLock() + defer t.mu.RUnlock() - ft.clock.removeWait(ft.until) + t.clock.removeWait(t.until) return true } diff --git a/pkg/tcpip/faketime/faketime_test.go b/pkg/tcpip/faketime/faketime_test.go new file mode 100644 index 000000000..c2704df2c --- /dev/null +++ b/pkg/tcpip/faketime/faketime_test.go @@ -0,0 +1,95 @@ +// 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 faketime_test + +import ( + "testing" + "time" + + "gvisor.dev/gvisor/pkg/tcpip/faketime" +) + +func TestManualClockAdvance(t *testing.T) { + const timeout = time.Millisecond + clock := faketime.NewManualClock() + start := clock.NowMonotonic() + clock.Advance(timeout) + if got, want := time.Duration(clock.NowMonotonic()-start)*time.Nanosecond, timeout; got != want { + t.Errorf("got = %d, want = %d", got, want) + } +} + +func TestManualClockAfterFunc(t *testing.T) { + const ( + timeout1 = time.Millisecond // timeout for counter1 + timeout2 = 2 * time.Millisecond // timeout for counter2 + ) + tests := []struct { + name string + advance time.Duration + wantCounter1 int + wantCounter2 int + }{ + { + name: "before timeout1", + advance: timeout1 - 1, + wantCounter1: 0, + wantCounter2: 0, + }, + { + name: "timeout1", + advance: timeout1, + wantCounter1: 1, + wantCounter2: 0, + }, + { + name: "timeout2", + advance: timeout2, + wantCounter1: 1, + wantCounter2: 1, + }, + { + name: "after timeout2", + advance: timeout2 + 1, + wantCounter1: 1, + wantCounter2: 1, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + clock := faketime.NewManualClock() + counter1 := 0 + counter2 := 0 + clock.AfterFunc(timeout1, func() { + counter1++ + }) + clock.AfterFunc(timeout2, func() { + counter2++ + }) + start := clock.NowMonotonic() + clock.Advance(test.advance) + if got, want := counter1, test.wantCounter1; got != want { + t.Errorf("got counter1 = %d, want = %d", got, want) + } + if got, want := counter2, test.wantCounter2; got != want { + t.Errorf("got counter2 = %d, want = %d", got, want) + } + if got, want := time.Duration(clock.NowMonotonic()-start)*time.Nanosecond, test.advance; got != want { + t.Errorf("got elapsed = %d, want = %d", got, want) + } + }) + } +} diff --git a/pkg/tcpip/header/icmpv4.go b/pkg/tcpip/header/icmpv4.go index be03fb086..504408878 100644 --- a/pkg/tcpip/header/icmpv4.go +++ b/pkg/tcpip/header/icmpv4.go @@ -31,6 +31,27 @@ const ( // ICMPv4MinimumSize is the minimum size of a valid ICMP packet. ICMPv4MinimumSize = 8 + // ICMPv4MinimumErrorPayloadSize Is the smallest number of bytes of an + // errant packet's transport layer that an ICMP error type packet should + // attempt to send as per RFC 792 (see each type) and RFC 1122 + // section 3.2.2 which states: + // Every ICMP error message includes the Internet header and at + // least the first 8 data octets of the datagram that triggered + // the error; more than 8 octets MAY be sent; this header and data + // MUST be unchanged from the received datagram. + // + // RFC 792 shows: + // 0 1 2 3 + // 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 + // +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + // | Type | Code | Checksum | + // +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + // | unused | + // +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + // | Internet Header + 64 bits of Original Data Datagram | + // +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + ICMPv4MinimumErrorPayloadSize = 8 + // ICMPv4ProtocolNumber is the ICMP transport protocol number. ICMPv4ProtocolNumber tcpip.TransportProtocolNumber = 1 @@ -39,15 +60,19 @@ const ( icmpv4ChecksumOffset = 2 // icmpv4MTUOffset is the offset of the MTU field - // in a ICMPv4FragmentationNeeded message. + // in an ICMPv4FragmentationNeeded message. icmpv4MTUOffset = 6 // icmpv4IdentOffset is the offset of the ident field - // in a ICMPv4EchoRequest/Reply message. + // in an ICMPv4EchoRequest/Reply message. icmpv4IdentOffset = 4 + // icmpv4PointerOffset is the offset of the pointer field + // in an ICMPv4ParamProblem message. + icmpv4PointerOffset = 4 + // icmpv4SequenceOffset is the offset of the sequence field - // in a ICMPv4EchoRequest/Reply message. + // in an ICMPv4EchoRequest/Reply message. icmpv4SequenceOffset = 6 ) @@ -72,15 +97,23 @@ const ( ICMPv4InfoReply ICMPv4Type = 16 ) +// ICMP codes for ICMPv4 Time Exceeded messages as defined in RFC 792. +const ( + ICMPv4TTLExceeded ICMPv4Code = 0 +) + // ICMP codes for ICMPv4 Destination Unreachable messages as defined in RFC 792. const ( - ICMPv4TTLExceeded ICMPv4Code = 0 + ICMPv4NetUnreachable ICMPv4Code = 0 ICMPv4HostUnreachable ICMPv4Code = 1 ICMPv4ProtoUnreachable ICMPv4Code = 2 ICMPv4PortUnreachable ICMPv4Code = 3 ICMPv4FragmentationNeeded ICMPv4Code = 4 ) +// ICMPv4UnusedCode is a code to use in ICMP messages where no code is needed. +const ICMPv4UnusedCode ICMPv4Code = 0 + // Type is the ICMP type field. func (b ICMPv4) Type() ICMPv4Type { return ICMPv4Type(b[0]) } diff --git a/pkg/tcpip/header/icmpv6.go b/pkg/tcpip/header/icmpv6.go index 20b01d8f4..6be31beeb 100644 --- a/pkg/tcpip/header/icmpv6.go +++ b/pkg/tcpip/header/icmpv6.go @@ -54,9 +54,17 @@ const ( // address. ICMPv6NeighborAdvertSize = ICMPv6HeaderSize + NDPNAMinimumSize + NDPLinkLayerAddressSize - // ICMPv6EchoMinimumSize is the minimum size of a valid ICMP echo packet. + // ICMPv6EchoMinimumSize is the minimum size of a valid echo packet. ICMPv6EchoMinimumSize = 8 + // ICMPv6ErrorHeaderSize is the size of an ICMP error packet header, + // as per RFC 4443, Apendix A, item 4 and the errata. + // ... all ICMP error messages shall have exactly + // 32 bits of type-specific data, so that receivers can reliably find + // the embedded invoking packet even when they don't recognize the + // ICMP message Type. + ICMPv6ErrorHeaderSize = 8 + // ICMPv6DstUnreachableMinimumSize is the minimum size of a valid ICMP // destination unreachable packet. ICMPv6DstUnreachableMinimumSize = ICMPv6MinimumSize @@ -69,6 +77,10 @@ const ( // in an ICMPv6 message. icmpv6ChecksumOffset = 2 + // icmpv6PointerOffset is the offset of the pointer + // in an ICMPv6 Parameter problem message. + icmpv6PointerOffset = 4 + // icmpv6MTUOffset is the offset of the MTU field in an ICMPv6 // PacketTooBig message. icmpv6MTUOffset = 4 @@ -89,9 +101,10 @@ const ( NDPHopLimit = 255 ) -// ICMPv6Type is the ICMP type field described in RFC 4443 and friends. +// ICMPv6Type is the ICMP type field described in RFC 4443. type ICMPv6Type byte +// Values for use in the Type field of ICMPv6 packet from RFC 4433. const ( ICMPv6DstUnreachable ICMPv6Type = 1 ICMPv6PacketTooBig ICMPv6Type = 2 @@ -109,7 +122,18 @@ const ( ICMPv6RedirectMsg ICMPv6Type = 137 ) -// ICMPv6Code is the ICMP code field described in RFC 4443. +// IsErrorType returns true if the receiver is an ICMP error type. +func (typ ICMPv6Type) IsErrorType() bool { + // Per RFC 4443 section 2.1: + // ICMPv6 messages are grouped into two classes: error messages and + // informational messages. Error messages are identified as such by a + // zero in the high-order bit of their message Type field values. Thus, + // error messages have message types from 0 to 127; informational + // messages have message types from 128 to 255. + return typ&0x80 == 0 +} + +// ICMPv6Code is the ICMP Code field described in RFC 4443. type ICMPv6Code byte // ICMP codes used with Destination Unreachable (Type 1). As per RFC 4443 @@ -132,9 +156,14 @@ const ( // ICMP codes used with Parameter Problem (Type 4). As per RFC 4443 section 3.4. const ( + // ICMPv6ErroneousHeader indicates an erroneous header field was encountered. ICMPv6ErroneousHeader ICMPv6Code = 0 - ICMPv6UnknownHeader ICMPv6Code = 1 - ICMPv6UnknownOption ICMPv6Code = 2 + + // ICMPv6UnknownHeader indicates an unrecognized Next Header type encountered. + ICMPv6UnknownHeader ICMPv6Code = 1 + + // ICMPv6UnknownOption indicates an unrecognized IPv6 option was encountered. + ICMPv6UnknownOption ICMPv6Code = 2 ) // ICMPv6UnusedCode is the code value used with ICMPv6 messages which don't use @@ -153,6 +182,16 @@ func (b ICMPv6) Code() ICMPv6Code { return ICMPv6Code(b[1]) } // SetCode sets the ICMP code field. func (b ICMPv6) SetCode(c ICMPv6Code) { b[1] = byte(c) } +// TypeSpecific returns the type specific data field. +func (b ICMPv6) TypeSpecific() uint32 { + return binary.BigEndian.Uint32(b[icmpv6PointerOffset:]) +} + +// SetTypeSpecific sets the type specific data field. +func (b ICMPv6) SetTypeSpecific(val uint32) { + binary.BigEndian.PutUint32(b[icmpv6PointerOffset:], val) +} + // Checksum is the ICMP checksum field. func (b ICMPv6) Checksum() uint16 { return binary.BigEndian.Uint16(b[icmpv6ChecksumOffset:]) diff --git a/pkg/tcpip/header/ipv4.go b/pkg/tcpip/header/ipv4.go index e8816c3f4..b07d9991d 100644 --- a/pkg/tcpip/header/ipv4.go +++ b/pkg/tcpip/header/ipv4.go @@ -80,7 +80,8 @@ type IPv4Fields struct { type IPv4 []byte const ( - // IPv4MinimumSize is the minimum size of a valid IPv4 packet. + // IPv4MinimumSize is the minimum size of a valid IPv4 packet; + // i.e. a packet header with no options. IPv4MinimumSize = 20 // IPv4MaximumHeaderSize is the maximum size of an IPv4 header. Given @@ -327,7 +328,7 @@ func IsV4MulticastAddress(addr tcpip.Address) bool { } // IsV4LoopbackAddress determines if the provided address is an IPv4 loopback -// address (belongs to 127.0.0.1/8 subnet). +// address (belongs to 127.0.0.0/8 subnet). See RFC 1122 section 3.2.1.3. func IsV4LoopbackAddress(addr tcpip.Address) bool { if len(addr) != IPv4AddressSize { return false diff --git a/pkg/tcpip/header/ipv6.go b/pkg/tcpip/header/ipv6.go index 0761a1807..ef454b313 100644 --- a/pkg/tcpip/header/ipv6.go +++ b/pkg/tcpip/header/ipv6.go @@ -34,6 +34,9 @@ const ( hopLimit = 7 v6SrcAddr = 8 v6DstAddr = v6SrcAddr + IPv6AddressSize + + // IPv6FixedHeaderSize is the size of the fixed header. + IPv6FixedHeaderSize = v6DstAddr + IPv6AddressSize ) // IPv6Fields contains the fields of an IPv6 packet. It is used to describe the @@ -69,7 +72,7 @@ type IPv6 []byte const ( // IPv6MinimumSize is the minimum size of a valid IPv6 packet. - IPv6MinimumSize = 40 + IPv6MinimumSize = IPv6FixedHeaderSize // IPv6AddressSize is the size, in bytes, of an IPv6 address. IPv6AddressSize = 16 diff --git a/pkg/tcpip/header/ipv6_extension_headers.go b/pkg/tcpip/header/ipv6_extension_headers.go index 3499d8399..583c2c5d3 100644 --- a/pkg/tcpip/header/ipv6_extension_headers.go +++ b/pkg/tcpip/header/ipv6_extension_headers.go @@ -149,6 +149,19 @@ func (b ipv6OptionsExtHdr) Iter() IPv6OptionsExtHdrOptionsIterator { // obtained before modification is no longer used. type IPv6OptionsExtHdrOptionsIterator struct { reader bytes.Reader + + // optionOffset is the number of bytes from the first byte of the + // options field to the beginning of the current option. + optionOffset uint32 + + // nextOptionOffset is the offset of the next option. + nextOptionOffset uint32 +} + +// OptionOffset returns the number of bytes parsed while processing the +// option field of the current Extension Header. +func (i *IPv6OptionsExtHdrOptionsIterator) OptionOffset() uint32 { + return i.optionOffset } // IPv6OptionUnknownAction is the action that must be taken if the processing @@ -226,6 +239,7 @@ func (*IPv6UnknownExtHdrOption) isIPv6ExtHdrOption() {} // the options data, or an error occured. func (i *IPv6OptionsExtHdrOptionsIterator) Next() (IPv6ExtHdrOption, bool, error) { for { + i.optionOffset = i.nextOptionOffset temp, err := i.reader.ReadByte() if err != nil { // If we can't read the first byte of a new option, then we know the @@ -238,6 +252,7 @@ func (i *IPv6OptionsExtHdrOptionsIterator) Next() (IPv6ExtHdrOption, bool, error // know the option does not have Length and Data fields. End processing of // the Pad1 option and continue processing the buffer as a new option. if id == ipv6Pad1ExtHdrOptionIdentifier { + i.nextOptionOffset = i.optionOffset + 1 continue } @@ -254,41 +269,40 @@ func (i *IPv6OptionsExtHdrOptionsIterator) Next() (IPv6ExtHdrOption, bool, error return nil, true, fmt.Errorf("error when reading the option's Length field for option with id = %d: %w", id, io.ErrUnexpectedEOF) } - // Special-case the variable length padding option to avoid a copy. - if id == ipv6PadNExtHdrOptionIdentifier { - // Do we have enough bytes in the reader for the PadN option? - if n := i.reader.Len(); n < int(length) { - // Reset the reader to effectively consume the remaining buffer. - i.reader.Reset(nil) - - // We return the same error as if we failed to read a non-padding option - // so consumers of this iterator don't need to differentiate between - // padding and non-padding options. - return nil, true, fmt.Errorf("read %d out of %d option data bytes for option with id = %d: %w", n, length, id, io.ErrUnexpectedEOF) - } + // Do we have enough bytes in the reader for the next option? + if n := i.reader.Len(); n < int(length) { + // Reset the reader to effectively consume the remaining buffer. + i.reader.Reset(nil) + + // We return the same error as if we failed to read a non-padding option + // so consumers of this iterator don't need to differentiate between + // padding and non-padding options. + return nil, true, fmt.Errorf("read %d out of %d option data bytes for option with id = %d: %w", n, length, id, io.ErrUnexpectedEOF) + } + + i.nextOptionOffset = i.optionOffset + uint32(length) + 1 /* option ID */ + 1 /* length byte */ + switch id { + case ipv6PadNExtHdrOptionIdentifier: + // Special-case the variable length padding option to avoid a copy. if _, err := i.reader.Seek(int64(length), io.SeekCurrent); err != nil { panic(fmt.Sprintf("error when skipping PadN (N = %d) option's data bytes: %s", length, err)) } - - // End processing of the PadN option and continue processing the buffer as - // a new option. continue - } - - bytes := make([]byte, length) - if n, err := io.ReadFull(&i.reader, bytes); err != nil { - // io.ReadFull may return io.EOF if i.reader has been exhausted. We use - // io.ErrUnexpectedEOF instead as the io.EOF is unexpected given the - // Length field found in the option. - if err == io.EOF { - err = io.ErrUnexpectedEOF + default: + bytes := make([]byte, length) + if n, err := io.ReadFull(&i.reader, bytes); err != nil { + // io.ReadFull may return io.EOF if i.reader has been exhausted. We use + // io.ErrUnexpectedEOF instead as the io.EOF is unexpected given the + // Length field found in the option. + if err == io.EOF { + err = io.ErrUnexpectedEOF + } + + return nil, true, fmt.Errorf("read %d out of %d option data bytes for option with id = %d: %w", n, length, id, err) } - - return nil, true, fmt.Errorf("read %d out of %d option data bytes for option with id = %d: %w", n, length, id, err) + return &IPv6UnknownExtHdrOption{Identifier: id, Data: bytes}, false, nil } - - return &IPv6UnknownExtHdrOption{Identifier: id, Data: bytes}, false, nil } } @@ -382,6 +396,29 @@ type IPv6PayloadIterator struct { // Indicates to the iterator that it should return the remaining payload as a // raw payload on the next call to Next. forceRaw bool + + // headerOffset is the offset of the beginning of the current extension + // header starting from the beginning of the fixed header. + headerOffset uint32 + + // parseOffset is the byte offset into the current extension header of the + // field we are currently examining. It can be added to the header offset + // if the absolute offset within the packet is required. + parseOffset uint32 + + // nextOffset is the offset of the next header. + nextOffset uint32 +} + +// HeaderOffset returns the offset to the start of the extension +// header most recently processed. +func (i IPv6PayloadIterator) HeaderOffset() uint32 { + return i.headerOffset +} + +// ParseOffset returns the number of bytes successfully parsed. +func (i IPv6PayloadIterator) ParseOffset() uint32 { + return i.headerOffset + i.parseOffset } // MakeIPv6PayloadIterator returns an iterator over the IPv6 payload containing @@ -397,7 +434,8 @@ func MakeIPv6PayloadIterator(nextHdrIdentifier IPv6ExtensionHeaderIdentifier, pa nextHdrIdentifier: nextHdrIdentifier, payload: payload.Clone(nil), // We need a buffer of size 1 for calls to bufio.Reader.ReadByte. - reader: *bufio.NewReaderSize(io.MultiReader(readerPs...), 1), + reader: *bufio.NewReaderSize(io.MultiReader(readerPs...), 1), + nextOffset: IPv6FixedHeaderSize, } } @@ -434,6 +472,8 @@ func (i *IPv6PayloadIterator) AsRawHeader(consume bool) IPv6RawPayloadHeader { // 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) { + i.headerOffset = i.nextOffset + i.parseOffset = 0 // 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. @@ -461,7 +501,7 @@ func (i *IPv6PayloadIterator) Next() (IPv6PayloadHeader, bool, error) { return IPv6RoutingExtHdr(bytes), false, nil case IPv6FragmentExtHdrIdentifier: var data [6]byte - // We ignore the returned bytes becauase we know the fragment extension + // We ignore the returned bytes because we know the fragment extension // header specific data will fit in data. nextHdrIdentifier, _, err := i.nextHeaderData(true /* fragmentHdr */, data[:]) if err != nil { @@ -519,10 +559,12 @@ func (i *IPv6PayloadIterator) nextHeaderData(fragmentHdr bool, bytes []byte) (IP 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) } + i.parseOffset++ var length uint8 length, err = i.reader.ReadByte() i.payload.TrimFront(1) + if err != nil { if fragmentHdr { return 0, nil, fmt.Errorf("error when reading the Length field for extension header with id = %d: %w", i.nextHdrIdentifier, err) @@ -534,6 +576,17 @@ func (i *IPv6PayloadIterator) nextHeaderData(fragmentHdr bool, bytes []byte) (IP length = 0 } + // Make parseOffset point to the first byte of the Extension Header + // specific data. + i.parseOffset++ + + // length is in 8 byte chunks but doesn't include the first one. + // See RFC 8200 for each header type, sections 4.3-4.6 and the requirement + // in section 4.8 for new extension headers at the top of page 24. + // [ Hdr Ext Len ] ... Length of the Destination Options header in 8-octet + // units, not including the first 8 octets. + i.nextOffset += uint32((length + 1) * ipv6ExtHdrLenBytesPerUnit) + bytesLen := int(length)*ipv6ExtHdrLenBytesPerUnit + ipv6ExtHdrLenBytesExcluded if bytes == nil { bytes = make([]byte, bytesLen) diff --git a/pkg/tcpip/header/parse/BUILD b/pkg/tcpip/header/parse/BUILD new file mode 100644 index 000000000..2adee9288 --- /dev/null +++ b/pkg/tcpip/header/parse/BUILD @@ -0,0 +1,15 @@ +load("//tools:defs.bzl", "go_library") + +package(licenses = ["notice"]) + +go_library( + name = "parse", + srcs = ["parse.go"], + visibility = ["//visibility:public"], + deps = [ + "//pkg/tcpip", + "//pkg/tcpip/buffer", + "//pkg/tcpip/header", + "//pkg/tcpip/stack", + ], +) diff --git a/pkg/tcpip/header/parse/parse.go b/pkg/tcpip/header/parse/parse.go new file mode 100644 index 000000000..5ca75c834 --- /dev/null +++ b/pkg/tcpip/header/parse/parse.go @@ -0,0 +1,168 @@ +// 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 parse provides utilities to parse packets. +package parse + +import ( + "fmt" + + "gvisor.dev/gvisor/pkg/tcpip" + "gvisor.dev/gvisor/pkg/tcpip/buffer" + "gvisor.dev/gvisor/pkg/tcpip/header" + "gvisor.dev/gvisor/pkg/tcpip/stack" +) + +// ARP populates pkt's network header with an ARP header found in +// pkt.Data. +// +// Returns true if the header was successfully parsed. +func ARP(pkt *stack.PacketBuffer) bool { + _, ok := pkt.NetworkHeader().Consume(header.ARPSize) + if ok { + pkt.NetworkProtocolNumber = header.ARPProtocolNumber + } + return ok +} + +// IPv4 parses an IPv4 packet found in pkt.Data and populates pkt's network +// header with the IPv4 header. +// +// Returns true if the header was successfully parsed. +func IPv4(pkt *stack.PacketBuffer) bool { + hdr, ok := pkt.Data.PullUp(header.IPv4MinimumSize) + if !ok { + return 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 false + } + ipHdr = header.IPv4(hdr) + + pkt.NetworkProtocolNumber = header.IPv4ProtocolNumber + pkt.Data.CapLength(int(ipHdr.TotalLength()) - len(hdr)) + return true +} + +// IPv6 parses an IPv6 packet found in pkt.Data and populates pkt's network +// header with the IPv6 header. +func IPv6(pkt *stack.PacketBuffer) (proto tcpip.TransportProtocolNumber, fragID uint32, fragOffset uint16, fragMore bool, ok bool) { + hdr, ok := pkt.Data.PullUp(header.IPv6MinimumSize) + if !ok { + return 0, 0, 0, false, false + } + ipHdr := header.IPv6(hdr) + + // dataClone consists of: + // - Any IPv6 header bytes after the first 40 (i.e. extensions). + // - The transport header, if present. + // - Any other payload data. + views := [8]buffer.View{} + dataClone := pkt.Data.Clone(views[:]) + dataClone.TrimFront(header.IPv6MinimumSize) + it := header.MakeIPv6PayloadIterator(header.IPv6ExtensionHeaderIdentifier(ipHdr.NextHeader()), dataClone) + + // Iterate over the IPv6 extensions to find their length. + var nextHdr tcpip.TransportProtocolNumber + var extensionsSize int + +traverseExtensions: + for { + extHdr, done, err := it.Next() + if err != nil { + break + } + + // If we exhaust the extension list, the entire packet is the IPv6 header + // and (possibly) extensions. + if done { + extensionsSize = dataClone.Size() + break + } + + switch extHdr := extHdr.(type) { + case header.IPv6FragmentExtHdr: + if fragID == 0 && fragOffset == 0 && !fragMore { + fragID = extHdr.ID() + fragOffset = extHdr.FragmentOffset() + fragMore = extHdr.More() + } + + case header.IPv6RawPayloadHeader: + // We've found the payload after any extensions. + extensionsSize = dataClone.Size() - extHdr.Buf.Size() + nextHdr = tcpip.TransportProtocolNumber(extHdr.Identifier) + break traverseExtensions + + default: + // Any other extension is a no-op, keep looping until we find the payload. + } + } + + // Put the IPv6 header with extensions in pkt.NetworkHeader(). + hdr, ok = pkt.NetworkHeader().Consume(header.IPv6MinimumSize + extensionsSize) + if !ok { + panic(fmt.Sprintf("pkt.Data should have at least %d bytes, but only has %d.", header.IPv6MinimumSize+extensionsSize, pkt.Data.Size())) + } + ipHdr = header.IPv6(hdr) + pkt.Data.CapLength(int(ipHdr.PayloadLength())) + pkt.NetworkProtocolNumber = header.IPv6ProtocolNumber + + return nextHdr, fragID, fragOffset, fragMore, true +} + +// UDP parses a UDP packet found in pkt.Data and populates pkt's transport +// header with the UDP header. +// +// Returns true if the header was successfully parsed. +func UDP(pkt *stack.PacketBuffer) bool { + _, ok := pkt.TransportHeader().Consume(header.UDPMinimumSize) + pkt.TransportProtocolNumber = header.UDPProtocolNumber + return ok +} + +// TCP parses a TCP packet found in pkt.Data and populates pkt's transport +// header with the TCP header. +// +// Returns true if the header was successfully parsed. +func TCP(pkt *stack.PacketBuffer) bool { + // TCP header is variable length, peek at it first. + hdrLen := header.TCPMinimumSize + hdr, ok := pkt.Data.PullUp(hdrLen) + if !ok { + return false + } + + // If the header has options, pull those up as well. + if offset := int(header.TCP(hdr).DataOffset()); offset > header.TCPMinimumSize && offset <= pkt.Data.Size() { + // TODO(gvisor.dev/issue/2404): Figure out whether to reject this kind of + // packets. + hdrLen = offset + } + + _, ok = pkt.TransportHeader().Consume(hdrLen) + pkt.TransportProtocolNumber = header.TCPProtocolNumber + return ok +} diff --git a/pkg/tcpip/link/sniffer/BUILD b/pkg/tcpip/link/sniffer/BUILD index 7cbc305e7..4aac12a8c 100644 --- a/pkg/tcpip/link/sniffer/BUILD +++ b/pkg/tcpip/link/sniffer/BUILD @@ -14,6 +14,7 @@ go_library( "//pkg/tcpip", "//pkg/tcpip/buffer", "//pkg/tcpip/header", + "//pkg/tcpip/header/parse", "//pkg/tcpip/link/nested", "//pkg/tcpip/stack", ], diff --git a/pkg/tcpip/link/sniffer/sniffer.go b/pkg/tcpip/link/sniffer/sniffer.go index 4fb127978..560477926 100644 --- a/pkg/tcpip/link/sniffer/sniffer.go +++ b/pkg/tcpip/link/sniffer/sniffer.go @@ -31,6 +31,7 @@ import ( "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/header" + "gvisor.dev/gvisor/pkg/tcpip/header/parse" "gvisor.dev/gvisor/pkg/tcpip/link/nested" "gvisor.dev/gvisor/pkg/tcpip/stack" ) @@ -195,49 +196,52 @@ func logPacket(prefix string, protocol tcpip.NetworkProtocolNumber, pkt *stack.P var transProto uint8 src := tcpip.Address("unknown") dst := tcpip.Address("unknown") - id := 0 - size := uint16(0) + var size uint16 + var id uint32 var fragmentOffset uint16 var moreFragments bool - // Examine the packet using a new VV. Backing storage must not be written. - vv := buffer.NewVectorisedView(pkt.Size(), pkt.Views()) - + // Clone the packet buffer to not modify the original. + // + // We don't clone the original packet buffer so that the new packet buffer + // does not have any of its headers set. + pkt = stack.NewPacketBuffer(stack.PacketBufferOptions{Data: buffer.NewVectorisedView(pkt.Size(), pkt.Views())}) switch protocol { case header.IPv4ProtocolNumber: - hdr, ok := vv.PullUp(header.IPv4MinimumSize) - if !ok { + if ok := parse.IPv4(pkt); !ok { return } - ipv4 := header.IPv4(hdr) + + ipv4 := header.IPv4(pkt.NetworkHeader().View()) fragmentOffset = ipv4.FragmentOffset() moreFragments = ipv4.Flags()&header.IPv4FlagMoreFragments == header.IPv4FlagMoreFragments src = ipv4.SourceAddress() dst = ipv4.DestinationAddress() transProto = ipv4.Protocol() size = ipv4.TotalLength() - uint16(ipv4.HeaderLength()) - vv.TrimFront(int(ipv4.HeaderLength())) - id = int(ipv4.ID()) + id = uint32(ipv4.ID()) case header.IPv6ProtocolNumber: - hdr, ok := vv.PullUp(header.IPv6MinimumSize) + proto, fragID, fragOffset, fragMore, ok := parse.IPv6(pkt) if !ok { return } - ipv6 := header.IPv6(hdr) + + ipv6 := header.IPv6(pkt.NetworkHeader().View()) src = ipv6.SourceAddress() dst = ipv6.DestinationAddress() - transProto = ipv6.NextHeader() + transProto = uint8(proto) size = ipv6.PayloadLength() - vv.TrimFront(header.IPv6MinimumSize) + id = fragID + moreFragments = fragMore + fragmentOffset = fragOffset case header.ARPProtocolNumber: - hdr, ok := vv.PullUp(header.ARPSize) - if !ok { + if parse.ARP(pkt) { return } - vv.TrimFront(header.ARPSize) - arp := header.ARP(hdr) + + arp := header.ARP(pkt.NetworkHeader().View()) log.Infof( "%s arp %s (%s) -> %s (%s) valid:%t", prefix, @@ -259,7 +263,7 @@ func logPacket(prefix string, protocol tcpip.NetworkProtocolNumber, pkt *stack.P switch tcpip.TransportProtocolNumber(transProto) { case header.ICMPv4ProtocolNumber: transName = "icmp" - hdr, ok := vv.PullUp(header.ICMPv4MinimumSize) + hdr, ok := pkt.Data.PullUp(header.ICMPv4MinimumSize) if !ok { break } @@ -296,7 +300,7 @@ func logPacket(prefix string, protocol tcpip.NetworkProtocolNumber, pkt *stack.P case header.ICMPv6ProtocolNumber: transName = "icmp" - hdr, ok := vv.PullUp(header.ICMPv6MinimumSize) + hdr, ok := pkt.Data.PullUp(header.ICMPv6MinimumSize) if !ok { break } @@ -331,11 +335,11 @@ func logPacket(prefix string, protocol tcpip.NetworkProtocolNumber, pkt *stack.P case header.UDPProtocolNumber: transName = "udp" - hdr, ok := vv.PullUp(header.UDPMinimumSize) - if !ok { + if ok := parse.UDP(pkt); !ok { break } - udp := header.UDP(hdr) + + udp := header.UDP(pkt.TransportHeader().View()) if fragmentOffset == 0 { srcPort = udp.SourcePort() dstPort = udp.DestinationPort() @@ -345,19 +349,19 @@ func logPacket(prefix string, protocol tcpip.NetworkProtocolNumber, pkt *stack.P case header.TCPProtocolNumber: transName = "tcp" - hdr, ok := vv.PullUp(header.TCPMinimumSize) - if !ok { + if ok := parse.TCP(pkt); !ok { break } - tcp := header.TCP(hdr) + + tcp := header.TCP(pkt.TransportHeader().View()) if fragmentOffset == 0 { offset := int(tcp.DataOffset()) if offset < header.TCPMinimumSize { details += fmt.Sprintf("invalid packet: tcp data offset too small %d", offset) break } - if offset > vv.Size() && !moreFragments { - details += fmt.Sprintf("invalid packet: tcp data offset %d larger than packet buffer length %d", offset, vv.Size()) + if size := pkt.Data.Size() + len(tcp); offset > size && !moreFragments { + details += fmt.Sprintf("invalid packet: tcp data offset %d larger than tcp packet length %d", offset, size) break } diff --git a/pkg/tcpip/network/BUILD b/pkg/tcpip/network/BUILD index 46083925c..59710352b 100644 --- a/pkg/tcpip/network/BUILD +++ b/pkg/tcpip/network/BUILD @@ -9,6 +9,7 @@ go_test( "ip_test.go", ], deps = [ + "//pkg/sync", "//pkg/tcpip", "//pkg/tcpip/buffer", "//pkg/tcpip/header", @@ -17,6 +18,7 @@ go_test( "//pkg/tcpip/network/ipv4", "//pkg/tcpip/network/ipv6", "//pkg/tcpip/stack", + "//pkg/tcpip/transport/icmp", "//pkg/tcpip/transport/tcp", "//pkg/tcpip/transport/udp", ], diff --git a/pkg/tcpip/network/arp/BUILD b/pkg/tcpip/network/arp/BUILD index 82c073e32..b40dde96b 100644 --- a/pkg/tcpip/network/arp/BUILD +++ b/pkg/tcpip/network/arp/BUILD @@ -10,6 +10,7 @@ go_library( "//pkg/tcpip", "//pkg/tcpip/buffer", "//pkg/tcpip/header", + "//pkg/tcpip/header/parse", "//pkg/tcpip/stack", ], ) diff --git a/pkg/tcpip/network/arp/arp.go b/pkg/tcpip/network/arp/arp.go index 7aaee08c4..b47a7be51 100644 --- a/pkg/tcpip/network/arp/arp.go +++ b/pkg/tcpip/network/arp/arp.go @@ -15,20 +15,15 @@ // Package arp implements the ARP network protocol. It is used to resolve // IPv4 addresses into link-local MAC addresses, and advertises IPv4 // addresses of its stack with the local network. -// -// To use it in the networking stack, pass arp.NewProtocol() as one of the -// network protocols when calling stack.New. Then add an "arp" address to every -// NIC on the stack that should respond to ARP requests. That is: -// -// if err := s.AddAddress(1, arp.ProtocolNumber, "arp"); err != nil { -// // handle err -// } package arp import ( + "sync/atomic" + "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/header" + "gvisor.dev/gvisor/pkg/tcpip/header/parse" "gvisor.dev/gvisor/pkg/tcpip/stack" ) @@ -40,15 +35,57 @@ const ( ProtocolAddress = tcpip.Address("arp") ) -// endpoint implements stack.NetworkEndpoint. +var _ stack.AddressableEndpoint = (*endpoint)(nil) +var _ stack.NetworkEndpoint = (*endpoint)(nil) + type endpoint struct { - protocol *protocol - nicID tcpip.NICID + stack.AddressableEndpointState + + protocol *protocol + + // enabled is set to 1 when the NIC is enabled and 0 when it is disabled. + // + // Must be accessed using atomic operations. + enabled uint32 + + nic stack.NetworkInterface linkEP stack.LinkEndpoint linkAddrCache stack.LinkAddressCache nud stack.NUDHandler } +func (e *endpoint) Enable() *tcpip.Error { + if !e.nic.Enabled() { + return tcpip.ErrNotPermitted + } + + e.setEnabled(true) + return nil +} + +func (e *endpoint) Enabled() bool { + return e.nic.Enabled() && e.isEnabled() +} + +// isEnabled returns true if the endpoint is enabled, regardless of the +// enabled status of the NIC. +func (e *endpoint) isEnabled() bool { + return atomic.LoadUint32(&e.enabled) == 1 +} + +// setEnabled sets the enabled status for the endpoint. +func (e *endpoint) setEnabled(v bool) { + if v { + atomic.StoreUint32(&e.enabled, 1) + } else { + atomic.StoreUint32(&e.enabled, 0) + } +} + +func (e *endpoint) Disable() { + e.setEnabled(false) +} + // DefaultTTL is unused for ARP. It implements stack.NetworkEndpoint. func (e *endpoint) DefaultTTL() uint8 { return 0 @@ -59,19 +96,13 @@ func (e *endpoint) MTU() uint32 { return lmtu - uint32(e.MaxHeaderLength()) } -func (e *endpoint) NICID() tcpip.NICID { - return e.nicID -} - -func (e *endpoint) Capabilities() stack.LinkEndpointCapabilities { - return e.linkEP.Capabilities() -} - func (e *endpoint) MaxHeaderLength() uint16 { return e.linkEP.MaxHeaderLength() + header.ARPSize } -func (e *endpoint) Close() {} +func (e *endpoint) Close() { + e.AddressableEndpointState.Cleanup() +} func (e *endpoint) WritePacket(*stack.Route, *stack.GSO, stack.NetworkHeaderParams, *stack.PacketBuffer) *tcpip.Error { return tcpip.ErrNotSupported @@ -92,6 +123,10 @@ func (e *endpoint) WriteHeaderIncludedPacket(r *stack.Route, pkt *stack.PacketBu } func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { + if !e.isEnabled() { + return + } + h := header.ARP(pkt.NetworkHeader().View()) if !h.IsValid() { return @@ -102,15 +137,15 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { localAddr := tcpip.Address(h.ProtocolAddressTarget()) if e.nud == nil { - if e.linkAddrCache.CheckLocalAddress(e.nicID, header.IPv4ProtocolNumber, localAddr) == 0 { + if e.linkAddrCache.CheckLocalAddress(e.nic.ID(), header.IPv4ProtocolNumber, localAddr) == 0 { return // we have no useful answer, ignore the request } addr := tcpip.Address(h.ProtocolAddressSender()) linkAddr := tcpip.LinkAddress(h.HardwareAddressSender()) - e.linkAddrCache.AddLinkAddress(e.nicID, addr, linkAddr) + e.linkAddrCache.AddLinkAddress(e.nic.ID(), addr, linkAddr) } else { - if r.Stack().CheckLocalAddress(e.nicID, header.IPv4ProtocolNumber, localAddr) == 0 { + if r.Stack().CheckLocalAddress(e.nic.ID(), header.IPv4ProtocolNumber, localAddr) == 0 { return // we have no useful answer, ignore the request } @@ -136,7 +171,7 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { linkAddr := tcpip.LinkAddress(h.HardwareAddressSender()) if e.nud == nil { - e.linkAddrCache.AddLinkAddress(e.nicID, addr, linkAddr) + e.linkAddrCache.AddLinkAddress(e.nic.ID(), addr, linkAddr) return } @@ -168,14 +203,16 @@ func (*protocol) ParseAddresses(v buffer.View) (src, dst tcpip.Address) { return tcpip.Address(h.ProtocolAddressSender()), ProtocolAddress } -func (p *protocol) NewEndpoint(nicID tcpip.NICID, linkAddrCache stack.LinkAddressCache, nud stack.NUDHandler, dispatcher stack.TransportDispatcher, sender stack.LinkEndpoint, st *stack.Stack) stack.NetworkEndpoint { - return &endpoint{ +func (p *protocol) NewEndpoint(nic stack.NetworkInterface, linkAddrCache stack.LinkAddressCache, nud stack.NUDHandler, dispatcher stack.TransportDispatcher) stack.NetworkEndpoint { + e := &endpoint{ protocol: p, - nicID: nicID, - linkEP: sender, + nic: nic, + linkEP: nic.LinkEndpoint(), linkAddrCache: linkAddrCache, nud: nud, } + e.AddressableEndpointState.Init(e) + return e } // LinkAddressProtocol implements stack.LinkAddressResolver.LinkAddressProtocol. @@ -234,14 +271,14 @@ func (*protocol) Wait() {} // Parse implements stack.NetworkProtocol.Parse. func (*protocol) Parse(pkt *stack.PacketBuffer) (proto tcpip.TransportProtocolNumber, hasTransportHdr bool, ok bool) { - _, ok = pkt.NetworkHeader().Consume(header.ARPSize) - if !ok { - return 0, false, false - } - return 0, false, true + return 0, false, parse.ARP(pkt) } // NewProtocol returns an ARP network protocol. -func NewProtocol() stack.NetworkProtocol { +// +// Note, to make sure that the ARP endpoint receives ARP packets, the "arp" +// address must be added to every NIC that should respond to ARP requests. See +// ProtocolAddress for more details. +func NewProtocol(*stack.Stack) stack.NetworkProtocol { return &protocol{} } diff --git a/pkg/tcpip/network/arp/arp_test.go b/pkg/tcpip/network/arp/arp_test.go index 9c9a859e3..626af975a 100644 --- a/pkg/tcpip/network/arp/arp_test.go +++ b/pkg/tcpip/network/arp/arp_test.go @@ -176,8 +176,8 @@ func newTestContext(t *testing.T, useNeighborCache bool) *testContext { } s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), arp.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{icmp.NewProtocol4()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, arp.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{icmp.NewProtocol4}, NUDConfigs: c, NUDDisp: &d, UseNeighborCache: useNeighborCache, @@ -442,7 +442,7 @@ func TestLinkAddressRequest(t *testing.T) { } for _, test := range tests { - p := arp.NewProtocol() + p := arp.NewProtocol(nil) linkRes, ok := p.(stack.LinkAddressResolver) if !ok { t.Fatal("expected ARP protocol to implement stack.LinkAddressResolver") diff --git a/pkg/tcpip/network/fragmentation/BUILD b/pkg/tcpip/network/fragmentation/BUILD index 96c5f42f8..e247f06a4 100644 --- a/pkg/tcpip/network/fragmentation/BUILD +++ b/pkg/tcpip/network/fragmentation/BUILD @@ -43,5 +43,6 @@ go_test( library = ":fragmentation", deps = [ "//pkg/tcpip/buffer", + "//pkg/tcpip/faketime", ], ) diff --git a/pkg/tcpip/network/fragmentation/fragmentation.go b/pkg/tcpip/network/fragmentation/fragmentation.go index 6a4843f92..e1909fab0 100644 --- a/pkg/tcpip/network/fragmentation/fragmentation.go +++ b/pkg/tcpip/network/fragmentation/fragmentation.go @@ -81,6 +81,8 @@ type Fragmentation struct { size int timeout time.Duration blockSize uint16 + clock tcpip.Clock + releaseJob *tcpip.Job } // NewFragmentation creates a new Fragmentation. @@ -97,7 +99,7 @@ type Fragmentation struct { // reassemblingTimeout specifies the maximum time allowed to reassemble a packet. // Fragments are lazily evicted only when a new a packet with an // already existing fragmentation-id arrives after the timeout. -func NewFragmentation(blockSize uint16, highMemoryLimit, lowMemoryLimit int, reassemblingTimeout time.Duration) *Fragmentation { +func NewFragmentation(blockSize uint16, highMemoryLimit, lowMemoryLimit int, reassemblingTimeout time.Duration, clock tcpip.Clock) *Fragmentation { if lowMemoryLimit >= highMemoryLimit { lowMemoryLimit = highMemoryLimit } @@ -110,13 +112,17 @@ func NewFragmentation(blockSize uint16, highMemoryLimit, lowMemoryLimit int, rea blockSize = minBlockSize } - return &Fragmentation{ + f := &Fragmentation{ reassemblers: make(map[FragmentID]*reassembler), highLimit: highMemoryLimit, lowLimit: lowMemoryLimit, timeout: reassemblingTimeout, blockSize: blockSize, + clock: clock, } + f.releaseJob = tcpip.NewJob(f.clock, &f.mu, f.releaseReassemblersLocked) + + return f } // Process processes an incoming fragment belonging to an ID and returns a @@ -155,15 +161,17 @@ func (f *Fragmentation) Process( f.mu.Lock() r, ok := f.reassemblers[id] - if ok && r.tooOld(f.timeout) { - // This is very likely to be an id-collision or someone performing a slow-rate attack. - f.release(r) - ok = false - } if !ok { - r = newReassembler(id) + r = newReassembler(id, f.clock) f.reassemblers[id] = r + wasEmpty := f.rList.Empty() f.rList.PushFront(r) + if wasEmpty { + // If we have just pushed a first reassembler into an empty list, we + // should kickstart the release job. The release job will keep + // rescheduling itself until the list becomes empty. + f.releaseReassemblersLocked() + } } f.mu.Unlock() @@ -211,3 +219,27 @@ func (f *Fragmentation) release(r *reassembler) { f.size = 0 } } + +// releaseReassemblersLocked releases already-expired reassemblers, then +// schedules the job to call back itself for the remaining reassemblers if +// any. This function must be called with f.mu locked. +func (f *Fragmentation) releaseReassemblersLocked() { + now := f.clock.NowMonotonic() + for { + // The reassembler at the end of the list is the oldest. + r := f.rList.Back() + if r == nil { + // The list is empty. + break + } + elapsed := time.Duration(now-r.creationTime) * time.Nanosecond + if f.timeout > elapsed { + // If the oldest reassembler has not expired, schedule the release + // job so that this function is called back when it has expired. + f.releaseJob.Schedule(f.timeout - elapsed) + break + } + // If the oldest reassembler has already expired, release it. + f.release(r) + } +} diff --git a/pkg/tcpip/network/fragmentation/fragmentation_test.go b/pkg/tcpip/network/fragmentation/fragmentation_test.go index 416604659..189b223c5 100644 --- a/pkg/tcpip/network/fragmentation/fragmentation_test.go +++ b/pkg/tcpip/network/fragmentation/fragmentation_test.go @@ -21,6 +21,7 @@ import ( "time" "gvisor.dev/gvisor/pkg/tcpip/buffer" + "gvisor.dev/gvisor/pkg/tcpip/faketime" ) // vv is a helper to build VectorisedView from different strings. @@ -95,7 +96,7 @@ var processTestCases = []struct { func TestFragmentationProcess(t *testing.T) { for _, c := range processTestCases { t.Run(c.comment, func(t *testing.T) { - f := NewFragmentation(minBlockSize, 1024, 512, DefaultReassembleTimeout) + f := NewFragmentation(minBlockSize, 1024, 512, DefaultReassembleTimeout, &faketime.NullClock{}) firstFragmentProto := c.in[0].proto for i, in := range c.in { vv, proto, done, err := f.Process(in.id, in.first, in.last, in.more, in.proto, in.vv) @@ -131,25 +132,126 @@ func TestFragmentationProcess(t *testing.T) { } func TestReassemblingTimeout(t *testing.T) { - timeout := time.Millisecond - f := NewFragmentation(minBlockSize, 1024, 512, timeout) - // Send first fragment with id = 0, first = 0, last = 0, and more = true. - f.Process(FragmentID{}, 0, 0, true, 0xFF, vv(1, "0")) - // Sleep more than the timeout. - time.Sleep(2 * timeout) - // Send another fragment that completes a packet. - // However, no packet should be reassembled because the fragment arrived after the timeout. - _, _, done, err := f.Process(FragmentID{}, 1, 1, false, 0xFF, vv(1, "1")) - if err != nil { - t.Fatalf("f.Process(0, 1, 1, false, 0xFF, vv(1, \"1\")) failed: %v", err) + const ( + reassemblyTimeout = time.Millisecond + protocol = 0xff + ) + + type fragment struct { + first uint16 + last uint16 + more bool + data string } - if done { - t.Errorf("Fragmentation does not respect the reassembling timeout.") + + type event struct { + // name is a nickname of this event. + name string + + // clockAdvance is a duration to advance the clock. The clock advances + // before a fragment specified in the fragment field is processed. + clockAdvance time.Duration + + // fragment is a fragment to process. This can be nil if there is no + // fragment to process. + fragment *fragment + + // expectDone is true if the fragmentation instance should report the + // reassembly is done after the fragment is processd. + expectDone bool + + // sizeAfterEvent is the expected size of the fragmentation instance after + // the event. + sizeAfterEvent int + } + + half1 := &fragment{first: 0, last: 0, more: true, data: "0"} + half2 := &fragment{first: 1, last: 1, more: false, data: "1"} + + tests := []struct { + name string + events []event + }{ + { + name: "half1 and half2 are reassembled successfully", + events: []event{ + { + name: "half1", + fragment: half1, + expectDone: false, + sizeAfterEvent: 1, + }, + { + name: "half2", + fragment: half2, + expectDone: true, + sizeAfterEvent: 0, + }, + }, + }, + { + name: "half1 timeout, half2 timeout", + events: []event{ + { + name: "half1", + fragment: half1, + expectDone: false, + sizeAfterEvent: 1, + }, + { + name: "half1 just before reassembly timeout", + clockAdvance: reassemblyTimeout - 1, + sizeAfterEvent: 1, + }, + { + name: "half1 reassembly timeout", + clockAdvance: 1, + sizeAfterEvent: 0, + }, + { + name: "half2", + fragment: half2, + expectDone: false, + sizeAfterEvent: 1, + }, + { + name: "half2 just before reassembly timeout", + clockAdvance: reassemblyTimeout - 1, + sizeAfterEvent: 1, + }, + { + name: "half2 reassembly timeout", + clockAdvance: 1, + sizeAfterEvent: 0, + }, + }, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + clock := faketime.NewManualClock() + f := NewFragmentation(minBlockSize, HighFragThreshold, LowFragThreshold, reassemblyTimeout, clock) + for _, event := range test.events { + clock.Advance(event.clockAdvance) + if frag := event.fragment; frag != nil { + _, _, done, err := f.Process(FragmentID{}, frag.first, frag.last, frag.more, protocol, vv(len(frag.data), frag.data)) + if err != nil { + t.Fatalf("%s: f.Process failed: %s", event.name, err) + } + if done != event.expectDone { + t.Fatalf("%s: got done = %t, want = %t", event.name, done, event.expectDone) + } + } + if got, want := f.size, event.sizeAfterEvent; got != want { + t.Errorf("%s: got f.size = %d, want = %d", event.name, got, want) + } + } + }) } } func TestMemoryLimits(t *testing.T) { - f := NewFragmentation(minBlockSize, 3, 1, DefaultReassembleTimeout) + f := NewFragmentation(minBlockSize, 3, 1, DefaultReassembleTimeout, &faketime.NullClock{}) // Send first fragment with id = 0. f.Process(FragmentID{ID: 0}, 0, 0, true, 0xFF, vv(1, "0")) // Send first fragment with id = 1. @@ -173,7 +275,7 @@ func TestMemoryLimits(t *testing.T) { } func TestMemoryLimitsIgnoresDuplicates(t *testing.T) { - f := NewFragmentation(minBlockSize, 1, 0, DefaultReassembleTimeout) + f := NewFragmentation(minBlockSize, 1, 0, DefaultReassembleTimeout, &faketime.NullClock{}) // Send first fragment with id = 0. f.Process(FragmentID{}, 0, 0, true, 0xFF, vv(1, "0")) // Send the same packet again. @@ -268,7 +370,7 @@ func TestErrors(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - f := NewFragmentation(test.blockSize, HighFragThreshold, LowFragThreshold, DefaultReassembleTimeout) + f := NewFragmentation(test.blockSize, HighFragThreshold, LowFragThreshold, DefaultReassembleTimeout, &faketime.NullClock{}) _, _, done, err := f.Process(FragmentID{}, test.first, test.last, test.more, 0, vv(len(test.data), test.data)) if !errors.Is(err, test.err) { t.Errorf("got Process(_, %d, %d, %t, _, %q) = (_, _, _, %v), want = (_, _, _, %v)", test.first, test.last, test.more, test.data, err, test.err) diff --git a/pkg/tcpip/network/fragmentation/reassembler.go b/pkg/tcpip/network/fragmentation/reassembler.go index f044867dc..9bb051a30 100644 --- a/pkg/tcpip/network/fragmentation/reassembler.go +++ b/pkg/tcpip/network/fragmentation/reassembler.go @@ -18,9 +18,9 @@ import ( "container/heap" "fmt" "math" - "time" "gvisor.dev/gvisor/pkg/sync" + "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" ) @@ -40,15 +40,15 @@ type reassembler struct { deleted int heap fragHeap done bool - creationTime time.Time + creationTime int64 } -func newReassembler(id FragmentID) *reassembler { +func newReassembler(id FragmentID, clock tcpip.Clock) *reassembler { r := &reassembler{ id: id, holes: make([]hole, 0, 16), heap: make(fragHeap, 0, 8), - creationTime: time.Now(), + creationTime: clock.NowMonotonic(), } r.holes = append(r.holes, hole{ first: 0, @@ -116,10 +116,6 @@ func (r *reassembler) process(first, last uint16, more bool, proto uint8, vv buf return res, r.proto, true, consumed, nil } -func (r *reassembler) tooOld(timeout time.Duration) bool { - return time.Now().Sub(r.creationTime) > timeout -} - func (r *reassembler) checkDoneOrMark() bool { r.mu.Lock() prev := r.done diff --git a/pkg/tcpip/network/fragmentation/reassembler_test.go b/pkg/tcpip/network/fragmentation/reassembler_test.go index dff7c9dcb..a0a04a027 100644 --- a/pkg/tcpip/network/fragmentation/reassembler_test.go +++ b/pkg/tcpip/network/fragmentation/reassembler_test.go @@ -18,6 +18,8 @@ import ( "math" "reflect" "testing" + + "gvisor.dev/gvisor/pkg/tcpip/faketime" ) type updateHolesInput struct { @@ -94,7 +96,7 @@ var holesTestCases = []struct { func TestUpdateHoles(t *testing.T) { for _, c := range holesTestCases { - r := newReassembler(FragmentID{}) + r := newReassembler(FragmentID{}, &faketime.NullClock{}) for _, i := range c.in { r.updateHoles(i.first, i.last, i.more) } diff --git a/pkg/tcpip/network/ip_test.go b/pkg/tcpip/network/ip_test.go index e45dd17f8..6861cfdaf 100644 --- a/pkg/tcpip/network/ip_test.go +++ b/pkg/tcpip/network/ip_test.go @@ -17,6 +17,7 @@ package ip_test import ( "testing" + "gvisor.dev/gvisor/pkg/sync" "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/header" @@ -25,26 +26,35 @@ import ( "gvisor.dev/gvisor/pkg/tcpip/network/ipv4" "gvisor.dev/gvisor/pkg/tcpip/network/ipv6" "gvisor.dev/gvisor/pkg/tcpip/stack" + "gvisor.dev/gvisor/pkg/tcpip/transport/icmp" "gvisor.dev/gvisor/pkg/tcpip/transport/tcp" "gvisor.dev/gvisor/pkg/tcpip/transport/udp" ) const ( - localIpv4Addr = "\x0a\x00\x00\x01" - localIpv4PrefixLen = 24 - remoteIpv4Addr = "\x0a\x00\x00\x02" - ipv4SubnetAddr = "\x0a\x00\x00\x00" - ipv4SubnetMask = "\xff\xff\xff\x00" - ipv4Gateway = "\x0a\x00\x00\x03" - localIpv6Addr = "\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01" - localIpv6PrefixLen = 120 - remoteIpv6Addr = "\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02" - ipv6SubnetAddr = "\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00" - ipv6SubnetMask = "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\x00" - ipv6Gateway = "\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x03" - nicID = 1 + localIPv4Addr = "\x0a\x00\x00\x01" + remoteIPv4Addr = "\x0a\x00\x00\x02" + ipv4SubnetAddr = "\x0a\x00\x00\x00" + ipv4SubnetMask = "\xff\xff\xff\x00" + ipv4Gateway = "\x0a\x00\x00\x03" + localIPv6Addr = "\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01" + remoteIPv6Addr = "\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02" + ipv6SubnetAddr = "\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00" + ipv6SubnetMask = "\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\x00" + ipv6Gateway = "\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x03" + nicID = 1 ) +var localIPv4AddrWithPrefix = tcpip.AddressWithPrefix{ + Address: localIPv4Addr, + PrefixLen: 24, +} + +var localIPv6AddrWithPrefix = tcpip.AddressWithPrefix{ + Address: localIPv6Addr, + PrefixLen: 120, +} + // testObject implements two interfaces: LinkEndpoint and TransportDispatcher. // The former is used to pretend that it's a link endpoint so that we can // inspect packets written by the network endpoints. The latter is used to @@ -98,9 +108,10 @@ func (t *testObject) checkValues(protocol tcpip.TransportProtocolNumber, vv buff // DeliverTransportPacket is called by network endpoints after parsing incoming // packets. This is used by the test object to verify that the results of the // parsing are expected. -func (t *testObject) DeliverTransportPacket(r *stack.Route, protocol tcpip.TransportProtocolNumber, pkt *stack.PacketBuffer) { +func (t *testObject) DeliverTransportPacket(r *stack.Route, protocol tcpip.TransportProtocolNumber, pkt *stack.PacketBuffer) stack.TransportPacketDisposition { t.checkValues(protocol, pkt.Data, r.RemoteAddress, r.LocalAddress) t.dataCalls++ + return stack.TransportPacketHandled } // DeliverTransportControlPacket is called by network endpoints after parsing @@ -194,8 +205,8 @@ func (*testObject) AddHeader(local, remote tcpip.LinkAddress, protocol tcpip.Net func buildIPv4Route(local, remote tcpip.Address) (stack.Route, *tcpip.Error) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol(), tcp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol, tcp.NewProtocol}, }) s.CreateNIC(nicID, loopback.New()) s.AddAddress(nicID, ipv4.ProtocolNumber, local) @@ -210,8 +221,8 @@ func buildIPv4Route(local, remote tcpip.Address) (stack.Route, *tcpip.Error) { func buildIPv6Route(local, remote tcpip.Address) (stack.Route, *tcpip.Error) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol(), tcp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol, tcp.NewProtocol}, }) s.CreateNIC(nicID, loopback.New()) s.AddAddress(nicID, ipv6.ProtocolNumber, local) @@ -224,33 +235,294 @@ func buildIPv6Route(local, remote tcpip.Address) (stack.Route, *tcpip.Error) { return s.FindRoute(nicID, local, remote, ipv6.ProtocolNumber, false /* multicastLoop */) } -func buildDummyStack(t *testing.T) *stack.Stack { +func buildDummyStackWithLinkEndpoint(t *testing.T) (*stack.Stack, *channel.Endpoint) { t.Helper() s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol(), tcp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol, tcp.NewProtocol}, }) e := channel.New(0, 1280, "") if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(%d, _) = %s", nicID, err) } - if err := s.AddAddress(nicID, header.IPv4ProtocolNumber, localIpv4Addr); err != nil { - t.Fatalf("AddAddress(%d, %d, %s) = %s", nicID, header.IPv4ProtocolNumber, localIpv4Addr, err) + v4Addr := tcpip.ProtocolAddress{Protocol: header.IPv4ProtocolNumber, AddressWithPrefix: localIPv4AddrWithPrefix} + if err := s.AddProtocolAddress(nicID, v4Addr); err != nil { + t.Fatalf("AddProtocolAddress(%d, %#v) = %s", nicID, v4Addr, err) } - if err := s.AddAddress(nicID, header.IPv6ProtocolNumber, localIpv6Addr); err != nil { - t.Fatalf("AddAddress(%d, %d, %s) = %s", nicID, header.IPv6ProtocolNumber, localIpv6Addr, err) + v6Addr := tcpip.ProtocolAddress{Protocol: header.IPv6ProtocolNumber, AddressWithPrefix: localIPv6AddrWithPrefix} + if err := s.AddProtocolAddress(nicID, v6Addr); err != nil { + t.Fatalf("AddProtocolAddress(%d, %#v) = %s", nicID, v6Addr, err) } + return s, e +} + +func buildDummyStack(t *testing.T) *stack.Stack { + t.Helper() + + s, _ := buildDummyStackWithLinkEndpoint(t) return s } +var _ stack.NetworkInterface = (*testInterface)(nil) + +type testInterface struct { + tester testObject + + mu struct { + sync.RWMutex + disabled bool + } +} + +func (*testInterface) ID() tcpip.NICID { + return nicID +} + +func (*testInterface) IsLoopback() bool { + return false +} + +func (*testInterface) Name() string { + return "" +} + +func (t *testInterface) Enabled() bool { + t.mu.RLock() + defer t.mu.RUnlock() + return !t.mu.disabled +} + +func (t *testInterface) setEnabled(v bool) { + t.mu.Lock() + defer t.mu.Unlock() + t.mu.disabled = !v +} + +func (t *testInterface) LinkEndpoint() stack.LinkEndpoint { + return &t.tester +} + +func TestSourceAddressValidation(t *testing.T) { + rxIPv4ICMP := func(e *channel.Endpoint, src tcpip.Address) { + totalLen := header.IPv4MinimumSize + header.ICMPv4MinimumSize + hdr := buffer.NewPrependable(totalLen) + pkt := header.ICMPv4(hdr.Prepend(header.ICMPv4MinimumSize)) + pkt.SetType(header.ICMPv4Echo) + pkt.SetCode(0) + pkt.SetChecksum(0) + pkt.SetChecksum(^header.Checksum(pkt, 0)) + ip := header.IPv4(hdr.Prepend(header.IPv4MinimumSize)) + ip.Encode(&header.IPv4Fields{ + IHL: header.IPv4MinimumSize, + TotalLength: uint16(totalLen), + Protocol: uint8(icmp.ProtocolNumber4), + TTL: ipv4.DefaultTTL, + SrcAddr: src, + DstAddr: localIPv4Addr, + }) + + e.InjectInbound(header.IPv4ProtocolNumber, stack.NewPacketBuffer(stack.PacketBufferOptions{ + Data: hdr.View().ToVectorisedView(), + })) + } + + rxIPv6ICMP := func(e *channel.Endpoint, src tcpip.Address) { + totalLen := header.IPv6MinimumSize + header.ICMPv6MinimumSize + hdr := buffer.NewPrependable(totalLen) + pkt := header.ICMPv6(hdr.Prepend(header.ICMPv6MinimumSize)) + pkt.SetType(header.ICMPv6EchoRequest) + pkt.SetCode(0) + pkt.SetChecksum(0) + pkt.SetChecksum(header.ICMPv6Checksum(pkt, src, localIPv6Addr, buffer.VectorisedView{})) + ip := header.IPv6(hdr.Prepend(header.IPv6MinimumSize)) + ip.Encode(&header.IPv6Fields{ + PayloadLength: header.ICMPv6MinimumSize, + NextHeader: uint8(icmp.ProtocolNumber6), + HopLimit: ipv6.DefaultTTL, + SrcAddr: src, + DstAddr: localIPv6Addr, + }) + + e.InjectInbound(header.IPv6ProtocolNumber, stack.NewPacketBuffer(stack.PacketBufferOptions{ + Data: hdr.View().ToVectorisedView(), + })) + } + + tests := []struct { + name string + srcAddress tcpip.Address + rxICMP func(*channel.Endpoint, tcpip.Address) + valid bool + }{ + { + name: "IPv4 valid", + srcAddress: "\x01\x02\x03\x04", + rxICMP: rxIPv4ICMP, + valid: true, + }, + { + name: "IPv6 valid", + srcAddress: "\x01\x02\x03\x04\x05\x06\x07\x08\x09\x0a\x0b\x0c\x0d\x0e\x0f\x10", + rxICMP: rxIPv6ICMP, + valid: true, + }, + { + name: "IPv4 unspecified", + srcAddress: header.IPv4Any, + rxICMP: rxIPv4ICMP, + valid: true, + }, + { + name: "IPv6 unspecified", + srcAddress: header.IPv4Any, + rxICMP: rxIPv6ICMP, + valid: true, + }, + { + name: "IPv4 multicast", + srcAddress: "\xe0\x00\x00\x01", + rxICMP: rxIPv4ICMP, + valid: false, + }, + { + name: "IPv6 multicast", + srcAddress: "\xff\x02\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01", + rxICMP: rxIPv6ICMP, + valid: false, + }, + { + name: "IPv4 broadcast", + srcAddress: header.IPv4Broadcast, + rxICMP: rxIPv4ICMP, + valid: false, + }, + { + name: "IPv4 subnet broadcast", + srcAddress: func() tcpip.Address { + subnet := localIPv4AddrWithPrefix.Subnet() + return subnet.Broadcast() + }(), + rxICMP: rxIPv4ICMP, + valid: false, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + s, e := buildDummyStackWithLinkEndpoint(t) + test.rxICMP(e, test.srcAddress) + + var wantValid uint64 + if test.valid { + wantValid = 1 + } + + if got, want := s.Stats().IP.InvalidSourceAddressesReceived.Value(), 1-wantValid; got != want { + t.Errorf("got s.Stats().IP.InvalidSourceAddressesReceived.Value() = %d, want = %d", got, want) + } + if got := s.Stats().IP.PacketsDelivered.Value(); got != wantValid { + t.Errorf("got s.Stats().IP.PacketsDelivered.Value() = %d, want = %d", got, wantValid) + } + }) + } +} + +func TestEnableWhenNICDisabled(t *testing.T) { + tests := []struct { + name string + protocolFactory stack.NetworkProtocolFactory + protoNum tcpip.NetworkProtocolNumber + }{ + { + name: "IPv4", + protocolFactory: ipv4.NewProtocol, + protoNum: ipv4.ProtocolNumber, + }, + { + name: "IPv6", + protocolFactory: ipv6.NewProtocol, + protoNum: ipv6.ProtocolNumber, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + var nic testInterface + nic.setEnabled(false) + + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{test.protocolFactory}, + }) + p := s.NetworkProtocolInstance(test.protoNum) + + // We pass nil for all parameters except the NetworkInterface and Stack + // since Enable only depends on these. + ep := p.NewEndpoint(&nic, nil, nil, nil) + + // The endpoint should initially be disabled, regardless the NIC's enabled + // status. + if ep.Enabled() { + t.Fatal("got ep.Enabled() = true, want = false") + } + nic.setEnabled(true) + if ep.Enabled() { + t.Fatal("got ep.Enabled() = true, want = false") + } + + // Attempting to enable the endpoint while the NIC is disabled should + // fail. + nic.setEnabled(false) + if err := ep.Enable(); err != tcpip.ErrNotPermitted { + t.Fatalf("got ep.Enable() = %s, want = %s", err, tcpip.ErrNotPermitted) + } + // ep should consider the NIC's enabled status when determining its own + // enabled status so we "enable" the NIC to read just the endpoint's + // enabled status. + nic.setEnabled(true) + if ep.Enabled() { + t.Fatal("got ep.Enabled() = true, want = false") + } + + // Enabling the interface after the NIC has been enabled should succeed. + if err := ep.Enable(); err != nil { + t.Fatalf("ep.Enable(): %s", err) + } + if !ep.Enabled() { + t.Fatal("got ep.Enabled() = false, want = true") + } + + // ep should consider the NIC's enabled status when determining its own + // enabled status. + nic.setEnabled(false) + if ep.Enabled() { + t.Fatal("got ep.Enabled() = true, want = false") + } + + // Disabling the endpoint when the NIC is enabled should make the endpoint + // disabled. + nic.setEnabled(true) + ep.Disable() + if ep.Enabled() { + t.Fatal("got ep.Enabled() = true, want = false") + } + }) + } +} + func TestIPv4Send(t *testing.T) { - o := testObject{t: t, v4: true} - proto := ipv4.NewProtocol() - ep := proto.NewEndpoint(nicID, nil, nil, nil, &o, buildDummyStack(t)) + s := buildDummyStack(t) + proto := s.NetworkProtocolInstance(ipv4.ProtocolNumber) + nic := testInterface{ + tester: testObject{ + t: t, + v4: true, + }, + } + ep := proto.NewEndpoint(&nic, nil, nil, nil) defer ep.Close() // Allocate and initialize the payload view. @@ -266,12 +538,12 @@ func TestIPv4Send(t *testing.T) { }) // Issue the write. - o.protocol = 123 - o.srcAddr = localIpv4Addr - o.dstAddr = remoteIpv4Addr - o.contents = payload + nic.tester.protocol = 123 + nic.tester.srcAddr = localIPv4Addr + nic.tester.dstAddr = remoteIPv4Addr + nic.tester.contents = payload - r, err := buildIPv4Route(localIpv4Addr, remoteIpv4Addr) + r, err := buildIPv4Route(localIPv4Addr, remoteIPv4Addr) if err != nil { t.Fatalf("could not find route: %v", err) } @@ -285,11 +557,21 @@ func TestIPv4Send(t *testing.T) { } func TestIPv4Receive(t *testing.T) { - o := testObject{t: t, v4: true} - proto := ipv4.NewProtocol() - ep := proto.NewEndpoint(nicID, nil, nil, &o, nil, buildDummyStack(t)) + s := buildDummyStack(t) + proto := s.NetworkProtocolInstance(ipv4.ProtocolNumber) + nic := testInterface{ + tester: testObject{ + t: t, + v4: true, + }, + } + ep := proto.NewEndpoint(&nic, nil, nil, &nic.tester) defer ep.Close() + if err := ep.Enable(); err != nil { + t.Fatalf("ep.Enable(): %s", err) + } + totalLen := header.IPv4MinimumSize + 30 view := buffer.NewView(totalLen) ip := header.IPv4(view) @@ -298,8 +580,8 @@ func TestIPv4Receive(t *testing.T) { TotalLength: uint16(totalLen), TTL: 20, Protocol: 10, - SrcAddr: remoteIpv4Addr, - DstAddr: localIpv4Addr, + SrcAddr: remoteIPv4Addr, + DstAddr: localIPv4Addr, }) // Make payload be non-zero. @@ -308,12 +590,12 @@ func TestIPv4Receive(t *testing.T) { } // Give packet to ipv4 endpoint, dispatcher will validate that it's ok. - o.protocol = 10 - o.srcAddr = remoteIpv4Addr - o.dstAddr = localIpv4Addr - o.contents = view[header.IPv4MinimumSize:totalLen] + nic.tester.protocol = 10 + nic.tester.srcAddr = remoteIPv4Addr + nic.tester.dstAddr = localIPv4Addr + nic.tester.contents = view[header.IPv4MinimumSize:totalLen] - r, err := buildIPv4Route(localIpv4Addr, remoteIpv4Addr) + r, err := buildIPv4Route(localIPv4Addr, remoteIPv4Addr) if err != nil { t.Fatalf("could not find route: %v", err) } @@ -324,8 +606,8 @@ func TestIPv4Receive(t *testing.T) { t.Fatalf("failed to parse packet: %x", pkt.Data.ToView()) } ep.HandlePacket(&r, pkt) - if o.dataCalls != 1 { - t.Fatalf("Bad number of data calls: got %x, want 1", o.dataCalls) + if nic.tester.dataCalls != 1 { + t.Fatalf("Bad number of data calls: got %x, want 1", nic.tester.dataCalls) } } @@ -349,17 +631,26 @@ func TestIPv4ReceiveControl(t *testing.T) { {"Non-zero fragment offset", 0, 100, header.ICMPv4PortUnreachable, stack.ControlPortUnreachable, 0, 0}, {"Zero-length packet", 0, 0, header.ICMPv4PortUnreachable, stack.ControlPortUnreachable, 0, 2*header.IPv4MinimumSize + header.ICMPv4MinimumSize + 8}, } - r, err := buildIPv4Route(localIpv4Addr, "\x0a\x00\x00\xbb") + r, err := buildIPv4Route(localIPv4Addr, "\x0a\x00\x00\xbb") if err != nil { t.Fatal(err) } for _, c := range cases { t.Run(c.name, func(t *testing.T) { - o := testObject{t: t} - proto := ipv4.NewProtocol() - ep := proto.NewEndpoint(nicID, nil, nil, &o, nil, buildDummyStack(t)) + s := buildDummyStack(t) + proto := s.NetworkProtocolInstance(ipv4.ProtocolNumber) + nic := testInterface{ + tester: testObject{ + t: t, + }, + } + ep := proto.NewEndpoint(&nic, nil, nil, &nic.tester) defer ep.Close() + if err := ep.Enable(); err != nil { + t.Fatalf("ep.Enable(): %s", err) + } + const dataOffset = header.IPv4MinimumSize*2 + header.ICMPv4MinimumSize view := buffer.NewView(dataOffset + 8) @@ -371,7 +662,7 @@ func TestIPv4ReceiveControl(t *testing.T) { TTL: 20, Protocol: uint8(header.ICMPv4ProtocolNumber), SrcAddr: "\x0a\x00\x00\xbb", - DstAddr: localIpv4Addr, + DstAddr: localIPv4Addr, }) // Create the ICMP header. @@ -389,8 +680,8 @@ func TestIPv4ReceiveControl(t *testing.T) { TTL: 20, Protocol: 10, FragmentOffset: c.fragmentOffset, - SrcAddr: localIpv4Addr, - DstAddr: remoteIpv4Addr, + SrcAddr: localIPv4Addr, + DstAddr: remoteIPv4Addr, }) // Make payload be non-zero. @@ -400,27 +691,37 @@ func TestIPv4ReceiveControl(t *testing.T) { // Give packet to IPv4 endpoint, dispatcher will validate that // it's ok. - o.protocol = 10 - o.srcAddr = remoteIpv4Addr - o.dstAddr = localIpv4Addr - o.contents = view[dataOffset:] - o.typ = c.expectedTyp - o.extra = c.expectedExtra + nic.tester.protocol = 10 + nic.tester.srcAddr = remoteIPv4Addr + nic.tester.dstAddr = localIPv4Addr + nic.tester.contents = view[dataOffset:] + nic.tester.typ = c.expectedTyp + nic.tester.extra = c.expectedExtra ep.HandlePacket(&r, truncatedPacket(view, c.trunc, header.IPv4MinimumSize)) - if want := c.expectedCount; o.controlCalls != want { - t.Fatalf("Bad number of control calls for %q case: got %v, want %v", c.name, o.controlCalls, want) + if want := c.expectedCount; nic.tester.controlCalls != want { + t.Fatalf("Bad number of control calls for %q case: got %v, want %v", c.name, nic.tester.controlCalls, want) } }) } } func TestIPv4FragmentationReceive(t *testing.T) { - o := testObject{t: t, v4: true} - proto := ipv4.NewProtocol() - ep := proto.NewEndpoint(nicID, nil, nil, &o, nil, buildDummyStack(t)) + s := buildDummyStack(t) + proto := s.NetworkProtocolInstance(ipv4.ProtocolNumber) + nic := testInterface{ + tester: testObject{ + t: t, + v4: true, + }, + } + ep := proto.NewEndpoint(&nic, nil, nil, &nic.tester) defer ep.Close() + if err := ep.Enable(); err != nil { + t.Fatalf("ep.Enable(): %s", err) + } + totalLen := header.IPv4MinimumSize + 24 frag1 := buffer.NewView(totalLen) @@ -432,8 +733,8 @@ func TestIPv4FragmentationReceive(t *testing.T) { Protocol: 10, FragmentOffset: 0, Flags: header.IPv4FlagMoreFragments, - SrcAddr: remoteIpv4Addr, - DstAddr: localIpv4Addr, + SrcAddr: remoteIPv4Addr, + DstAddr: localIPv4Addr, }) // Make payload be non-zero. for i := header.IPv4MinimumSize; i < totalLen; i++ { @@ -448,8 +749,8 @@ func TestIPv4FragmentationReceive(t *testing.T) { TTL: 20, Protocol: 10, FragmentOffset: 24, - SrcAddr: remoteIpv4Addr, - DstAddr: localIpv4Addr, + SrcAddr: remoteIPv4Addr, + DstAddr: localIPv4Addr, }) // Make payload be non-zero. for i := header.IPv4MinimumSize; i < totalLen; i++ { @@ -457,12 +758,12 @@ func TestIPv4FragmentationReceive(t *testing.T) { } // Give packet to ipv4 endpoint, dispatcher will validate that it's ok. - o.protocol = 10 - o.srcAddr = remoteIpv4Addr - o.dstAddr = localIpv4Addr - o.contents = append(frag1[header.IPv4MinimumSize:totalLen], frag2[header.IPv4MinimumSize:totalLen]...) + nic.tester.protocol = 10 + nic.tester.srcAddr = remoteIPv4Addr + nic.tester.dstAddr = localIPv4Addr + nic.tester.contents = append(frag1[header.IPv4MinimumSize:totalLen], frag2[header.IPv4MinimumSize:totalLen]...) - r, err := buildIPv4Route(localIpv4Addr, remoteIpv4Addr) + r, err := buildIPv4Route(localIPv4Addr, remoteIPv4Addr) if err != nil { t.Fatalf("could not find route: %v", err) } @@ -475,8 +776,8 @@ func TestIPv4FragmentationReceive(t *testing.T) { t.Fatalf("failed to parse packet: %x", pkt.Data.ToView()) } ep.HandlePacket(&r, pkt) - if o.dataCalls != 0 { - t.Fatalf("Bad number of data calls: got %x, want 0", o.dataCalls) + if nic.tester.dataCalls != 0 { + t.Fatalf("Bad number of data calls: got %x, want 0", nic.tester.dataCalls) } // Send second segment. @@ -487,17 +788,26 @@ func TestIPv4FragmentationReceive(t *testing.T) { t.Fatalf("failed to parse packet: %x", pkt.Data.ToView()) } ep.HandlePacket(&r, pkt) - if o.dataCalls != 1 { - t.Fatalf("Bad number of data calls: got %x, want 1", o.dataCalls) + if nic.tester.dataCalls != 1 { + t.Fatalf("Bad number of data calls: got %x, want 1", nic.tester.dataCalls) } } func TestIPv6Send(t *testing.T) { - o := testObject{t: t} - proto := ipv6.NewProtocol() - ep := proto.NewEndpoint(nicID, nil, nil, &o, channel.New(0, 1280, ""), buildDummyStack(t)) + s := buildDummyStack(t) + proto := s.NetworkProtocolInstance(ipv6.ProtocolNumber) + nic := testInterface{ + tester: testObject{ + t: t, + }, + } + ep := proto.NewEndpoint(&nic, nil, nil, nil) defer ep.Close() + if err := ep.Enable(); err != nil { + t.Fatalf("ep.Enable(): %s", err) + } + // Allocate and initialize the payload view. payload := buffer.NewView(100) for i := 0; i < len(payload); i++ { @@ -511,12 +821,12 @@ func TestIPv6Send(t *testing.T) { }) // Issue the write. - o.protocol = 123 - o.srcAddr = localIpv6Addr - o.dstAddr = remoteIpv6Addr - o.contents = payload + nic.tester.protocol = 123 + nic.tester.srcAddr = localIPv6Addr + nic.tester.dstAddr = remoteIPv6Addr + nic.tester.contents = payload - r, err := buildIPv6Route(localIpv6Addr, remoteIpv6Addr) + r, err := buildIPv6Route(localIPv6Addr, remoteIPv6Addr) if err != nil { t.Fatalf("could not find route: %v", err) } @@ -530,11 +840,20 @@ func TestIPv6Send(t *testing.T) { } func TestIPv6Receive(t *testing.T) { - o := testObject{t: t} - proto := ipv6.NewProtocol() - ep := proto.NewEndpoint(nicID, nil, nil, &o, nil, buildDummyStack(t)) + s := buildDummyStack(t) + proto := s.NetworkProtocolInstance(ipv6.ProtocolNumber) + nic := testInterface{ + tester: testObject{ + t: t, + }, + } + ep := proto.NewEndpoint(&nic, nil, nil, &nic.tester) defer ep.Close() + if err := ep.Enable(); err != nil { + t.Fatalf("ep.Enable(): %s", err) + } + totalLen := header.IPv6MinimumSize + 30 view := buffer.NewView(totalLen) ip := header.IPv6(view) @@ -542,8 +861,8 @@ func TestIPv6Receive(t *testing.T) { PayloadLength: uint16(totalLen - header.IPv6MinimumSize), NextHeader: 10, HopLimit: 20, - SrcAddr: remoteIpv6Addr, - DstAddr: localIpv6Addr, + SrcAddr: remoteIPv6Addr, + DstAddr: localIPv6Addr, }) // Make payload be non-zero. @@ -552,12 +871,12 @@ func TestIPv6Receive(t *testing.T) { } // Give packet to ipv6 endpoint, dispatcher will validate that it's ok. - o.protocol = 10 - o.srcAddr = remoteIpv6Addr - o.dstAddr = localIpv6Addr - o.contents = view[header.IPv6MinimumSize:totalLen] + nic.tester.protocol = 10 + nic.tester.srcAddr = remoteIPv6Addr + nic.tester.dstAddr = localIPv6Addr + nic.tester.contents = view[header.IPv6MinimumSize:totalLen] - r, err := buildIPv6Route(localIpv6Addr, remoteIpv6Addr) + r, err := buildIPv6Route(localIPv6Addr, remoteIPv6Addr) if err != nil { t.Fatalf("could not find route: %v", err) } @@ -569,8 +888,8 @@ func TestIPv6Receive(t *testing.T) { t.Fatalf("failed to parse packet: %x", pkt.Data.ToView()) } ep.HandlePacket(&r, pkt) - if o.dataCalls != 1 { - t.Fatalf("Bad number of data calls: got %x, want 1", o.dataCalls) + if nic.tester.dataCalls != 1 { + t.Fatalf("Bad number of data calls: got %x, want 1", nic.tester.dataCalls) } } @@ -601,7 +920,7 @@ func TestIPv6ReceiveControl(t *testing.T) { {"Zero-length packet", 0, nil, header.ICMPv6DstUnreachable, header.ICMPv6PortUnreachable, stack.ControlPortUnreachable, 0, 2*header.IPv6MinimumSize + header.ICMPv6DstUnreachableMinimumSize + 8}, } r, err := buildIPv6Route( - localIpv6Addr, + localIPv6Addr, "\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\xaa", ) if err != nil { @@ -609,11 +928,20 @@ func TestIPv6ReceiveControl(t *testing.T) { } for _, c := range cases { t.Run(c.name, func(t *testing.T) { - o := testObject{t: t} - proto := ipv6.NewProtocol() - ep := proto.NewEndpoint(nicID, nil, nil, &o, nil, buildDummyStack(t)) + s := buildDummyStack(t) + proto := s.NetworkProtocolInstance(ipv6.ProtocolNumber) + nic := testInterface{ + tester: testObject{ + t: t, + }, + } + ep := proto.NewEndpoint(&nic, nil, nil, &nic.tester) defer ep.Close() + if err := ep.Enable(); err != nil { + t.Fatalf("ep.Enable(): %s", err) + } + dataOffset := header.IPv6MinimumSize*2 + header.ICMPv6MinimumSize if c.fragmentOffset != nil { dataOffset += header.IPv6FragmentHeaderSize @@ -627,7 +955,7 @@ func TestIPv6ReceiveControl(t *testing.T) { NextHeader: uint8(header.ICMPv6ProtocolNumber), HopLimit: 20, SrcAddr: outerSrcAddr, - DstAddr: localIpv6Addr, + DstAddr: localIPv6Addr, }) // Create the ICMP header. @@ -643,8 +971,8 @@ func TestIPv6ReceiveControl(t *testing.T) { PayloadLength: 100, NextHeader: 10, HopLimit: 20, - SrcAddr: localIpv6Addr, - DstAddr: remoteIpv6Addr, + SrcAddr: localIPv6Addr, + DstAddr: remoteIPv6Addr, }) // Build the fragmentation header if needed. @@ -666,19 +994,19 @@ func TestIPv6ReceiveControl(t *testing.T) { // Give packet to IPv6 endpoint, dispatcher will validate that // it's ok. - o.protocol = 10 - o.srcAddr = remoteIpv6Addr - o.dstAddr = localIpv6Addr - o.contents = view[dataOffset:] - o.typ = c.expectedTyp - o.extra = c.expectedExtra + nic.tester.protocol = 10 + nic.tester.srcAddr = remoteIPv6Addr + nic.tester.dstAddr = localIPv6Addr + nic.tester.contents = view[dataOffset:] + nic.tester.typ = c.expectedTyp + nic.tester.extra = c.expectedExtra // Set ICMPv6 checksum. - icmp.SetChecksum(header.ICMPv6Checksum(icmp, outerSrcAddr, localIpv6Addr, buffer.VectorisedView{})) + icmp.SetChecksum(header.ICMPv6Checksum(icmp, outerSrcAddr, localIPv6Addr, buffer.VectorisedView{})) ep.HandlePacket(&r, truncatedPacket(view, c.trunc, header.IPv6MinimumSize)) - if want := c.expectedCount; o.controlCalls != want { - t.Fatalf("Bad number of control calls for %q case: got %v, want %v", c.name, o.controlCalls, want) + if want := c.expectedCount; nic.tester.controlCalls != want { + t.Fatalf("Bad number of control calls for %q case: got %v, want %v", c.name, nic.tester.controlCalls, want) } }) } diff --git a/pkg/tcpip/network/ipv4/BUILD b/pkg/tcpip/network/ipv4/BUILD index c82593e71..ee2c23e91 100644 --- a/pkg/tcpip/network/ipv4/BUILD +++ b/pkg/tcpip/network/ipv4/BUILD @@ -10,9 +10,11 @@ go_library( ], visibility = ["//visibility:public"], deps = [ + "//pkg/sync", "//pkg/tcpip", "//pkg/tcpip/buffer", "//pkg/tcpip/header", + "//pkg/tcpip/header/parse", "//pkg/tcpip/network/fragmentation", "//pkg/tcpip/network/hash", "//pkg/tcpip/stack", @@ -26,12 +28,14 @@ go_test( deps = [ "//pkg/tcpip", "//pkg/tcpip/buffer", + "//pkg/tcpip/checker", "//pkg/tcpip/header", "//pkg/tcpip/link/channel", "//pkg/tcpip/link/sniffer", "//pkg/tcpip/network/ipv4", "//pkg/tcpip/network/testutil", "//pkg/tcpip/stack", + "//pkg/tcpip/transport/icmp", "//pkg/tcpip/transport/tcp", "//pkg/tcpip/transport/udp", "//pkg/waiter", diff --git a/pkg/tcpip/network/ipv4/icmp.go b/pkg/tcpip/network/ipv4/icmp.go index b5659a36b..a8985ff5d 100644 --- a/pkg/tcpip/network/ipv4/icmp.go +++ b/pkg/tcpip/network/ipv4/icmp.go @@ -15,6 +15,9 @@ package ipv4 import ( + "fmt" + + "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/header" "gvisor.dev/gvisor/pkg/tcpip/stack" @@ -39,7 +42,7 @@ func (e *endpoint) handleControl(typ stack.ControlType, extra uint32, pkt *stack // Drop packet if it doesn't have the basic IPv4 header or if the // original source address doesn't match an address we own. src := hdr.SourceAddress() - if e.stack.CheckLocalAddress(e.NICID(), ProtocolNumber, src) == 0 { + if e.protocol.stack.CheckLocalAddress(e.nic.ID(), ProtocolNumber, src) == 0 { return } @@ -105,11 +108,11 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer) { // source address MUST be one of its own IP addresses (but not a broadcast // or multicast address). localAddr := r.LocalAddress - if r.IsInboundBroadcast() || header.IsV4MulticastAddress(r.LocalAddress) { + if r.IsInboundBroadcast() || header.IsV4MulticastAddress(localAddr) { localAddr = "" } - r, err := r.Stack().FindRoute(e.NICID(), localAddr, r.RemoteAddress, ProtocolNumber, false /* multicastLoop */) + r, err := r.Stack().FindRoute(e.nic.ID(), localAddr, r.RemoteAddress, ProtocolNumber, false /* multicastLoop */) if err != nil { // If we cannot find a route to the destination, silently drop the packet. return @@ -131,7 +134,10 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer) { ReserveHeaderBytes: int(r.MaxHeaderLength()), Data: dataVV, }) - + // TODO(gvisor.dev/issue/3810): When adding protocol numbers into the header + // information we will have to change this code to handle the ICMP header + // no longer being in the data buffer. + replyPkt.TransportProtocolNumber = header.ICMPv4ProtocolNumber // Send out the reply packet. sent := stats.ICMP.V4PacketsSent if err := r.WritePacket(nil /* gso */, stack.NetworkHeaderParams{ @@ -193,3 +199,177 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer) { received.Invalid.Increment() } } + +// ======= ICMP Error packet generation ========= + +// icmpReason is a marker interface for IPv4 specific ICMP errors. +type icmpReason interface { + isICMPReason() +} + +// icmpReasonPortUnreachable is an error where the transport protocol has no +// listener and no alternative means to inform the sender. +type icmpReasonPortUnreachable struct{} + +func (*icmpReasonPortUnreachable) isICMPReason() {} + +// icmpReasonProtoUnreachable is an error where the transport protocol is +// not supported. +type icmpReasonProtoUnreachable struct{} + +func (*icmpReasonProtoUnreachable) isICMPReason() {} + +// returnError takes an error descriptor and generates the appropriate ICMP +// error packet for IPv4 and sends it back to the remote device that sent +// the problematic packet. It incorporates as much of that packet as +// possible as well as any error metadata as is available. returnError +// expects pkt to hold a valid IPv4 packet as per the wire format. +func returnError(r *stack.Route, reason icmpReason, pkt *stack.PacketBuffer) *tcpip.Error { + sent := r.Stats().ICMP.V4PacketsSent + if !r.Stack().AllowICMPMessage() { + sent.RateLimited.Increment() + return nil + } + + // We check we are responding only when we are allowed to. + // See RFC 1812 section 4.3.2.7 (shown below). + // + // ========= + // 4.3.2.7 When Not to Send ICMP Errors + // + // An ICMP error message MUST NOT be sent as the result of receiving: + // + // o An ICMP error message, or + // + // o A packet which fails the IP header validation tests described in + // Section [5.2.2] (except where that section specifically permits + // the sending of an ICMP error message), or + // + // o A packet destined to an IP broadcast or IP multicast address, or + // + // o A packet sent as a Link Layer broadcast or multicast, or + // + // o Any fragment of a datagram other then the first fragment (i.e., a + // packet for which the fragment offset in the IP header is nonzero). + // + // TODO(gvisor.dev/issues/4058): Make sure we don't send ICMP errors in + // response to a non-initial fragment, but it currently can not happen. + + if r.IsInboundBroadcast() || header.IsV4MulticastAddress(r.LocalAddress) || r.RemoteAddress == header.IPv4Any { + return nil + } + + networkHeader := pkt.NetworkHeader().View() + transportHeader := pkt.TransportHeader().View() + + // Don't respond to icmp error packets. + if header.IPv4(networkHeader).Protocol() == uint8(header.ICMPv4ProtocolNumber) { + // TODO(gvisor.dev/issue/3810): + // Unfortunately the current stack pretty much always has ICMPv4 headers + // in the Data section of the packet but there is no guarantee that is the + // case. If this is the case grab the header to make it like all other + // packet types. When this is cleaned up the Consume should be removed. + if transportHeader.IsEmpty() { + var ok bool + transportHeader, ok = pkt.TransportHeader().Consume(header.ICMPv4MinimumSize) + if !ok { + return nil + } + } else if transportHeader.Size() < header.ICMPv4MinimumSize { + return nil + } + // We need to decide to explicitly name the packets we can respond to or + // the ones we can not respond to. The decision is somewhat arbitrary and + // if problems arise this could be reversed. It was judged less of a breach + // of protocol to not respond to unknown non-error packets than to respond + // to unknown error packets so we take the first approach. + switch header.ICMPv4(transportHeader).Type() { + case + header.ICMPv4EchoReply, + header.ICMPv4Echo, + header.ICMPv4Timestamp, + header.ICMPv4TimestampReply, + header.ICMPv4InfoRequest, + header.ICMPv4InfoReply: + default: + // Assume any type we don't know about may be an error type. + return nil + } + } + + // Now work out how much of the triggering packet we should return. + // As per RFC 1812 Section 4.3.2.3 + // + // ICMP datagram SHOULD contain as much of the original + // datagram as possible without the length of the ICMP + // datagram exceeding 576 bytes. + // + // NOTE: The above RFC referenced is different from the original + // recommendation in RFC 1122 and RFC 792 where it mentioned that at + // least 8 bytes of the payload must be included. Today linux and other + // systems implement the RFC 1812 definition and not the original + // requirement. We treat 8 bytes as the minimum but will try send more. + mtu := int(r.MTU()) + if mtu > header.IPv4MinimumProcessableDatagramSize { + mtu = header.IPv4MinimumProcessableDatagramSize + } + headerLen := int(r.MaxHeaderLength()) + header.ICMPv4MinimumSize + available := int(mtu) - headerLen + + if available < header.IPv4MinimumSize+header.ICMPv4MinimumErrorPayloadSize { + return nil + } + + payloadLen := networkHeader.Size() + transportHeader.Size() + pkt.Data.Size() + if payloadLen > available { + payloadLen = available + } + + // The buffers used by pkt may be used elsewhere in the system. + // For example, an AF_RAW or AF_PACKET socket may use what the transport + // protocol considers an unreachable destination. Thus we deep copy pkt to + // prevent multiple ownership and SR errors. The new copy is a vectorized + // view with the entire incoming IP packet reassembled and truncated as + // required. This is now the payload of the new ICMP packet and no longer + // considered a packet in its own right. + newHeader := append(buffer.View(nil), networkHeader...) + newHeader = append(newHeader, transportHeader...) + payload := newHeader.ToVectorisedView() + payload.AppendView(pkt.Data.ToView()) + payload.CapLength(payloadLen) + + icmpPkt := stack.NewPacketBuffer(stack.PacketBufferOptions{ + ReserveHeaderBytes: headerLen, + Data: payload, + }) + + icmpPkt.TransportProtocolNumber = header.ICMPv4ProtocolNumber + + icmpHdr := header.ICMPv4(icmpPkt.TransportHeader().Push(header.ICMPv4MinimumSize)) + switch reason.(type) { + case *icmpReasonPortUnreachable: + icmpHdr.SetCode(header.ICMPv4PortUnreachable) + case *icmpReasonProtoUnreachable: + icmpHdr.SetCode(header.ICMPv4ProtoUnreachable) + default: + panic(fmt.Sprintf("unsupported ICMP type %T", reason)) + } + icmpHdr.SetType(header.ICMPv4DstUnreachable) + icmpHdr.SetChecksum(header.ICMPv4Checksum(icmpHdr, icmpPkt.Data)) + counter := sent.DstUnreachable + + if err := r.WritePacket( + nil, /* gso */ + stack.NetworkHeaderParams{ + Protocol: header.ICMPv4ProtocolNumber, + TTL: r.DefaultTTL(), + TOS: stack.DefaultTOS, + }, + icmpPkt, + ); err != nil { + sent.Dropped.Increment() + return err + } + counter.Increment() + return nil +} diff --git a/pkg/tcpip/network/ipv4/ipv4.go b/pkg/tcpip/network/ipv4/ipv4.go index f4394749d..1f6e14c3f 100644 --- a/pkg/tcpip/network/ipv4/ipv4.go +++ b/pkg/tcpip/network/ipv4/ipv4.go @@ -12,20 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Package ipv4 contains the implementation of the ipv4 network protocol. To use -// it in the networking stack, this package must be added to the project, and -// activated on the stack by passing ipv4.NewProtocol() as one of the network -// protocols when calling stack.New(). Then endpoints can be created by passing -// ipv4.ProtocolNumber as the network protocol number when calling -// Stack.NewEndpoint(). +// Package ipv4 contains the implementation of the ipv4 network protocol. package ipv4 import ( + "fmt" "sync/atomic" + "gvisor.dev/gvisor/pkg/sync" "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/header" + "gvisor.dev/gvisor/pkg/tcpip/header/parse" "gvisor.dev/gvisor/pkg/tcpip/network/fragmentation" "gvisor.dev/gvisor/pkg/tcpip/network/hash" "gvisor.dev/gvisor/pkg/tcpip/stack" @@ -50,22 +48,115 @@ const ( fragmentblockSize = 8 ) +var ipv4BroadcastAddr = header.IPv4Broadcast.WithPrefix() + +var _ stack.GroupAddressableEndpoint = (*endpoint)(nil) +var _ stack.AddressableEndpoint = (*endpoint)(nil) +var _ stack.NetworkEndpoint = (*endpoint)(nil) + type endpoint struct { - nicID tcpip.NICID + nic stack.NetworkInterface linkEP stack.LinkEndpoint dispatcher stack.TransportDispatcher protocol *protocol - stack *stack.Stack + + // enabled is set to 1 when the enpoint is enabled and 0 when it is + // disabled. + // + // Must be accessed using atomic operations. + enabled uint32 + + mu struct { + sync.RWMutex + + addressableEndpointState stack.AddressableEndpointState + } } // NewEndpoint creates a new ipv4 endpoint. -func (p *protocol) NewEndpoint(nicID tcpip.NICID, _ stack.LinkAddressCache, _ stack.NUDHandler, dispatcher stack.TransportDispatcher, linkEP stack.LinkEndpoint, st *stack.Stack) stack.NetworkEndpoint { - return &endpoint{ - nicID: nicID, - linkEP: linkEP, +func (p *protocol) NewEndpoint(nic stack.NetworkInterface, _ stack.LinkAddressCache, _ stack.NUDHandler, dispatcher stack.TransportDispatcher) stack.NetworkEndpoint { + e := &endpoint{ + nic: nic, + linkEP: nic.LinkEndpoint(), dispatcher: dispatcher, protocol: p, - stack: st, + } + e.mu.addressableEndpointState.Init(e) + return e +} + +// Enable implements stack.NetworkEndpoint. +func (e *endpoint) Enable() *tcpip.Error { + e.mu.Lock() + defer e.mu.Unlock() + + // If the NIC is not enabled, the endpoint can't do anything meaningful so + // don't enable the endpoint. + if !e.nic.Enabled() { + return tcpip.ErrNotPermitted + } + + // If the endpoint is already enabled, there is nothing for it to do. + if !e.setEnabled(true) { + return nil + } + + // Create an endpoint to receive broadcast packets on this interface. + ep, err := e.mu.addressableEndpointState.AddAndAcquirePermanentAddress(ipv4BroadcastAddr, stack.NeverPrimaryEndpoint, stack.AddressConfigStatic, false /* deprecated */) + if err != nil { + return err + } + // We have no need for the address endpoint. + ep.DecRef() + + // As per RFC 1122 section 3.3.7, all hosts should join the all-hosts + // multicast group. Note, the IANA calls the all-hosts multicast group the + // all-systems multicast group. + _, err = e.mu.addressableEndpointState.JoinGroup(header.IPv4AllSystems) + return err +} + +// Enabled implements stack.NetworkEndpoint. +func (e *endpoint) Enabled() bool { + return e.nic.Enabled() && e.isEnabled() +} + +// isEnabled returns true if the endpoint is enabled, regardless of the +// enabled status of the NIC. +func (e *endpoint) isEnabled() bool { + return atomic.LoadUint32(&e.enabled) == 1 +} + +// setEnabled sets the enabled status for the endpoint. +// +// Returns true if the enabled status was updated. +func (e *endpoint) setEnabled(v bool) bool { + if v { + return atomic.SwapUint32(&e.enabled, 1) == 0 + } + return atomic.SwapUint32(&e.enabled, 0) == 1 +} + +// Disable implements stack.NetworkEndpoint. +func (e *endpoint) Disable() { + e.mu.Lock() + defer e.mu.Unlock() + e.disableLocked() +} + +func (e *endpoint) disableLocked() { + if !e.setEnabled(false) { + return + } + + // The endpoint may have already left the multicast group. + if _, err := e.mu.addressableEndpointState.LeaveGroup(header.IPv4AllSystems); err != nil && err != tcpip.ErrBadLocalAddress { + panic(fmt.Sprintf("unexpected error when leaving group = %s: %s", header.IPv4AllSystems, err)) + } + + // The address may have already been removed. + if err := e.mu.addressableEndpointState.RemovePermanentAddress(ipv4BroadcastAddr.Address); err != nil && err != tcpip.ErrBadLocalAddress { + panic(fmt.Sprintf("unexpected error when removing address = %s: %s", ipv4BroadcastAddr.Address, err)) } } @@ -80,16 +171,6 @@ func (e *endpoint) MTU() uint32 { return calculateMTU(e.linkEP.MTU()) } -// Capabilities implements stack.NetworkEndpoint.Capabilities. -func (e *endpoint) Capabilities() stack.LinkEndpointCapabilities { - return e.linkEP.Capabilities() -} - -// NICID returns the ID of the NIC this endpoint belongs to. -func (e *endpoint) NICID() tcpip.NICID { - return e.nicID -} - // MaxHeaderLength returns the maximum length needed by ipv4 headers (and // underlying protocols). func (e *endpoint) MaxHeaderLength() uint16 { @@ -197,6 +278,7 @@ func (e *endpoint) writePacketFragments(r *stack.Route, gso *stack.GSO, mtu int, // Send out the fragment. if err := e.linkEP.WritePacket(r, gso, ProtocolNumber, fragPkt); err != nil { + r.Stats().IP.OutgoingPacketErrors.IncrementBy(uint64(n - i)) return err } r.Stats().IP.PacketsSent.Increment() @@ -231,21 +313,24 @@ func (e *endpoint) WritePacket(r *stack.Route, gso *stack.GSO, params stack.Netw // iptables filtering. All packets that reach here are locally // generated. - nicName := e.stack.FindNICNameFromID(e.NICID()) - ipt := e.stack.IPTables() + nicName := e.protocol.stack.FindNICNameFromID(e.nic.ID()) + ipt := e.protocol.stack.IPTables() if ok := ipt.Check(stack.Output, pkt, gso, r, "", nicName); !ok { // iptables is telling us to drop the packet. + r.Stats().IP.IPTablesOutputDropped.Increment() 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 the packet is manipulated as per NAT Output 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()) + ep, err := e.protocol.stack.FindNetworkEndpoint(header.IPv4ProtocolNumber, netHeader.DestinationAddress()) if err == nil { route := r.ReverseRoute(netHeader.SourceAddress(), netHeader.DestinationAddress()) ep.HandlePacket(&route, pkt) @@ -265,6 +350,7 @@ func (e *endpoint) WritePacket(r *stack.Route, gso *stack.GSO, params stack.Netw return e.writePacketFragments(r, gso, int(e.linkEP.MTU()), pkt) } if err := e.linkEP.WritePacket(r, gso, ProtocolNumber, pkt); err != nil { + r.Stats().IP.OutgoingPacketErrors.Increment() return err } r.Stats().IP.PacketsSent.Increment() @@ -285,20 +371,24 @@ func (e *endpoint) WritePackets(r *stack.Route, gso *stack.GSO, pkts stack.Packe pkt = pkt.Next() } - nicName := e.stack.FindNICNameFromID(e.NICID()) + nicName := e.protocol.stack.FindNICNameFromID(e.nic.ID()) // iptables filtering. All packets that reach here are locally // generated. - ipt := e.stack.IPTables() + ipt := e.protocol.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)) + if err != nil { + r.Stats().IP.OutgoingPacketErrors.IncrementBy(uint64(pkts.Len() - n)) + } return n, err } + r.Stats().IP.IPTablesOutputDropped.IncrementBy(uint64(len(dropped))) - // Slow Path as we are dropping some packets in the batch degrade to + // 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() { @@ -307,7 +397,7 @@ func (e *endpoint) WritePackets(r *stack.Route, gso *stack.GSO, pkts stack.Packe } if _, ok := natPkts[pkt]; ok { netHeader := header.IPv4(pkt.NetworkHeader().View()) - if ep, err := e.stack.FindNetworkEndpoint(header.IPv4ProtocolNumber, netHeader.DestinationAddress()); err == nil { + if ep, err := e.protocol.stack.FindNetworkEndpoint(header.IPv4ProtocolNumber, netHeader.DestinationAddress()); err == nil { src := netHeader.SourceAddress() dst := netHeader.DestinationAddress() route := r.ReverseRoute(src, dst) @@ -318,12 +408,16 @@ func (e *endpoint) WritePackets(r *stack.Route, gso *stack.GSO, pkts stack.Packe } if err := e.linkEP.WritePacket(r, gso, ProtocolNumber, pkt); err != nil { r.Stats().IP.PacketsSent.IncrementBy(uint64(n)) - return n, err + r.Stats().IP.OutgoingPacketErrors.IncrementBy(uint64(pkts.Len() - n - len(dropped))) + // Dropped packets aren't errors, so include them in + // the return value. + return n + len(dropped), err } n++ } r.Stats().IP.PacketsSent.IncrementBy(uint64(n)) - return n, nil + // Dropped packets aren't errors, so include them in the return value. + return n + len(dropped), nil } // WriteHeaderIncludedPacket writes a packet already containing a network @@ -373,25 +467,42 @@ func (e *endpoint) WriteHeaderIncludedPacket(r *stack.Route, pkt *stack.PacketBu return nil } + if err := e.linkEP.WritePacket(r, nil /* gso */, ProtocolNumber, pkt); err != nil { + r.Stats().IP.OutgoingPacketErrors.Increment() + return err + } r.Stats().IP.PacketsSent.Increment() - - return e.linkEP.WritePacket(r, nil /* gso */, ProtocolNumber, pkt) + return nil } // HandlePacket is called by the link layer when new ipv4 packets arrive for // this endpoint. func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { + if !e.isEnabled() { + return + } + 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 } + // As per RFC 1122 section 3.2.1.3: + // When a host sends any datagram, the IP source address MUST + // be one of its own IP addresses (but not a broadcast or + // multicast address). + if r.IsOutboundBroadcast() || header.IsV4MulticastAddress(r.RemoteAddress) { + r.Stats().IP.InvalidSourceAddressesReceived.Increment() + return + } + // iptables filtering. All packets that reach here are intended for // this machine and will not be forwarded. - ipt := e.stack.IPTables() + ipt := e.protocol.stack.IPTables() if ok := ipt.Check(stack.Input, pkt, nil, nil, "", ""); !ok { // iptables is telling us to drop the packet. + r.Stats().IP.IPTablesInputDropped.Increment() return } @@ -444,27 +555,165 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { return } } + + r.Stats().IP.PacketsDelivered.Increment() p := h.TransportProtocol() if p == header.ICMPv4ProtocolNumber { + // TODO(gvisor.dev/issues/3810): when we sort out ICMP and transport + // headers, the setting of the transport number here should be + // unnecessary and removed. + pkt.TransportProtocolNumber = p e.handleICMP(r, pkt) return } - r.Stats().IP.PacketsDelivered.Increment() - e.dispatcher.DeliverTransportPacket(r, p, pkt) + + switch res := e.dispatcher.DeliverTransportPacket(r, p, pkt); res { + case stack.TransportPacketHandled: + case stack.TransportPacketDestinationPortUnreachable: + // As per RFC: 1122 Section 3.2.2.1 A host SHOULD generate Destination + // Unreachable messages with code: + // 3 (Port Unreachable), when the designated transport protocol + // (e.g., UDP) is unable to demultiplex the datagram but has no + // protocol mechanism to inform the sender. + _ = returnError(r, &icmpReasonPortUnreachable{}, pkt) + case stack.TransportPacketProtocolUnreachable: + // As per RFC: 1122 Section 3.2.2.1 + // A host SHOULD generate Destination Unreachable messages with code: + // 2 (Protocol Unreachable), when the designated transport protocol + // is not supported + _ = returnError(r, &icmpReasonProtoUnreachable{}, pkt) + default: + panic(fmt.Sprintf("unrecognized result from DeliverTransportPacket = %d", res)) + } } // Close cleans up resources associated with the endpoint. -func (e *endpoint) Close() {} +func (e *endpoint) Close() { + e.mu.Lock() + defer e.mu.Unlock() + + e.disableLocked() + e.mu.addressableEndpointState.Cleanup() +} + +// AddAndAcquirePermanentAddress implements stack.AddressableEndpoint. +func (e *endpoint) AddAndAcquirePermanentAddress(addr tcpip.AddressWithPrefix, peb stack.PrimaryEndpointBehavior, configType stack.AddressConfigType, deprecated bool) (stack.AddressEndpoint, *tcpip.Error) { + e.mu.Lock() + defer e.mu.Unlock() + return e.mu.addressableEndpointState.AddAndAcquirePermanentAddress(addr, peb, configType, deprecated) +} + +// RemovePermanentAddress implements stack.AddressableEndpoint. +func (e *endpoint) RemovePermanentAddress(addr tcpip.Address) *tcpip.Error { + e.mu.Lock() + defer e.mu.Unlock() + return e.mu.addressableEndpointState.RemovePermanentAddress(addr) +} + +// MainAddress implements stack.AddressableEndpoint. +func (e *endpoint) MainAddress() tcpip.AddressWithPrefix { + e.mu.RLock() + defer e.mu.RUnlock() + return e.mu.addressableEndpointState.MainAddress() +} + +// AcquireAssignedAddress implements stack.AddressableEndpoint. +func (e *endpoint) AcquireAssignedAddress(localAddr tcpip.Address, allowTemp bool, tempPEB stack.PrimaryEndpointBehavior) stack.AddressEndpoint { + e.mu.Lock() + defer e.mu.Unlock() + + loopback := e.nic.IsLoopback() + addressEndpoint := e.mu.addressableEndpointState.ReadOnly().AddrOrMatching(localAddr, allowTemp, func(addressEndpoint stack.AddressEndpoint) bool { + subnet := addressEndpoint.AddressWithPrefix().Subnet() + // IPv4 has a notion of a subnet broadcast address and considers the + // loopback interface bound to an address's whole subnet (on linux). + return subnet.IsBroadcast(localAddr) || (loopback && subnet.Contains(localAddr)) + }) + if addressEndpoint != nil { + return addressEndpoint + } + + if !allowTemp { + return nil + } + + addr := localAddr.WithPrefix() + addressEndpoint, err := e.mu.addressableEndpointState.AddAndAcquireTemporaryAddress(addr, tempPEB) + if err != nil { + // AddAddress only returns an error if the address is already assigned, + // but we just checked above if the address exists so we expect no error. + panic(fmt.Sprintf("e.mu.addressableEndpointState.AddAndAcquireTemporaryAddress(%s, %d): %s", addr, tempPEB, err)) + } + return addressEndpoint +} + +// AcquireOutgoingPrimaryAddress implements stack.AddressableEndpoint. +func (e *endpoint) AcquireOutgoingPrimaryAddress(remoteAddr tcpip.Address, allowExpired bool) stack.AddressEndpoint { + e.mu.RLock() + defer e.mu.RUnlock() + return e.mu.addressableEndpointState.AcquireOutgoingPrimaryAddress(remoteAddr, allowExpired) +} + +// PrimaryAddresses implements stack.AddressableEndpoint. +func (e *endpoint) PrimaryAddresses() []tcpip.AddressWithPrefix { + e.mu.RLock() + defer e.mu.RUnlock() + return e.mu.addressableEndpointState.PrimaryAddresses() +} + +// PermanentAddresses implements stack.AddressableEndpoint. +func (e *endpoint) PermanentAddresses() []tcpip.AddressWithPrefix { + e.mu.RLock() + defer e.mu.RUnlock() + return e.mu.addressableEndpointState.PermanentAddresses() +} + +// JoinGroup implements stack.GroupAddressableEndpoint. +func (e *endpoint) JoinGroup(addr tcpip.Address) (bool, *tcpip.Error) { + if !header.IsV4MulticastAddress(addr) { + return false, tcpip.ErrBadAddress + } + + e.mu.Lock() + defer e.mu.Unlock() + return e.mu.addressableEndpointState.JoinGroup(addr) +} + +// LeaveGroup implements stack.GroupAddressableEndpoint. +func (e *endpoint) LeaveGroup(addr tcpip.Address) (bool, *tcpip.Error) { + e.mu.Lock() + defer e.mu.Unlock() + return e.mu.addressableEndpointState.LeaveGroup(addr) +} + +// IsInGroup implements stack.GroupAddressableEndpoint. +func (e *endpoint) IsInGroup(addr tcpip.Address) bool { + e.mu.RLock() + defer e.mu.RUnlock() + return e.mu.addressableEndpointState.IsInGroup(addr) +} + +var _ stack.ForwardingNetworkProtocol = (*protocol)(nil) +var _ stack.NetworkProtocol = (*protocol)(nil) type protocol struct { - ids []uint32 - hashIV uint32 + stack *stack.Stack // defaultTTL is the current default TTL for the protocol. Only the - // uint8 portion of it is meaningful and it must be accessed - // atomically. + // uint8 portion of it is meaningful. + // + // Must be accessed using atomic operations. defaultTTL uint32 + // forwarding is set to 1 when the protocol has forwarding enabled and 0 + // when it is disabled. + // + // Must be accessed using atomic operations. + forwarding uint32 + + ids []uint32 + hashIV uint32 + fragmentation *fragmentation.Fragmentation } @@ -527,37 +776,28 @@ func (*protocol) Close() {} // Wait implements stack.TransportProtocol.Wait. func (*protocol) Wait() {} -// Parse implements stack.TransportProtocol.Parse. +// Parse implements stack.NetworkProtocol.Parse. func (*protocol) Parse(pkt *stack.PacketBuffer) (proto tcpip.TransportProtocolNumber, hasTransportHdr bool, ok bool) { - hdr, ok := pkt.Data.PullUp(header.IPv4MinimumSize) - if !ok { + if ok := parse.IPv4(pkt); !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) + ipHdr := header.IPv4(pkt.NetworkHeader().View()) + return ipHdr.TransportProtocol(), !ipHdr.More() && ipHdr.FragmentOffset() == 0, true +} - // If this is a fragment, don't bother parsing the transport header. - parseTransportHeader := true - if ipHdr.More() || ipHdr.FragmentOffset() != 0 { - parseTransportHeader = false - } +// Forwarding implements stack.ForwardingNetworkProtocol. +func (p *protocol) Forwarding() bool { + return uint8(atomic.LoadUint32(&p.forwarding)) == 1 +} - pkt.NetworkProtocolNumber = header.IPv4ProtocolNumber - pkt.Data.CapLength(int(ipHdr.TotalLength()) - len(hdr)) - return ipHdr.TransportProtocol(), parseTransportHeader, true +// SetForwarding implements stack.ForwardingNetworkProtocol. +func (p *protocol) SetForwarding(v bool) { + if v { + atomic.StoreUint32(&p.forwarding, 1) + } else { + atomic.StoreUint32(&p.forwarding, 0) + } } // calculateMTU calculates the network-layer payload MTU based on the link-layer @@ -581,7 +821,7 @@ func hashRoute(r *stack.Route, protocol tcpip.TransportProtocolNumber, hashIV ui } // NewProtocol returns an IPv4 network protocol. -func NewProtocol() stack.NetworkProtocol { +func NewProtocol(s *stack.Stack) stack.NetworkProtocol { ids := make([]uint32, buckets) // Randomly initialize hashIV and the ids. @@ -592,9 +832,10 @@ func NewProtocol() stack.NetworkProtocol { hashIV := r[buckets] return &protocol{ + stack: s, ids: ids, hashIV: hashIV, defaultTTL: DefaultTTL, - fragmentation: fragmentation.NewFragmentation(fragmentblockSize, fragmentation.HighFragThreshold, fragmentation.LowFragThreshold, fragmentation.DefaultReassembleTimeout), + fragmentation: fragmentation.NewFragmentation(fragmentblockSize, fragmentation.HighFragThreshold, fragmentation.LowFragThreshold, fragmentation.DefaultReassembleTimeout, s.Clock()), } } diff --git a/pkg/tcpip/network/ipv4/ipv4_test.go b/pkg/tcpip/network/ipv4/ipv4_test.go index 5e50558e8..33cd5a3eb 100644 --- a/pkg/tcpip/network/ipv4/ipv4_test.go +++ b/pkg/tcpip/network/ipv4/ipv4_test.go @@ -17,17 +17,21 @@ package ipv4_test import ( "bytes" "encoding/hex" + "math" + "net" "testing" "github.com/google/go-cmp/cmp" "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" + "gvisor.dev/gvisor/pkg/tcpip/checker" "gvisor.dev/gvisor/pkg/tcpip/header" "gvisor.dev/gvisor/pkg/tcpip/link/channel" "gvisor.dev/gvisor/pkg/tcpip/link/sniffer" "gvisor.dev/gvisor/pkg/tcpip/network/ipv4" "gvisor.dev/gvisor/pkg/tcpip/network/testutil" "gvisor.dev/gvisor/pkg/tcpip/stack" + "gvisor.dev/gvisor/pkg/tcpip/transport/icmp" "gvisor.dev/gvisor/pkg/tcpip/transport/tcp" "gvisor.dev/gvisor/pkg/tcpip/transport/udp" "gvisor.dev/gvisor/pkg/waiter" @@ -35,8 +39,8 @@ import ( func TestExcludeBroadcast(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, }) const defaultMTU = 65536 @@ -91,6 +95,246 @@ func TestExcludeBroadcast(t *testing.T) { }) } +// TestIPv4Sanity sends IP/ICMP packets with various problems to the stack and +// checks the response. +func TestIPv4Sanity(t *testing.T) { + const ( + defaultMTU = header.IPv6MinimumMTU + ttl = 255 + nicID = 1 + randomSequence = 123 + randomIdent = 42 + ) + var ( + ipv4Addr = tcpip.AddressWithPrefix{ + Address: tcpip.Address(net.ParseIP("192.168.1.58").To4()), + PrefixLen: 24, + } + remoteIPv4Addr = tcpip.Address(net.ParseIP("10.0.0.1").To4()) + ) + + tests := []struct { + name string + headerLength uint8 + minTotalLength uint16 + transportProtocol uint8 + TTL uint8 + shouldFail bool + expectICMP bool + ICMPType header.ICMPv4Type + ICMPCode header.ICMPv4Code + }{ + { + name: "valid", + headerLength: header.IPv4MinimumSize, + minTotalLength: defaultMTU, + transportProtocol: uint8(header.ICMPv4ProtocolNumber), + TTL: ttl, + shouldFail: false, + }, + // The TTL tests check that we are not rejecting an incoming packet + // with a zero or one TTL, which has been a point of confusion in the + // past as RFC 791 says: "If this field contains the value zero, then the + // datagram must be destroyed". However RFC 1122 section 3.2.1.7 clarifies + // for the case of the destination host, stating as follows. + // + // A host MUST NOT send a datagram with a Time-to-Live (TTL) + // value of zero. + // + // A host MUST NOT discard a datagram just because it was + // received with TTL less than 2. + { + name: "zero TTL", + headerLength: header.IPv4MinimumSize, + minTotalLength: defaultMTU, + transportProtocol: uint8(header.ICMPv4ProtocolNumber), + TTL: 0, + shouldFail: false, + }, + { + name: "one TTL", + headerLength: header.IPv4MinimumSize, + minTotalLength: defaultMTU, + transportProtocol: uint8(header.ICMPv4ProtocolNumber), + TTL: 1, + shouldFail: false, + }, + { + name: "bad header length", + headerLength: header.IPv4MinimumSize - 1, + minTotalLength: defaultMTU, + transportProtocol: uint8(header.ICMPv4ProtocolNumber), + TTL: ttl, + shouldFail: true, + expectICMP: false, + }, + { + name: "bad total length (0)", + headerLength: header.IPv4MinimumSize, + minTotalLength: 0, + transportProtocol: uint8(header.ICMPv4ProtocolNumber), + TTL: ttl, + shouldFail: true, + expectICMP: false, + }, + { + name: "bad total length (ip - 1)", + headerLength: header.IPv4MinimumSize, + minTotalLength: uint16(header.IPv4MinimumSize - 1), + transportProtocol: uint8(header.ICMPv4ProtocolNumber), + TTL: ttl, + shouldFail: true, + expectICMP: false, + }, + { + name: "bad total length (ip + icmp - 1)", + headerLength: header.IPv4MinimumSize, + minTotalLength: uint16(header.IPv4MinimumSize + header.ICMPv4MinimumSize - 1), + transportProtocol: uint8(header.ICMPv4ProtocolNumber), + TTL: ttl, + shouldFail: true, + expectICMP: false, + }, + { + name: "bad protocol", + headerLength: header.IPv4MinimumSize, + minTotalLength: defaultMTU, + transportProtocol: 99, + TTL: ttl, + shouldFail: true, + expectICMP: true, + ICMPType: header.ICMPv4DstUnreachable, + ICMPCode: header.ICMPv4ProtoUnreachable, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{icmp.NewProtocol4}, + }) + // We expect at most a single packet in response to our ICMP Echo Request. + e := channel.New(1, defaultMTU, "") + if err := s.CreateNIC(nicID, e); err != nil { + t.Fatalf("CreateNIC(%d, _): %s", nicID, err) + } + ipv4ProtoAddr := tcpip.ProtocolAddress{Protocol: header.IPv4ProtocolNumber, AddressWithPrefix: ipv4Addr} + if err := s.AddProtocolAddress(nicID, ipv4ProtoAddr); err != nil { + t.Fatalf("AddProtocolAddress(%d, %#v): %s", nicID, ipv4ProtoAddr, err) + } + + // Default routes for IPv4 so ICMP can find a route to the remote + // node when attempting to send the ICMP Echo Reply. + s.SetRouteTable([]tcpip.Route{ + tcpip.Route{ + Destination: header.IPv4EmptySubnet, + NIC: nicID, + }, + }) + + ipHeaderLength := header.IPv4MinimumSize + totalLen := uint16(ipHeaderLength + header.ICMPv4MinimumSize) + hdr := buffer.NewPrependable(int(totalLen)) + icmp := header.ICMPv4(hdr.Prepend(header.ICMPv4MinimumSize)) + + // Specify ident/seq to make sure we get the same in the response. + icmp.SetIdent(randomIdent) + icmp.SetSequence(randomSequence) + icmp.SetType(header.ICMPv4Echo) + icmp.SetCode(header.ICMPv4UnusedCode) + icmp.SetChecksum(0) + icmp.SetChecksum(^header.Checksum(icmp, 0)) + ip := header.IPv4(hdr.Prepend(ipHeaderLength)) + if test.minTotalLength < totalLen { + totalLen = test.minTotalLength + } + ip.Encode(&header.IPv4Fields{ + IHL: test.headerLength, + TotalLength: totalLen, + Protocol: test.transportProtocol, + TTL: test.TTL, + SrcAddr: remoteIPv4Addr, + DstAddr: ipv4Addr.Address, + }) + requestPkt := stack.NewPacketBuffer(stack.PacketBufferOptions{ + Data: hdr.View().ToVectorisedView(), + }) + e.InjectInbound(header.IPv4ProtocolNumber, requestPkt) + reply, ok := e.Read() + if !ok { + if test.shouldFail { + if test.expectICMP { + t.Fatal("expected ICMP error response missing") + } + return // Expected silent failure. + } + t.Fatal("expected ICMP echo reply missing") + } + + // Check the route that brought the packet to us. + if reply.Route.LocalAddress != ipv4Addr.Address { + t.Errorf("got pkt.Route.LocalAddress = %s, want = %s", reply.Route.LocalAddress, ipv4Addr.Address) + } + if reply.Route.RemoteAddress != remoteIPv4Addr { + t.Errorf("got pkt.Route.RemoteAddress = %s, want = %s", reply.Route.RemoteAddress, remoteIPv4Addr) + } + + // Make sure it's all in one buffer. + vv := buffer.NewVectorisedView(reply.Pkt.Size(), reply.Pkt.Views()) + replyIPHeader := header.IPv4(vv.ToView()) + + // At this stage we only know it's an IP header so verify that much. + checker.IPv4(t, replyIPHeader, + checker.SrcAddr(ipv4Addr.Address), + checker.DstAddr(remoteIPv4Addr), + ) + + // All expected responses are ICMP packets. + if got, want := replyIPHeader.Protocol(), uint8(header.ICMPv4ProtocolNumber); got != want { + t.Fatalf("not ICMP response, got protocol %d, want = %d", got, want) + } + replyICMPHeader := header.ICMPv4(replyIPHeader.Payload()) + + // Sanity check the response. + switch replyICMPHeader.Type() { + case header.ICMPv4DstUnreachable: + checker.IPv4(t, replyIPHeader, + checker.IPFullLength(uint16(header.IPv4MinimumSize+header.ICMPv4MinimumSize+requestPkt.Size())), + checker.IPv4HeaderLength(ipHeaderLength), + checker.ICMPv4( + checker.ICMPv4Code(test.ICMPCode), + checker.ICMPv4Checksum(), + checker.ICMPv4Payload([]byte(hdr.View())), + ), + ) + if !test.shouldFail || !test.expectICMP { + t.Fatalf("unexpected packet rejection, got ICMP error packet type %d, code %d", + header.ICMPv4DstUnreachable, replyICMPHeader.Code()) + } + return + case header.ICMPv4EchoReply: + checker.IPv4(t, replyIPHeader, + checker.IPv4HeaderLength(ipHeaderLength), + checker.IPFullLength(uint16(requestPkt.Size())), + checker.ICMPv4( + checker.ICMPv4Code(test.ICMPCode), + checker.ICMPv4Seq(randomSequence), + checker.ICMPv4Ident(randomIdent), + checker.ICMPv4Checksum(), + ), + ) + if test.shouldFail { + t.Fatalf("unexpected Echo Reply packet\n") + } + default: + t.Fatalf("unexpected ICMP response, got type %d, want = %d or %d", + replyICMPHeader.Type(), header.ICMPv4EchoReply, header.ICMPv4DstUnreachable) + } + }) + } +} + // comparePayloads compared the contents of all the packets against the contents // of the source packet. func compareFragments(t *testing.T, packets []*stack.PacketBuffer, sourcePacketInfo *stack.PacketBuffer, mtu uint32) { @@ -160,47 +404,6 @@ func compareFragments(t *testing.T, packets []*stack.PacketBuffer, sourcePacketI } } -type testRoute struct { - stack.Route - - linkEP *testutil.TestEndpoint -} - -func buildTestRoute(t *testing.T, ep *channel.Endpoint, packetCollectorErrors []*tcpip.Error) testRoute { - // Make the packet and write it. - s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol()}, - }) - testEP := testutil.NewTestEndpoint(ep, packetCollectorErrors) - s.CreateNIC(1, testEP) - const ( - src = "\x10\x00\x00\x01" - dst = "\x10\x00\x00\x02" - ) - s.AddAddress(1, ipv4.ProtocolNumber, src) - { - subnet, err := tcpip.NewSubnet(dst, tcpip.AddressMask(header.IPv4Broadcast)) - if err != nil { - t.Fatal(err) - } - s.SetRouteTable([]tcpip.Route{{ - Destination: subnet, - NIC: 1, - }}) - } - r, err := s.FindRoute(0, src, dst, ipv4.ProtocolNumber, false /* multicastLoop */) - if err != nil { - t.Fatalf("s.FindRoute got %v, want %v", err, nil) - } - t.Cleanup(func() { - testEP.Close() - }) - return testRoute{ - Route: r, - linkEP: testEP, - } -} - func TestFragmentation(t *testing.T) { var manyPayloadViewsSizes [1000]int for i := range manyPayloadViewsSizes { @@ -228,7 +431,8 @@ func TestFragmentation(t *testing.T) { for _, ft := range fragTests { t.Run(ft.description, func(t *testing.T) { - r := buildTestRoute(t, channel.New(0, ft.mtu, ""), nil) + ep := testutil.NewMockLinkEndpoint(ft.mtu, nil, math.MaxInt32) + r := buildRoute(t, ep) pkt := testutil.MakeRandPkt(ft.transportHeaderLength, ft.extraHeaderReserveLength, ft.payloadViewsSizes, header.IPv4ProtocolNumber) source := pkt.Clone() err := r.WritePacket(ft.gso, stack.NetworkHeaderParams{ @@ -237,16 +441,19 @@ func TestFragmentation(t *testing.T) { TOS: stack.DefaultTOS, }, pkt) if err != nil { - t.Errorf("err got %v, want %v", err, nil) + t.Errorf("got err = %s, want = nil", err) } - if got, want := len(r.linkEP.WrittenPackets), ft.expectedFrags; got != want { - t.Errorf("len(r.linkEP.WrittenPackets) got %d, want %d", got, want) + if got := len(ep.WrittenPackets); got != ft.expectedFrags { + t.Errorf("got len(ep.WrittenPackets) = %d, want = %d", got, ft.expectedFrags) + } + if got, want := len(ep.WrittenPackets), int(r.Stats().IP.PacketsSent.Value()); got != want { + t.Errorf("no errors yet got len(ep.WrittenPackets) = %d, want = %d", got, want) } - if got, want := len(r.linkEP.WrittenPackets), int(r.Stats().IP.PacketsSent.Value()); got != want { - t.Errorf("no errors yet len(r.linkEP.WrittenPackets) got %d, want %d", got, want) + if got := r.Stats().IP.OutgoingPacketErrors.Value(); got != 0 { + t.Errorf("got r.Stats().IP.OutgoingPacketErrors.Value() = %d, want = 0", got) } - compareFragments(t, r.linkEP.WrittenPackets, source, ft.mtu) + compareFragments(t, ep.WrittenPackets, source, ft.mtu) }) } } @@ -259,35 +466,66 @@ func TestFragmentationErrors(t *testing.T) { mtu uint32 transportHeaderLength int payloadViewsSizes []int - packetCollectorErrors []*tcpip.Error + err *tcpip.Error + allowPackets int + fragmentCount int }{ - {"NoFrag", 2000, 0, []int{1000}, []*tcpip.Error{tcpip.ErrAborted}}, - {"ErrorOnFirstFrag", 500, 0, []int{1000}, []*tcpip.Error{tcpip.ErrAborted}}, - {"ErrorOnSecondFrag", 500, 0, []int{1000}, []*tcpip.Error{nil, tcpip.ErrAborted}}, - {"ErrorOnFirstFragMTUSmallerThanHeader", 500, 1000, []int{500}, []*tcpip.Error{tcpip.ErrAborted}}, + { + description: "NoFrag", + mtu: 2000, + transportHeaderLength: 0, + payloadViewsSizes: []int{1000}, + err: tcpip.ErrAborted, + allowPackets: 0, + fragmentCount: 1, + }, + { + description: "ErrorOnFirstFrag", + mtu: 500, + transportHeaderLength: 0, + payloadViewsSizes: []int{1000}, + err: tcpip.ErrAborted, + allowPackets: 0, + fragmentCount: 3, + }, + { + description: "ErrorOnSecondFrag", + mtu: 500, + transportHeaderLength: 0, + payloadViewsSizes: []int{1000}, + err: tcpip.ErrAborted, + allowPackets: 1, + fragmentCount: 3, + }, + { + description: "ErrorOnFirstFragMTUSmallerThanHeader", + mtu: 500, + transportHeaderLength: 1000, + payloadViewsSizes: []int{500}, + err: tcpip.ErrAborted, + allowPackets: 0, + fragmentCount: 4, + }, } for _, ft := range fragTests { t.Run(ft.description, func(t *testing.T) { - r := buildTestRoute(t, channel.New(0, ft.mtu, ""), ft.packetCollectorErrors) + ep := testutil.NewMockLinkEndpoint(ft.mtu, ft.err, ft.allowPackets) + r := buildRoute(t, ep) pkt := testutil.MakeRandPkt(ft.transportHeaderLength, header.IPv4MinimumSize, ft.payloadViewsSizes, header.IPv4ProtocolNumber) err := r.WritePacket(&stack.GSO{}, stack.NetworkHeaderParams{ Protocol: tcp.ProtocolNumber, TTL: 42, TOS: stack.DefaultTOS, }, pkt) - for i := 0; i < len(ft.packetCollectorErrors)-1; i++ { - if got, want := ft.packetCollectorErrors[i], (*tcpip.Error)(nil); got != want { - t.Errorf("ft.packetCollectorErrors[%d] got %v, want %v", i, got, want) - } + if err != ft.err { + t.Errorf("got WritePacket() = %s, want = %s", err, ft.err) } - // We only need to check that last error because all the ones before are - // nil. - if got, want := err, ft.packetCollectorErrors[len(ft.packetCollectorErrors)-1]; got != want { - t.Errorf("err got %v, want %v", got, want) + if got, want := len(ep.WrittenPackets), int(r.Stats().IP.PacketsSent.Value()); err != nil && got != want { + t.Errorf("got len(ep.WrittenPackets) = %d, want = %d", got, want) } - if got, want := len(r.linkEP.WrittenPackets), int(r.Stats().IP.PacketsSent.Value())+1; err != nil && got != want { - t.Errorf("after linkEP error len(result) got %d, want %d", got, want) + if got, want := int(r.Stats().IP.OutgoingPacketErrors.Value()), ft.fragmentCount-ft.allowPackets; got != want { + t.Errorf("got r.Stats().IP.OutgoingPacketErrors.Value() = %d, want = %d", got, want) } }) } @@ -561,8 +799,8 @@ func TestInvalidFragments(t *testing.T) { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ - ipv4.NewProtocol(), + NetworkProtocols: []stack.NetworkProtocolFactory{ + ipv4.NewProtocol, }, }) e := channel.New(0, 1500, linkAddr) @@ -973,8 +1211,8 @@ func TestReceiveFragments(t *testing.T) { t.Run(test.name, func(t *testing.T) { // Setup a stack and endpoint. s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, }) e := channel.New(0, 1280, tcpip.LinkAddress("\xf0\x00")) if err := s.CreateNIC(nicID, e); err != nil { @@ -1046,3 +1284,189 @@ func TestReceiveFragments(t *testing.T) { }) } } + +func TestWriteStats(t *testing.T) { + const nPackets = 3 + tests := []struct { + name string + setup func(*testing.T, *stack.Stack) + allowPackets int + expectSent int + expectDropped int + expectWritten int + }{ + { + name: "Accept all", + // No setup needed, tables accept everything by default. + setup: func(*testing.T, *stack.Stack) {}, + allowPackets: math.MaxInt32, + expectSent: nPackets, + expectDropped: 0, + expectWritten: nPackets, + }, { + name: "Accept all with error", + // No setup needed, tables accept everything by default. + setup: func(*testing.T, *stack.Stack) {}, + allowPackets: nPackets - 1, + expectSent: nPackets - 1, + expectDropped: 0, + expectWritten: nPackets - 1, + }, { + name: "Drop all", + setup: func(t *testing.T, stk *stack.Stack) { + // Install Output DROP rule. + t.Helper() + ipt := stk.IPTables() + filter, ok := ipt.GetTable(stack.FilterTable, false /* ipv6 */) + if !ok { + t.Fatalf("failed to find filter table") + } + ruleIdx := filter.BuiltinChains[stack.Output] + filter.Rules[ruleIdx].Target = &stack.DropTarget{} + if err := ipt.ReplaceTable(stack.FilterTable, filter, false /* ipv6 */); err != nil { + t.Fatalf("failed to replace table: %s", err) + } + }, + allowPackets: math.MaxInt32, + expectSent: 0, + expectDropped: nPackets, + expectWritten: nPackets, + }, { + name: "Drop some", + setup: func(t *testing.T, stk *stack.Stack) { + // Install Output DROP rule that matches only 1 + // of the 3 packets. + t.Helper() + ipt := stk.IPTables() + filter, ok := ipt.GetTable(stack.FilterTable, false /* ipv6 */) + if !ok { + t.Fatalf("failed to find filter table") + } + // We'll match and DROP the last packet. + ruleIdx := filter.BuiltinChains[stack.Output] + filter.Rules[ruleIdx].Target = &stack.DropTarget{} + filter.Rules[ruleIdx].Matchers = []stack.Matcher{&limitedMatcher{nPackets - 1}} + // Make sure the next rule is ACCEPT. + filter.Rules[ruleIdx+1].Target = &stack.AcceptTarget{} + if err := ipt.ReplaceTable(stack.FilterTable, filter, false /* ipv6 */); err != nil { + t.Fatalf("failed to replace table: %s", err) + } + }, + allowPackets: math.MaxInt32, + expectSent: nPackets - 1, + expectDropped: 1, + expectWritten: nPackets, + }, + } + + // Parameterize the tests to run with both WritePacket and WritePackets. + writers := []struct { + name string + writePackets func(*stack.Route, stack.PacketBufferList) (int, *tcpip.Error) + }{ + { + name: "WritePacket", + writePackets: func(rt *stack.Route, pkts stack.PacketBufferList) (int, *tcpip.Error) { + nWritten := 0 + for pkt := pkts.Front(); pkt != nil; pkt = pkt.Next() { + if err := rt.WritePacket(nil, stack.NetworkHeaderParams{}, pkt); err != nil { + return nWritten, err + } + nWritten++ + } + return nWritten, nil + }, + }, { + name: "WritePackets", + writePackets: func(rt *stack.Route, pkts stack.PacketBufferList) (int, *tcpip.Error) { + return rt.WritePackets(nil, pkts, stack.NetworkHeaderParams{}) + }, + }, + } + + for _, writer := range writers { + t.Run(writer.name, func(t *testing.T) { + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + ep := testutil.NewMockLinkEndpoint(header.IPv4MinimumSize+header.UDPMinimumSize, tcpip.ErrInvalidEndpointState, test.allowPackets) + rt := buildRoute(t, ep) + + var pkts stack.PacketBufferList + for i := 0; i < nPackets; i++ { + pkt := stack.NewPacketBuffer(stack.PacketBufferOptions{ + ReserveHeaderBytes: header.UDPMinimumSize + int(rt.MaxHeaderLength()), + Data: buffer.NewView(0).ToVectorisedView(), + }) + pkt.TransportHeader().Push(header.UDPMinimumSize) + pkts.PushBack(pkt) + } + + test.setup(t, rt.Stack()) + + nWritten, _ := writer.writePackets(&rt, pkts) + + if got := int(rt.Stats().IP.PacketsSent.Value()); got != test.expectSent { + t.Errorf("sent %d packets, but expected to send %d", got, test.expectSent) + } + if got := int(rt.Stats().IP.IPTablesOutputDropped.Value()); got != test.expectDropped { + t.Errorf("dropped %d packets, but expected to drop %d", got, test.expectDropped) + } + if nWritten != test.expectWritten { + t.Errorf("wrote %d packets, but expected WritePackets to return %d", nWritten, test.expectWritten) + } + }) + } + }) + } +} + +func buildRoute(t *testing.T, ep stack.LinkEndpoint) stack.Route { + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, + }) + if err := s.CreateNIC(1, ep); err != nil { + t.Fatalf("CreateNIC(1, _) failed: %s", err) + } + const ( + src = "\x10\x00\x00\x01" + dst = "\x10\x00\x00\x02" + ) + if err := s.AddAddress(1, ipv4.ProtocolNumber, src); err != nil { + t.Fatalf("AddAddress(1, %d, _) failed: %s", ipv4.ProtocolNumber, err) + } + { + subnet, err := tcpip.NewSubnet(dst, tcpip.AddressMask(header.IPv4Broadcast)) + if err != nil { + t.Fatalf("NewSubnet(_, _) failed: %v", err) + } + s.SetRouteTable([]tcpip.Route{{ + Destination: subnet, + NIC: 1, + }}) + } + rt, err := s.FindRoute(1, src, dst, ipv4.ProtocolNumber, false /* multicastLoop */) + if err != nil { + t.Fatalf("got FindRoute(1, _, _, %d, false) = %s, want = nil", ipv4.ProtocolNumber, err) + } + return rt +} + +// limitedMatcher is an iptables matcher that matches after a certain number of +// packets are checked against it. +type limitedMatcher struct { + limit int +} + +// Name implements Matcher.Name. +func (*limitedMatcher) Name() string { + return "limitedMatcher" +} + +// Match implements Matcher.Match. +func (lm *limitedMatcher) Match(stack.Hook, *stack.PacketBuffer, string) (bool, bool) { + if lm.limit == 0 { + return true, false + } + lm.limit-- + return false, false +} diff --git a/pkg/tcpip/network/ipv6/BUILD b/pkg/tcpip/network/ipv6/BUILD index bcc64994e..97adbcbd4 100644 --- a/pkg/tcpip/network/ipv6/BUILD +++ b/pkg/tcpip/network/ipv6/BUILD @@ -5,14 +5,18 @@ package(licenses = ["notice"]) go_library( name = "ipv6", srcs = [ + "dhcpv6configurationfromndpra_string.go", "icmp.go", "ipv6.go", + "ndp.go", ], visibility = ["//visibility:public"], deps = [ + "//pkg/sync", "//pkg/tcpip", "//pkg/tcpip/buffer", "//pkg/tcpip/header", + "//pkg/tcpip/header/parse", "//pkg/tcpip/network/fragmentation", "//pkg/tcpip/stack", ], @@ -34,6 +38,7 @@ go_test( "//pkg/tcpip/header", "//pkg/tcpip/link/channel", "//pkg/tcpip/link/sniffer", + "//pkg/tcpip/network/testutil", "//pkg/tcpip/stack", "//pkg/tcpip/transport/icmp", "//pkg/tcpip/transport/udp", diff --git a/pkg/tcpip/stack/dhcpv6configurationfromndpra_string.go b/pkg/tcpip/network/ipv6/dhcpv6configurationfromndpra_string.go index d199ded6a..09ba133b1 100644 --- a/pkg/tcpip/stack/dhcpv6configurationfromndpra_string.go +++ b/pkg/tcpip/network/ipv6/dhcpv6configurationfromndpra_string.go @@ -14,7 +14,7 @@ // Code generated by "stringer -type DHCPv6ConfigurationFromNDPRA"; DO NOT EDIT. -package stack +package ipv6 import "strconv" diff --git a/pkg/tcpip/network/ipv6/icmp.go b/pkg/tcpip/network/ipv6/icmp.go index 2b83c421e..8e9def6b8 100644 --- a/pkg/tcpip/network/ipv6/icmp.go +++ b/pkg/tcpip/network/ipv6/icmp.go @@ -15,6 +15,8 @@ package ipv6 import ( + "fmt" + "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/header" @@ -39,7 +41,7 @@ func (e *endpoint) handleControl(typ stack.ControlType, extra uint32, pkt *stack // Drop packet if it doesn't have the basic IPv6 header or if the // original source address doesn't match an address we own. src := hdr.SourceAddress() - if e.stack.CheckLocalAddress(e.NICID(), ProtocolNumber, src) == 0 { + if e.protocol.stack.CheckLocalAddress(e.nic.ID(), ProtocolNumber, src) == 0 { return } @@ -207,14 +209,7 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer, hasFragme return } - s := r.Stack() - if isTentative, err := s.IsAddrTentative(e.nicID, targetAddr); err != nil { - // We will only get an error if the NIC is unrecognized, which should not - // happen. For now, drop this packet. - // - // TODO(b/141002840): Handle this better? - return - } else if isTentative { + if e.hasTentativeAddr(targetAddr) { // If the target address is tentative and the source of the packet is a // unicast (specified) address, then the source of the packet is // attempting to perform address resolution on the target. In this case, @@ -227,7 +222,20 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer, hasFragme // stack know so it can handle such a scenario and do nothing further with // the NS. if r.RemoteAddress == header.IPv6Any { - s.DupTentativeAddrDetected(e.nicID, targetAddr) + // We would get an error if the address no longer exists or the address + // is no longer tentative (DAD resolved between the call to + // hasTentativeAddr and this point). Both of these are valid scenarios: + // 1) An address may be removed at any time. + // 2) As per RFC 4862 section 5.4, DAD is not a perfect: + // "Note that the method for detecting duplicates + // is not completely reliable, and it is possible that duplicate + // addresses will still exist" + // + // TODO(gvisor.dev/issue/4046): Handle the scenario when a duplicate + // address is detected for an assigned address. + if err := e.dupTentativeAddrDetected(targetAddr); err != nil && err != tcpip.ErrBadAddress && err != tcpip.ErrInvalidEndpointState { + panic(fmt.Sprintf("unexpected error handling duplicate tentative address: %s", err)) + } } // Do not handle neighbor solicitations targeted to an address that is @@ -240,7 +248,7 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer, hasFragme // section 5.4.3. // Is the NS targeting us? - if s.CheckLocalAddress(e.nicID, ProtocolNumber, targetAddr) == 0 { + if r.Stack().CheckLocalAddress(e.nic.ID(), ProtocolNumber, targetAddr) == 0 { return } @@ -275,7 +283,7 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer, hasFragme } else if e.nud != nil { e.nud.HandleProbe(r.RemoteAddress, r.LocalAddress, header.IPv6ProtocolNumber, sourceLinkAddr, e.protocol) } else { - e.linkAddrCache.AddLinkAddress(e.nicID, r.RemoteAddress, sourceLinkAddr) + e.linkAddrCache.AddLinkAddress(e.nic.ID(), r.RemoteAddress, sourceLinkAddr) } // ICMPv6 Neighbor Solicit messages are always sent to @@ -318,6 +326,7 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer, hasFragme ReserveHeaderBytes: int(r.MaxHeaderLength()) + header.ICMPv6NeighborAdvertMinimumSize + int(optsSerializer.Length()), }) packet := header.ICMPv6(pkt.TransportHeader().Push(header.ICMPv6NeighborAdvertSize)) + pkt.TransportProtocolNumber = header.ICMPv6ProtocolNumber packet.SetType(header.ICMPv6NeighborAdvert) na := header.NDPNeighborAdvert(packet.NDPPayload()) na.SetSolicitedFlag(solicited) @@ -352,20 +361,26 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer, hasFragme // NDP datagrams are very small and ToView() will not incur allocations. na := header.NDPNeighborAdvert(payload.ToView()) targetAddr := na.TargetAddress() - s := r.Stack() - - if isTentative, err := s.IsAddrTentative(e.nicID, targetAddr); err != nil { - // We will only get an error if the NIC is unrecognized, which should not - // happen. For now short-circuit this packet. - // - // TODO(b/141002840): Handle this better? - return - } else if isTentative { + if e.hasTentativeAddr(targetAddr) { // We just got an NA from a node that owns an address we are performing // DAD on, implying the address is not unique. In this case we let the // stack know so it can handle such a scenario and do nothing furthur with // the NDP NA. - s.DupTentativeAddrDetected(e.nicID, targetAddr) + // + // We would get an error if the address no longer exists or the address + // is no longer tentative (DAD resolved between the call to + // hasTentativeAddr and this point). Both of these are valid scenarios: + // 1) An address may be removed at any time. + // 2) As per RFC 4862 section 5.4, DAD is not a perfect: + // "Note that the method for detecting duplicates + // is not completely reliable, and it is possible that duplicate + // addresses will still exist" + // + // TODO(gvisor.dev/issue/4046): Handle the scenario when a duplicate + // address is detected for an assigned address. + if err := e.dupTentativeAddrDetected(targetAddr); err != nil && err != tcpip.ErrBadAddress && err != tcpip.ErrInvalidEndpointState { + panic(fmt.Sprintf("unexpected error handling duplicate tentative address: %s", err)) + } return } @@ -395,7 +410,7 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer, hasFragme // address cache with the link address for the target of the message. if len(targetLinkAddr) != 0 { if e.nud == nil { - e.linkAddrCache.AddLinkAddress(e.nicID, targetAddr, targetLinkAddr) + e.linkAddrCache.AddLinkAddress(e.nic.ID(), targetAddr, targetLinkAddr) return } @@ -423,7 +438,7 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer, hasFragme localAddr = "" } - r, err := r.Stack().FindRoute(e.NICID(), localAddr, r.RemoteAddress, ProtocolNumber, false /* multicastLoop */) + r, err := r.Stack().FindRoute(e.nic.ID(), localAddr, r.RemoteAddress, ProtocolNumber, false /* multicastLoop */) if err != nil { // If we cannot find a route to the destination, silently drop the packet. return @@ -438,6 +453,7 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer, hasFragme Data: pkt.Data, }) packet := header.ICMPv6(replyPkt.TransportHeader().Push(header.ICMPv6EchoMinimumSize)) + pkt.TransportProtocolNumber = header.ICMPv6ProtocolNumber copy(packet, icmpHdr) packet.SetType(header.ICMPv6EchoReply) packet.SetChecksum(header.ICMPv6Checksum(packet, r.LocalAddress, r.RemoteAddress, pkt.Data)) @@ -477,7 +493,7 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer, hasFragme stack := r.Stack() // Is the networking stack operating as a router? - if !stack.Forwarding() { + if !stack.Forwarding(ProtocolNumber) { // ... No, silently drop the packet. received.RouterOnlyPacketsDroppedByHost.Increment() return @@ -566,9 +582,9 @@ func (e *endpoint) handleICMP(r *stack.Route, pkt *stack.PacketBuffer, hasFragme e.nud.HandleProbe(routerAddr, r.LocalAddress, header.IPv6ProtocolNumber, sourceLinkAddr, e.protocol) } - // Tell the NIC to handle the RA. - stack := r.Stack() - stack.HandleNDPRA(e.nicID, routerAddr, ra) + e.mu.Lock() + e.mu.ndp.handleRA(routerAddr, ra) + e.mu.Unlock() case header.ICMPv6RedirectMsg: // TODO(gvisor.dev/issue/2285): Call `e.nud.HandleProbe` after validating @@ -637,6 +653,7 @@ func (*protocol) LinkAddressRequest(addr, localAddr tcpip.Address, remoteLinkAdd ReserveHeaderBytes: int(linkEP.MaxHeaderLength()) + header.IPv6MinimumSize + header.ICMPv6NeighborAdvertSize, }) icmpHdr := header.ICMPv6(pkt.TransportHeader().Push(header.ICMPv6NeighborAdvertSize)) + pkt.TransportProtocolNumber = header.ICMPv6ProtocolNumber icmpHdr.SetType(header.ICMPv6NeighborSolicit) copy(icmpHdr[icmpV6OptOffset-len(addr):], addr) icmpHdr[icmpV6OptOffset] = ndpOptSrcLinkAddr @@ -665,3 +682,159 @@ func (*protocol) ResolveStaticAddress(addr tcpip.Address) (tcpip.LinkAddress, bo } return tcpip.LinkAddress([]byte(nil)), false } + +// ======= ICMP Error packet generation ========= + +// icmpReason is a marker interface for IPv6 specific ICMP errors. +type icmpReason interface { + isICMPReason() +} + +// icmpReasonParameterProblem is an error during processing of extension headers +// or the fixed header defined in RFC 4443 section 3.4. +type icmpReasonParameterProblem struct { + code header.ICMPv6Code + + // respondToMulticast indicates that we are sending a packet that falls under + // the exception outlined by RFC 4443 section 2.4 point e.3 exception 2: + // + // (e.3) A packet destined to an IPv6 multicast address. (There are + // two exceptions to this rule: (1) the Packet Too Big Message + // (Section 3.2) to allow Path MTU discovery to work for IPv6 + // multicast, and (2) the Parameter Problem Message, Code 2 + // (Section 3.4) reporting an unrecognized IPv6 option (see + // Section 4.2 of [IPv6]) that has the Option Type highest- + // order two bits set to 10). + respondToMulticast bool + + // pointer is defined in the RFC 4443 setion 3.4 which reads: + // + // Pointer Identifies the octet offset within the invoking packet + // where the error was detected. + // + // The pointer will point beyond the end of the ICMPv6 + // packet if the field in error is beyond what can fit + // in the maximum size of an ICMPv6 error message. + pointer uint32 +} + +func (*icmpReasonParameterProblem) isICMPReason() {} + +// icmpReasonPortUnreachable is an error where the transport protocol has no +// listener and no alternative means to inform the sender. +type icmpReasonPortUnreachable struct{} + +func (*icmpReasonPortUnreachable) isICMPReason() {} + +// returnError takes an error descriptor and generates the appropriate ICMP +// error packet for IPv6 and sends it. +func returnError(r *stack.Route, reason icmpReason, pkt *stack.PacketBuffer) *tcpip.Error { + stats := r.Stats().ICMP + sent := stats.V6PacketsSent + if !r.Stack().AllowICMPMessage() { + sent.RateLimited.Increment() + return nil + } + + // Only send ICMP error if the address is not a multicast v6 + // address and the source is not the unspecified address. + // + // There are exceptions to this rule. + // See: point e.3) RFC 4443 section-2.4 + // + // (e) An ICMPv6 error message MUST NOT be originated as a result of + // receiving the following: + // + // (e.1) An ICMPv6 error message. + // + // (e.2) An ICMPv6 redirect message [IPv6-DISC]. + // + // (e.3) A packet destined to an IPv6 multicast address. (There are + // two exceptions to this rule: (1) the Packet Too Big Message + // (Section 3.2) to allow Path MTU discovery to work for IPv6 + // multicast, and (2) the Parameter Problem Message, Code 2 + // (Section 3.4) reporting an unrecognized IPv6 option (see + // Section 4.2 of [IPv6]) that has the Option Type highest- + // order two bits set to 10). + // + var allowResponseToMulticast bool + if reason, ok := reason.(*icmpReasonParameterProblem); ok { + allowResponseToMulticast = reason.respondToMulticast + } + + if (!allowResponseToMulticast && header.IsV6MulticastAddress(r.LocalAddress)) || r.RemoteAddress == header.IPv6Any { + return nil + } + + network, transport := pkt.NetworkHeader().View(), pkt.TransportHeader().View() + + if pkt.TransportProtocolNumber == header.ICMPv6ProtocolNumber { + // TODO(gvisor.dev/issues/3810): Sort this out when ICMP headers are stored. + // Unfortunately at this time ICMP Packets do not have a transport + // header separated out. It is in the Data part so we need to + // separate it out now. We will just pretend it is a minimal length + // ICMP packet as we don't really care if any later bits of a + // larger ICMP packet are in the header view or in the Data view. + transport, ok := pkt.TransportHeader().Consume(header.ICMPv6MinimumSize) + if !ok { + return nil + } + typ := header.ICMPv6(transport).Type() + if typ.IsErrorType() || typ == header.ICMPv6RedirectMsg { + return nil + } + } + + // As per RFC 4443 section 2.4 + // + // (c) Every ICMPv6 error message (type < 128) MUST include + // as much of the IPv6 offending (invoking) packet (the + // packet that caused the error) as possible without making + // the error message packet exceed the minimum IPv6 MTU + // [IPv6]. + mtu := int(r.MTU()) + if mtu > header.IPv6MinimumMTU { + mtu = header.IPv6MinimumMTU + } + headerLen := int(r.MaxHeaderLength()) + header.ICMPv6ErrorHeaderSize + available := int(mtu) - headerLen + if available < header.IPv6MinimumSize { + return nil + } + payloadLen := network.Size() + transport.Size() + pkt.Data.Size() + if payloadLen > available { + payloadLen = available + } + payload := buffer.NewVectorisedView(pkt.Size(), pkt.Views()) + payload.CapLength(payloadLen) + + newPkt := stack.NewPacketBuffer(stack.PacketBufferOptions{ + ReserveHeaderBytes: headerLen, + Data: payload, + }) + newPkt.TransportProtocolNumber = header.ICMPv6ProtocolNumber + + icmpHdr := header.ICMPv6(newPkt.TransportHeader().Push(header.ICMPv6DstUnreachableMinimumSize)) + var counter *tcpip.StatCounter + switch reason := reason.(type) { + case *icmpReasonParameterProblem: + icmpHdr.SetType(header.ICMPv6ParamProblem) + icmpHdr.SetCode(reason.code) + icmpHdr.SetTypeSpecific(reason.pointer) + counter = sent.ParamProblem + case *icmpReasonPortUnreachable: + icmpHdr.SetType(header.ICMPv6DstUnreachable) + icmpHdr.SetCode(header.ICMPv6PortUnreachable) + counter = sent.DstUnreachable + default: + panic(fmt.Sprintf("unsupported ICMP type %T", reason)) + } + icmpHdr.SetChecksum(header.ICMPv6Checksum(icmpHdr, r.LocalAddress, r.RemoteAddress, newPkt.Data)) + err := r.WritePacket(nil /* gso */, stack.NetworkHeaderParams{Protocol: header.ICMPv6ProtocolNumber, TTL: r.DefaultTTL(), TOS: stack.DefaultTOS}, newPkt) + if err != nil { + sent.Dropped.Increment() + return err + } + counter.Increment() + return nil +} diff --git a/pkg/tcpip/network/ipv6/icmp_test.go b/pkg/tcpip/network/ipv6/icmp_test.go index 8112ed051..31370c1d4 100644 --- a/pkg/tcpip/network/ipv6/icmp_test.go +++ b/pkg/tcpip/network/ipv6/icmp_test.go @@ -75,7 +75,8 @@ type stubDispatcher struct { stack.TransportDispatcher } -func (*stubDispatcher) DeliverTransportPacket(*stack.Route, tcpip.TransportProtocolNumber, *stack.PacketBuffer) { +func (*stubDispatcher) DeliverTransportPacket(*stack.Route, tcpip.TransportProtocolNumber, *stack.PacketBuffer) stack.TransportPacketDisposition { + return stack.TransportPacketHandled } type stubLinkAddressCache struct { @@ -102,6 +103,30 @@ func (*stubNUDHandler) HandleConfirmation(addr tcpip.Address, linkAddr tcpip.Lin func (*stubNUDHandler) HandleUpperLevelConfirmation(addr tcpip.Address) { } +var _ stack.NetworkInterface = (*testInterface)(nil) + +type testInterface struct{} + +func (*testInterface) ID() tcpip.NICID { + return 0 +} + +func (*testInterface) IsLoopback() bool { + return false +} + +func (*testInterface) Name() string { + return "" +} + +func (*testInterface) Enabled() bool { + return true +} + +func (*testInterface) LinkEndpoint() stack.LinkEndpoint { + return nil +} + func TestICMPCounts(t *testing.T) { tests := []struct { name string @@ -120,8 +145,8 @@ func TestICMPCounts(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{icmp.NewProtocol6()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{icmp.NewProtocol6}, UseNeighborCache: test.useNeighborCache, }) { @@ -149,9 +174,13 @@ func TestICMPCounts(t *testing.T) { if netProto == nil { t.Fatalf("cannot find protocol instance for network protocol %d", ProtocolNumber) } - ep := netProto.NewEndpoint(0, &stubLinkAddressCache{}, &stubNUDHandler{}, &stubDispatcher{}, nil, s) + ep := netProto.NewEndpoint(&testInterface{}, &stubLinkAddressCache{}, &stubNUDHandler{}, &stubDispatcher{}) defer ep.Close() + if err := ep.Enable(); err != nil { + t.Fatalf("ep.Enable(): %s", err) + } + r, err := s.FindRoute(nicID, lladdr0, lladdr1, ProtocolNumber, false /* multicastLoop */) if err != nil { t.Fatalf("FindRoute(%d, %s, %s, _, false) = (_, %s), want = (_, nil)", nicID, lladdr0, lladdr1, err) @@ -258,8 +287,8 @@ func TestICMPCounts(t *testing.T) { func TestICMPCountsWithNeighborCache(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{icmp.NewProtocol6()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{icmp.NewProtocol6}, UseNeighborCache: true, }) { @@ -287,9 +316,13 @@ func TestICMPCountsWithNeighborCache(t *testing.T) { if netProto == nil { t.Fatalf("cannot find protocol instance for network protocol %d", ProtocolNumber) } - ep := netProto.NewEndpoint(0, nil, &stubNUDHandler{}, &stubDispatcher{}, nil, s) + ep := netProto.NewEndpoint(&testInterface{}, nil, &stubNUDHandler{}, &stubDispatcher{}) defer ep.Close() + if err := ep.Enable(); err != nil { + t.Fatalf("ep.Enable(): %s", err) + } + r, err := s.FindRoute(nicID, lladdr0, lladdr1, ProtocolNumber, false /* multicastLoop */) if err != nil { t.Fatalf("FindRoute(%d, %s, %s, _, false) = (_, %s), want = (_, nil)", nicID, lladdr0, lladdr1, err) @@ -423,12 +456,12 @@ func (e endpointWithResolutionCapability) Capabilities() stack.LinkEndpointCapab func newTestContext(t *testing.T) *testContext { c := &testContext{ s0: stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{icmp.NewProtocol6()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{icmp.NewProtocol6}, }), s1: stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{icmp.NewProtocol6()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{icmp.NewProtocol6}, }), } @@ -723,12 +756,12 @@ func TestICMPChecksumValidationSimple(t *testing.T) { e.LinkEPCapabilities |= stack.CapabilityResolutionRequired s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, UseNeighborCache: test.useNeighborCache, }) if isRouter { // Enabling forwarding makes the stack act as a router. - s.SetForwarding(true) + s.SetForwarding(ProtocolNumber, true) } if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(_, _) = %s", err) @@ -919,7 +952,7 @@ func TestICMPChecksumValidationWithPayload(t *testing.T) { t.Run(typ.name, func(t *testing.T) { e := channel.New(10, 1280, linkAddr0) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, }) if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(_, _) = %s", err) @@ -1097,7 +1130,7 @@ func TestICMPChecksumValidationWithPayloadMultipleViews(t *testing.T) { t.Run(typ.name, func(t *testing.T) { e := channel.New(10, 1280, linkAddr0) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, }) if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(%d, _) = %s", nicID, err) @@ -1203,7 +1236,10 @@ func TestLinkAddressRequest(t *testing.T) { } for _, test := range tests { - p := NewProtocol() + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, + }) + p := s.NetworkProtocolInstance(ProtocolNumber) linkRes, ok := p.(stack.LinkAddressResolver) if !ok { t.Fatalf("expected IPv6 protocol to implement stack.LinkAddressResolver") diff --git a/pkg/tcpip/network/ipv6/ipv6.go b/pkg/tcpip/network/ipv6/ipv6.go index e821a8bff..c8a3e0b34 100644 --- a/pkg/tcpip/network/ipv6/ipv6.go +++ b/pkg/tcpip/network/ipv6/ipv6.go @@ -1,4 +1,4 @@ -// Copyright 2018 The gVisor Authors. +// 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. @@ -12,21 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Package ipv6 contains the implementation of the ipv6 network protocol. To use -// it in the networking stack, this package must be added to the project, and -// activated on the stack by passing ipv6.NewProtocol() as one of the network -// protocols when calling stack.New(). Then endpoints can be created by passing -// ipv6.ProtocolNumber as the network protocol number when calling -// Stack.NewEndpoint(). +// Package ipv6 contains the implementation of the ipv6 network protocol. package ipv6 import ( "fmt" + "sort" "sync/atomic" + "gvisor.dev/gvisor/pkg/sync" "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/header" + "gvisor.dev/gvisor/pkg/tcpip/header/parse" "gvisor.dev/gvisor/pkg/tcpip/network/fragmentation" "gvisor.dev/gvisor/pkg/tcpip/stack" ) @@ -44,14 +42,302 @@ const ( DefaultTTL = 64 ) +var _ stack.GroupAddressableEndpoint = (*endpoint)(nil) +var _ stack.AddressableEndpoint = (*endpoint)(nil) +var _ stack.NetworkEndpoint = (*endpoint)(nil) +var _ stack.NDPEndpoint = (*endpoint)(nil) +var _ NDPEndpoint = (*endpoint)(nil) + type endpoint struct { - nicID tcpip.NICID + nic stack.NetworkInterface linkEP stack.LinkEndpoint linkAddrCache stack.LinkAddressCache nud stack.NUDHandler dispatcher stack.TransportDispatcher protocol *protocol stack *stack.Stack + + // enabled is set to 1 when the endpoint is enabled and 0 when it is + // disabled. + // + // Must be accessed using atomic operations. + enabled uint32 + + mu struct { + sync.RWMutex + + addressableEndpointState stack.AddressableEndpointState + ndp ndpState + } +} + +// NICNameFromID is a function that returns a stable name for the specified NIC, +// even if different NIC IDs are used to refer to the same NIC in different +// program runs. It is used when generating opaque interface identifiers (IIDs). +// If the NIC was created with a name, it is passed to NICNameFromID. +// +// NICNameFromID SHOULD return unique NIC names so unique opaque IIDs are +// generated for the same prefix on differnt NICs. +type NICNameFromID func(tcpip.NICID, string) string + +// OpaqueInterfaceIdentifierOptions holds the options related to the generation +// of opaque interface indentifiers (IIDs) as defined by RFC 7217. +type OpaqueInterfaceIdentifierOptions struct { + // NICNameFromID is a function that returns a stable name for a specified NIC, + // even if the NIC ID changes over time. + // + // Must be specified to generate the opaque IID. + NICNameFromID NICNameFromID + + // SecretKey is a pseudo-random number used as the secret key when generating + // opaque IIDs as defined by RFC 7217. The key SHOULD be at least + // header.OpaqueIIDSecretKeyMinBytes bytes and MUST follow minimum randomness + // requirements for security as outlined by RFC 4086. SecretKey MUST NOT + // change between program runs, unless explicitly changed. + // + // OpaqueInterfaceIdentifierOptions takes ownership of SecretKey. SecretKey + // MUST NOT be modified after Stack is created. + // + // May be nil, but a nil value is highly discouraged to maintain + // some level of randomness between nodes. + SecretKey []byte +} + +// InvalidateDefaultRouter implements stack.NDPEndpoint. +func (e *endpoint) InvalidateDefaultRouter(rtr tcpip.Address) { + e.mu.Lock() + defer e.mu.Unlock() + e.mu.ndp.invalidateDefaultRouter(rtr) +} + +// SetNDPConfigurations implements NDPEndpoint. +func (e *endpoint) SetNDPConfigurations(c NDPConfigurations) { + c.validate() + e.mu.Lock() + defer e.mu.Unlock() + e.mu.ndp.configs = c +} + +// hasTentativeAddr returns true if addr is tentative on e. +func (e *endpoint) hasTentativeAddr(addr tcpip.Address) bool { + e.mu.RLock() + addressEndpoint := e.getAddressRLocked(addr) + e.mu.RUnlock() + return addressEndpoint != nil && addressEndpoint.GetKind() == stack.PermanentTentative +} + +// dupTentativeAddrDetected attempts to inform e that a tentative addr is a +// duplicate on a link. +// +// dupTentativeAddrDetected removes the tentative address if it exists. If the +// address was generated via SLAAC, an attempt is made to generate a new +// address. +func (e *endpoint) dupTentativeAddrDetected(addr tcpip.Address) *tcpip.Error { + e.mu.Lock() + defer e.mu.Unlock() + + addressEndpoint := e.getAddressRLocked(addr) + if addressEndpoint == nil { + return tcpip.ErrBadAddress + } + + if addressEndpoint.GetKind() != stack.PermanentTentative { + return tcpip.ErrInvalidEndpointState + } + + // If the address is a SLAAC address, do not invalidate its SLAAC prefix as an + // attempt will be made to generate a new address for it. + if err := e.removePermanentEndpointLocked(addressEndpoint, false /* allowSLAACInvalidation */); err != nil { + return err + } + + prefix := addressEndpoint.AddressWithPrefix().Subnet() + + switch t := addressEndpoint.ConfigType(); t { + case stack.AddressConfigStatic: + case stack.AddressConfigSlaac: + e.mu.ndp.regenerateSLAACAddr(prefix) + case stack.AddressConfigSlaacTemp: + // Do not reset the generation attempts counter for the prefix as the + // temporary address is being regenerated in response to a DAD conflict. + e.mu.ndp.regenerateTempSLAACAddr(prefix, false /* resetGenAttempts */) + default: + panic(fmt.Sprintf("unrecognized address config type = %d", t)) + } + + return nil +} + +// transitionForwarding transitions the endpoint's forwarding status to +// forwarding. +// +// Must only be called when the forwarding status changes. +func (e *endpoint) transitionForwarding(forwarding bool) { + e.mu.Lock() + defer e.mu.Unlock() + + if !e.Enabled() { + return + } + + if forwarding { + // When transitioning into an IPv6 router, host-only state (NDP discovered + // routers, discovered on-link prefixes, and auto-generated addresses) is + // cleaned up/invalidated and NDP router solicitations are stopped. + e.mu.ndp.stopSolicitingRouters() + e.mu.ndp.cleanupState(true /* hostOnly */) + } else { + // When transitioning into an IPv6 host, NDP router solicitations are + // started. + e.mu.ndp.startSolicitingRouters() + } +} + +// Enable implements stack.NetworkEndpoint. +func (e *endpoint) Enable() *tcpip.Error { + e.mu.Lock() + defer e.mu.Unlock() + + // If the NIC is not enabled, the endpoint can't do anything meaningful so + // don't enable the endpoint. + if !e.nic.Enabled() { + return tcpip.ErrNotPermitted + } + + // If the endpoint is already enabled, there is nothing for it to do. + if !e.setEnabled(true) { + return nil + } + + // Join the IPv6 All-Nodes Multicast group if the stack is configured to + // use IPv6. This is required to ensure that this node properly receives + // and responds to the various NDP messages that are destined to the + // all-nodes multicast address. An example is the Neighbor Advertisement + // when we perform Duplicate Address Detection, or Router Advertisement + // when we do Router Discovery. See RFC 4862, section 5.4.2 and RFC 4861 + // section 4.2 for more information. + // + // Also auto-generate an IPv6 link-local address based on the endpoint's + // link address if it is configured to do so. Note, each interface is + // required to have IPv6 link-local unicast address, as per RFC 4291 + // section 2.1. + + // Join the All-Nodes multicast group before starting DAD as responses to DAD + // (NDP NS) messages may be sent to the All-Nodes multicast group if the + // source address of the NDP NS is the unspecified address, as per RFC 4861 + // section 7.2.4. + if _, err := e.mu.addressableEndpointState.JoinGroup(header.IPv6AllNodesMulticastAddress); err != nil { + return err + } + + // Perform DAD on the all the unicast IPv6 endpoints that are in the permanent + // state. + // + // Addresses may have aleady completed DAD but in the time since the endpoint + // was last enabled, other devices may have acquired the same addresses. + var err *tcpip.Error + e.mu.addressableEndpointState.ReadOnly().ForEach(func(addressEndpoint stack.AddressEndpoint) bool { + addr := addressEndpoint.AddressWithPrefix().Address + if !header.IsV6UnicastAddress(addr) { + return true + } + + switch addressEndpoint.GetKind() { + case stack.Permanent: + addressEndpoint.SetKind(stack.PermanentTentative) + fallthrough + case stack.PermanentTentative: + err = e.mu.ndp.startDuplicateAddressDetection(addr, addressEndpoint) + return err == nil + default: + return true + } + }) + if err != nil { + return err + } + + // Do not auto-generate an IPv6 link-local address for loopback devices. + if e.protocol.autoGenIPv6LinkLocal && !e.nic.IsLoopback() { + // The valid and preferred lifetime is infinite for the auto-generated + // link-local address. + e.mu.ndp.doSLAAC(header.IPv6LinkLocalPrefix.Subnet(), header.NDPInfiniteLifetime, header.NDPInfiniteLifetime) + } + + // If we are operating as a router, then do not solicit routers since we + // won't process the RAs anyway. + // + // Routers do not process Router Advertisements (RA) the same way a host + // does. That is, routers do not learn from RAs (e.g. on-link prefixes + // and default routers). Therefore, soliciting RAs from other routers on + // a link is unnecessary for routers. + if !e.protocol.Forwarding() { + e.mu.ndp.startSolicitingRouters() + } + + return nil +} + +// Enabled implements stack.NetworkEndpoint. +func (e *endpoint) Enabled() bool { + return e.nic.Enabled() && e.isEnabled() +} + +// isEnabled returns true if the endpoint is enabled, regardless of the +// enabled status of the NIC. +func (e *endpoint) isEnabled() bool { + return atomic.LoadUint32(&e.enabled) == 1 +} + +// setEnabled sets the enabled status for the endpoint. +// +// Returns true if the enabled status was updated. +func (e *endpoint) setEnabled(v bool) bool { + if v { + return atomic.SwapUint32(&e.enabled, 1) == 0 + } + return atomic.SwapUint32(&e.enabled, 0) == 1 +} + +// Disable implements stack.NetworkEndpoint. +func (e *endpoint) Disable() { + e.mu.Lock() + defer e.mu.Unlock() + e.disableLocked() +} + +func (e *endpoint) disableLocked() { + if !e.setEnabled(false) { + return + } + + e.mu.ndp.stopSolicitingRouters() + e.mu.ndp.cleanupState(false /* hostOnly */) + e.stopDADForPermanentAddressesLocked() + + // The endpoint may have already left the multicast group. + if _, err := e.mu.addressableEndpointState.LeaveGroup(header.IPv6AllNodesMulticastAddress); err != nil && err != tcpip.ErrBadLocalAddress { + panic(fmt.Sprintf("unexpected error when leaving group = %s: %s", header.IPv6AllNodesMulticastAddress, err)) + } +} + +// stopDADForPermanentAddressesLocked stops DAD for all permaneent addresses. +// +// Precondition: e.mu must be write locked. +func (e *endpoint) stopDADForPermanentAddressesLocked() { + // Stop DAD for all the tentative unicast addresses. + e.mu.addressableEndpointState.ReadOnly().ForEach(func(addressEndpoint stack.AddressEndpoint) bool { + if addressEndpoint.GetKind() != stack.PermanentTentative { + return true + } + + addr := addressEndpoint.AddressWithPrefix().Address + if header.IsV6UnicastAddress(addr) { + e.mu.ndp.stopDuplicateAddressDetection(addr) + } + + return true + }) } // DefaultTTL is the default hop limit for this endpoint. @@ -65,16 +351,6 @@ func (e *endpoint) MTU() uint32 { return calculateMTU(e.linkEP.MTU()) } -// NICID returns the ID of the NIC this endpoint belongs to. -func (e *endpoint) NICID() tcpip.NICID { - return e.nicID -} - -// Capabilities implements stack.NetworkEndpoint.Capabilities. -func (e *endpoint) Capabilities() stack.LinkEndpointCapabilities { - return e.linkEP.Capabilities() -} - // MaxHeaderLength returns the maximum length needed by ipv6 headers (and // underlying protocols). func (e *endpoint) MaxHeaderLength() uint16 { @@ -107,6 +383,32 @@ func (e *endpoint) addIPHeader(r *stack.Route, pkt *stack.PacketBuffer, params s 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.protocol.stack.FindNICNameFromID(e.nic.ID()) + ipt := e.protocol.stack.IPTables() + if ok := ipt.Check(stack.Output, pkt, gso, r, "", nicName); !ok { + // iptables is telling us to drop the packet. + r.Stats().IP.IPTablesOutputDropped.Increment() + return nil + } + + // If the packet is manipulated as per NAT Output 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.IPv6(pkt.NetworkHeader().View()) + if ep, err := e.protocol.stack.FindNetworkEndpoint(header.IPv6ProtocolNumber, netHeader.DestinationAddress()); err == nil { + route := r.ReverseRoute(netHeader.SourceAddress(), netHeader.DestinationAddress()) + ep.HandlePacket(&route, pkt) + return nil + } + } + if r.Loop&stack.PacketLoop != 0 { loopedR := r.MakeLoopedRoute() @@ -121,8 +423,12 @@ func (e *endpoint) WritePacket(r *stack.Route, gso *stack.GSO, params stack.Netw return nil } + if err := e.linkEP.WritePacket(r, gso, ProtocolNumber, pkt); err != nil { + r.Stats().IP.OutgoingPacketErrors.Increment() + return err + } r.Stats().IP.PacketsSent.Increment() - return e.linkEP.WritePacket(r, gso, ProtocolNumber, pkt) + return nil } // WritePackets implements stack.LinkEndpoint.WritePackets. @@ -138,9 +444,54 @@ func (e *endpoint) WritePackets(r *stack.Route, gso *stack.GSO, pkts stack.Packe e.addIPHeader(r, pb, params) } - n, err := e.linkEP.WritePackets(r, gso, pkts, ProtocolNumber) + // iptables filtering. All packets that reach here are locally + // generated. + nicName := e.protocol.stack.FindNICNameFromID(e.nic.ID()) + ipt := e.protocol.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)) + if err != nil { + r.Stats().IP.OutgoingPacketErrors.IncrementBy(uint64(pkts.Len() - n)) + } + return n, err + } + r.Stats().IP.IPTablesOutputDropped.IncrementBy(uint64(len(dropped))) + + // 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.IPv6(pkt.NetworkHeader().View()) + if ep, err := e.protocol.stack.FindNetworkEndpoint(header.IPv6ProtocolNumber, 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)) + r.Stats().IP.OutgoingPacketErrors.IncrementBy(uint64(pkts.Len() - n + len(dropped))) + // Dropped packets aren't errors, so include them in + // the return value. + return n + len(dropped), err + } + n++ + } + r.Stats().IP.PacketsSent.IncrementBy(uint64(n)) - return n, err + // Dropped packets aren't errors, so include them in the return value. + return n + len(dropped), nil } // WriteHeaderIncludedPacker implements stack.NetworkEndpoint. It is not yet @@ -153,12 +504,24 @@ func (*endpoint) WriteHeaderIncludedPacket(r *stack.Route, pkt *stack.PacketBuff // HandlePacket is called by the link layer when new ipv6 packets arrive for // this endpoint. func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { + if !e.isEnabled() { + return + } + h := header.IPv6(pkt.NetworkHeader().View()) if !h.IsValid(pkt.Data.Size() + pkt.NetworkHeader().View().Size() + pkt.TransportHeader().View().Size()) { r.Stats().IP.MalformedPacketsReceived.Increment() return } + // As per RFC 4291 section 2.7: + // Multicast addresses must not be used as source addresses in IPv6 + // packets or appear in any Routing header. + if header.IsV6MulticastAddress(r.RemoteAddress) { + r.Stats().IP.InvalidSourceAddressesReceived.Increment() + return + } + // vv consists of: // - Any IPv6 header bytes after the first 40 (i.e. extensions). // - The transport header, if present. @@ -169,7 +532,19 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { it := header.MakeIPv6PayloadIterator(header.IPv6ExtensionHeaderIdentifier(h.NextHeader()), vv) hasFragmentHeader := false - for firstHeader := true; ; firstHeader = false { + // iptables filtering. All packets that reach here are intended for + // this machine and need not be forwarded. + ipt := e.protocol.stack.IPTables() + if ok := ipt.Check(stack.Input, pkt, nil, nil, "", ""); !ok { + // iptables is telling us to drop the packet. + r.Stats().IP.IPTablesInputDropped.Increment() + return + } + + for { + // Keep track of the start of the previous header so we can report the + // special case of a Hop by Hop at a location other than at the start. + previousHeaderStart := it.HeaderOffset() extHdr, done, err := it.Next() if err != nil { r.Stats().IP.MalformedPacketsReceived.Increment() @@ -183,11 +558,11 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { case header.IPv6HopByHopOptionsExtHdr: // As per RFC 8200 section 4.1, the Hop By Hop extension header is // restricted to appear immediately after an IPv6 fixed header. - // - // TODO(b/152019344): Send an ICMPv6 Parameter Problem, Code 1 - // (unrecognized next header) error in response to an extension header's - // Next Header field with the Hop By Hop extension header identifier. - if !firstHeader { + if previousHeaderStart != 0 { + _ = returnError(r, &icmpReasonParameterProblem{ + code: header.ICMPv6UnknownHeader, + pointer: previousHeaderStart, + }, pkt) return } @@ -209,13 +584,25 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { case header.IPv6OptionUnknownActionSkip: case header.IPv6OptionUnknownActionDiscard: return - case header.IPv6OptionUnknownActionDiscardSendICMP: - // TODO(b/152019344): Send an ICMPv6 Parameter Problem Code 2 for - // unrecognized IPv6 extension header options. - return case header.IPv6OptionUnknownActionDiscardSendICMPNoMulticastDest: - // TODO(b/152019344): Send an ICMPv6 Parameter Problem Code 2 for - // unrecognized IPv6 extension header options. + if header.IsV6MulticastAddress(r.LocalAddress) { + return + } + fallthrough + case header.IPv6OptionUnknownActionDiscardSendICMP: + // This case satisfies a requirement of RFC 8200 section 4.2 + // which states that an unknown option starting with bits [10] should: + // + // discard the packet and, regardless of whether or not the + // packet's Destination Address was a multicast address, send an + // ICMP Parameter Problem, Code 2, message to the packet's + // Source Address, pointing to the unrecognized Option Type. + // + _ = returnError(r, &icmpReasonParameterProblem{ + code: header.ICMPv6UnknownOption, + pointer: it.ParseOffset() + optsIt.OptionOffset(), + respondToMulticast: true, + }, pkt) return default: panic(fmt.Sprintf("unrecognized action for an unrecognized Hop By Hop extension header option = %d", opt)) @@ -226,16 +613,20 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { // 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. + // Problem, Code 0 to the packet's Source Address, pointing to the + // unrecognized Routing Type. + // + // 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 { + _ = returnError(r, &icmpReasonParameterProblem{ + code: header.ICMPv6ErroneousHeader, + pointer: it.ParseOffset(), + }, pkt) return } @@ -268,7 +659,7 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { it, done, err := it.Next() if err != nil { r.Stats().IP.MalformedPacketsReceived.Increment() - r.Stats().IP.MalformedPacketsReceived.Increment() + r.Stats().IP.MalformedFragmentsReceived.Increment() return } if done { @@ -370,13 +761,25 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { case header.IPv6OptionUnknownActionSkip: case header.IPv6OptionUnknownActionDiscard: return - case header.IPv6OptionUnknownActionDiscardSendICMP: - // TODO(b/152019344): Send an ICMPv6 Parameter Problem Code 2 for - // unrecognized IPv6 extension header options. - return case header.IPv6OptionUnknownActionDiscardSendICMPNoMulticastDest: - // TODO(b/152019344): Send an ICMPv6 Parameter Problem Code 2 for - // unrecognized IPv6 extension header options. + if header.IsV6MulticastAddress(r.LocalAddress) { + return + } + fallthrough + case header.IPv6OptionUnknownActionDiscardSendICMP: + // This case satisfies a requirement of RFC 8200 section 4.2 + // which states that an unknown option starting with bits [10] should: + // + // discard the packet and, regardless of whether or not the + // packet's Destination Address was a multicast address, send an + // ICMP Parameter Problem, Code 2, message to the packet's + // Source Address, pointing to the unrecognized Option Type. + // + _ = returnError(r, &icmpReasonParameterProblem{ + code: header.ICMPv6UnknownOption, + pointer: it.ParseOffset() + optsIt.OptionOffset(), + respondToMulticast: true, + }, pkt) return default: panic(fmt.Sprintf("unrecognized action for an unrecognized Destination extension header option = %d", opt)) @@ -395,21 +798,55 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { extHdr.Buf.TrimFront(pkt.TransportHeader().View().Size()) pkt.Data = extHdr.Buf + r.Stats().IP.PacketsDelivered.Increment() if p := tcpip.TransportProtocolNumber(extHdr.Identifier); p == header.ICMPv6ProtocolNumber { + pkt.TransportProtocolNumber = p e.handleICMP(r, pkt, hasFragmentHeader) } 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) + switch res := e.dispatcher.DeliverTransportPacket(r, p, pkt); res { + case stack.TransportPacketHandled: + case stack.TransportPacketDestinationPortUnreachable: + // As per RFC 4443 section 3.1: + // A destination node SHOULD originate a Destination Unreachable + // message with Code 4 in response to a packet for which the + // transport protocol (e.g., UDP) has no listener, if that transport + // protocol has no alternative means to inform the sender. + _ = returnError(r, &icmpReasonPortUnreachable{}, pkt) + case stack.TransportPacketProtocolUnreachable: + // As per RFC 8200 section 4. (page 7): + // Extension headers are numbered from IANA IP Protocol Numbers + // [IANA-PN], the same values used for IPv4 and IPv6. When + // processing a sequence of Next Header values in a packet, the + // first one that is not an extension header [IANA-EH] indicates + // that the next item in the packet is the corresponding upper-layer + // header. + // With more related information on page 8: + // If, as a result of processing a header, the destination node is + // required to proceed to the next header but the Next Header value + // in the current header is unrecognized by the node, it should + // discard the packet and send an ICMP Parameter Problem message to + // the source of the packet, with an ICMP Code value of 1 + // ("unrecognized Next Header type encountered") and the ICMP + // Pointer field containing the offset of the unrecognized value + // within the original packet. + // + // Which when taken together indicate that an unknown protocol should + // be treated as an unrecognized next header value. + _ = returnError(r, &icmpReasonParameterProblem{ + code: header.ICMPv6UnknownHeader, + pointer: it.ParseOffset(), + }, pkt) + default: + panic(fmt.Sprintf("unrecognized result from DeliverTransportPacket = %d", res)) + } } 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. + _ = returnError(r, &icmpReasonParameterProblem{ + code: header.ICMPv6UnknownHeader, + pointer: it.ParseOffset(), + }, pkt) r.Stats().UnknownProtocolRcvdPackets.Increment() return } @@ -417,19 +854,340 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt *stack.PacketBuffer) { } // Close cleans up resources associated with the endpoint. -func (*endpoint) Close() {} +func (e *endpoint) Close() { + e.mu.Lock() + e.disableLocked() + e.mu.ndp.removeSLAACAddresses(false /* keepLinkLocal */) + e.stopDADForPermanentAddressesLocked() + e.mu.addressableEndpointState.Cleanup() + e.mu.Unlock() + + e.protocol.forgetEndpoint(e) +} // NetworkProtocolNumber implements stack.NetworkEndpoint.NetworkProtocolNumber. func (e *endpoint) NetworkProtocolNumber() tcpip.NetworkProtocolNumber { return e.protocol.Number() } +// AddAndAcquirePermanentAddress implements stack.AddressableEndpoint. +func (e *endpoint) AddAndAcquirePermanentAddress(addr tcpip.AddressWithPrefix, peb stack.PrimaryEndpointBehavior, configType stack.AddressConfigType, deprecated bool) (stack.AddressEndpoint, *tcpip.Error) { + // TODO(b/169350103): add checks here after making sure we no longer receive + // an empty address. + e.mu.Lock() + defer e.mu.Unlock() + return e.addAndAcquirePermanentAddressLocked(addr, peb, configType, deprecated) +} + +// addAndAcquirePermanentAddressLocked is like AddAndAcquirePermanentAddress but +// with locking requirements. +// +// addAndAcquirePermanentAddressLocked also joins the passed address's +// solicited-node multicast group and start duplicate address detection. +// +// Precondition: e.mu must be write locked. +func (e *endpoint) addAndAcquirePermanentAddressLocked(addr tcpip.AddressWithPrefix, peb stack.PrimaryEndpointBehavior, configType stack.AddressConfigType, deprecated bool) (stack.AddressEndpoint, *tcpip.Error) { + addressEndpoint, err := e.mu.addressableEndpointState.AddAndAcquirePermanentAddress(addr, peb, configType, deprecated) + if err != nil { + return nil, err + } + + if !header.IsV6UnicastAddress(addr.Address) { + return addressEndpoint, nil + } + + snmc := header.SolicitedNodeAddr(addr.Address) + if _, err := e.mu.addressableEndpointState.JoinGroup(snmc); err != nil { + return nil, err + } + + addressEndpoint.SetKind(stack.PermanentTentative) + + if e.Enabled() { + if err := e.mu.ndp.startDuplicateAddressDetection(addr.Address, addressEndpoint); err != nil { + return nil, err + } + } + + return addressEndpoint, nil +} + +// RemovePermanentAddress implements stack.AddressableEndpoint. +func (e *endpoint) RemovePermanentAddress(addr tcpip.Address) *tcpip.Error { + e.mu.Lock() + defer e.mu.Unlock() + + addressEndpoint := e.getAddressRLocked(addr) + if addressEndpoint == nil || !addressEndpoint.GetKind().IsPermanent() { + return tcpip.ErrBadLocalAddress + } + + return e.removePermanentEndpointLocked(addressEndpoint, true) +} + +// removePermanentEndpointLocked is like removePermanentAddressLocked except +// it works with a stack.AddressEndpoint. +// +// Precondition: e.mu must be write locked. +func (e *endpoint) removePermanentEndpointLocked(addressEndpoint stack.AddressEndpoint, allowSLAACInvalidation bool) *tcpip.Error { + addr := addressEndpoint.AddressWithPrefix() + unicast := header.IsV6UnicastAddress(addr.Address) + if unicast { + e.mu.ndp.stopDuplicateAddressDetection(addr.Address) + + // If we are removing an address generated via SLAAC, cleanup + // its SLAAC resources and notify the integrator. + switch addressEndpoint.ConfigType() { + case stack.AddressConfigSlaac: + e.mu.ndp.cleanupSLAACAddrResourcesAndNotify(addr, allowSLAACInvalidation) + case stack.AddressConfigSlaacTemp: + e.mu.ndp.cleanupTempSLAACAddrResourcesAndNotify(addr, allowSLAACInvalidation) + } + } + + if err := e.mu.addressableEndpointState.RemovePermanentEndpoint(addressEndpoint); err != nil { + return err + } + + if !unicast { + return nil + } + + snmc := header.SolicitedNodeAddr(addr.Address) + if _, err := e.mu.addressableEndpointState.LeaveGroup(snmc); err != nil && err != tcpip.ErrBadLocalAddress { + return err + } + + return nil +} + +// hasPermanentAddressLocked returns true if the endpoint has a permanent +// address equal to the passed address. +// +// Precondition: e.mu must be read or write locked. +func (e *endpoint) hasPermanentAddressRLocked(addr tcpip.Address) bool { + addressEndpoint := e.getAddressRLocked(addr) + if addressEndpoint == nil { + return false + } + return addressEndpoint.GetKind().IsPermanent() +} + +// getAddressRLocked returns the endpoint for the passed address. +// +// Precondition: e.mu must be read or write locked. +func (e *endpoint) getAddressRLocked(localAddr tcpip.Address) stack.AddressEndpoint { + return e.mu.addressableEndpointState.ReadOnly().Lookup(localAddr) +} + +// MainAddress implements stack.AddressableEndpoint. +func (e *endpoint) MainAddress() tcpip.AddressWithPrefix { + e.mu.RLock() + defer e.mu.RUnlock() + return e.mu.addressableEndpointState.MainAddress() +} + +// AcquireAssignedAddress implements stack.AddressableEndpoint. +func (e *endpoint) AcquireAssignedAddress(localAddr tcpip.Address, allowTemp bool, tempPEB stack.PrimaryEndpointBehavior) stack.AddressEndpoint { + e.mu.Lock() + defer e.mu.Unlock() + return e.acquireAddressOrCreateTempLocked(localAddr, allowTemp, tempPEB) +} + +// acquireAddressOrCreateTempLocked is like AcquireAssignedAddress but with +// locking requirements. +// +// Precondition: e.mu must be write locked. +func (e *endpoint) acquireAddressOrCreateTempLocked(localAddr tcpip.Address, allowTemp bool, tempPEB stack.PrimaryEndpointBehavior) stack.AddressEndpoint { + return e.mu.addressableEndpointState.AcquireAssignedAddress(localAddr, allowTemp, tempPEB) +} + +// AcquireOutgoingPrimaryAddress implements stack.AddressableEndpoint. +func (e *endpoint) AcquireOutgoingPrimaryAddress(remoteAddr tcpip.Address, allowExpired bool) stack.AddressEndpoint { + e.mu.RLock() + defer e.mu.RUnlock() + return e.acquireOutgoingPrimaryAddressRLocked(remoteAddr, allowExpired) +} + +// acquireOutgoingPrimaryAddressRLocked is like AcquireOutgoingPrimaryAddress +// but with locking requirements. +// +// Precondition: e.mu must be read locked. +func (e *endpoint) acquireOutgoingPrimaryAddressRLocked(remoteAddr tcpip.Address, allowExpired bool) stack.AddressEndpoint { + // addrCandidate is a candidate for Source Address Selection, as per + // RFC 6724 section 5. + type addrCandidate struct { + addressEndpoint stack.AddressEndpoint + scope header.IPv6AddressScope + } + + if len(remoteAddr) == 0 { + return e.mu.addressableEndpointState.AcquireOutgoingPrimaryAddress(remoteAddr, allowExpired) + } + + // Create a candidate set of available addresses we can potentially use as a + // source address. + var cs []addrCandidate + e.mu.addressableEndpointState.ReadOnly().ForEachPrimaryEndpoint(func(addressEndpoint stack.AddressEndpoint) { + // If r is not valid for outgoing connections, it is not a valid endpoint. + if !addressEndpoint.IsAssigned(allowExpired) { + return + } + + addr := addressEndpoint.AddressWithPrefix().Address + scope, err := header.ScopeForIPv6Address(addr) + if err != nil { + // Should never happen as we got r from the primary IPv6 endpoint list and + // ScopeForIPv6Address only returns an error if addr is not an IPv6 + // address. + panic(fmt.Sprintf("header.ScopeForIPv6Address(%s): %s", addr, err)) + } + + cs = append(cs, addrCandidate{ + addressEndpoint: addressEndpoint, + scope: scope, + }) + }) + + remoteScope, err := header.ScopeForIPv6Address(remoteAddr) + if err != nil { + // primaryIPv6Endpoint should never be called with an invalid IPv6 address. + panic(fmt.Sprintf("header.ScopeForIPv6Address(%s): %s", remoteAddr, err)) + } + + // Sort the addresses as per RFC 6724 section 5 rules 1-3. + // + // TODO(b/146021396): Implement rules 4-8 of RFC 6724 section 5. + sort.Slice(cs, func(i, j int) bool { + sa := cs[i] + sb := cs[j] + + // Prefer same address as per RFC 6724 section 5 rule 1. + if sa.addressEndpoint.AddressWithPrefix().Address == remoteAddr { + return true + } + if sb.addressEndpoint.AddressWithPrefix().Address == remoteAddr { + return false + } + + // Prefer appropriate scope as per RFC 6724 section 5 rule 2. + if sa.scope < sb.scope { + return sa.scope >= remoteScope + } else if sb.scope < sa.scope { + return sb.scope < remoteScope + } + + // Avoid deprecated addresses as per RFC 6724 section 5 rule 3. + if saDep, sbDep := sa.addressEndpoint.Deprecated(), sb.addressEndpoint.Deprecated(); saDep != sbDep { + // If sa is not deprecated, it is preferred over sb. + return sbDep + } + + // Prefer temporary addresses as per RFC 6724 section 5 rule 7. + if saTemp, sbTemp := sa.addressEndpoint.ConfigType() == stack.AddressConfigSlaacTemp, sb.addressEndpoint.ConfigType() == stack.AddressConfigSlaacTemp; saTemp != sbTemp { + return saTemp + } + + // sa and sb are equal, return the endpoint that is closest to the front of + // the primary endpoint list. + return i < j + }) + + // Return the most preferred address that can have its reference count + // incremented. + for _, c := range cs { + if c.addressEndpoint.IncRef() { + return c.addressEndpoint + } + } + + return nil +} + +// PrimaryAddresses implements stack.AddressableEndpoint. +func (e *endpoint) PrimaryAddresses() []tcpip.AddressWithPrefix { + e.mu.RLock() + defer e.mu.RUnlock() + return e.mu.addressableEndpointState.PrimaryAddresses() +} + +// PermanentAddresses implements stack.AddressableEndpoint. +func (e *endpoint) PermanentAddresses() []tcpip.AddressWithPrefix { + e.mu.RLock() + defer e.mu.RUnlock() + return e.mu.addressableEndpointState.PermanentAddresses() +} + +// JoinGroup implements stack.GroupAddressableEndpoint. +func (e *endpoint) JoinGroup(addr tcpip.Address) (bool, *tcpip.Error) { + if !header.IsV6MulticastAddress(addr) { + return false, tcpip.ErrBadAddress + } + + e.mu.Lock() + defer e.mu.Unlock() + return e.mu.addressableEndpointState.JoinGroup(addr) +} + +// LeaveGroup implements stack.GroupAddressableEndpoint. +func (e *endpoint) LeaveGroup(addr tcpip.Address) (bool, *tcpip.Error) { + e.mu.Lock() + defer e.mu.Unlock() + return e.mu.addressableEndpointState.LeaveGroup(addr) +} + +// IsInGroup implements stack.GroupAddressableEndpoint. +func (e *endpoint) IsInGroup(addr tcpip.Address) bool { + e.mu.RLock() + defer e.mu.RUnlock() + return e.mu.addressableEndpointState.IsInGroup(addr) +} + +var _ stack.ForwardingNetworkProtocol = (*protocol)(nil) +var _ stack.NetworkProtocol = (*protocol)(nil) + type protocol struct { + stack *stack.Stack + + mu struct { + sync.RWMutex + + eps map[*endpoint]struct{} + } + // defaultTTL is the current default TTL for the protocol. Only the - // uint8 portion of it is meaningful and it must be accessed - // atomically. - defaultTTL uint32 + // uint8 portion of it is meaningful. + // + // Must be accessed using atomic operations. + defaultTTL uint32 + + // forwarding is set to 1 when the protocol has forwarding enabled and 0 + // when it is disabled. + // + // Must be accessed using atomic operations. + forwarding uint32 + fragmentation *fragmentation.Fragmentation + + // ndpDisp is the NDP event dispatcher that is used to send the netstack + // integrator NDP related events. + ndpDisp NDPDispatcher + + // ndpConfigs is the default NDP configurations used by an IPv6 endpoint. + ndpConfigs NDPConfigurations + + // opaqueIIDOpts hold the options for generating opaque interface identifiers + // (IIDs) as outlined by RFC 7217. + opaqueIIDOpts OpaqueInterfaceIdentifierOptions + + // tempIIDSeed is used to seed the initial temporary interface identifier + // history value used to generate IIDs for temporary SLAAC addresses. + tempIIDSeed []byte + + // autoGenIPv6LinkLocal determines whether or not the stack attempts to + // auto-generate an IPv6 link-local address for newly enabled non-loopback + // NICs. See the AutoGenIPv6LinkLocal field of Options for more details. + autoGenIPv6LinkLocal bool } // Number returns the ipv6 protocol number. @@ -454,16 +1212,36 @@ func (*protocol) ParseAddresses(v buffer.View) (src, dst tcpip.Address) { } // NewEndpoint creates a new ipv6 endpoint. -func (p *protocol) NewEndpoint(nicID tcpip.NICID, linkAddrCache stack.LinkAddressCache, nud stack.NUDHandler, dispatcher stack.TransportDispatcher, linkEP stack.LinkEndpoint, st *stack.Stack) stack.NetworkEndpoint { - return &endpoint{ - nicID: nicID, - linkEP: linkEP, +func (p *protocol) NewEndpoint(nic stack.NetworkInterface, linkAddrCache stack.LinkAddressCache, nud stack.NUDHandler, dispatcher stack.TransportDispatcher) stack.NetworkEndpoint { + e := &endpoint{ + nic: nic, + linkEP: nic.LinkEndpoint(), linkAddrCache: linkAddrCache, nud: nud, dispatcher: dispatcher, protocol: p, - stack: st, } + e.mu.addressableEndpointState.Init(e) + e.mu.ndp = ndpState{ + ep: e, + configs: p.ndpConfigs, + dad: make(map[tcpip.Address]dadState), + defaultRouters: make(map[tcpip.Address]defaultRouterState), + onLinkPrefixes: make(map[tcpip.Subnet]onLinkPrefixState), + slaacPrefixes: make(map[tcpip.Subnet]slaacPrefixState), + } + e.mu.ndp.initializeTempAddrState() + + p.mu.Lock() + defer p.mu.Unlock() + p.mu.eps[e] = struct{}{} + return e +} + +func (p *protocol) forgetEndpoint(e *endpoint) { + p.mu.Lock() + defer p.mu.Unlock() + delete(p.mu.eps, e) } // SetOption implements NetworkProtocol.SetOption. @@ -504,75 +1282,43 @@ func (*protocol) Close() {} // Wait implements stack.TransportProtocol.Wait. func (*protocol) Wait() {} -// Parse implements stack.TransportProtocol.Parse. +// Parse implements stack.NetworkProtocol.Parse. func (*protocol) Parse(pkt *stack.PacketBuffer) (proto tcpip.TransportProtocolNumber, hasTransportHdr bool, ok bool) { - hdr, ok := pkt.Data.PullUp(header.IPv6MinimumSize) + proto, _, fragOffset, fragMore, ok := parse.IPv6(pkt) if !ok { return 0, false, false } - ipHdr := header.IPv6(hdr) - // dataClone consists of: - // - Any IPv6 header bytes after the first 40 (i.e. extensions). - // - The transport header, if present. - // - Any other payload data. - views := [8]buffer.View{} - dataClone := pkt.Data.Clone(views[:]) - dataClone.TrimFront(header.IPv6MinimumSize) - it := header.MakeIPv6PayloadIterator(header.IPv6ExtensionHeaderIdentifier(ipHdr.NextHeader()), dataClone) + return proto, !fragMore && fragOffset == 0, true +} - // Iterate over the IPv6 extensions to find their length. - // - // Parsing occurs again in HandlePacket because we don't track the - // extensions in PacketBuffer. Unfortunately, that means HandlePacket - // has to do the parsing work again. - var nextHdr tcpip.TransportProtocolNumber - foundNext := true - extensionsSize := 0 -traverseExtensions: - for extHdr, done, err := it.Next(); ; extHdr, done, err = it.Next() { - if err != nil { - break - } - // If we exhaust the extension list, the entire packet is the IPv6 header - // and (possibly) extensions. - if done { - extensionsSize = dataClone.Size() - foundNext = false - break - } +// Forwarding implements stack.ForwardingNetworkProtocol. +func (p *protocol) Forwarding() bool { + return uint8(atomic.LoadUint32(&p.forwarding)) == 1 +} - switch extHdr := extHdr.(type) { - case header.IPv6FragmentExtHdr: - // If this is an atomic fragment, we don't have to treat it specially. - if !extHdr.More() && extHdr.FragmentOffset() == 0 { - continue - } - // This is a non-atomic fragment and has to be re-assembled before we can - // examine the payload for a transport header. - foundNext = false +// setForwarding sets the forwarding status for the protocol. +// +// Returns true if the forwarding status was updated. +func (p *protocol) setForwarding(v bool) bool { + if v { + return atomic.SwapUint32(&p.forwarding, 1) == 0 + } + return atomic.SwapUint32(&p.forwarding, 0) == 1 +} - case header.IPv6RawPayloadHeader: - // We've found the payload after any extensions. - extensionsSize = dataClone.Size() - extHdr.Buf.Size() - nextHdr = tcpip.TransportProtocolNumber(extHdr.Identifier) - break traverseExtensions +// SetForwarding implements stack.ForwardingNetworkProtocol. +func (p *protocol) SetForwarding(v bool) { + p.mu.Lock() + defer p.mu.Unlock() - default: - // Any other extension is a no-op, keep looping until we find the payload. - } + if !p.setForwarding(v) { + return } - // Put the IPv6 header with extensions in pkt.NetworkHeader(). - hdr, ok = pkt.NetworkHeader().Consume(header.IPv6MinimumSize + extensionsSize) - if !ok { - panic(fmt.Sprintf("pkt.Data should have at least %d bytes, but only has %d.", header.IPv6MinimumSize+extensionsSize, pkt.Data.Size())) + for ep := range p.mu.eps { + ep.transitionForwarding(v) } - ipHdr = header.IPv6(hdr) - pkt.Data.CapLength(int(ipHdr.PayloadLength())) - pkt.NetworkProtocolNumber = header.IPv6ProtocolNumber - - return nextHdr, foundNext, true } // calculateMTU calculates the network-layer payload MTU based on the link-layer @@ -585,10 +1331,69 @@ func calculateMTU(mtu uint32) uint32 { return maxPayloadSize } -// NewProtocol returns an IPv6 network protocol. -func NewProtocol() stack.NetworkProtocol { - return &protocol{ - defaultTTL: DefaultTTL, - fragmentation: fragmentation.NewFragmentation(header.IPv6FragmentExtHdrFragmentOffsetBytesPerUnit, fragmentation.HighFragThreshold, fragmentation.LowFragThreshold, fragmentation.DefaultReassembleTimeout), +// Options holds options to configure a new protocol. +type Options struct { + // NDPConfigs is the default NDP configurations used by interfaces. + NDPConfigs NDPConfigurations + + // AutoGenIPv6LinkLocal determines whether or not the stack attempts to + // auto-generate an IPv6 link-local address for newly enabled non-loopback + // NICs. + // + // Note, setting this to true does not mean that a link-local address is + // assigned right away, or at all. If Duplicate Address Detection is enabled, + // an address is only assigned if it successfully resolves. If it fails, no + // further attempts are made to auto-generate an IPv6 link-local adddress. + // + // The generated link-local address follows RFC 4291 Appendix A guidelines. + AutoGenIPv6LinkLocal bool + + // NDPDisp is the NDP event dispatcher that an integrator can provide to + // receive NDP related events. + NDPDisp NDPDispatcher + + // OpaqueIIDOpts hold the options for generating opaque interface + // identifiers (IIDs) as outlined by RFC 7217. + OpaqueIIDOpts OpaqueInterfaceIdentifierOptions + + // TempIIDSeed is used to seed the initial temporary interface identifier + // history value used to generate IIDs for temporary SLAAC addresses. + // + // Temporary SLAAC adresses are short-lived addresses which are unpredictable + // and random from the perspective of other nodes on the network. It is + // recommended that the seed be a random byte buffer of at least + // header.IIDSize bytes to make sure that temporary SLAAC addresses are + // sufficiently random. It should follow minimum randomness requirements for + // security as outlined by RFC 4086. + // + // Note: using a nil value, the same seed across netstack program runs, or a + // seed that is too small would reduce randomness and increase predictability, + // defeating the purpose of temporary SLAAC addresses. + TempIIDSeed []byte +} + +// NewProtocolWithOptions returns an IPv6 network protocol. +func NewProtocolWithOptions(opts Options) stack.NetworkProtocolFactory { + opts.NDPConfigs.validate() + + return func(s *stack.Stack) stack.NetworkProtocol { + p := &protocol{ + stack: s, + fragmentation: fragmentation.NewFragmentation(header.IPv6FragmentExtHdrFragmentOffsetBytesPerUnit, fragmentation.HighFragThreshold, fragmentation.LowFragThreshold, fragmentation.DefaultReassembleTimeout, s.Clock()), + + ndpDisp: opts.NDPDisp, + ndpConfigs: opts.NDPConfigs, + opaqueIIDOpts: opts.OpaqueIIDOpts, + tempIIDSeed: opts.TempIIDSeed, + autoGenIPv6LinkLocal: opts.AutoGenIPv6LinkLocal, + } + p.mu.eps = make(map[*endpoint]struct{}) + p.SetDefaultTTL(DefaultTTL) + return p } } + +// NewProtocol is equivalent to NewProtocolWithOptions with an empty Options. +func NewProtocol(s *stack.Stack) stack.NetworkProtocol { + return NewProtocolWithOptions(Options{})(s) +} diff --git a/pkg/tcpip/network/ipv6/ipv6_test.go b/pkg/tcpip/network/ipv6/ipv6_test.go index 5f9822c49..d7f82973b 100644 --- a/pkg/tcpip/network/ipv6/ipv6_test.go +++ b/pkg/tcpip/network/ipv6/ipv6_test.go @@ -21,8 +21,10 @@ import ( "github.com/google/go-cmp/cmp" "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" + "gvisor.dev/gvisor/pkg/tcpip/checker" "gvisor.dev/gvisor/pkg/tcpip/header" "gvisor.dev/gvisor/pkg/tcpip/link/channel" + "gvisor.dev/gvisor/pkg/tcpip/network/testutil" "gvisor.dev/gvisor/pkg/tcpip/stack" "gvisor.dev/gvisor/pkg/tcpip/transport/icmp" "gvisor.dev/gvisor/pkg/tcpip/transport/udp" @@ -140,18 +142,18 @@ func testReceiveUDP(t *testing.T, s *stack.Stack, e *channel.Endpoint, src, dst func TestReceiveOnAllNodesMulticastAddr(t *testing.T) { tests := []struct { name string - protocolFactory stack.TransportProtocol + protocolFactory stack.TransportProtocolFactory rxf func(t *testing.T, s *stack.Stack, e *channel.Endpoint, src, dst tcpip.Address, want uint64) }{ - {"ICMP", icmp.NewProtocol6(), testReceiveICMP}, - {"UDP", udp.NewProtocol(), testReceiveUDP}, + {"ICMP", icmp.NewProtocol6, testReceiveICMP}, + {"UDP", udp.NewProtocol, testReceiveUDP}, } for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{test.protocolFactory}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{test.protocolFactory}, }) e := channel.New(10, 1280, linkAddr1) if err := s.CreateNIC(1, e); err != nil { @@ -173,11 +175,11 @@ func TestReceiveOnSolicitedNodeAddr(t *testing.T) { tests := []struct { name string - protocolFactory stack.TransportProtocol + protocolFactory stack.TransportProtocolFactory rxf func(t *testing.T, s *stack.Stack, e *channel.Endpoint, src, dst tcpip.Address, want uint64) }{ - {"ICMP", icmp.NewProtocol6(), testReceiveICMP}, - {"UDP", udp.NewProtocol(), testReceiveUDP}, + {"ICMP", icmp.NewProtocol6, testReceiveICMP}, + {"UDP", udp.NewProtocol, testReceiveUDP}, } snmc := header.SolicitedNodeAddr(addr2) @@ -185,8 +187,8 @@ func TestReceiveOnSolicitedNodeAddr(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{test.protocolFactory}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{test.protocolFactory}, }) e := channel.New(1, 1280, linkAddr1) if err := s.CreateNIC(nicID, e); err != nil { @@ -272,7 +274,7 @@ func TestAddIpv6Address(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, }) if err := s.CreateNIC(1, &stubLinkEndpoint{}); err != nil { t.Fatalf("CreateNIC(_) = %s", err) @@ -300,11 +302,18 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { name string extHdr func(nextHdr uint8) ([]byte, uint8) shouldAccept bool + // Should we expect an ICMP response and if so, with what contents? + expectICMP bool + ICMPType header.ICMPv6Type + ICMPCode header.ICMPv6Code + pointer uint32 + multicast bool }{ { name: "None", extHdr: func(nextHdr uint8) ([]byte, uint8) { return []byte{}, nextHdr }, shouldAccept: true, + expectICMP: false, }, { name: "hopbyhop with unknown option skippable action", @@ -335,9 +344,10 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { }, hopByHopExtHdrID }, shouldAccept: false, + expectICMP: false, }, { - name: "hopbyhop with unknown option discard and send icmp action", + name: "hopbyhop with unknown option discard and send icmp action (unicast)", extHdr: func(nextHdr uint8) ([]byte, uint8) { return []byte{ nextHdr, 1, @@ -347,12 +357,38 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { // Discard & send ICMP if option is unknown. 191, 6, 1, 2, 3, 4, 5, 6, + //^ Unknown option. }, hopByHopExtHdrID }, shouldAccept: false, + expectICMP: true, + ICMPType: header.ICMPv6ParamProblem, + ICMPCode: header.ICMPv6UnknownOption, + pointer: header.IPv6FixedHeaderSize + 8, }, { - name: "hopbyhop with unknown option discard and send icmp action unless multicast dest", + name: "hopbyhop with unknown option discard and send icmp action (multicast)", + extHdr: func(nextHdr uint8) ([]byte, uint8) { + return []byte{ + nextHdr, 1, + + // Skippable unknown. + 63, 4, 1, 2, 3, 4, + + // Discard & send ICMP if option is unknown. + 191, 6, 1, 2, 3, 4, 5, 6, + //^ Unknown option. + }, hopByHopExtHdrID + }, + multicast: true, + shouldAccept: false, + expectICMP: true, + ICMPType: header.ICMPv6ParamProblem, + ICMPCode: header.ICMPv6UnknownOption, + pointer: header.IPv6FixedHeaderSize + 8, + }, + { + name: "hopbyhop with unknown option discard and send icmp action unless multicast dest (unicast)", extHdr: func(nextHdr uint8) ([]byte, uint8) { return []byte{ nextHdr, 1, @@ -363,39 +399,97 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { // Discard & send ICMP unless packet is for multicast destination if // option is unknown. 255, 6, 1, 2, 3, 4, 5, 6, + //^ Unknown option. }, hopByHopExtHdrID }, + expectICMP: true, + ICMPType: header.ICMPv6ParamProblem, + ICMPCode: header.ICMPv6UnknownOption, + pointer: header.IPv6FixedHeaderSize + 8, + }, + { + name: "hopbyhop with unknown option discard and send icmp action unless multicast dest (multicast)", + extHdr: func(nextHdr uint8) ([]byte, uint8) { + return []byte{ + nextHdr, 1, + + // Skippable unknown. + 63, 4, 1, 2, 3, 4, + + // Discard & send ICMP unless packet is for multicast destination if + // option is unknown. + 255, 6, 1, 2, 3, 4, 5, 6, + //^ Unknown option. + }, hopByHopExtHdrID + }, + multicast: true, shouldAccept: false, + expectICMP: false, }, { - name: "routing with zero segments left", - extHdr: func(nextHdr uint8) ([]byte, uint8) { return []byte{nextHdr, 0, 1, 0, 2, 3, 4, 5}, routingExtHdrID }, + name: "routing with zero segments left", + extHdr: func(nextHdr uint8) ([]byte, uint8) { + return []byte{ + nextHdr, 0, + 1, 0, 2, 3, 4, 5, + }, routingExtHdrID + }, shouldAccept: true, }, { - name: "routing with non-zero segments left", - extHdr: func(nextHdr uint8) ([]byte, uint8) { return []byte{nextHdr, 0, 1, 1, 2, 3, 4, 5}, routingExtHdrID }, + name: "routing with non-zero segments left", + extHdr: func(nextHdr uint8) ([]byte, uint8) { + return []byte{ + nextHdr, 0, + 1, 1, 2, 3, 4, 5, + }, routingExtHdrID + }, shouldAccept: false, + expectICMP: true, + ICMPType: header.ICMPv6ParamProblem, + ICMPCode: header.ICMPv6ErroneousHeader, + pointer: header.IPv6FixedHeaderSize + 2, }, { - name: "atomic fragment with zero ID", - extHdr: func(nextHdr uint8) ([]byte, uint8) { return []byte{nextHdr, 0, 0, 0, 0, 0, 0, 0}, fragmentExtHdrID }, + name: "atomic fragment with zero ID", + extHdr: func(nextHdr uint8) ([]byte, uint8) { + return []byte{ + nextHdr, 0, + 0, 0, 0, 0, 0, 0, + }, fragmentExtHdrID + }, shouldAccept: true, }, { - name: "atomic fragment with non-zero ID", - extHdr: func(nextHdr uint8) ([]byte, uint8) { return []byte{nextHdr, 0, 0, 0, 1, 2, 3, 4}, fragmentExtHdrID }, + name: "atomic fragment with non-zero ID", + extHdr: func(nextHdr uint8) ([]byte, uint8) { + return []byte{ + nextHdr, 0, + 0, 0, 1, 2, 3, 4, + }, fragmentExtHdrID + }, shouldAccept: true, + expectICMP: false, }, { - name: "fragment", - extHdr: func(nextHdr uint8) ([]byte, uint8) { return []byte{nextHdr, 0, 1, 0, 1, 2, 3, 4}, fragmentExtHdrID }, + name: "fragment", + extHdr: func(nextHdr uint8) ([]byte, uint8) { + return []byte{ + nextHdr, 0, + 1, 0, 1, 2, 3, 4, + }, fragmentExtHdrID + }, shouldAccept: false, + expectICMP: false, }, { - name: "No next header", - extHdr: func(nextHdr uint8) ([]byte, uint8) { return []byte{}, noNextHdrID }, + name: "No next header", + extHdr: func(nextHdr uint8) ([]byte, uint8) { + return []byte{}, + noNextHdrID + }, shouldAccept: false, + expectICMP: false, }, { name: "destination with unknown option skippable action", @@ -411,6 +505,7 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { }, destinationExtHdrID }, shouldAccept: true, + expectICMP: false, }, { name: "destination with unknown option discard action", @@ -426,9 +521,30 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { }, destinationExtHdrID }, shouldAccept: false, + expectICMP: false, + }, + { + name: "destination with unknown option discard and send icmp action (unicast)", + extHdr: func(nextHdr uint8) ([]byte, uint8) { + return []byte{ + nextHdr, 1, + + // Skippable unknown. + 63, 4, 1, 2, 3, 4, + + // Discard & send ICMP if option is unknown. + 191, 6, 1, 2, 3, 4, 5, 6, + //^ 191 is an unknown option. + }, destinationExtHdrID + }, + shouldAccept: false, + expectICMP: true, + ICMPType: header.ICMPv6ParamProblem, + ICMPCode: header.ICMPv6UnknownOption, + pointer: header.IPv6FixedHeaderSize + 8, }, { - name: "destination with unknown option discard and send icmp action", + name: "destination with unknown option discard and send icmp action (muilticast)", extHdr: func(nextHdr uint8) ([]byte, uint8) { return []byte{ nextHdr, 1, @@ -438,12 +554,18 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { // Discard & send ICMP if option is unknown. 191, 6, 1, 2, 3, 4, 5, 6, + //^ 191 is an unknown option. }, destinationExtHdrID }, + multicast: true, shouldAccept: false, + expectICMP: true, + ICMPType: header.ICMPv6ParamProblem, + ICMPCode: header.ICMPv6UnknownOption, + pointer: header.IPv6FixedHeaderSize + 8, }, { - name: "destination with unknown option discard and send icmp action unless multicast dest", + name: "destination with unknown option discard and send icmp action unless multicast dest (unicast)", extHdr: func(nextHdr uint8) ([]byte, uint8) { return []byte{ nextHdr, 1, @@ -454,22 +576,33 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { // Discard & send ICMP unless packet is for multicast destination if // option is unknown. 255, 6, 1, 2, 3, 4, 5, 6, + //^ 255 is unknown. }, destinationExtHdrID }, shouldAccept: false, + expectICMP: true, + ICMPType: header.ICMPv6ParamProblem, + ICMPCode: header.ICMPv6UnknownOption, + pointer: header.IPv6FixedHeaderSize + 8, }, { - name: "routing - atomic fragment", + name: "destination with unknown option discard and send icmp action unless multicast dest (multicast)", extHdr: func(nextHdr uint8) ([]byte, uint8) { return []byte{ - // Routing extension header. - fragmentExtHdrID, 0, 1, 0, 2, 3, 4, 5, + nextHdr, 1, - // Fragment extension header. - nextHdr, 0, 0, 0, 1, 2, 3, 4, - }, routingExtHdrID + // Skippable unknown. + 63, 4, 1, 2, 3, 4, + + // Discard & send ICMP unless packet is for multicast destination if + // option is unknown. + 255, 6, 1, 2, 3, 4, 5, 6, + //^ 255 is unknown. + }, destinationExtHdrID }, - shouldAccept: true, + shouldAccept: false, + expectICMP: false, + multicast: true, }, { name: "atomic fragment - routing", @@ -503,12 +636,42 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { return []byte{ // Routing extension header. hopByHopExtHdrID, 0, 1, 0, 2, 3, 4, 5, + // ^^^ The HopByHop extension header may not appear after the first + // extension header. // Hop By Hop extension header with skippable unknown option. nextHdr, 0, 62, 4, 1, 2, 3, 4, }, routingExtHdrID }, shouldAccept: false, + expectICMP: true, + ICMPType: header.ICMPv6ParamProblem, + ICMPCode: header.ICMPv6UnknownHeader, + pointer: header.IPv6FixedHeaderSize, + }, + { + name: "routing - hop by hop (with send icmp unknown)", + extHdr: func(nextHdr uint8) ([]byte, uint8) { + return []byte{ + // Routing extension header. + hopByHopExtHdrID, 0, 1, 0, 2, 3, 4, 5, + // ^^^ The HopByHop extension header may not appear after the first + // extension header. + + nextHdr, 1, + + // Skippable unknown. + 63, 4, 1, 2, 3, 4, + + // Skippable unknown. + 191, 6, 1, 2, 3, 4, 5, 6, + }, routingExtHdrID + }, + shouldAccept: false, + expectICMP: true, + ICMPType: header.ICMPv6ParamProblem, + ICMPCode: header.ICMPv6UnknownHeader, + pointer: header.IPv6FixedHeaderSize, }, { name: "No next header", @@ -552,6 +715,7 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { }, hopByHopExtHdrID }, shouldAccept: false, + expectICMP: false, }, { name: "hopbyhop (with skippable unknown) - routing - atomic fragment - destination (with discard unknown)", @@ -572,16 +736,17 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { }, hopByHopExtHdrID }, shouldAccept: false, + expectICMP: false, }, } for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, }) - e := channel.New(0, 1280, linkAddr1) + e := channel.New(1, 1280, linkAddr1) if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(%d, _) = %s", nicID, err) } @@ -589,6 +754,14 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { t.Fatalf("AddAddress(%d, %d, %s) = %s", nicID, ProtocolNumber, addr2, err) } + // Add a default route so that a return packet knows where to go. + s.SetRouteTable([]tcpip.Route{ + { + Destination: header.IPv6EmptySubnet, + NIC: nicID, + }, + }) + wq := waiter.Queue{} we, ch := waiter.NewChannelEntry(nil) wq.EventRegister(&we, waiter.EventIn) @@ -630,12 +803,16 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { // Serialize IPv6 fixed header. payloadLength := hdr.UsedLength() ip := header.IPv6(hdr.Prepend(header.IPv6MinimumSize)) + dstAddr := tcpip.Address(addr2) + if test.multicast { + dstAddr = header.IPv6AllNodesMulticastAddress + } ip.Encode(&header.IPv6Fields{ PayloadLength: uint16(payloadLength), NextHeader: ipv6NextHdr, HopLimit: 255, SrcAddr: addr1, - DstAddr: addr2, + DstAddr: dstAddr, }) e.InjectInbound(ProtocolNumber, stack.NewPacketBuffer(stack.PacketBufferOptions{ @@ -649,6 +826,44 @@ func TestReceiveIPv6ExtHdrs(t *testing.T) { t.Errorf("got UDP Rx Packets = %d, want = 0", got) } + if !test.expectICMP { + if p, ok := e.Read(); ok { + t.Fatalf("unexpected packet received: %#v", p) + } + return + } + + // ICMP required. + p, ok := e.Read() + if !ok { + t.Fatalf("expected packet wasn't written out") + } + + // Pack the output packet into a single buffer.View as the checkers + // assume that. + vv := buffer.NewVectorisedView(p.Pkt.Size(), p.Pkt.Views()) + pkt := vv.ToView() + if got, want := len(pkt), header.IPv6FixedHeaderSize+header.ICMPv6MinimumSize+hdr.UsedLength(); got != want { + t.Fatalf("got an ICMP packet of size = %d, want = %d", got, want) + } + + ipHdr := header.IPv6(pkt) + checker.IPv6(t, ipHdr, checker.ICMPv6( + checker.ICMPv6Type(test.ICMPType), + checker.ICMPv6Code(test.ICMPCode))) + + // We know we are looking at no extension headers in the error ICMP + // packets. + icmpPkt := header.ICMPv6(ipHdr.Payload()) + // We know we sent small packets that won't be truncated when reflected + // back to us. + originalPacket := icmpPkt.Payload() + if got, want := icmpPkt.TypeSpecific(), test.pointer; got != want { + t.Errorf("unexpected ICMPv6 pointer, got = %d, want = %d\n", got, want) + } + if diff := cmp.Diff(hdr.View(), buffer.View(originalPacket)); diff != "" { + t.Errorf("ICMPv6 payload mismatch (-want +got):\n%s", diff) + } return } @@ -1548,8 +1763,8 @@ func TestReceiveIPv6Fragments(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, }) e := channel.New(0, 1280, linkAddr1) if err := s.CreateNIC(nicID, e); err != nil { @@ -1667,8 +1882,8 @@ func TestInvalidIPv6Fragments(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ - NewProtocol(), + NetworkProtocols: []stack.NetworkProtocolFactory{ + NewProtocol, }, }) e := channel.New(0, 1500, linkAddr1) @@ -1709,3 +1924,215 @@ func TestInvalidIPv6Fragments(t *testing.T) { }) } } + +func TestWriteStats(t *testing.T) { + const nPackets = 3 + tests := []struct { + name string + setup func(*testing.T, *stack.Stack) + allowPackets int + expectSent int + expectDropped int + expectWritten int + }{ + { + name: "Accept all", + // No setup needed, tables accept everything by default. + setup: func(*testing.T, *stack.Stack) {}, + allowPackets: math.MaxInt32, + expectSent: nPackets, + expectDropped: 0, + expectWritten: nPackets, + }, { + name: "Accept all with error", + // No setup needed, tables accept everything by default. + setup: func(*testing.T, *stack.Stack) {}, + allowPackets: nPackets - 1, + expectSent: nPackets - 1, + expectDropped: 0, + expectWritten: nPackets - 1, + }, { + name: "Drop all", + setup: func(t *testing.T, stk *stack.Stack) { + // Install Output DROP rule. + t.Helper() + ipt := stk.IPTables() + filter, ok := ipt.GetTable(stack.FilterTable, true /* ipv6 */) + if !ok { + t.Fatalf("failed to find filter table") + } + ruleIdx := filter.BuiltinChains[stack.Output] + filter.Rules[ruleIdx].Target = &stack.DropTarget{} + if err := ipt.ReplaceTable(stack.FilterTable, filter, true /* ipv6 */); err != nil { + t.Fatalf("failed to replace table: %v", err) + } + }, + allowPackets: math.MaxInt32, + expectSent: 0, + expectDropped: nPackets, + expectWritten: nPackets, + }, { + name: "Drop some", + setup: func(t *testing.T, stk *stack.Stack) { + // Install Output DROP rule that matches only 1 + // of the 3 packets. + t.Helper() + ipt := stk.IPTables() + filter, ok := ipt.GetTable(stack.FilterTable, true /* ipv6 */) + if !ok { + t.Fatalf("failed to find filter table") + } + // We'll match and DROP the last packet. + ruleIdx := filter.BuiltinChains[stack.Output] + filter.Rules[ruleIdx].Target = &stack.DropTarget{} + filter.Rules[ruleIdx].Matchers = []stack.Matcher{&limitedMatcher{nPackets - 1}} + // Make sure the next rule is ACCEPT. + filter.Rules[ruleIdx+1].Target = &stack.AcceptTarget{} + if err := ipt.ReplaceTable(stack.FilterTable, filter, true /* ipv6 */); err != nil { + t.Fatalf("failed to replace table: %v", err) + } + }, + allowPackets: math.MaxInt32, + expectSent: nPackets - 1, + expectDropped: 1, + expectWritten: nPackets, + }, + } + + writers := []struct { + name string + writePackets func(*stack.Route, stack.PacketBufferList) (int, *tcpip.Error) + }{ + { + name: "WritePacket", + writePackets: func(rt *stack.Route, pkts stack.PacketBufferList) (int, *tcpip.Error) { + nWritten := 0 + for pkt := pkts.Front(); pkt != nil; pkt = pkt.Next() { + if err := rt.WritePacket(nil, stack.NetworkHeaderParams{}, pkt); err != nil { + return nWritten, err + } + nWritten++ + } + return nWritten, nil + }, + }, { + name: "WritePackets", + writePackets: func(rt *stack.Route, pkts stack.PacketBufferList) (int, *tcpip.Error) { + return rt.WritePackets(nil, pkts, stack.NetworkHeaderParams{}) + }, + }, + } + + for _, writer := range writers { + t.Run(writer.name, func(t *testing.T) { + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + ep := testutil.NewMockLinkEndpoint(header.IPv6MinimumMTU, tcpip.ErrInvalidEndpointState, test.allowPackets) + rt := buildRoute(t, ep) + + var pkts stack.PacketBufferList + for i := 0; i < nPackets; i++ { + pkt := stack.NewPacketBuffer(stack.PacketBufferOptions{ + ReserveHeaderBytes: header.UDPMinimumSize + int(rt.MaxHeaderLength()), + Data: buffer.NewView(0).ToVectorisedView(), + }) + pkt.TransportHeader().Push(header.UDPMinimumSize) + pkts.PushBack(pkt) + } + + test.setup(t, rt.Stack()) + + nWritten, _ := writer.writePackets(&rt, pkts) + + if got := int(rt.Stats().IP.PacketsSent.Value()); got != test.expectSent { + t.Errorf("sent %d packets, but expected to send %d", got, test.expectSent) + } + if got := int(rt.Stats().IP.IPTablesOutputDropped.Value()); got != test.expectDropped { + t.Errorf("dropped %d packets, but expected to drop %d", got, test.expectDropped) + } + if nWritten != test.expectWritten { + t.Errorf("wrote %d packets, but expected WritePackets to return %d", nWritten, test.expectWritten) + } + }) + } + }) + } +} + +func buildRoute(t *testing.T, ep stack.LinkEndpoint) stack.Route { + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, + }) + if err := s.CreateNIC(1, ep); err != nil { + t.Fatalf("CreateNIC(1, _) failed: %s", err) + } + const ( + src = "\xfc\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01" + dst = "\xfc\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02" + ) + if err := s.AddAddress(1, ProtocolNumber, src); err != nil { + t.Fatalf("AddAddress(1, %d, _) failed: %s", ProtocolNumber, err) + } + { + subnet, err := tcpip.NewSubnet(dst, tcpip.AddressMask("\xfc\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff")) + if err != nil { + t.Fatalf("NewSubnet(_, _) failed: %v", err) + } + s.SetRouteTable([]tcpip.Route{{ + Destination: subnet, + NIC: 1, + }}) + } + rt, err := s.FindRoute(1, src, dst, ProtocolNumber, false /* multicastLoop */) + if err != nil { + t.Fatalf("got FindRoute(1, _, _, %d, false) = %s, want = nil", ProtocolNumber, err) + } + return rt +} + +// limitedMatcher is an iptables matcher that matches after a certain number of +// packets are checked against it. +type limitedMatcher struct { + limit int +} + +// Name implements Matcher.Name. +func (*limitedMatcher) Name() string { + return "limitedMatcher" +} + +// Match implements Matcher.Match. +func (lm *limitedMatcher) Match(stack.Hook, *stack.PacketBuffer, string) (bool, bool) { + if lm.limit == 0 { + return true, false + } + lm.limit-- + return false, false +} + +func TestClearEndpointFromProtocolOnClose(t *testing.T) { + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, + }) + proto := s.NetworkProtocolInstance(ProtocolNumber).(*protocol) + ep := proto.NewEndpoint(&testInterface{}, nil, nil, nil).(*endpoint) + { + proto.mu.Lock() + _, hasEP := proto.mu.eps[ep] + proto.mu.Unlock() + if !hasEP { + t.Fatalf("expected protocol to have ep = %p in set of endpoints", ep) + } + } + + ep.Close() + + { + proto.mu.Lock() + _, hasEP := proto.mu.eps[ep] + proto.mu.Unlock() + if hasEP { + t.Fatalf("unexpectedly found ep = %p in set of protocol's endpoints", ep) + } + } +} diff --git a/pkg/tcpip/stack/ndp.go b/pkg/tcpip/network/ipv6/ndp.go index b0873d1af..48a4c65e3 100644 --- a/pkg/tcpip/stack/ndp.go +++ b/pkg/tcpip/network/ipv6/ndp.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package stack +package ipv6 import ( "fmt" @@ -23,9 +23,27 @@ import ( "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/header" + "gvisor.dev/gvisor/pkg/tcpip/stack" ) const ( + // defaultRetransmitTimer is the default amount of time to wait between + // sending reachability probes. + // + // Default taken from RETRANS_TIMER of RFC 4861 section 10. + defaultRetransmitTimer = time.Second + + // minimumRetransmitTimer is the minimum amount of time to wait between + // sending reachability probes. + // + // Note, RFC 4861 does not impose a minimum Retransmit Timer, but we do here + // to make sure the messages are not sent all at once. We also come to this + // value because in the RetransmitTimer field of a Router Advertisement, a + // value of 0 means unspecified, so the smallest valid value is 1. Note, the + // unit of the RetransmitTimer field in the Router Advertisement is + // milliseconds. + minimumRetransmitTimer = time.Millisecond + // defaultDupAddrDetectTransmits is the default number of NDP Neighbor // Solicitation messages to send when doing Duplicate Address Detection // for a tentative address. @@ -34,7 +52,7 @@ const ( defaultDupAddrDetectTransmits = 1 // defaultMaxRtrSolicitations is the default number of Router - // Solicitation messages to send when a NIC becomes enabled. + // Solicitation messages to send when an IPv6 endpoint becomes enabled. // // Default = 3 (from RFC 4861 section 10). defaultMaxRtrSolicitations = 3 @@ -131,7 +149,7 @@ const ( minRegenAdvanceDuration = time.Duration(0) // maxSLAACAddrLocalRegenAttempts is the maximum number of times to attempt - // SLAAC address regenerations in response to a NIC-local conflict. + // SLAAC address regenerations in response to an IPv6 endpoint-local conflict. maxSLAACAddrLocalRegenAttempts = 10 ) @@ -163,7 +181,7 @@ var ( // This is exported as a variable (instead of a constant) so tests // can update it to a smaller value. // - // This value guarantees that a temporary address will be preferred for at + // This value guarantees that a temporary address is preferred for at // least 1hr if the SLAAC prefix is valid for at least that time. MinMaxTempAddrPreferredLifetime = defaultRegenAdvanceDuration + MaxDesyncFactor + time.Hour @@ -173,11 +191,17 @@ var ( // This is exported as a variable (instead of a constant) so tests // can update it to a smaller value. // - // This value guarantees that a temporary address will be valid for at least + // This value guarantees that a temporary address is valid for at least // 2hrs if the SLAAC prefix is valid for at least that time. MinMaxTempAddrValidLifetime = 2 * time.Hour ) +// NDPEndpoint is an endpoint that supports NDP. +type NDPEndpoint interface { + // SetNDPConfigurations sets the NDP configurations. + SetNDPConfigurations(NDPConfigurations) +} + // DHCPv6ConfigurationFromNDPRA is a configuration available via DHCPv6 that an // NDP Router Advertisement informed the Stack about. type DHCPv6ConfigurationFromNDPRA int @@ -192,7 +216,7 @@ const ( // DHCPv6ManagedAddress indicates that addresses are available via DHCPv6. // // DHCPv6ManagedAddress also implies DHCPv6OtherConfigurations because DHCPv6 - // will return all available configuration information. + // returns all available configuration information when serving addresses. DHCPv6ManagedAddress // DHCPv6OtherConfigurations indicates that other configuration information is @@ -207,19 +231,18 @@ const ( // NDPDispatcher is the interface integrators of netstack must implement to // receive and handle NDP related events. type NDPDispatcher interface { - // OnDuplicateAddressDetectionStatus will be called when the DAD process - // for an address (addr) on a NIC (with ID nicID) completes. resolved - // will be set to true if DAD completed successfully (no duplicate addr - // detected); false otherwise (addr was detected to be a duplicate on - // the link the NIC is a part of, or it was stopped for some other - // reason, such as the address being removed). If an error occured - // during DAD, err will be set and resolved must be ignored. + // OnDuplicateAddressDetectionStatus is called when the DAD process for an + // address (addr) on a NIC (with ID nicID) completes. resolved is set to true + // if DAD completed successfully (no duplicate addr detected); false otherwise + // (addr was detected to be a duplicate on the link the NIC is a part of, or + // it was stopped for some other reason, such as the address being removed). + // If an error occured during DAD, err is set and resolved must be ignored. // // This function is not permitted to block indefinitely. This function // is also not permitted to call into the stack. OnDuplicateAddressDetectionStatus(nicID tcpip.NICID, addr tcpip.Address, resolved bool, err *tcpip.Error) - // OnDefaultRouterDiscovered will be called when a new default router is + // OnDefaultRouterDiscovered is called when a new default router is // discovered. Implementations must return true if the newly discovered // router should be remembered. // @@ -227,56 +250,55 @@ type NDPDispatcher interface { // is also not permitted to call into the stack. OnDefaultRouterDiscovered(nicID tcpip.NICID, addr tcpip.Address) bool - // OnDefaultRouterInvalidated will be called when a discovered default - // router that was remembered is invalidated. + // OnDefaultRouterInvalidated is called when a discovered default router that + // was remembered is invalidated. // // This function is not permitted to block indefinitely. This function // is also not permitted to call into the stack. OnDefaultRouterInvalidated(nicID tcpip.NICID, addr tcpip.Address) - // OnOnLinkPrefixDiscovered will be called when a new on-link prefix is - // discovered. Implementations must return true if the newly discovered - // on-link prefix should be remembered. + // OnOnLinkPrefixDiscovered is called when a new on-link prefix is discovered. + // Implementations must return true if the newly discovered on-link prefix + // should be remembered. // // This function is not permitted to block indefinitely. This function // is also not permitted to call into the stack. OnOnLinkPrefixDiscovered(nicID tcpip.NICID, prefix tcpip.Subnet) bool - // OnOnLinkPrefixInvalidated will be called when a discovered on-link - // prefix that was remembered is invalidated. + // OnOnLinkPrefixInvalidated is called when a discovered on-link prefix that + // was remembered is invalidated. // // This function is not permitted to block indefinitely. This function // is also not permitted to call into the stack. OnOnLinkPrefixInvalidated(nicID tcpip.NICID, prefix tcpip.Subnet) - // OnAutoGenAddress will be called when a new prefix with its - // autonomous address-configuration flag set has been received and SLAAC - // has been performed. Implementations may prevent the stack from - // assigning the address to the NIC by returning false. + // OnAutoGenAddress is called when a new prefix with its autonomous address- + // configuration flag set is received and SLAAC was performed. Implementations + // may prevent the stack from assigning the address to the NIC by returning + // false. // // This function is not permitted to block indefinitely. It must not // call functions on the stack itself. OnAutoGenAddress(tcpip.NICID, tcpip.AddressWithPrefix) bool - // OnAutoGenAddressDeprecated will be called when an auto-generated - // address (as part of SLAAC) has been deprecated, but is still - // considered valid. Note, if an address is invalidated at the same - // time it is deprecated, the deprecation event MAY be omitted. + // OnAutoGenAddressDeprecated is called when an auto-generated address (SLAAC) + // is deprecated, but is still considered valid. Note, if an address is + // invalidated at the same ime it is deprecated, the deprecation event may not + // be received. // // This function is not permitted to block indefinitely. It must not // call functions on the stack itself. OnAutoGenAddressDeprecated(tcpip.NICID, tcpip.AddressWithPrefix) - // OnAutoGenAddressInvalidated will be called when an auto-generated - // address (as part of SLAAC) has been invalidated. + // OnAutoGenAddressInvalidated is called when an auto-generated address + // (SLAAC) is invalidated. // // This function is not permitted to block indefinitely. It must not // call functions on the stack itself. OnAutoGenAddressInvalidated(tcpip.NICID, tcpip.AddressWithPrefix) - // OnRecursiveDNSServerOption will be called when an NDP option with - // recursive DNS servers has been received. Note, addrs may contain - // link-local addresses. + // OnRecursiveDNSServerOption is called when the stack learns of DNS servers + // through NDP. Note, the addresses may contain link-local addresses. // // It is up to the caller to use the DNS Servers only for their valid // lifetime. OnRecursiveDNSServerOption may be called for new or @@ -288,8 +310,8 @@ type NDPDispatcher interface { // call functions on the stack itself. OnRecursiveDNSServerOption(nicID tcpip.NICID, addrs []tcpip.Address, lifetime time.Duration) - // OnDNSSearchListOption will be called when an NDP option with a DNS - // search list has been received. + // OnDNSSearchListOption is called when the stack learns of DNS search lists + // through NDP. // // It is up to the caller to use the domain names in the search list // for only their valid lifetime. OnDNSSearchListOption may be called @@ -298,8 +320,8 @@ type NDPDispatcher interface { // be increased, decreased or completely invalidated when lifetime = 0. OnDNSSearchListOption(nicID tcpip.NICID, domainNames []string, lifetime time.Duration) - // OnDHCPv6Configuration will be called with an updated configuration that is - // available via DHCPv6 for a specified NIC. + // OnDHCPv6Configuration is called with an updated configuration that is + // available via DHCPv6 for the passed NIC. // // This function is not permitted to block indefinitely. It must not // call functions on the stack itself. @@ -320,7 +342,7 @@ type NDPConfigurations struct { // Must be greater than or equal to 1ms. RetransmitTimer time.Duration - // The number of Router Solicitation messages to send when the NIC + // The number of Router Solicitation messages to send when the IPv6 endpoint // becomes enabled. MaxRtrSolicitations uint8 @@ -335,24 +357,22 @@ type NDPConfigurations struct { // Must be greater than or equal to 0s. MaxRtrSolicitationDelay time.Duration - // HandleRAs determines whether or not Router Advertisements will be - // processed. + // HandleRAs determines whether or not Router Advertisements are processed. HandleRAs bool - // DiscoverDefaultRouters determines whether or not default routers will - // be discovered from Router Advertisements. This configuration is - // ignored if HandleRAs is false. + // DiscoverDefaultRouters determines whether or not default routers are + // discovered from Router Advertisements, as per RFC 4861 section 6. This + // configuration is ignored if HandleRAs is false. DiscoverDefaultRouters bool - // DiscoverOnLinkPrefixes determines whether or not on-link prefixes - // will be discovered from Router Advertisements' Prefix Information - // option. This configuration is ignored if HandleRAs is false. + // DiscoverOnLinkPrefixes determines whether or not on-link prefixes are + // discovered from Router Advertisements' Prefix Information option, as per + // RFC 4861 section 6. This configuration is ignored if HandleRAs is false. DiscoverOnLinkPrefixes bool - // AutoGenGlobalAddresses determines whether or not global IPv6 - // addresses will be generated for a NIC in response to receiving a new - // Prefix Information option with its Autonomous Address - // AutoConfiguration flag set, as a host, as per RFC 4862 (SLAAC). + // AutoGenGlobalAddresses determines whether or not an IPv6 endpoint performs + // SLAAC to auto-generate global SLAAC addresses in response to Prefix + // Information options, as per RFC 4862. // // Note, if an address was already generated for some unique prefix, as // part of SLAAC, this option does not affect whether or not the @@ -366,12 +386,12 @@ type NDPConfigurations struct { // // If the method used to generate the address does not support creating // alternative addresses (e.g. IIDs based on the modified EUI64 of a NIC's - // MAC address), then no attempt will be made to resolve the conflict. + // MAC address), then no attempt is made to resolve the conflict. AutoGenAddressConflictRetries uint8 // AutoGenTempGlobalAddresses determines whether or not temporary SLAAC - // addresses will be generated for a NIC as part of SLAAC privacy extensions, - // RFC 4941. + // addresses are generated for an IPv6 endpoint as part of SLAAC privacy + // extensions, as per RFC 4941. // // Ignored if AutoGenGlobalAddresses is false. AutoGenTempGlobalAddresses bool @@ -410,7 +430,7 @@ func DefaultNDPConfigurations() NDPConfigurations { } // validate modifies an NDPConfigurations with valid values. If invalid values -// are present in c, the corresponding default values will be used instead. +// are present in c, the corresponding default values are used instead. func (c *NDPConfigurations) validate() { if c.RetransmitTimer < minimumRetransmitTimer { c.RetransmitTimer = defaultRetransmitTimer @@ -439,8 +459,8 @@ func (c *NDPConfigurations) validate() { // ndpState is the per-interface NDP state. type ndpState struct { - // The NIC this ndpState is for. - nic *NIC + // The IPv6 endpoint this ndpState is for. + ep *endpoint // configs is the per-interface NDP configurations. configs NDPConfigurations @@ -458,8 +478,8 @@ type ndpState struct { // Used to let the Router Solicitation timer know that it has been stopped. // // Must only be read from or written to while protected by the lock of - // the NIC this ndpState is associated with. MUST be set when the timer is - // set. + // the IPv6 endpoint this ndpState is associated with. MUST be set when the + // timer is set. done *bool } @@ -492,7 +512,7 @@ type dadState struct { // Used to let the DAD timer know that it has been stopped. // // Must only be read from or written to while protected by the lock of - // the NIC this dadState is associated with. + // the IPv6 endpoint this dadState is associated with. done *bool } @@ -537,7 +557,7 @@ type tempSLAACAddrState struct { // The address's endpoint. // // Must not be nil. - ref *referencedNetworkEndpoint + addressEndpoint stack.AddressEndpoint // Has a new temporary SLAAC address already been regenerated? regenerated bool @@ -567,10 +587,10 @@ type slaacPrefixState struct { // // May only be nil when the address is being (re-)generated. Otherwise, // must not be nil as all SLAAC prefixes must have a stable address. - ref *referencedNetworkEndpoint + addressEndpoint stack.AddressEndpoint - // The number of times an address has been generated locally where the NIC - // already had the generated address. + // The number of times an address has been generated locally where the IPv6 + // endpoint already had the generated address. localGenerationFailures uint8 } @@ -578,11 +598,12 @@ type slaacPrefixState struct { tempAddrs map[tcpip.Address]tempSLAACAddrState // The next two fields are used by both stable and temporary addresses - // generated for a SLAAC prefix. This is safe as only 1 address will be - // in the generation and DAD process at any time. That is, no two addresses - // will be generated at the same time for a given SLAAC prefix. + // generated for a SLAAC prefix. This is safe as only 1 address is in the + // generation and DAD process at any time. That is, no two addresses are + // generated at the same time for a given SLAAC prefix. - // The number of times an address has been generated and added to the NIC. + // The number of times an address has been generated and added to the IPv6 + // endpoint. // // Addresses may be regenerated in reseponse to a DAD conflicts. generationAttempts uint8 @@ -597,16 +618,16 @@ type slaacPrefixState struct { // This function must only be called by IPv6 addresses that are currently // tentative. // -// The NIC that ndp belongs to MUST be locked. -func (ndp *ndpState) startDuplicateAddressDetection(addr tcpip.Address, ref *referencedNetworkEndpoint) *tcpip.Error { +// The IPv6 endpoint that ndp belongs to MUST be locked. +func (ndp *ndpState) startDuplicateAddressDetection(addr tcpip.Address, addressEndpoint stack.AddressEndpoint) *tcpip.Error { // addr must be a valid unicast IPv6 address. if !header.IsV6UnicastAddress(addr) { return tcpip.ErrAddressFamilyNotSupported } - if ref.getKind() != permanentTentative { + if addressEndpoint.GetKind() != stack.PermanentTentative { // The endpoint should be marked as tentative since we are starting DAD. - panic(fmt.Sprintf("ndpdad: addr %s is not tentative on NIC(%d)", addr, ndp.nic.ID())) + panic(fmt.Sprintf("ndpdad: addr %s is not tentative on NIC(%d)", addr, ndp.ep.nic.ID())) } // Should not attempt to perform DAD on an address that is currently in the @@ -617,18 +638,18 @@ func (ndp *ndpState) startDuplicateAddressDetection(addr tcpip.Address, ref *ref // existed, we would get an error since we attempted to add a duplicate // address, or its reference count would have been increased without doing // the work that would have been done for an address that was brand new. - // See NIC.addAddressLocked. - panic(fmt.Sprintf("ndpdad: already performing DAD for addr %s on NIC(%d)", addr, ndp.nic.ID())) + // See endpoint.addAddressLocked. + panic(fmt.Sprintf("ndpdad: already performing DAD for addr %s on NIC(%d)", addr, ndp.ep.nic.ID())) } remaining := ndp.configs.DupAddrDetectTransmits if remaining == 0 { - ref.setKind(permanent) + addressEndpoint.SetKind(stack.Permanent) // Consider DAD to have resolved even if no DAD messages were actually // transmitted. - if ndpDisp := ndp.nic.stack.ndpDisp; ndpDisp != nil { - ndpDisp.OnDuplicateAddressDetectionStatus(ndp.nic.ID(), addr, true, nil) + if ndpDisp := ndp.ep.protocol.ndpDisp; ndpDisp != nil { + ndpDisp.OnDuplicateAddressDetectionStatus(ndp.ep.nic.ID(), addr, true, nil) } return nil @@ -637,25 +658,25 @@ func (ndp *ndpState) startDuplicateAddressDetection(addr tcpip.Address, ref *ref var done bool var timer tcpip.Timer // We initially start a timer to fire immediately because some of the DAD work - // cannot be done while holding the NIC's lock. This is effectively the same - // as starting a goroutine but we use a timer that fires immediately so we can - // reset it for the next DAD iteration. - timer = ndp.nic.stack.Clock().AfterFunc(0, func() { - ndp.nic.mu.Lock() - defer ndp.nic.mu.Unlock() + // cannot be done while holding the IPv6 endpoint's lock. This is effectively + // the same as starting a goroutine but we use a timer that fires immediately + // so we can reset it for the next DAD iteration. + timer = ndp.ep.protocol.stack.Clock().AfterFunc(0, func() { + ndp.ep.mu.Lock() + defer ndp.ep.mu.Unlock() if done { // If we reach this point, it means that the DAD timer fired after - // another goroutine already obtained the NIC lock and stopped DAD - // before this function obtained the NIC lock. Simply return here and do - // nothing further. + // another goroutine already obtained the IPv6 endpoint lock and stopped + // DAD before this function obtained the NIC lock. Simply return here and + // do nothing further. return } - if ref.getKind() != permanentTentative { + if addressEndpoint.GetKind() != stack.PermanentTentative { // The endpoint should still be marked as tentative since we are still // performing DAD on it. - panic(fmt.Sprintf("ndpdad: addr %s is no longer tentative on NIC(%d)", addr, ndp.nic.ID())) + panic(fmt.Sprintf("ndpdad: addr %s is no longer tentative on NIC(%d)", addr, ndp.ep.nic.ID())) } dadDone := remaining == 0 @@ -663,33 +684,34 @@ func (ndp *ndpState) startDuplicateAddressDetection(addr tcpip.Address, ref *ref var err *tcpip.Error if !dadDone { // Use the unspecified address as the source address when performing DAD. - ref := ndp.nic.getRefOrCreateTempLocked(header.IPv6ProtocolNumber, header.IPv6Any, NeverPrimaryEndpoint) + addressEndpoint := ndp.ep.acquireAddressOrCreateTempLocked(header.IPv6Any, true /* createTemp */, stack.NeverPrimaryEndpoint) // Do not hold the lock when sending packets which may be a long running // task or may block link address resolution. We know this is safe // because immediately after obtaining the lock again, we check if DAD - // has been stopped before doing any work with the NIC. Note, DAD would be - // stopped if the NIC was disabled or removed, or if the address was - // removed. - ndp.nic.mu.Unlock() - err = ndp.sendDADPacket(addr, ref) - ndp.nic.mu.Lock() + // has been stopped before doing any work with the IPv6 endpoint. Note, + // DAD would be stopped if the IPv6 endpoint was disabled or closed, or if + // the address was removed. + ndp.ep.mu.Unlock() + err = ndp.sendDADPacket(addr, addressEndpoint) + ndp.ep.mu.Lock() + addressEndpoint.DecRef() } if done { // If we reach this point, it means that DAD was stopped after we released - // the NIC's read lock and before we obtained the write lock. + // the IPv6 endpoint's read lock and before we obtained the write lock. return } if dadDone { // DAD has resolved. - ref.setKind(permanent) + addressEndpoint.SetKind(stack.Permanent) } else if err == nil { // DAD is not done and we had no errors when sending the last NDP NS, // schedule the next DAD timer. remaining-- - timer.Reset(ndp.nic.stack.ndpConfigs.RetransmitTimer) + timer.Reset(ndp.configs.RetransmitTimer) return } @@ -698,16 +720,16 @@ func (ndp *ndpState) startDuplicateAddressDetection(addr tcpip.Address, ref *ref // integrator know DAD has completed. delete(ndp.dad, addr) - if ndpDisp := ndp.nic.stack.ndpDisp; ndpDisp != nil { - ndpDisp.OnDuplicateAddressDetectionStatus(ndp.nic.ID(), addr, dadDone, err) + if ndpDisp := ndp.ep.protocol.ndpDisp; ndpDisp != nil { + ndpDisp.OnDuplicateAddressDetectionStatus(ndp.ep.nic.ID(), addr, dadDone, err) } // If DAD resolved for a stable SLAAC address, attempt generation of a // temporary SLAAC address. - if dadDone && ref.configType == slaac { + if dadDone && addressEndpoint.ConfigType() == stack.AddressConfigSlaac { // Reset the generation attempts counter as we are starting the generation // of a new address for the SLAAC prefix. - ndp.regenerateTempSLAACAddr(ref.addrWithPrefix().Subnet(), true /* resetGenAttempts */) + ndp.regenerateTempSLAACAddr(addressEndpoint.AddressWithPrefix().Subnet(), true /* resetGenAttempts */) } }) @@ -722,28 +744,31 @@ func (ndp *ndpState) startDuplicateAddressDetection(addr tcpip.Address, ref *ref // sendDADPacket sends a NS message to see if any nodes on ndp's NIC's link owns // addr. // -// addr must be a tentative IPv6 address on ndp's NIC. +// addr must be a tentative IPv6 address on ndp's IPv6 endpoint. // -// The NIC ndp belongs to MUST NOT be locked. -func (ndp *ndpState) sendDADPacket(addr tcpip.Address, ref *referencedNetworkEndpoint) *tcpip.Error { +// The IPv6 endpoint that ndp belongs to MUST NOT be locked. +func (ndp *ndpState) sendDADPacket(addr tcpip.Address, addressEndpoint stack.AddressEndpoint) *tcpip.Error { snmc := header.SolicitedNodeAddr(addr) - r := makeRoute(header.IPv6ProtocolNumber, ref.address(), snmc, ndp.nic.linkEP.LinkAddress(), ref, false, false) + r, err := ndp.ep.protocol.stack.FindRoute(ndp.ep.nic.ID(), header.IPv6Any, snmc, ProtocolNumber, false /* multicastLoop */) + if err != nil { + return err + } defer r.Release() // Route should resolve immediately since snmc is a multicast address so a // remote link address can be calculated without a resolution process. if c, err := r.Resolve(nil); err != nil { // Do not consider the NIC being unknown or disabled as a fatal error. - // Since this method is required to be called when the NIC is not locked, - // the NIC could have been disabled or removed by another goroutine. + // Since this method is required to be called when the IPv6 endpoint is not + // locked, the NIC could have been disabled or removed by another goroutine. if err == tcpip.ErrUnknownNICID || err != tcpip.ErrInvalidEndpointState { return err } - panic(fmt.Sprintf("ndp: error when resolving route to send NDP NS for DAD (%s -> %s on NIC(%d)): %s", header.IPv6Any, snmc, ndp.nic.ID(), err)) + panic(fmt.Sprintf("ndp: error when resolving route to send NDP NS for DAD (%s -> %s on NIC(%d)): %s", header.IPv6Any, snmc, ndp.ep.nic.ID(), err)) } else if c != nil { - panic(fmt.Sprintf("ndp: route resolution not immediate for route to send NDP NS for DAD (%s -> %s on NIC(%d))", header.IPv6Any, snmc, ndp.nic.ID())) + panic(fmt.Sprintf("ndp: route resolution not immediate for route to send NDP NS for DAD (%s -> %s on NIC(%d))", header.IPv6Any, snmc, ndp.ep.nic.ID())) } icmpData := header.ICMPv6(buffer.NewView(header.ICMPv6NeighborSolicitMinimumSize)) @@ -752,17 +777,16 @@ func (ndp *ndpState) sendDADPacket(addr tcpip.Address, ref *referencedNetworkEnd ns.SetTargetAddress(addr) icmpData.SetChecksum(header.ICMPv6Checksum(icmpData, r.LocalAddress, r.RemoteAddress, buffer.VectorisedView{})) - pkt := NewPacketBuffer(PacketBufferOptions{ + pkt := stack.NewPacketBuffer(stack.PacketBufferOptions{ ReserveHeaderBytes: int(r.MaxHeaderLength()), Data: buffer.View(icmpData).ToVectorisedView(), }) sent := r.Stats().ICMP.V6PacketsSent if err := r.WritePacket(nil, - NetworkHeaderParams{ + stack.NetworkHeaderParams{ Protocol: header.ICMPv6ProtocolNumber, TTL: header.NDPHopLimit, - TOS: DefaultTOS, }, pkt, ); err != nil { sent.Dropped.Increment() @@ -778,11 +802,9 @@ func (ndp *ndpState) sendDADPacket(addr tcpip.Address, ref *referencedNetworkEnd // such a state forever, unless some other external event resolves the DAD // process (receiving an NA from the true owner of addr, or an NS for addr // (implying another node is attempting to use addr)). It is up to the caller -// of this function to handle such a scenario. Normally, addr will be removed -// from n right after this function returns or the address successfully -// resolved. +// of this function to handle such a scenario. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) stopDuplicateAddressDetection(addr tcpip.Address) { dad, ok := ndp.dad[addr] if !ok { @@ -801,30 +823,30 @@ func (ndp *ndpState) stopDuplicateAddressDetection(addr tcpip.Address) { delete(ndp.dad, addr) // Let the integrator know DAD did not resolve. - if ndpDisp := ndp.nic.stack.ndpDisp; ndpDisp != nil { - ndpDisp.OnDuplicateAddressDetectionStatus(ndp.nic.ID(), addr, false, nil) + if ndpDisp := ndp.ep.protocol.ndpDisp; ndpDisp != nil { + ndpDisp.OnDuplicateAddressDetectionStatus(ndp.ep.nic.ID(), addr, false, nil) } } // handleRA handles a Router Advertisement message that arrived on the NIC // this ndp is for. Does nothing if the NIC is configured to not handle RAs. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) handleRA(ip tcpip.Address, ra header.NDPRouterAdvert) { - // Is the NIC configured to handle RAs at all? + // Is the IPv6 endpoint configured to handle RAs at all? // // Currently, the stack does not determine router interface status on a - // per-interface basis; it is a stack-wide configuration, so we check - // stack's forwarding flag to determine if the NIC is a routing - // interface. - if !ndp.configs.HandleRAs || ndp.nic.stack.forwarding { + // per-interface basis; it is a protocol-wide configuration, so we check the + // protocol's forwarding flag to determine if the IPv6 endpoint is forwarding + // packets. + if !ndp.configs.HandleRAs || ndp.ep.protocol.Forwarding() { return } // Only worry about the DHCPv6 configuration if we have an NDPDispatcher as we // only inform the dispatcher on configuration changes. We do nothing else // with the information. - if ndpDisp := ndp.nic.stack.ndpDisp; ndpDisp != nil { + if ndpDisp := ndp.ep.protocol.ndpDisp; ndpDisp != nil { var configuration DHCPv6ConfigurationFromNDPRA switch { case ra.ManagedAddrConfFlag(): @@ -839,11 +861,11 @@ func (ndp *ndpState) handleRA(ip tcpip.Address, ra header.NDPRouterAdvert) { if ndp.dhcpv6Configuration != configuration { ndp.dhcpv6Configuration = configuration - ndpDisp.OnDHCPv6Configuration(ndp.nic.ID(), configuration) + ndpDisp.OnDHCPv6Configuration(ndp.ep.nic.ID(), configuration) } } - // Is the NIC configured to discover default routers? + // Is the IPv6 endpoint configured to discover default routers? if ndp.configs.DiscoverDefaultRouters { rtr, ok := ndp.defaultRouters[ip] rl := ra.RouterLifetime() @@ -881,20 +903,20 @@ func (ndp *ndpState) handleRA(ip tcpip.Address, ra header.NDPRouterAdvert) { for opt, done, _ := it.Next(); !done; opt, done, _ = it.Next() { switch opt := opt.(type) { case header.NDPRecursiveDNSServer: - if ndp.nic.stack.ndpDisp == nil { + if ndp.ep.protocol.ndpDisp == nil { continue } addrs, _ := opt.Addresses() - ndp.nic.stack.ndpDisp.OnRecursiveDNSServerOption(ndp.nic.ID(), addrs, opt.Lifetime()) + ndp.ep.protocol.ndpDisp.OnRecursiveDNSServerOption(ndp.ep.nic.ID(), addrs, opt.Lifetime()) case header.NDPDNSSearchList: - if ndp.nic.stack.ndpDisp == nil { + if ndp.ep.protocol.ndpDisp == nil { continue } domainNames, _ := opt.DomainNames() - ndp.nic.stack.ndpDisp.OnDNSSearchListOption(ndp.nic.ID(), domainNames, opt.Lifetime()) + ndp.ep.protocol.ndpDisp.OnDNSSearchListOption(ndp.ep.nic.ID(), domainNames, opt.Lifetime()) case header.NDPPrefixInformation: prefix := opt.Subnet() @@ -928,7 +950,7 @@ func (ndp *ndpState) handleRA(ip tcpip.Address, ra header.NDPRouterAdvert) { // invalidateDefaultRouter invalidates a discovered default router. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) invalidateDefaultRouter(ip tcpip.Address) { rtr, ok := ndp.defaultRouters[ip] @@ -942,32 +964,32 @@ func (ndp *ndpState) invalidateDefaultRouter(ip tcpip.Address) { delete(ndp.defaultRouters, ip) // Let the integrator know a discovered default router is invalidated. - if ndpDisp := ndp.nic.stack.ndpDisp; ndpDisp != nil { - ndpDisp.OnDefaultRouterInvalidated(ndp.nic.ID(), ip) + if ndpDisp := ndp.ep.protocol.ndpDisp; ndpDisp != nil { + ndpDisp.OnDefaultRouterInvalidated(ndp.ep.nic.ID(), ip) } } // rememberDefaultRouter remembers a newly discovered default router with IPv6 // link-local address ip with lifetime rl. // -// The router identified by ip MUST NOT already be known by the NIC. +// The router identified by ip MUST NOT already be known by the IPv6 endpoint. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) rememberDefaultRouter(ip tcpip.Address, rl time.Duration) { - ndpDisp := ndp.nic.stack.ndpDisp + ndpDisp := ndp.ep.protocol.ndpDisp if ndpDisp == nil { return } // Inform the integrator when we discovered a default router. - if !ndpDisp.OnDefaultRouterDiscovered(ndp.nic.ID(), ip) { + if !ndpDisp.OnDefaultRouterDiscovered(ndp.ep.nic.ID(), ip) { // Informed by the integrator to not remember the router, do // nothing further. return } state := defaultRouterState{ - invalidationJob: ndp.nic.stack.newJob(&ndp.nic.mu, func() { + invalidationJob: ndp.ep.protocol.stack.NewJob(&ndp.ep.mu, func() { ndp.invalidateDefaultRouter(ip) }), } @@ -982,22 +1004,22 @@ func (ndp *ndpState) rememberDefaultRouter(ip tcpip.Address, rl time.Duration) { // // The prefix identified by prefix MUST NOT already be known. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) rememberOnLinkPrefix(prefix tcpip.Subnet, l time.Duration) { - ndpDisp := ndp.nic.stack.ndpDisp + ndpDisp := ndp.ep.protocol.ndpDisp if ndpDisp == nil { return } // Inform the integrator when we discovered an on-link prefix. - if !ndpDisp.OnOnLinkPrefixDiscovered(ndp.nic.ID(), prefix) { + if !ndpDisp.OnOnLinkPrefixDiscovered(ndp.ep.nic.ID(), prefix) { // Informed by the integrator to not remember the prefix, do // nothing further. return } state := onLinkPrefixState{ - invalidationJob: ndp.nic.stack.newJob(&ndp.nic.mu, func() { + invalidationJob: ndp.ep.protocol.stack.NewJob(&ndp.ep.mu, func() { ndp.invalidateOnLinkPrefix(prefix) }), } @@ -1011,7 +1033,7 @@ func (ndp *ndpState) rememberOnLinkPrefix(prefix tcpip.Subnet, l time.Duration) // invalidateOnLinkPrefix invalidates a discovered on-link prefix. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) invalidateOnLinkPrefix(prefix tcpip.Subnet) { s, ok := ndp.onLinkPrefixes[prefix] @@ -1025,8 +1047,8 @@ func (ndp *ndpState) invalidateOnLinkPrefix(prefix tcpip.Subnet) { delete(ndp.onLinkPrefixes, prefix) // Let the integrator know a discovered on-link prefix is invalidated. - if ndpDisp := ndp.nic.stack.ndpDisp; ndpDisp != nil { - ndpDisp.OnOnLinkPrefixInvalidated(ndp.nic.ID(), prefix) + if ndpDisp := ndp.ep.protocol.ndpDisp; ndpDisp != nil { + ndpDisp.OnOnLinkPrefixInvalidated(ndp.ep.nic.ID(), prefix) } } @@ -1036,7 +1058,7 @@ func (ndp *ndpState) invalidateOnLinkPrefix(prefix tcpip.Subnet) { // handleOnLinkPrefixInformation assumes that the prefix this pi is for is // not the link-local prefix and the on-link flag is set. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) handleOnLinkPrefixInformation(pi header.NDPPrefixInformation) { prefix := pi.Subnet() prefixState, ok := ndp.onLinkPrefixes[prefix] @@ -1089,7 +1111,7 @@ func (ndp *ndpState) handleOnLinkPrefixInformation(pi header.NDPPrefixInformatio // handleAutonomousPrefixInformation assumes that the prefix this pi is for is // not the link-local prefix and the autonomous flag is set. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) handleAutonomousPrefixInformation(pi header.NDPPrefixInformation) { vl := pi.ValidLifetime() pl := pi.PreferredLifetime() @@ -1125,7 +1147,7 @@ func (ndp *ndpState) handleAutonomousPrefixInformation(pi header.NDPPrefixInform // // pl is the new preferred lifetime. vl is the new valid lifetime. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) doSLAAC(prefix tcpip.Subnet, pl, vl time.Duration) { // If we do not already have an address for this prefix and the valid // lifetime is 0, no need to do anything further, as per RFC 4862 @@ -1142,15 +1164,15 @@ func (ndp *ndpState) doSLAAC(prefix tcpip.Subnet, pl, vl time.Duration) { } state := slaacPrefixState{ - deprecationJob: ndp.nic.stack.newJob(&ndp.nic.mu, func() { + deprecationJob: ndp.ep.protocol.stack.NewJob(&ndp.ep.mu, func() { state, ok := ndp.slaacPrefixes[prefix] if !ok { panic(fmt.Sprintf("ndp: must have a slaacPrefixes entry for the deprecated SLAAC prefix %s", prefix)) } - ndp.deprecateSLAACAddress(state.stableAddr.ref) + ndp.deprecateSLAACAddress(state.stableAddr.addressEndpoint) }), - invalidationJob: ndp.nic.stack.newJob(&ndp.nic.mu, func() { + invalidationJob: ndp.ep.protocol.stack.NewJob(&ndp.ep.mu, func() { state, ok := ndp.slaacPrefixes[prefix] if !ok { panic(fmt.Sprintf("ndp: must have a slaacPrefixes entry for the invalidated SLAAC prefix %s", prefix)) @@ -1189,7 +1211,7 @@ func (ndp *ndpState) doSLAAC(prefix tcpip.Subnet, pl, vl time.Duration) { } // If the address is assigned (DAD resolved), generate a temporary address. - if state.stableAddr.ref.getKind() == permanent { + if state.stableAddr.addressEndpoint.GetKind() == stack.Permanent { // Reset the generation attempts counter as we are starting the generation // of a new address for the SLAAC prefix. ndp.generateTempSLAACAddr(prefix, &state, true /* resetGenAttempts */) @@ -1198,32 +1220,27 @@ func (ndp *ndpState) doSLAAC(prefix tcpip.Subnet, pl, vl time.Duration) { ndp.slaacPrefixes[prefix] = state } -// addSLAACAddr adds a SLAAC address to the NIC. +// addAndAcquireSLAACAddr adds a SLAAC address to the IPv6 endpoint. // -// The NIC that ndp belongs to MUST be locked. -func (ndp *ndpState) addSLAACAddr(addr tcpip.AddressWithPrefix, configType networkEndpointConfigType, deprecated bool) *referencedNetworkEndpoint { +// The IPv6 endpoint that ndp belongs to MUST be locked. +func (ndp *ndpState) addAndAcquireSLAACAddr(addr tcpip.AddressWithPrefix, configType stack.AddressConfigType, deprecated bool) stack.AddressEndpoint { // Inform the integrator that we have a new SLAAC address. - ndpDisp := ndp.nic.stack.ndpDisp + ndpDisp := ndp.ep.protocol.ndpDisp if ndpDisp == nil { return nil } - if !ndpDisp.OnAutoGenAddress(ndp.nic.ID(), addr) { + if !ndpDisp.OnAutoGenAddress(ndp.ep.nic.ID(), addr) { // Informed by the integrator not to add the address. return nil } - protocolAddr := tcpip.ProtocolAddress{ - Protocol: header.IPv6ProtocolNumber, - AddressWithPrefix: addr, - } - - ref, err := ndp.nic.addAddressLocked(protocolAddr, FirstPrimaryEndpoint, permanent, configType, deprecated) + addressEndpoint, err := ndp.ep.addAndAcquirePermanentAddressLocked(addr, stack.FirstPrimaryEndpoint, configType, deprecated) if err != nil { - panic(fmt.Sprintf("ndp: error when adding SLAAC address %+v: %s", protocolAddr, err)) + panic(fmt.Sprintf("ndp: error when adding SLAAC address %+v: %s", addr, err)) } - return ref + return addressEndpoint } // generateSLAACAddr generates a SLAAC address for prefix. @@ -1232,10 +1249,10 @@ func (ndp *ndpState) addSLAACAddr(addr tcpip.AddressWithPrefix, configType netwo // // Panics if the prefix is not a SLAAC prefix or it already has an address. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) generateSLAACAddr(prefix tcpip.Subnet, state *slaacPrefixState) bool { - if r := state.stableAddr.ref; r != nil { - panic(fmt.Sprintf("ndp: SLAAC prefix %s already has a permenant address %s", prefix, r.addrWithPrefix())) + if addressEndpoint := state.stableAddr.addressEndpoint; addressEndpoint != nil { + panic(fmt.Sprintf("ndp: SLAAC prefix %s already has a permenant address %s", prefix, addressEndpoint.AddressWithPrefix())) } // If we have already reached the maximum address generation attempts for the @@ -1255,11 +1272,11 @@ func (ndp *ndpState) generateSLAACAddr(prefix tcpip.Subnet, state *slaacPrefixSt } dadCounter := state.generationAttempts + state.stableAddr.localGenerationFailures - if oIID := ndp.nic.stack.opaqueIIDOpts; oIID.NICNameFromID != nil { + if oIID := ndp.ep.protocol.opaqueIIDOpts; oIID.NICNameFromID != nil { addrBytes = header.AppendOpaqueInterfaceIdentifier( addrBytes[:header.IIDOffsetInIPv6Address], prefix, - oIID.NICNameFromID(ndp.nic.ID(), ndp.nic.name), + oIID.NICNameFromID(ndp.ep.nic.ID(), ndp.ep.nic.Name()), dadCounter, oIID.SecretKey, ) @@ -1272,7 +1289,7 @@ func (ndp *ndpState) generateSLAACAddr(prefix tcpip.Subnet, state *slaacPrefixSt // // TODO(b/141011931): Validate a LinkEndpoint's link address (provided by // LinkEndpoint.LinkAddress) before reaching this point. - linkAddr := ndp.nic.linkEP.LinkAddress() + linkAddr := ndp.ep.linkEP.LinkAddress() if !header.IsValidUnicastEthernetAddress(linkAddr) { return false } @@ -1291,15 +1308,15 @@ func (ndp *ndpState) generateSLAACAddr(prefix tcpip.Subnet, state *slaacPrefixSt PrefixLen: validPrefixLenForAutoGen, } - if !ndp.nic.hasPermanentAddrLocked(generatedAddr.Address) { + if !ndp.ep.hasPermanentAddressRLocked(generatedAddr.Address) { break } state.stableAddr.localGenerationFailures++ } - if ref := ndp.addSLAACAddr(generatedAddr, slaac, time.Since(state.preferredUntil) >= 0 /* deprecated */); ref != nil { - state.stableAddr.ref = ref + if addressEndpoint := ndp.addAndAcquireSLAACAddr(generatedAddr, stack.AddressConfigSlaac, time.Since(state.preferredUntil) >= 0 /* deprecated */); addressEndpoint != nil { + state.stableAddr.addressEndpoint = addressEndpoint state.generationAttempts++ return true } @@ -1309,10 +1326,9 @@ func (ndp *ndpState) generateSLAACAddr(prefix tcpip.Subnet, state *slaacPrefixSt // regenerateSLAACAddr regenerates an address for a SLAAC prefix. // -// If generating a new address for the prefix fails, the prefix will be -// invalidated. +// If generating a new address for the prefix fails, the prefix is invalidated. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) regenerateSLAACAddr(prefix tcpip.Subnet) { state, ok := ndp.slaacPrefixes[prefix] if !ok { @@ -1332,7 +1348,7 @@ func (ndp *ndpState) regenerateSLAACAddr(prefix tcpip.Subnet) { // generateTempSLAACAddr generates a new temporary SLAAC address. // -// If resetGenAttempts is true, the prefix's generation counter will be reset. +// If resetGenAttempts is true, the prefix's generation counter is reset. // // Returns true if a new address was generated. func (ndp *ndpState) generateTempSLAACAddr(prefix tcpip.Subnet, prefixState *slaacPrefixState, resetGenAttempts bool) bool { @@ -1353,7 +1369,7 @@ func (ndp *ndpState) generateTempSLAACAddr(prefix tcpip.Subnet, prefixState *sla return false } - stableAddr := prefixState.stableAddr.ref.address() + stableAddr := prefixState.stableAddr.addressEndpoint.AddressWithPrefix().Address now := time.Now() // As per RFC 4941 section 3.3 step 4, the valid lifetime of a temporary @@ -1392,7 +1408,8 @@ func (ndp *ndpState) generateTempSLAACAddr(prefix tcpip.Subnet, prefixState *sla return false } - // Attempt to generate a new address that is not already assigned to the NIC. + // Attempt to generate a new address that is not already assigned to the IPv6 + // endpoint. var generatedAddr tcpip.AddressWithPrefix for i := 0; ; i++ { // If we were unable to generate an address after the maximum SLAAC address @@ -1402,7 +1419,7 @@ func (ndp *ndpState) generateTempSLAACAddr(prefix tcpip.Subnet, prefixState *sla } generatedAddr = header.GenerateTempIPv6SLAACAddr(ndp.temporaryIIDHistory[:], stableAddr) - if !ndp.nic.hasPermanentAddrLocked(generatedAddr.Address) { + if !ndp.ep.hasPermanentAddressRLocked(generatedAddr.Address) { break } } @@ -1410,13 +1427,13 @@ func (ndp *ndpState) generateTempSLAACAddr(prefix tcpip.Subnet, prefixState *sla // As per RFC RFC 4941 section 3.3 step 5, we MUST NOT create a temporary // address with a zero preferred lifetime. The checks above ensure this // so we know the address is not deprecated. - ref := ndp.addSLAACAddr(generatedAddr, slaacTemp, false /* deprecated */) - if ref == nil { + addressEndpoint := ndp.addAndAcquireSLAACAddr(generatedAddr, stack.AddressConfigSlaacTemp, false /* deprecated */) + if addressEndpoint == nil { return false } state := tempSLAACAddrState{ - deprecationJob: ndp.nic.stack.newJob(&ndp.nic.mu, func() { + deprecationJob: ndp.ep.protocol.stack.NewJob(&ndp.ep.mu, func() { prefixState, ok := ndp.slaacPrefixes[prefix] if !ok { panic(fmt.Sprintf("ndp: must have a slaacPrefixes entry for %s to deprecate temporary address %s", prefix, generatedAddr)) @@ -1427,9 +1444,9 @@ func (ndp *ndpState) generateTempSLAACAddr(prefix tcpip.Subnet, prefixState *sla panic(fmt.Sprintf("ndp: must have a tempAddr entry to deprecate temporary address %s", generatedAddr)) } - ndp.deprecateSLAACAddress(tempAddrState.ref) + ndp.deprecateSLAACAddress(tempAddrState.addressEndpoint) }), - invalidationJob: ndp.nic.stack.newJob(&ndp.nic.mu, func() { + invalidationJob: ndp.ep.protocol.stack.NewJob(&ndp.ep.mu, func() { prefixState, ok := ndp.slaacPrefixes[prefix] if !ok { panic(fmt.Sprintf("ndp: must have a slaacPrefixes entry for %s to invalidate temporary address %s", prefix, generatedAddr)) @@ -1442,7 +1459,7 @@ func (ndp *ndpState) generateTempSLAACAddr(prefix tcpip.Subnet, prefixState *sla ndp.invalidateTempSLAACAddr(prefixState.tempAddrs, generatedAddr.Address, tempAddrState) }), - regenJob: ndp.nic.stack.newJob(&ndp.nic.mu, func() { + regenJob: ndp.ep.protocol.stack.NewJob(&ndp.ep.mu, func() { prefixState, ok := ndp.slaacPrefixes[prefix] if !ok { panic(fmt.Sprintf("ndp: must have a slaacPrefixes entry for %s to regenerate temporary address after %s", prefix, generatedAddr)) @@ -1465,8 +1482,8 @@ func (ndp *ndpState) generateTempSLAACAddr(prefix tcpip.Subnet, prefixState *sla prefixState.tempAddrs[generatedAddr.Address] = tempAddrState ndp.slaacPrefixes[prefix] = prefixState }), - createdAt: now, - ref: ref, + createdAt: now, + addressEndpoint: addressEndpoint, } state.deprecationJob.Schedule(pl) @@ -1481,7 +1498,7 @@ func (ndp *ndpState) generateTempSLAACAddr(prefix tcpip.Subnet, prefixState *sla // regenerateTempSLAACAddr regenerates a temporary address for a SLAAC prefix. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) regenerateTempSLAACAddr(prefix tcpip.Subnet, resetGenAttempts bool) { state, ok := ndp.slaacPrefixes[prefix] if !ok { @@ -1496,14 +1513,14 @@ func (ndp *ndpState) regenerateTempSLAACAddr(prefix tcpip.Subnet, resetGenAttemp // // pl is the new preferred lifetime. vl is the new valid lifetime. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) refreshSLAACPrefixLifetimes(prefix tcpip.Subnet, prefixState *slaacPrefixState, pl, vl time.Duration) { // If the preferred lifetime is zero, then the prefix should be deprecated. deprecated := pl == 0 if deprecated { - ndp.deprecateSLAACAddress(prefixState.stableAddr.ref) + ndp.deprecateSLAACAddress(prefixState.stableAddr.addressEndpoint) } else { - prefixState.stableAddr.ref.deprecated = false + prefixState.stableAddr.addressEndpoint.SetDeprecated(false) } // If prefix was preferred for some finite lifetime before, cancel the @@ -1565,7 +1582,7 @@ func (ndp *ndpState) refreshSLAACPrefixLifetimes(prefix tcpip.Subnet, prefixStat // If DAD is not yet complete on the stable address, there is no need to do // work with temporary addresses. - if prefixState.stableAddr.ref.getKind() != permanent { + if prefixState.stableAddr.addressEndpoint.GetKind() != stack.Permanent { return } @@ -1608,9 +1625,9 @@ func (ndp *ndpState) refreshSLAACPrefixLifetimes(prefix tcpip.Subnet, prefixStat newPreferredLifetime := preferredUntil.Sub(now) tempAddrState.deprecationJob.Cancel() if newPreferredLifetime <= 0 { - ndp.deprecateSLAACAddress(tempAddrState.ref) + ndp.deprecateSLAACAddress(tempAddrState.addressEndpoint) } else { - tempAddrState.ref.deprecated = false + tempAddrState.addressEndpoint.SetDeprecated(false) tempAddrState.deprecationJob.Schedule(newPreferredLifetime) } @@ -1635,8 +1652,8 @@ func (ndp *ndpState) refreshSLAACPrefixLifetimes(prefix tcpip.Subnet, prefixStat // due to an update in preferred lifetime. // // If each temporay address has already been regenerated, no new temporary - // address will be generated. To ensure continuation of temporary SLAAC - // addresses, we manually try to regenerate an address here. + // address is generated. To ensure continuation of temporary SLAAC addresses, + // we manually try to regenerate an address here. if len(regenForAddr) != 0 || allAddressesRegenerated { // Reset the generation attempts counter as we are starting the generation // of a new address for the SLAAC prefix. @@ -1647,57 +1664,58 @@ func (ndp *ndpState) refreshSLAACPrefixLifetimes(prefix tcpip.Subnet, prefixStat } } -// deprecateSLAACAddress marks ref as deprecated and notifies the stack's NDP -// dispatcher that ref has been deprecated. +// deprecateSLAACAddress marks the address as deprecated and notifies the NDP +// dispatcher that address has been deprecated. // -// deprecateSLAACAddress does nothing if ref is already deprecated. +// deprecateSLAACAddress does nothing if the address is already deprecated. // -// The NIC that ndp belongs to MUST be locked. -func (ndp *ndpState) deprecateSLAACAddress(ref *referencedNetworkEndpoint) { - if ref.deprecated { +// The IPv6 endpoint that ndp belongs to MUST be locked. +func (ndp *ndpState) deprecateSLAACAddress(addressEndpoint stack.AddressEndpoint) { + if addressEndpoint.Deprecated() { return } - ref.deprecated = true - if ndpDisp := ndp.nic.stack.ndpDisp; ndpDisp != nil { - ndpDisp.OnAutoGenAddressDeprecated(ndp.nic.ID(), ref.addrWithPrefix()) + addressEndpoint.SetDeprecated(true) + if ndpDisp := ndp.ep.protocol.ndpDisp; ndpDisp != nil { + ndpDisp.OnAutoGenAddressDeprecated(ndp.ep.nic.ID(), addressEndpoint.AddressWithPrefix()) } } // invalidateSLAACPrefix invalidates a SLAAC prefix. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) invalidateSLAACPrefix(prefix tcpip.Subnet, state slaacPrefixState) { - if r := state.stableAddr.ref; r != nil { + ndp.cleanupSLAACPrefixResources(prefix, state) + + if addressEndpoint := state.stableAddr.addressEndpoint; addressEndpoint != nil { // Since we are already invalidating the prefix, do not invalidate the // prefix when removing the address. - if err := ndp.nic.removePermanentIPv6EndpointLocked(r, false /* allowSLAACInvalidation */); err != nil { - panic(fmt.Sprintf("ndp: error removing stable SLAAC address %s: %s", r.addrWithPrefix(), err)) + if err := ndp.ep.removePermanentEndpointLocked(addressEndpoint, false /* allowSLAACInvalidation */); err != nil { + panic(fmt.Sprintf("ndp: error removing stable SLAAC address %s: %s", addressEndpoint.AddressWithPrefix(), err)) } } - - ndp.cleanupSLAACPrefixResources(prefix, state) } // cleanupSLAACAddrResourcesAndNotify cleans up an invalidated SLAAC address's // resources. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) cleanupSLAACAddrResourcesAndNotify(addr tcpip.AddressWithPrefix, invalidatePrefix bool) { - if ndpDisp := ndp.nic.stack.ndpDisp; ndpDisp != nil { - ndpDisp.OnAutoGenAddressInvalidated(ndp.nic.ID(), addr) + if ndpDisp := ndp.ep.protocol.ndpDisp; ndpDisp != nil { + ndpDisp.OnAutoGenAddressInvalidated(ndp.ep.nic.ID(), addr) } prefix := addr.Subnet() state, ok := ndp.slaacPrefixes[prefix] - if !ok || state.stableAddr.ref == nil || addr.Address != state.stableAddr.ref.address() { + if !ok || state.stableAddr.addressEndpoint == nil || addr.Address != state.stableAddr.addressEndpoint.AddressWithPrefix().Address { return } if !invalidatePrefix { // If the prefix is not being invalidated, disassociate the address from the // prefix and do nothing further. - state.stableAddr.ref = nil + state.stableAddr.addressEndpoint.DecRef() + state.stableAddr.addressEndpoint = nil ndp.slaacPrefixes[prefix] = state return } @@ -1709,14 +1727,17 @@ func (ndp *ndpState) cleanupSLAACAddrResourcesAndNotify(addr tcpip.AddressWithPr // // Panics if the SLAAC prefix is not known. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) cleanupSLAACPrefixResources(prefix tcpip.Subnet, state slaacPrefixState) { // Invalidate all temporary addresses. for tempAddr, tempAddrState := range state.tempAddrs { ndp.invalidateTempSLAACAddr(state.tempAddrs, tempAddr, tempAddrState) } - state.stableAddr.ref = nil + if state.stableAddr.addressEndpoint != nil { + state.stableAddr.addressEndpoint.DecRef() + state.stableAddr.addressEndpoint = nil + } state.deprecationJob.Cancel() state.invalidationJob.Cancel() delete(ndp.slaacPrefixes, prefix) @@ -1724,12 +1745,12 @@ func (ndp *ndpState) cleanupSLAACPrefixResources(prefix tcpip.Subnet, state slaa // invalidateTempSLAACAddr invalidates a temporary SLAAC address. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) invalidateTempSLAACAddr(tempAddrs map[tcpip.Address]tempSLAACAddrState, tempAddr tcpip.Address, tempAddrState tempSLAACAddrState) { // Since we are already invalidating the address, do not invalidate the // address when removing the address. - if err := ndp.nic.removePermanentIPv6EndpointLocked(tempAddrState.ref, false /* allowSLAACInvalidation */); err != nil { - panic(fmt.Sprintf("error removing temporary SLAAC address %s: %s", tempAddrState.ref.addrWithPrefix(), err)) + if err := ndp.ep.removePermanentEndpointLocked(tempAddrState.addressEndpoint, false /* allowSLAACInvalidation */); err != nil { + panic(fmt.Sprintf("error removing temporary SLAAC address %s: %s", tempAddrState.addressEndpoint.AddressWithPrefix(), err)) } ndp.cleanupTempSLAACAddrResources(tempAddrs, tempAddr, tempAddrState) @@ -1738,10 +1759,10 @@ func (ndp *ndpState) invalidateTempSLAACAddr(tempAddrs map[tcpip.Address]tempSLA // cleanupTempSLAACAddrResourcesAndNotify cleans up an invalidated temporary // SLAAC address's resources from ndp. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) cleanupTempSLAACAddrResourcesAndNotify(addr tcpip.AddressWithPrefix, invalidateAddr bool) { - if ndpDisp := ndp.nic.stack.ndpDisp; ndpDisp != nil { - ndpDisp.OnAutoGenAddressInvalidated(ndp.nic.ID(), addr) + if ndpDisp := ndp.ep.protocol.ndpDisp; ndpDisp != nil { + ndpDisp.OnAutoGenAddressInvalidated(ndp.ep.nic.ID(), addr) } if !invalidateAddr { @@ -1765,35 +1786,29 @@ func (ndp *ndpState) cleanupTempSLAACAddrResourcesAndNotify(addr tcpip.AddressWi // cleanupTempSLAACAddrResourcesAndNotify cleans up a temporary SLAAC address's // jobs and entry. // -// The NIC that ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) cleanupTempSLAACAddrResources(tempAddrs map[tcpip.Address]tempSLAACAddrState, tempAddr tcpip.Address, tempAddrState tempSLAACAddrState) { + tempAddrState.addressEndpoint.DecRef() + tempAddrState.addressEndpoint = nil tempAddrState.deprecationJob.Cancel() tempAddrState.invalidationJob.Cancel() tempAddrState.regenJob.Cancel() delete(tempAddrs, tempAddr) } -// cleanupState cleans up ndp's state. -// -// If hostOnly is true, then only host-specific state will be cleaned up. +// removeSLAACAddresses removes all SLAAC addresses. // -// cleanupState MUST be called with hostOnly set to true when ndp's NIC is -// transitioning from a host to a router. This function will invalidate all -// discovered on-link prefixes, discovered routers, and auto-generated -// addresses. -// -// If hostOnly is true, then the link-local auto-generated address will not be -// invalidated as routers are also expected to generate a link-local address. +// If keepLinkLocal is false, the SLAAC generated link-local address is removed. // -// The NIC that ndp belongs to MUST be locked. -func (ndp *ndpState) cleanupState(hostOnly bool) { +// The IPv6 endpoint that ndp belongs to MUST be locked. +func (ndp *ndpState) removeSLAACAddresses(keepLinkLocal bool) { linkLocalSubnet := header.IPv6LinkLocalPrefix.Subnet() - linkLocalPrefixes := 0 + var linkLocalPrefixes int for prefix, state := range ndp.slaacPrefixes { // RFC 4862 section 5 states that routers are also expected to generate a // link-local address so we do not invalidate them if we are cleaning up // host-only state. - if hostOnly && prefix == linkLocalSubnet { + if keepLinkLocal && prefix == linkLocalSubnet { linkLocalPrefixes++ continue } @@ -1804,6 +1819,21 @@ func (ndp *ndpState) cleanupState(hostOnly bool) { if got := len(ndp.slaacPrefixes); got != linkLocalPrefixes { panic(fmt.Sprintf("ndp: still have non-linklocal SLAAC prefixes after cleaning up; found = %d prefixes, of which %d are link-local", got, linkLocalPrefixes)) } +} + +// cleanupState cleans up ndp's state. +// +// If hostOnly is true, then only host-specific state is cleaned up. +// +// This function invalidates all discovered on-link prefixes, discovered +// routers, and auto-generated addresses. +// +// If hostOnly is true, then the link-local auto-generated address aren't +// invalidated as routers are also expected to generate a link-local address. +// +// The IPv6 endpoint that ndp belongs to MUST be locked. +func (ndp *ndpState) cleanupState(hostOnly bool) { + ndp.removeSLAACAddresses(hostOnly /* keepLinkLocal */) for prefix := range ndp.onLinkPrefixes { ndp.invalidateOnLinkPrefix(prefix) @@ -1827,7 +1857,7 @@ func (ndp *ndpState) cleanupState(hostOnly bool) { // startSolicitingRouters starts soliciting routers, as per RFC 4861 section // 6.3.7. If routers are already being solicited, this function does nothing. // -// The NIC ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) startSolicitingRouters() { if ndp.rtrSolicit.timer != nil { // We are already soliciting routers. @@ -1848,27 +1878,37 @@ func (ndp *ndpState) startSolicitingRouters() { var done bool ndp.rtrSolicit.done = &done - ndp.rtrSolicit.timer = ndp.nic.stack.Clock().AfterFunc(delay, func() { - ndp.nic.mu.Lock() + ndp.rtrSolicit.timer = ndp.ep.protocol.stack.Clock().AfterFunc(delay, func() { + ndp.ep.mu.Lock() if done { // If we reach this point, it means that the RS timer fired after another - // goroutine already obtained the NIC lock and stopped solicitations. - // Simply return here and do nothing further. - ndp.nic.mu.Unlock() + // goroutine already obtained the IPv6 endpoint lock and stopped + // solicitations. Simply return here and do nothing further. + ndp.ep.mu.Unlock() return } // As per RFC 4861 section 4.1, the source of the RS is an address assigned // to the sending interface, or the unspecified address if no address is // assigned to the sending interface. - ref := ndp.nic.primaryIPv6EndpointRLocked(header.IPv6AllRoutersMulticastAddress) - if ref == nil { - ref = ndp.nic.getRefOrCreateTempLocked(header.IPv6ProtocolNumber, header.IPv6Any, NeverPrimaryEndpoint) + addressEndpoint := ndp.ep.acquireOutgoingPrimaryAddressRLocked(header.IPv6AllRoutersMulticastAddress, false) + if addressEndpoint == nil { + // Incase this ends up creating a new temporary address, we need to hold + // onto the endpoint until a route is obtained. If we decrement the + // reference count before obtaing a route, the address's resources would + // be released and attempting to obtain a route after would fail. Once a + // route is obtainted, it is safe to decrement the reference count since + // obtaining a route increments the address's reference count. + addressEndpoint = ndp.ep.acquireAddressOrCreateTempLocked(header.IPv6Any, true /* createTemp */, stack.NeverPrimaryEndpoint) } - ndp.nic.mu.Unlock() + ndp.ep.mu.Unlock() - localAddr := ref.address() - r := makeRoute(header.IPv6ProtocolNumber, localAddr, header.IPv6AllRoutersMulticastAddress, ndp.nic.linkEP.LinkAddress(), ref, false, false) + localAddr := addressEndpoint.AddressWithPrefix().Address + r, err := ndp.ep.protocol.stack.FindRoute(ndp.ep.nic.ID(), localAddr, header.IPv6AllRoutersMulticastAddress, ProtocolNumber, false /* multicastLoop */) + addressEndpoint.DecRef() + if err != nil { + return + } defer r.Release() // Route should resolve immediately since @@ -1876,15 +1916,16 @@ func (ndp *ndpState) startSolicitingRouters() { // remote link address can be calculated without a resolution process. if c, err := r.Resolve(nil); err != nil { // Do not consider the NIC being unknown or disabled as a fatal error. - // Since this method is required to be called when the NIC is not locked, - // the NIC could have been disabled or removed by another goroutine. + // Since this method is required to be called when the IPv6 endpoint is + // not locked, the IPv6 endpoint could have been disabled or removed by + // another goroutine. if err == tcpip.ErrUnknownNICID || err == tcpip.ErrInvalidEndpointState { return } - panic(fmt.Sprintf("ndp: error when resolving route to send NDP RS (%s -> %s on NIC(%d)): %s", header.IPv6Any, header.IPv6AllRoutersMulticastAddress, ndp.nic.ID(), err)) + panic(fmt.Sprintf("ndp: error when resolving route to send NDP RS (%s -> %s on NIC(%d)): %s", header.IPv6Any, header.IPv6AllRoutersMulticastAddress, ndp.ep.nic.ID(), err)) } else if c != nil { - panic(fmt.Sprintf("ndp: route resolution not immediate for route to send NDP RS (%s -> %s on NIC(%d))", header.IPv6Any, header.IPv6AllRoutersMulticastAddress, ndp.nic.ID())) + panic(fmt.Sprintf("ndp: route resolution not immediate for route to send NDP RS (%s -> %s on NIC(%d))", header.IPv6Any, header.IPv6AllRoutersMulticastAddress, ndp.ep.nic.ID())) } // As per RFC 4861 section 4.1, an NDP RS SHOULD include the source @@ -1907,21 +1948,20 @@ func (ndp *ndpState) startSolicitingRouters() { rs.Options().Serialize(optsSerializer) icmpData.SetChecksum(header.ICMPv6Checksum(icmpData, r.LocalAddress, r.RemoteAddress, buffer.VectorisedView{})) - pkt := NewPacketBuffer(PacketBufferOptions{ + pkt := stack.NewPacketBuffer(stack.PacketBufferOptions{ ReserveHeaderBytes: int(r.MaxHeaderLength()), Data: buffer.View(icmpData).ToVectorisedView(), }) sent := r.Stats().ICMP.V6PacketsSent if err := r.WritePacket(nil, - NetworkHeaderParams{ + stack.NetworkHeaderParams{ Protocol: header.ICMPv6ProtocolNumber, TTL: header.NDPHopLimit, - TOS: DefaultTOS, }, pkt, ); err != nil { sent.Dropped.Increment() - log.Printf("startSolicitingRouters: error writing NDP router solicit message on NIC(%d); err = %s", ndp.nic.ID(), err) + log.Printf("startSolicitingRouters: error writing NDP router solicit message on NIC(%d); err = %s", ndp.ep.nic.ID(), err) // Don't send any more messages if we had an error. remaining = 0 } else { @@ -1929,19 +1969,19 @@ func (ndp *ndpState) startSolicitingRouters() { remaining-- } - ndp.nic.mu.Lock() + ndp.ep.mu.Lock() if done || remaining == 0 { ndp.rtrSolicit.timer = nil ndp.rtrSolicit.done = nil } else if ndp.rtrSolicit.timer != nil { // Note, we need to explicitly check to make sure that // the timer field is not nil because if it was nil but - // we still reached this point, then we know the NIC + // we still reached this point, then we know the IPv6 endpoint // was requested to stop soliciting routers so we don't // need to send the next Router Solicitation message. ndp.rtrSolicit.timer.Reset(ndp.configs.RtrSolicitationInterval) } - ndp.nic.mu.Unlock() + ndp.ep.mu.Unlock() }) } @@ -1949,7 +1989,7 @@ func (ndp *ndpState) startSolicitingRouters() { // stopSolicitingRouters stops soliciting routers. If routers are not currently // being solicited, this function does nothing. // -// The NIC ndp belongs to MUST be locked. +// The IPv6 endpoint that ndp belongs to MUST be locked. func (ndp *ndpState) stopSolicitingRouters() { if ndp.rtrSolicit.timer == nil { // Nothing to do. @@ -1965,7 +2005,7 @@ func (ndp *ndpState) stopSolicitingRouters() { // initializeTempAddrState initializes state related to temporary SLAAC // addresses. func (ndp *ndpState) initializeTempAddrState() { - header.InitialTempIID(ndp.temporaryIIDHistory[:], ndp.nic.stack.tempIIDSeed, ndp.nic.ID()) + header.InitialTempIID(ndp.temporaryIIDHistory[:], ndp.ep.protocol.tempIIDSeed, ndp.ep.nic.ID()) if MaxDesyncFactor != 0 { ndp.temporaryAddressDesyncFactor = time.Duration(rand.Int63n(int64(MaxDesyncFactor))) diff --git a/pkg/tcpip/network/ipv6/ndp_test.go b/pkg/tcpip/network/ipv6/ndp_test.go index 480c495fa..25464a03a 100644 --- a/pkg/tcpip/network/ipv6/ndp_test.go +++ b/pkg/tcpip/network/ipv6/ndp_test.go @@ -17,6 +17,7 @@ package ipv6 import ( "strings" "testing" + "time" "github.com/google/go-cmp/cmp" "gvisor.dev/gvisor/pkg/tcpip" @@ -35,8 +36,8 @@ func setupStackAndEndpoint(t *testing.T, llladdr, rlladdr tcpip.Address, useNeig t.Helper() s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{icmp.NewProtocol6()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{icmp.NewProtocol6}, UseNeighborCache: useNeighborCache, }) @@ -65,10 +66,94 @@ func setupStackAndEndpoint(t *testing.T, llladdr, rlladdr tcpip.Address, useNeig t.Fatalf("cannot find protocol instance for network protocol %d", ProtocolNumber) } - ep := netProto.NewEndpoint(0, &stubLinkAddressCache{}, &stubNUDHandler{}, &stubDispatcher{}, nil, s) + ep := netProto.NewEndpoint(&testInterface{}, &stubLinkAddressCache{}, &stubNUDHandler{}, &stubDispatcher{}) + if err := ep.Enable(); err != nil { + t.Fatalf("ep.Enable(): %s", err) + } + t.Cleanup(ep.Close) + return s, ep } +var _ NDPDispatcher = (*testNDPDispatcher)(nil) + +// testNDPDispatcher is an NDPDispatcher only allows default router discovery. +type testNDPDispatcher struct { + addr tcpip.Address +} + +func (*testNDPDispatcher) OnDuplicateAddressDetectionStatus(tcpip.NICID, tcpip.Address, bool, *tcpip.Error) { +} + +func (t *testNDPDispatcher) OnDefaultRouterDiscovered(_ tcpip.NICID, addr tcpip.Address) bool { + t.addr = addr + return true +} + +func (t *testNDPDispatcher) OnDefaultRouterInvalidated(_ tcpip.NICID, addr tcpip.Address) { + t.addr = addr +} + +func (*testNDPDispatcher) OnOnLinkPrefixDiscovered(tcpip.NICID, tcpip.Subnet) bool { + return false +} + +func (*testNDPDispatcher) OnOnLinkPrefixInvalidated(tcpip.NICID, tcpip.Subnet) { +} + +func (*testNDPDispatcher) OnAutoGenAddress(tcpip.NICID, tcpip.AddressWithPrefix) bool { + return false +} + +func (*testNDPDispatcher) OnAutoGenAddressDeprecated(tcpip.NICID, tcpip.AddressWithPrefix) { +} + +func (*testNDPDispatcher) OnAutoGenAddressInvalidated(tcpip.NICID, tcpip.AddressWithPrefix) { +} + +func (*testNDPDispatcher) OnRecursiveDNSServerOption(tcpip.NICID, []tcpip.Address, time.Duration) { +} + +func (*testNDPDispatcher) OnDNSSearchListOption(tcpip.NICID, []string, time.Duration) { +} + +func (*testNDPDispatcher) OnDHCPv6Configuration(tcpip.NICID, DHCPv6ConfigurationFromNDPRA) { +} + +func TestStackNDPEndpointInvalidateDefaultRouter(t *testing.T) { + var ndpDisp testNDPDispatcher + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocolWithOptions(Options{ + NDPDisp: &ndpDisp, + })}, + }) + + if err := s.CreateNIC(nicID, &stubLinkEndpoint{}); err != nil { + t.Fatalf("s.CreateNIC(%d, _): %s", nicID, err) + } + + ep, err := s.GetNetworkEndpoint(nicID, ProtocolNumber) + if err != nil { + t.Fatalf("s.GetNetworkEndpoint(%d, %d): %s", nicID, ProtocolNumber, err) + } + + ipv6EP := ep.(*endpoint) + ipv6EP.mu.Lock() + ipv6EP.mu.ndp.rememberDefaultRouter(lladdr1, time.Hour) + ipv6EP.mu.Unlock() + + if ndpDisp.addr != lladdr1 { + t.Fatalf("got ndpDisp.addr = %s, want = %s", ndpDisp.addr, lladdr1) + } + + ndpDisp.addr = "" + ndpEP := ep.(stack.NDPEndpoint) + ndpEP.InvalidateDefaultRouter(lladdr1) + if ndpDisp.addr != lladdr1 { + t.Fatalf("got ndpDisp.addr = %s, want = %s", ndpDisp.addr, lladdr1) + } +} + // TestNeighorSolicitationWithSourceLinkLayerOption tests that receiving a // valid NDP NS message with the Source Link Layer Address option results in a // new entry in the link address cache for the sender of the message. @@ -98,7 +183,7 @@ func TestNeighorSolicitationWithSourceLinkLayerOption(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, }) e := channel.New(0, 1280, linkAddr0) if err := s.CreateNIC(nicID, e); err != nil { @@ -202,7 +287,7 @@ func TestNeighorSolicitationWithSourceLinkLayerOptionUsingNeighborCache(t *testi for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, UseNeighborCache: true, }) e := channel.New(0, 1280, linkAddr0) @@ -475,7 +560,7 @@ func TestNeighorSolicitationResponse(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, UseNeighborCache: stackTyp.useNeighborCache, }) e := channel.New(1, 1280, nicLinkAddr) @@ -596,7 +681,7 @@ func TestNeighorAdvertisementWithTargetLinkLayerOption(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, }) e := channel.New(0, 1280, linkAddr0) if err := s.CreateNIC(nicID, e); err != nil { @@ -707,7 +792,7 @@ func TestNeighorAdvertisementWithTargetLinkLayerOptionUsingNeighborCache(t *test for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, UseNeighborCache: true, }) e := channel.New(0, 1280, linkAddr0) @@ -958,7 +1043,7 @@ func TestNDPValidation(t *testing.T) { if isRouter { // Enabling forwarding makes the stack act as a router. - s.SetForwarding(true) + s.SetForwarding(ProtocolNumber, true) } stats := s.Stats().ICMP.V6PacketsReceived @@ -1172,7 +1257,7 @@ func TestRouterAdvertValidation(t *testing.T) { e := channel.New(10, 1280, linkAddr1) e.LinkEPCapabilities |= stack.CapabilityResolutionRequired s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{NewProtocol}, UseNeighborCache: stackTyp.useNeighborCache, }) diff --git a/pkg/tcpip/network/testutil/BUILD b/pkg/tcpip/network/testutil/BUILD index e218563d0..c9e57dc0d 100644 --- a/pkg/tcpip/network/testutil/BUILD +++ b/pkg/tcpip/network/testutil/BUILD @@ -7,11 +7,14 @@ go_library( srcs = [ "testutil.go", ], - visibility = ["//pkg/tcpip/network/ipv4:__pkg__"], + visibility = [ + "//pkg/tcpip/network/ipv4:__pkg__", + "//pkg/tcpip/network/ipv6:__pkg__", + ], deps = [ "//pkg/tcpip", "//pkg/tcpip/buffer", - "//pkg/tcpip/link/channel", + "//pkg/tcpip/header", "//pkg/tcpip/stack", ], ) diff --git a/pkg/tcpip/network/testutil/testutil.go b/pkg/tcpip/network/testutil/testutil.go index bf5ce74be..7cc52985e 100644 --- a/pkg/tcpip/network/testutil/testutil.go +++ b/pkg/tcpip/network/testutil/testutil.go @@ -22,48 +22,100 @@ import ( "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" - "gvisor.dev/gvisor/pkg/tcpip/link/channel" + "gvisor.dev/gvisor/pkg/tcpip/header" "gvisor.dev/gvisor/pkg/tcpip/stack" ) -// TestEndpoint is an endpoint used for testing, it stores packets written to it -// and can mock errors. -type TestEndpoint struct { - *channel.Endpoint - - // WrittenPackets is where we store packets written via WritePacket(). +// MockLinkEndpoint is an endpoint used for testing, it stores packets written +// to it and can mock errors. +type MockLinkEndpoint struct { + // WrittenPackets is where packets written to the endpoint are stored. WrittenPackets []*stack.PacketBuffer - packetCollectorErrors []*tcpip.Error + mtu uint32 + err *tcpip.Error + allowPackets int } -// NewTestEndpoint creates a new TestEndpoint endpoint. +// NewMockLinkEndpoint creates a new MockLinkEndpoint. // -// packetCollectorErrors can be used to set error values and each call to -// WritePacket will remove the first one from the slice and return it until -// the slice is empty - at that point it will return nil every time. -func NewTestEndpoint(ep *channel.Endpoint, packetCollectorErrors []*tcpip.Error) *TestEndpoint { - return &TestEndpoint{ - Endpoint: ep, - WrittenPackets: make([]*stack.PacketBuffer, 0), - packetCollectorErrors: packetCollectorErrors, +// err is the error that will be returned once allowPackets packets are written +// to the endpoint. +func NewMockLinkEndpoint(mtu uint32, err *tcpip.Error, allowPackets int) *MockLinkEndpoint { + return &MockLinkEndpoint{ + mtu: mtu, + err: err, + allowPackets: allowPackets, + } +} + +// MTU implements LinkEndpoint.MTU. +func (ep *MockLinkEndpoint) MTU() uint32 { return ep.mtu } + +// Capabilities implements LinkEndpoint.Capabilities. +func (*MockLinkEndpoint) Capabilities() stack.LinkEndpointCapabilities { return 0 } + +// MaxHeaderLength implements LinkEndpoint.MaxHeaderLength. +func (*MockLinkEndpoint) MaxHeaderLength() uint16 { return 0 } + +// LinkAddress implements LinkEndpoint.LinkAddress. +func (*MockLinkEndpoint) LinkAddress() tcpip.LinkAddress { return "" } + +// WritePacket implements LinkEndpoint.WritePacket. +func (ep *MockLinkEndpoint) WritePacket(_ *stack.Route, _ *stack.GSO, _ tcpip.NetworkProtocolNumber, pkt *stack.PacketBuffer) *tcpip.Error { + if ep.allowPackets == 0 { + return ep.err } + ep.allowPackets-- + ep.WrittenPackets = append(ep.WrittenPackets, pkt) + return nil } -// WritePacket stores outbound packets and may return an error if one was -// injected. -func (e *TestEndpoint) WritePacket(_ *stack.Route, _ *stack.GSO, _ tcpip.NetworkProtocolNumber, pkt *stack.PacketBuffer) *tcpip.Error { - e.WrittenPackets = append(e.WrittenPackets, pkt) +// WritePackets implements LinkEndpoint.WritePackets. +func (ep *MockLinkEndpoint) WritePackets(r *stack.Route, gso *stack.GSO, pkts stack.PacketBufferList, protocol tcpip.NetworkProtocolNumber) (int, *tcpip.Error) { + var n int - if len(e.packetCollectorErrors) > 0 { - nextError := e.packetCollectorErrors[0] - e.packetCollectorErrors = e.packetCollectorErrors[1:] - return nextError + for pkt := pkts.Front(); pkt != nil; pkt = pkt.Next() { + if err := ep.WritePacket(r, gso, protocol, pkt); err != nil { + return n, err + } + n++ } + return n, nil +} + +// WriteRawPacket implements LinkEndpoint.WriteRawPacket. +func (ep *MockLinkEndpoint) WriteRawPacket(vv buffer.VectorisedView) *tcpip.Error { + if ep.allowPackets == 0 { + return ep.err + } + ep.allowPackets-- + + pkt := stack.NewPacketBuffer(stack.PacketBufferOptions{ + Data: vv, + }) + ep.WrittenPackets = append(ep.WrittenPackets, pkt) + return nil } +// Attach implements LinkEndpoint.Attach. +func (*MockLinkEndpoint) Attach(stack.NetworkDispatcher) {} + +// IsAttached implements LinkEndpoint.IsAttached. +func (*MockLinkEndpoint) IsAttached() bool { return false } + +// Wait implements LinkEndpoint.Wait. +func (*MockLinkEndpoint) Wait() {} + +// ARPHardwareType implements LinkEndpoint.ARPHardwareType. +func (*MockLinkEndpoint) ARPHardwareType() header.ARPHardwareType { return header.ARPHardwareNone } + +// AddHeader implements LinkEndpoint.AddHeader. +func (*MockLinkEndpoint) AddHeader(_, _ tcpip.LinkAddress, _ tcpip.NetworkProtocolNumber, _ *stack.PacketBuffer) { +} + // MakeRandPkt generates a randomized packet. transportHeaderLength indicates // how many random bytes will be copied in the Transport Header. // extraHeaderReserveLength indicates how much extra space will be reserved for diff --git a/pkg/tcpip/sample/tun_tcp_connect/main.go b/pkg/tcpip/sample/tun_tcp_connect/main.go index 91fc26722..51d428049 100644 --- a/pkg/tcpip/sample/tun_tcp_connect/main.go +++ b/pkg/tcpip/sample/tun_tcp_connect/main.go @@ -127,8 +127,8 @@ func main() { // Create the stack with ipv4 and tcp protocols, then add a tun-based // NIC and ipv4 address. s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{tcp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{tcp.NewProtocol}, }) mtu, err := rawfile.GetMTU(tunName) diff --git a/pkg/tcpip/sample/tun_tcp_echo/main.go b/pkg/tcpip/sample/tun_tcp_echo/main.go index 3f58a15ea..8e0ee1cd7 100644 --- a/pkg/tcpip/sample/tun_tcp_echo/main.go +++ b/pkg/tcpip/sample/tun_tcp_echo/main.go @@ -112,8 +112,8 @@ func main() { // Create the stack with ip and tcp protocols, then add a tun-based // NIC and address. s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol(), arp.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{tcp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol, arp.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{tcp.NewProtocol}, }) mtu, err := rawfile.GetMTU(tunName) diff --git a/pkg/tcpip/stack/BUILD b/pkg/tcpip/stack/BUILD index 900938dd1..2eaeab779 100644 --- a/pkg/tcpip/stack/BUILD +++ b/pkg/tcpip/stack/BUILD @@ -54,8 +54,8 @@ go_template_instance( go_library( name = "stack", srcs = [ + "addressable_endpoint_state.go", "conntrack.go", - "dhcpv6configurationfromndpra_string.go", "forwarder.go", "headertype_string.go", "icmp_rate_limit.go", @@ -65,7 +65,6 @@ go_library( "iptables_types.go", "linkaddrcache.go", "linkaddrentry_list.go", - "ndp.go", "neighbor_cache.go", "neighbor_entry.go", "neighbor_entry_list.go", @@ -106,6 +105,7 @@ go_test( name = "stack_x_test", size = "medium", srcs = [ + "addressable_endpoint_state_test.go", "ndp_test.go", "nud_test.go", "stack_test.go", @@ -116,6 +116,7 @@ go_test( deps = [ ":stack", "//pkg/rand", + "//pkg/sync", "//pkg/tcpip", "//pkg/tcpip/buffer", "//pkg/tcpip/checker", @@ -138,7 +139,6 @@ go_test( name = "stack_test", size = "small", srcs = [ - "fake_time_test.go", "forwarder_test.go", "linkaddrcache_test.go", "neighbor_cache_test.go", @@ -152,8 +152,8 @@ go_test( "//pkg/sync", "//pkg/tcpip", "//pkg/tcpip/buffer", + "//pkg/tcpip/faketime", "//pkg/tcpip/header", - "@com_github_dpjacques_clockwork//:go_default_library", "@com_github_google_go_cmp//cmp:go_default_library", "@com_github_google_go_cmp//cmp/cmpopts:go_default_library", ], diff --git a/pkg/tcpip/stack/addressable_endpoint_state.go b/pkg/tcpip/stack/addressable_endpoint_state.go new file mode 100644 index 000000000..4d3acab96 --- /dev/null +++ b/pkg/tcpip/stack/addressable_endpoint_state.go @@ -0,0 +1,753 @@ +// 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 stack + +import ( + "fmt" + + "gvisor.dev/gvisor/pkg/sync" + "gvisor.dev/gvisor/pkg/tcpip" +) + +var _ GroupAddressableEndpoint = (*AddressableEndpointState)(nil) +var _ AddressableEndpoint = (*AddressableEndpointState)(nil) + +// AddressableEndpointState is an implementation of an AddressableEndpoint. +type AddressableEndpointState struct { + networkEndpoint NetworkEndpoint + + // Lock ordering (from outer to inner lock ordering): + // + // AddressableEndpointState.mu + // addressState.mu + mu struct { + sync.RWMutex + + endpoints map[tcpip.Address]*addressState + primary []*addressState + + // groups holds the mapping between group addresses and the number of times + // they have been joined. + groups map[tcpip.Address]uint32 + } +} + +// Init initializes the AddressableEndpointState with networkEndpoint. +// +// Must be called before calling any other function on m. +func (a *AddressableEndpointState) Init(networkEndpoint NetworkEndpoint) { + a.networkEndpoint = networkEndpoint + + a.mu.Lock() + defer a.mu.Unlock() + a.mu.endpoints = make(map[tcpip.Address]*addressState) + a.mu.groups = make(map[tcpip.Address]uint32) +} + +// ReadOnlyAddressableEndpointState provides read-only access to an +// AddressableEndpointState. +type ReadOnlyAddressableEndpointState struct { + inner *AddressableEndpointState +} + +// AddrOrMatching returns an endpoint for the passed address that is consisdered +// bound to the wrapped AddressableEndpointState. +// +// If addr is an exact match with an existing address, that address is returned. +// Otherwise, f is called with each address and the address that f returns true +// for is returned. +// +// Returns nil of no address matches. +func (m ReadOnlyAddressableEndpointState) AddrOrMatching(addr tcpip.Address, spoofingOrPrimiscuous bool, f func(AddressEndpoint) bool) AddressEndpoint { + m.inner.mu.RLock() + defer m.inner.mu.RUnlock() + + if ep, ok := m.inner.mu.endpoints[addr]; ok { + if ep.IsAssigned(spoofingOrPrimiscuous) && ep.IncRef() { + return ep + } + } + + for _, ep := range m.inner.mu.endpoints { + if ep.IsAssigned(spoofingOrPrimiscuous) && f(ep) && ep.IncRef() { + return ep + } + } + + return nil +} + +// Lookup returns the AddressEndpoint for the passed address. +// +// Returns nil if the passed address is not associated with the +// AddressableEndpointState. +func (m ReadOnlyAddressableEndpointState) Lookup(addr tcpip.Address) AddressEndpoint { + m.inner.mu.RLock() + defer m.inner.mu.RUnlock() + + ep, ok := m.inner.mu.endpoints[addr] + if !ok { + return nil + } + return ep +} + +// ForEach calls f for each address pair. +// +// If f returns false, f is no longer be called. +func (m ReadOnlyAddressableEndpointState) ForEach(f func(AddressEndpoint) bool) { + m.inner.mu.RLock() + defer m.inner.mu.RUnlock() + + for _, ep := range m.inner.mu.endpoints { + if !f(ep) { + return + } + } +} + +// ForEachPrimaryEndpoint calls f for each primary address. +// +// If f returns false, f is no longer be called. +func (m ReadOnlyAddressableEndpointState) ForEachPrimaryEndpoint(f func(AddressEndpoint)) { + m.inner.mu.RLock() + defer m.inner.mu.RUnlock() + for _, ep := range m.inner.mu.primary { + f(ep) + } +} + +// ReadOnly returns a readonly reference to a. +func (a *AddressableEndpointState) ReadOnly() ReadOnlyAddressableEndpointState { + return ReadOnlyAddressableEndpointState{inner: a} +} + +func (a *AddressableEndpointState) releaseAddressState(addrState *addressState) { + a.mu.Lock() + defer a.mu.Unlock() + a.releaseAddressStateLocked(addrState) +} + +// releaseAddressState removes addrState from s's address state (primary and endpoints list). +// +// Preconditions: a.mu must be write locked. +func (a *AddressableEndpointState) releaseAddressStateLocked(addrState *addressState) { + oldPrimary := a.mu.primary + for i, s := range a.mu.primary { + if s == addrState { + a.mu.primary = append(a.mu.primary[:i], a.mu.primary[i+1:]...) + oldPrimary[len(oldPrimary)-1] = nil + break + } + } + delete(a.mu.endpoints, addrState.addr.Address) +} + +// AddAndAcquirePermanentAddress implements AddressableEndpoint. +func (a *AddressableEndpointState) AddAndAcquirePermanentAddress(addr tcpip.AddressWithPrefix, peb PrimaryEndpointBehavior, configType AddressConfigType, deprecated bool) (AddressEndpoint, *tcpip.Error) { + a.mu.Lock() + defer a.mu.Unlock() + ep, err := a.addAndAcquireAddressLocked(addr, peb, configType, deprecated, true /* permanent */) + // From https://golang.org/doc/faq#nil_error: + // + // Under the covers, interfaces are implemented as two elements, a type T and + // a value V. + // + // An interface value is nil only if the V and T are both unset, (T=nil, V is + // not set), In particular, a nil interface will always hold a nil type. If we + // store a nil pointer of type *int inside an interface value, the inner type + // will be *int regardless of the value of the pointer: (T=*int, V=nil). Such + // an interface value will therefore be non-nil even when the pointer value V + // inside is nil. + // + // Since addAndAcquireAddressLocked returns a nil value with a non-nil type, + // we need to explicitly return nil below if ep is (a typed) nil. + if ep == nil { + return nil, err + } + return ep, err +} + +// AddAndAcquireTemporaryAddress adds a temporary address. +// +// Returns tcpip.ErrDuplicateAddress if the address exists. +// +// The temporary address's endpoint is acquired and returned. +func (a *AddressableEndpointState) AddAndAcquireTemporaryAddress(addr tcpip.AddressWithPrefix, peb PrimaryEndpointBehavior) (AddressEndpoint, *tcpip.Error) { + a.mu.Lock() + defer a.mu.Unlock() + ep, err := a.addAndAcquireAddressLocked(addr, peb, AddressConfigStatic, false /* deprecated */, false /* permanent */) + // From https://golang.org/doc/faq#nil_error: + // + // Under the covers, interfaces are implemented as two elements, a type T and + // a value V. + // + // An interface value is nil only if the V and T are both unset, (T=nil, V is + // not set), In particular, a nil interface will always hold a nil type. If we + // store a nil pointer of type *int inside an interface value, the inner type + // will be *int regardless of the value of the pointer: (T=*int, V=nil). Such + // an interface value will therefore be non-nil even when the pointer value V + // inside is nil. + // + // Since addAndAcquireAddressLocked returns a nil value with a non-nil type, + // we need to explicitly return nil below if ep is (a typed) nil. + if ep == nil { + return nil, err + } + return ep, err +} + +// addAndAcquireAddressLocked adds, acquires and returns a permanent or +// temporary address. +// +// If the addressable endpoint already has the address in a non-permanent state, +// and addAndAcquireAddressLocked is adding a permanent address, that address is +// promoted in place and its properties set to the properties provided. If the +// address already exists in any other state, then tcpip.ErrDuplicateAddress is +// returned, regardless the kind of address that is being added. +// +// Precondition: a.mu must be write locked. +func (a *AddressableEndpointState) addAndAcquireAddressLocked(addr tcpip.AddressWithPrefix, peb PrimaryEndpointBehavior, configType AddressConfigType, deprecated, permanent bool) (*addressState, *tcpip.Error) { + // attemptAddToPrimary is false when the address is already in the primary + // address list. + attemptAddToPrimary := true + addrState, ok := a.mu.endpoints[addr.Address] + if ok { + if !permanent { + // We are adding a non-permanent address but the address exists. No need + // to go any further since we can only promote existing temporary/expired + // addresses to permanent. + return nil, tcpip.ErrDuplicateAddress + } + + addrState.mu.Lock() + if addrState.mu.kind.IsPermanent() { + addrState.mu.Unlock() + // We are adding a permanent address but a permanent address already + // exists. + return nil, tcpip.ErrDuplicateAddress + } + + if addrState.mu.refs == 0 { + panic(fmt.Sprintf("found an address that should have been released (ref count == 0); address = %s", addrState.addr)) + } + + // We now promote the address. + for i, s := range a.mu.primary { + if s == addrState { + switch peb { + case CanBePrimaryEndpoint: + // The address is already in the primary address list. + attemptAddToPrimary = false + case FirstPrimaryEndpoint: + if i == 0 { + // The address is already first in the primary address list. + attemptAddToPrimary = false + } else { + a.mu.primary = append(a.mu.primary[:i], a.mu.primary[i+1:]...) + } + case NeverPrimaryEndpoint: + a.mu.primary = append(a.mu.primary[:i], a.mu.primary[i+1:]...) + default: + panic(fmt.Sprintf("unrecognized primary endpoint behaviour = %d", peb)) + } + break + } + } + } + + if addrState == nil { + addrState = &addressState{ + addressableEndpointState: a, + addr: addr, + } + a.mu.endpoints[addr.Address] = addrState + addrState.mu.Lock() + // We never promote an address to temporary - it can only be added as such. + // If we are actaully adding a permanent address, it is promoted below. + addrState.mu.kind = Temporary + } + + // At this point we have an address we are either promoting from an expired or + // temporary address to permanent, promoting an expired address to temporary, + // or we are adding a new temporary or permanent address. + // + // The address MUST be write locked at this point. + defer addrState.mu.Unlock() + + if permanent { + if addrState.mu.kind.IsPermanent() { + panic(fmt.Sprintf("only non-permanent addresses should be promoted to permanent; address = %s", addrState.addr)) + } + + // Primary addresses are biased by 1. + addrState.mu.refs++ + addrState.mu.kind = Permanent + } + // Acquire the address before returning it. + addrState.mu.refs++ + addrState.mu.deprecated = deprecated + addrState.mu.configType = configType + + if attemptAddToPrimary { + switch peb { + case NeverPrimaryEndpoint: + case CanBePrimaryEndpoint: + a.mu.primary = append(a.mu.primary, addrState) + case FirstPrimaryEndpoint: + if cap(a.mu.primary) == len(a.mu.primary) { + a.mu.primary = append([]*addressState{addrState}, a.mu.primary...) + } else { + // Shift all the endpoints by 1 to make room for the new address at the + // front. We could have just created a new slice but this saves + // allocations when the slice has capacity for the new address. + primaryCount := len(a.mu.primary) + a.mu.primary = append(a.mu.primary, nil) + if n := copy(a.mu.primary[1:], a.mu.primary); n != primaryCount { + panic(fmt.Sprintf("copied %d elements; expected = %d elements", n, primaryCount)) + } + a.mu.primary[0] = addrState + } + default: + panic(fmt.Sprintf("unrecognized primary endpoint behaviour = %d", peb)) + } + } + + return addrState, nil +} + +// RemovePermanentAddress implements AddressableEndpoint. +func (a *AddressableEndpointState) RemovePermanentAddress(addr tcpip.Address) *tcpip.Error { + a.mu.Lock() + defer a.mu.Unlock() + + if _, ok := a.mu.groups[addr]; ok { + panic(fmt.Sprintf("group address = %s must be removed with LeaveGroup", addr)) + } + + return a.removePermanentAddressLocked(addr) +} + +// removePermanentAddressLocked is like RemovePermanentAddress but with locking +// requirements. +// +// Precondition: a.mu must be write locked. +func (a *AddressableEndpointState) removePermanentAddressLocked(addr tcpip.Address) *tcpip.Error { + addrState, ok := a.mu.endpoints[addr] + if !ok { + return tcpip.ErrBadLocalAddress + } + + return a.removePermanentEndpointLocked(addrState) +} + +// RemovePermanentEndpoint removes the passed endpoint if it is associated with +// a and permanent. +func (a *AddressableEndpointState) RemovePermanentEndpoint(ep AddressEndpoint) *tcpip.Error { + addrState, ok := ep.(*addressState) + if !ok || addrState.addressableEndpointState != a { + return tcpip.ErrInvalidEndpointState + } + + return a.removePermanentEndpointLocked(addrState) +} + +// removePermanentAddressLocked is like RemovePermanentAddress but with locking +// requirements. +// +// Precondition: a.mu must be write locked. +func (a *AddressableEndpointState) removePermanentEndpointLocked(addrState *addressState) *tcpip.Error { + if !addrState.GetKind().IsPermanent() { + return tcpip.ErrBadLocalAddress + } + + addrState.SetKind(PermanentExpired) + a.decAddressRefLocked(addrState) + return nil +} + +// decAddressRef decrements the address's reference count and releases it once +// the reference count hits 0. +func (a *AddressableEndpointState) decAddressRef(addrState *addressState) { + a.mu.Lock() + defer a.mu.Unlock() + a.decAddressRefLocked(addrState) +} + +// decAddressRefLocked is like decAddressRef but with locking requirements. +// +// Precondition: a.mu must be write locked. +func (a *AddressableEndpointState) decAddressRefLocked(addrState *addressState) { + addrState.mu.Lock() + defer addrState.mu.Unlock() + + if addrState.mu.refs == 0 { + panic(fmt.Sprintf("attempted to decrease ref count for AddressEndpoint w/ addr = %s when it is already released", addrState.addr)) + } + + addrState.mu.refs-- + + if addrState.mu.refs != 0 { + return + } + + // A non-expired permanent address must not have its reference count dropped + // to 0. + if addrState.mu.kind.IsPermanent() { + panic(fmt.Sprintf("permanent addresses should be removed through the AddressableEndpoint: addr = %s, kind = %d", addrState.addr, addrState.mu.kind)) + } + + a.releaseAddressStateLocked(addrState) +} + +// MainAddress implements AddressableEndpoint. +func (a *AddressableEndpointState) MainAddress() tcpip.AddressWithPrefix { + a.mu.RLock() + defer a.mu.RUnlock() + + ep := a.acquirePrimaryAddressRLocked(func(ep *addressState) bool { + return ep.GetKind() == Permanent + }) + if ep == nil { + return tcpip.AddressWithPrefix{} + } + + addr := ep.AddressWithPrefix() + a.decAddressRefLocked(ep) + return addr +} + +// acquirePrimaryAddressRLocked returns an acquired primary address that is +// valid according to isValid. +// +// Precondition: e.mu must be read locked +func (a *AddressableEndpointState) acquirePrimaryAddressRLocked(isValid func(*addressState) bool) *addressState { + var deprecatedEndpoint *addressState + for _, ep := range a.mu.primary { + if !isValid(ep) { + continue + } + + if !ep.Deprecated() { + if ep.IncRef() { + // ep is not deprecated, so return it immediately. + // + // If we kept track of a deprecated endpoint, decrement its reference + // count since it was incremented when we decided to keep track of it. + if deprecatedEndpoint != nil { + a.decAddressRefLocked(deprecatedEndpoint) + deprecatedEndpoint = nil + } + + return ep + } + } else if deprecatedEndpoint == nil && ep.IncRef() { + // We prefer an endpoint that is not deprecated, but we keep track of + // ep in case a doesn't have any non-deprecated endpoints. + // + // If we end up finding a more preferred endpoint, ep's reference count + // will be decremented. + deprecatedEndpoint = ep + } + } + + return deprecatedEndpoint +} + +// AcquireAssignedAddress implements AddressableEndpoint. +func (a *AddressableEndpointState) AcquireAssignedAddress(localAddr tcpip.Address, allowTemp bool, tempPEB PrimaryEndpointBehavior) AddressEndpoint { + a.mu.Lock() + defer a.mu.Unlock() + + if addrState, ok := a.mu.endpoints[localAddr]; ok { + if !addrState.IsAssigned(allowTemp) { + return nil + } + + if !addrState.IncRef() { + panic(fmt.Sprintf("failed to increase the reference count for address = %s", addrState.addr)) + } + + return addrState + } + + if !allowTemp { + return nil + } + + addr := localAddr.WithPrefix() + ep, err := a.addAndAcquireAddressLocked(addr, tempPEB, AddressConfigStatic, false /* deprecated */, false /* permanent */) + if err != nil { + // addAndAcquireAddressLocked only returns an error if the address is + // already assigned but we just checked above if the address exists so we + // expect no error. + panic(fmt.Sprintf("a.addAndAcquireAddressLocked(%s, %d, %d, false, false): %s", addr, tempPEB, AddressConfigStatic, err)) + } + // From https://golang.org/doc/faq#nil_error: + // + // Under the covers, interfaces are implemented as two elements, a type T and + // a value V. + // + // An interface value is nil only if the V and T are both unset, (T=nil, V is + // not set), In particular, a nil interface will always hold a nil type. If we + // store a nil pointer of type *int inside an interface value, the inner type + // will be *int regardless of the value of the pointer: (T=*int, V=nil). Such + // an interface value will therefore be non-nil even when the pointer value V + // inside is nil. + // + // Since addAndAcquireAddressLocked returns a nil value with a non-nil type, + // we need to explicitly return nil below if ep is (a typed) nil. + if ep == nil { + return nil + } + return ep +} + +// AcquireOutgoingPrimaryAddress implements AddressableEndpoint. +func (a *AddressableEndpointState) AcquireOutgoingPrimaryAddress(remoteAddr tcpip.Address, allowExpired bool) AddressEndpoint { + a.mu.RLock() + defer a.mu.RUnlock() + + ep := a.acquirePrimaryAddressRLocked(func(ep *addressState) bool { + return ep.IsAssigned(allowExpired) + }) + + // From https://golang.org/doc/faq#nil_error: + // + // Under the covers, interfaces are implemented as two elements, a type T and + // a value V. + // + // An interface value is nil only if the V and T are both unset, (T=nil, V is + // not set), In particular, a nil interface will always hold a nil type. If we + // store a nil pointer of type *int inside an interface value, the inner type + // will be *int regardless of the value of the pointer: (T=*int, V=nil). Such + // an interface value will therefore be non-nil even when the pointer value V + // inside is nil. + // + // Since acquirePrimaryAddressRLocked returns a nil value with a non-nil type, + // we need to explicitly return nil below if ep is (a typed) nil. + if ep == nil { + return nil + } + + return ep +} + +// PrimaryAddresses implements AddressableEndpoint. +func (a *AddressableEndpointState) PrimaryAddresses() []tcpip.AddressWithPrefix { + a.mu.RLock() + defer a.mu.RUnlock() + + var addrs []tcpip.AddressWithPrefix + for _, ep := range a.mu.primary { + // Don't include tentative, expired or temporary endpoints + // to avoid confusion and prevent the caller from using + // those. + switch ep.GetKind() { + case PermanentTentative, PermanentExpired, Temporary: + continue + } + + addrs = append(addrs, ep.AddressWithPrefix()) + } + + return addrs +} + +// PermanentAddresses implements AddressableEndpoint. +func (a *AddressableEndpointState) PermanentAddresses() []tcpip.AddressWithPrefix { + a.mu.RLock() + defer a.mu.RUnlock() + + var addrs []tcpip.AddressWithPrefix + for _, ep := range a.mu.endpoints { + if !ep.GetKind().IsPermanent() { + continue + } + + addrs = append(addrs, ep.AddressWithPrefix()) + } + + return addrs +} + +// JoinGroup implements GroupAddressableEndpoint. +func (a *AddressableEndpointState) JoinGroup(group tcpip.Address) (bool, *tcpip.Error) { + a.mu.Lock() + defer a.mu.Unlock() + + joins, ok := a.mu.groups[group] + if !ok { + ep, err := a.addAndAcquireAddressLocked(group.WithPrefix(), NeverPrimaryEndpoint, AddressConfigStatic, false /* deprecated */, true /* permanent */) + if err != nil { + return false, err + } + // We have no need for the address endpoint. + a.decAddressRefLocked(ep) + } + + a.mu.groups[group] = joins + 1 + return !ok, nil +} + +// LeaveGroup implements GroupAddressableEndpoint. +func (a *AddressableEndpointState) LeaveGroup(group tcpip.Address) (bool, *tcpip.Error) { + a.mu.Lock() + defer a.mu.Unlock() + + joins, ok := a.mu.groups[group] + if !ok { + return false, tcpip.ErrBadLocalAddress + } + + if joins == 1 { + a.removeGroupAddressLocked(group) + delete(a.mu.groups, group) + return true, nil + } + + a.mu.groups[group] = joins - 1 + return false, nil +} + +// IsInGroup implements GroupAddressableEndpoint. +func (a *AddressableEndpointState) IsInGroup(group tcpip.Address) bool { + a.mu.RLock() + defer a.mu.RUnlock() + _, ok := a.mu.groups[group] + return ok +} + +func (a *AddressableEndpointState) removeGroupAddressLocked(group tcpip.Address) { + if err := a.removePermanentAddressLocked(group); err != nil { + // removePermanentEndpointLocked would only return an error if group is + // not bound to the addressable endpoint, but we know it MUST be assigned + // since we have group in our map of groups. + panic(fmt.Sprintf("error removing group address = %s: %s", group, err)) + } +} + +// Cleanup forcefully leaves all groups and removes all permanent addresses. +func (a *AddressableEndpointState) Cleanup() { + a.mu.Lock() + defer a.mu.Unlock() + + for group := range a.mu.groups { + a.removeGroupAddressLocked(group) + } + a.mu.groups = make(map[tcpip.Address]uint32) + + for _, ep := range a.mu.endpoints { + // removePermanentEndpointLocked returns tcpip.ErrBadLocalAddress if ep is + // not a permanent address. + if err := a.removePermanentEndpointLocked(ep); err != nil && err != tcpip.ErrBadLocalAddress { + panic(fmt.Sprintf("unexpected error from removePermanentEndpointLocked(%s): %s", ep.addr, err)) + } + } +} + +var _ AddressEndpoint = (*addressState)(nil) + +// addressState holds state for an address. +type addressState struct { + addressableEndpointState *AddressableEndpointState + addr tcpip.AddressWithPrefix + + // Lock ordering (from outer to inner lock ordering): + // + // AddressableEndpointState.mu + // addressState.mu + mu struct { + sync.RWMutex + + refs uint32 + kind AddressKind + configType AddressConfigType + deprecated bool + } +} + +// AddressWithPrefix implements AddressEndpoint. +func (a *addressState) AddressWithPrefix() tcpip.AddressWithPrefix { + return a.addr +} + +// GetKind implements AddressEndpoint. +func (a *addressState) GetKind() AddressKind { + a.mu.RLock() + defer a.mu.RUnlock() + return a.mu.kind +} + +// SetKind implements AddressEndpoint. +func (a *addressState) SetKind(kind AddressKind) { + a.mu.Lock() + defer a.mu.Unlock() + a.mu.kind = kind +} + +// IsAssigned implements AddressEndpoint. +func (a *addressState) IsAssigned(allowExpired bool) bool { + if !a.addressableEndpointState.networkEndpoint.Enabled() { + return false + } + + switch a.GetKind() { + case PermanentTentative: + return false + case PermanentExpired: + return allowExpired + default: + return true + } +} + +// IncRef implements AddressEndpoint. +func (a *addressState) IncRef() bool { + a.mu.Lock() + defer a.mu.Unlock() + if a.mu.refs == 0 { + return false + } + + a.mu.refs++ + return true +} + +// DecRef implements AddressEndpoint. +func (a *addressState) DecRef() { + a.addressableEndpointState.decAddressRef(a) +} + +// ConfigType implements AddressEndpoint. +func (a *addressState) ConfigType() AddressConfigType { + a.mu.RLock() + defer a.mu.RUnlock() + return a.mu.configType +} + +// SetDeprecated implements AddressEndpoint. +func (a *addressState) SetDeprecated(d bool) { + a.mu.Lock() + defer a.mu.Unlock() + a.mu.deprecated = d +} + +// Deprecated implements AddressEndpoint. +func (a *addressState) Deprecated() bool { + a.mu.RLock() + defer a.mu.RUnlock() + return a.mu.deprecated +} diff --git a/pkg/tcpip/stack/addressable_endpoint_state_test.go b/pkg/tcpip/stack/addressable_endpoint_state_test.go new file mode 100644 index 000000000..26787d0a3 --- /dev/null +++ b/pkg/tcpip/stack/addressable_endpoint_state_test.go @@ -0,0 +1,77 @@ +// 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 stack_test + +import ( + "testing" + + "gvisor.dev/gvisor/pkg/tcpip" + "gvisor.dev/gvisor/pkg/tcpip/stack" +) + +// TestAddressableEndpointStateCleanup tests that cleaning up an addressable +// endpoint state removes permanent addresses and leaves groups. +func TestAddressableEndpointStateCleanup(t *testing.T) { + var ep fakeNetworkEndpoint + if err := ep.Enable(); err != nil { + t.Fatalf("ep.Enable(): %s", err) + } + + var s stack.AddressableEndpointState + s.Init(&ep) + + addr := tcpip.AddressWithPrefix{ + Address: "\x01", + PrefixLen: 8, + } + + { + ep, err := s.AddAndAcquirePermanentAddress(addr, stack.NeverPrimaryEndpoint, stack.AddressConfigStatic, false /* deprecated */) + if err != nil { + t.Fatalf("s.AddAndAcquirePermanentAddress(%s, %d, %d, false): %s", addr, stack.NeverPrimaryEndpoint, stack.AddressConfigStatic, err) + } + // We don't need the address endpoint. + ep.DecRef() + } + { + ep := s.AcquireAssignedAddress(addr.Address, false /* allowTemp */, stack.NeverPrimaryEndpoint) + if ep == nil { + t.Fatalf("got s.AcquireAssignedAddress(%s, false, NeverPrimaryEndpoint) = nil, want = non-nil", addr.Address) + } + ep.DecRef() + } + + group := tcpip.Address("\x02") + if added, err := s.JoinGroup(group); err != nil { + t.Fatalf("s.JoinGroup(%s): %s", group, err) + } else if !added { + t.Fatalf("got s.JoinGroup(%s) = false, want = true", group) + } + if !s.IsInGroup(group) { + t.Fatalf("got s.IsInGroup(%s) = false, want = true", group) + } + + s.Cleanup() + { + ep := s.AcquireAssignedAddress(addr.Address, false /* allowTemp */, stack.NeverPrimaryEndpoint) + if ep != nil { + ep.DecRef() + t.Fatalf("got s.AcquireAssignedAddress(%s, false, NeverPrimaryEndpoint) = %s, want = nil", addr.Address, ep.AddressWithPrefix()) + } + } + if s.IsInGroup(group) { + t.Fatalf("got s.IsInGroup(%s) = true, want = false", group) + } +} diff --git a/pkg/tcpip/stack/conntrack.go b/pkg/tcpip/stack/conntrack.go index 836682ea0..0cd1da11f 100644 --- a/pkg/tcpip/stack/conntrack.go +++ b/pkg/tcpip/stack/conntrack.go @@ -196,13 +196,14 @@ type bucket struct { // packetToTupleID converts packet to a tuple ID. It fails when pkt lacks a valid // TCP header. +// +// Preconditions: pkt.NetworkHeader() is valid. func packetToTupleID(pkt *PacketBuffer) (tupleID, *tcpip.Error) { - // TODO(gvisor.dev/issue/170): Need to support for other - // protocols as well. - netHeader := header.IPv4(pkt.NetworkHeader().View()) - if len(netHeader) < header.IPv4MinimumSize || netHeader.TransportProtocol() != header.TCPProtocolNumber { + netHeader := pkt.Network() + if netHeader.TransportProtocol() != header.TCPProtocolNumber { return tupleID{}, tcpip.ErrUnknownProtocol } + tcpHeader := header.TCP(pkt.TransportHeader().View()) if len(tcpHeader) < header.TCPMinimumSize { return tupleID{}, tcpip.ErrUnknownProtocol @@ -214,7 +215,7 @@ func packetToTupleID(pkt *PacketBuffer) (tupleID, *tcpip.Error) { dstAddr: netHeader.DestinationAddress(), dstPort: tcpHeader.DestinationPort(), transProto: netHeader.TransportProtocol(), - netProto: header.IPv4ProtocolNumber, + netProto: pkt.NetworkProtocolNumber, }, nil } @@ -268,7 +269,7 @@ func (ct *ConnTrack) connForTID(tid tupleID) (*conn, direction) { return nil, dirOriginal } -func (ct *ConnTrack) insertRedirectConn(pkt *PacketBuffer, hook Hook, rt RedirectTarget) *conn { +func (ct *ConnTrack) insertRedirectConn(pkt *PacketBuffer, hook Hook, rt *RedirectTarget) *conn { tid, err := packetToTupleID(pkt) if err != nil { return nil @@ -281,8 +282,8 @@ func (ct *ConnTrack) insertRedirectConn(pkt *PacketBuffer, hook Hook, rt Redirec // rule. This tuple will be used to manipulate the packet in // handlePacket. replyTID := tid.reply() - replyTID.srcAddr = rt.MinIP - replyTID.srcPort = rt.MinPort + replyTID.srcAddr = rt.Addr + replyTID.srcPort = rt.Port var manip manipType switch hook { case Prerouting: @@ -344,7 +345,7 @@ func handlePacketPrerouting(pkt *PacketBuffer, conn *conn, dir direction) { return } - netHeader := header.IPv4(pkt.NetworkHeader().View()) + netHeader := pkt.Network() tcpHeader := header.TCP(pkt.TransportHeader().View()) // For prerouting redirection, packets going in the original direction @@ -366,8 +367,12 @@ func handlePacketPrerouting(pkt *PacketBuffer, conn *conn, dir direction) { // support cases when they are validated, e.g. when we can't offload // receive checksumming. - netHeader.SetChecksum(0) - netHeader.SetChecksum(^netHeader.CalculateChecksum()) + // After modification, IPv4 packets need a valid checksum. + if pkt.NetworkProtocolNumber == header.IPv4ProtocolNumber { + netHeader := header.IPv4(pkt.NetworkHeader().View()) + netHeader.SetChecksum(0) + netHeader.SetChecksum(^netHeader.CalculateChecksum()) + } } // handlePacketOutput manipulates ports for packets in Output hook. @@ -377,7 +382,7 @@ func handlePacketOutput(pkt *PacketBuffer, conn *conn, gso *GSO, r *Route, dir d return } - netHeader := header.IPv4(pkt.NetworkHeader().View()) + netHeader := pkt.Network() tcpHeader := header.TCP(pkt.TransportHeader().View()) // For output redirection, packets going in the original direction @@ -396,7 +401,7 @@ func handlePacketOutput(pkt *PacketBuffer, conn *conn, gso *GSO, r *Route, dir d // Calculate the TCP checksum and set it. tcpHeader.SetChecksum(0) - length := uint16(pkt.Size()) - uint16(netHeader.HeaderLength()) + length := uint16(pkt.Size()) - uint16(len(pkt.NetworkHeader().View())) xsum := r.PseudoHeaderChecksum(header.TCPProtocolNumber, length) if gso != nil && gso.NeedsCsum { tcpHeader.SetChecksum(xsum) @@ -405,8 +410,11 @@ func handlePacketOutput(pkt *PacketBuffer, conn *conn, gso *GSO, r *Route, dir d tcpHeader.SetChecksum(^tcpHeader.CalculateChecksum(xsum)) } - netHeader.SetChecksum(0) - netHeader.SetChecksum(^netHeader.CalculateChecksum()) + if pkt.NetworkProtocolNumber == header.IPv4ProtocolNumber { + netHeader := header.IPv4(pkt.NetworkHeader().View()) + netHeader.SetChecksum(0) + netHeader.SetChecksum(^netHeader.CalculateChecksum()) + } } // handlePacket will manipulate the port and address of the packet if the @@ -422,7 +430,7 @@ func (ct *ConnTrack) handlePacket(pkt *PacketBuffer, hook Hook, gso *GSO, r *Rou } // TODO(gvisor.dev/issue/170): Support other transport protocols. - if nh := pkt.NetworkHeader().View(); nh.IsEmpty() || header.IPv4(nh).TransportProtocol() != header.TCPProtocolNumber { + if pkt.Network().TransportProtocol() != header.TCPProtocolNumber { return false } @@ -473,7 +481,7 @@ func (ct *ConnTrack) maybeInsertNoop(pkt *PacketBuffer, hook Hook) { } // We only track TCP connections. - if nh := pkt.NetworkHeader().View(); nh.IsEmpty() || header.IPv4(nh).TransportProtocol() != header.TCPProtocolNumber { + if pkt.Network().TransportProtocol() != header.TCPProtocolNumber { return } @@ -609,7 +617,7 @@ func (ct *ConnTrack) reapTupleLocked(tuple *tuple, bucket int, now time.Time) bo return true } -func (ct *ConnTrack) originalDst(epID TransportEndpointID) (tcpip.Address, uint16, *tcpip.Error) { +func (ct *ConnTrack) originalDst(epID TransportEndpointID, netProto tcpip.NetworkProtocolNumber) (tcpip.Address, uint16, *tcpip.Error) { // Lookup the connection. The reply's original destination // describes the original address. tid := tupleID{ @@ -618,7 +626,7 @@ func (ct *ConnTrack) originalDst(epID TransportEndpointID) (tcpip.Address, uint1 dstAddr: epID.RemoteAddress, dstPort: epID.RemotePort, transProto: header.TCPProtocolNumber, - netProto: header.IPv4ProtocolNumber, + netProto: netProto, } conn, _ := ct.connForTID(tid) if conn == nil { diff --git a/pkg/tcpip/stack/forwarder_test.go b/pkg/tcpip/stack/forwarder_test.go index 54759091a..4e4b00a92 100644 --- a/pkg/tcpip/stack/forwarder_test.go +++ b/pkg/tcpip/stack/forwarder_test.go @@ -20,6 +20,7 @@ import ( "testing" "time" + "gvisor.dev/gvisor/pkg/sync" "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/header" @@ -45,6 +46,8 @@ const ( // use the first three: destination address, source address, and transport // protocol. They're all one byte fields to simplify parsing. type fwdTestNetworkEndpoint struct { + AddressableEndpointState + nicID tcpip.NICID proto *fwdTestNetworkProtocol dispatcher TransportDispatcher @@ -53,12 +56,18 @@ type fwdTestNetworkEndpoint struct { var _ NetworkEndpoint = (*fwdTestNetworkEndpoint)(nil) -func (f *fwdTestNetworkEndpoint) MTU() uint32 { - return f.ep.MTU() - uint32(f.MaxHeaderLength()) +func (*fwdTestNetworkEndpoint) Enable() *tcpip.Error { + return nil +} + +func (*fwdTestNetworkEndpoint) Enabled() bool { + return true } -func (f *fwdTestNetworkEndpoint) NICID() tcpip.NICID { - return f.nicID +func (*fwdTestNetworkEndpoint) Disable() {} + +func (f *fwdTestNetworkEndpoint) MTU() uint32 { + return f.ep.MTU() - uint32(f.MaxHeaderLength()) } func (*fwdTestNetworkEndpoint) DefaultTTL() uint8 { @@ -78,10 +87,6 @@ func (f *fwdTestNetworkEndpoint) PseudoHeaderChecksum(protocol tcpip.TransportPr return 0 } -func (f *fwdTestNetworkEndpoint) Capabilities() LinkEndpointCapabilities { - return f.ep.Capabilities() -} - func (f *fwdTestNetworkEndpoint) NetworkProtocolNumber() tcpip.NetworkProtocolNumber { return f.proto.Number() } @@ -106,7 +111,9 @@ func (*fwdTestNetworkEndpoint) WriteHeaderIncludedPacket(r *Route, pkt *PacketBu return tcpip.ErrNotSupported } -func (*fwdTestNetworkEndpoint) Close() {} +func (f *fwdTestNetworkEndpoint) Close() { + f.AddressableEndpointState.Cleanup() +} // fwdTestNetworkProtocol is a network-layer protocol that implements Address // resolution. @@ -116,6 +123,11 @@ type fwdTestNetworkProtocol struct { addrResolveDelay time.Duration onLinkAddressResolved func(cache *linkAddrCache, neigh *neighborCache, addr tcpip.Address, _ tcpip.LinkAddress) onResolveStaticAddress func(tcpip.Address) (tcpip.LinkAddress, bool) + + mu struct { + sync.RWMutex + forwarding bool + } } var _ NetworkProtocol = (*fwdTestNetworkProtocol)(nil) @@ -145,13 +157,15 @@ func (*fwdTestNetworkProtocol) Parse(pkt *PacketBuffer) (tcpip.TransportProtocol return tcpip.TransportProtocolNumber(netHeader[protocolNumberOffset]), true, true } -func (f *fwdTestNetworkProtocol) NewEndpoint(nicID tcpip.NICID, _ LinkAddressCache, _ NUDHandler, dispatcher TransportDispatcher, ep LinkEndpoint, _ *Stack) NetworkEndpoint { - return &fwdTestNetworkEndpoint{ - nicID: nicID, +func (f *fwdTestNetworkProtocol) NewEndpoint(nic NetworkInterface, _ LinkAddressCache, _ NUDHandler, dispatcher TransportDispatcher) NetworkEndpoint { + e := &fwdTestNetworkEndpoint{ + nicID: nic.ID(), proto: f, dispatcher: dispatcher, - ep: ep, + ep: nic.LinkEndpoint(), } + e.AddressableEndpointState.Init(e) + return e } func (*fwdTestNetworkProtocol) SetOption(tcpip.SettableNetworkProtocolOption) *tcpip.Error { @@ -186,6 +200,21 @@ func (*fwdTestNetworkProtocol) LinkAddressProtocol() tcpip.NetworkProtocolNumber return fwdTestNetNumber } +// Forwarding implements stack.ForwardingNetworkProtocol. +func (f *fwdTestNetworkProtocol) Forwarding() bool { + f.mu.RLock() + defer f.mu.RUnlock() + return f.mu.forwarding + +} + +// SetForwarding implements stack.ForwardingNetworkProtocol. +func (f *fwdTestNetworkProtocol) SetForwarding(v bool) { + f.mu.Lock() + defer f.mu.Unlock() + f.mu.forwarding = v +} + // fwdTestPacketInfo holds all the information about an outbound packet. type fwdTestPacketInfo struct { RemoteLinkAddress tcpip.LinkAddress @@ -307,7 +336,7 @@ func (e *fwdTestLinkEndpoint) AddHeader(local, remote tcpip.LinkAddress, protoco func fwdTestNetFactory(t *testing.T, proto *fwdTestNetworkProtocol, useNeighborCache bool) (ep1, ep2 *fwdTestLinkEndpoint) { // Create a stack with the network protocol and two NICs. s := New(Options{ - NetworkProtocols: []NetworkProtocol{proto}, + NetworkProtocols: []NetworkProtocolFactory{func(*Stack) NetworkProtocol { return proto }}, UseNeighborCache: useNeighborCache, }) @@ -316,7 +345,7 @@ func fwdTestNetFactory(t *testing.T, proto *fwdTestNetworkProtocol, useNeighborC } // Enable forwarding. - s.SetForwarding(true) + s.SetForwarding(proto.Number(), true) // NIC 1 has the link address "a", and added the network address 1. ep1 = &fwdTestLinkEndpoint{ diff --git a/pkg/tcpip/stack/iptables.go b/pkg/tcpip/stack/iptables.go index 0e33cbe92..8d6d9a7f1 100644 --- a/pkg/tcpip/stack/iptables.go +++ b/pkg/tcpip/stack/iptables.go @@ -57,14 +57,14 @@ const reaperDelay = 5 * time.Second // all packets. func DefaultTables() *IPTables { return &IPTables{ - tables: [numTables]Table{ + v4Tables: [numTables]Table{ natID: Table{ Rules: []Rule{ - Rule{Target: AcceptTarget{}}, - Rule{Target: AcceptTarget{}}, - Rule{Target: AcceptTarget{}}, - Rule{Target: AcceptTarget{}}, - Rule{Target: ErrorTarget{}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv4ProtocolNumber}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv4ProtocolNumber}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv4ProtocolNumber}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv4ProtocolNumber}}, + Rule{Target: &ErrorTarget{NetworkProtocol: header.IPv4ProtocolNumber}}, }, BuiltinChains: [NumHooks]int{ Prerouting: 0, @@ -83,9 +83,9 @@ func DefaultTables() *IPTables { }, mangleID: Table{ Rules: []Rule{ - Rule{Target: AcceptTarget{}}, - Rule{Target: AcceptTarget{}}, - Rule{Target: ErrorTarget{}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv4ProtocolNumber}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv4ProtocolNumber}}, + Rule{Target: &ErrorTarget{NetworkProtocol: header.IPv4ProtocolNumber}}, }, BuiltinChains: [NumHooks]int{ Prerouting: 0, @@ -101,10 +101,75 @@ func DefaultTables() *IPTables { }, filterID: Table{ Rules: []Rule{ - Rule{Target: AcceptTarget{}}, - Rule{Target: AcceptTarget{}}, - Rule{Target: AcceptTarget{}}, - Rule{Target: ErrorTarget{}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv4ProtocolNumber}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv4ProtocolNumber}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv4ProtocolNumber}}, + Rule{Target: &ErrorTarget{NetworkProtocol: header.IPv4ProtocolNumber}}, + }, + BuiltinChains: [NumHooks]int{ + Prerouting: HookUnset, + Input: 0, + Forward: 1, + Output: 2, + Postrouting: HookUnset, + }, + Underflows: [NumHooks]int{ + Prerouting: HookUnset, + Input: 0, + Forward: 1, + Output: 2, + Postrouting: HookUnset, + }, + }, + }, + v6Tables: [numTables]Table{ + natID: Table{ + Rules: []Rule{ + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv6ProtocolNumber}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv6ProtocolNumber}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv6ProtocolNumber}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv6ProtocolNumber}}, + Rule{Target: &ErrorTarget{NetworkProtocol: header.IPv6ProtocolNumber}}, + }, + BuiltinChains: [NumHooks]int{ + Prerouting: 0, + Input: 1, + Forward: HookUnset, + Output: 2, + Postrouting: 3, + }, + Underflows: [NumHooks]int{ + Prerouting: 0, + Input: 1, + Forward: HookUnset, + Output: 2, + Postrouting: 3, + }, + }, + mangleID: Table{ + Rules: []Rule{ + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv6ProtocolNumber}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv6ProtocolNumber}}, + Rule{Target: &ErrorTarget{NetworkProtocol: header.IPv6ProtocolNumber}}, + }, + BuiltinChains: [NumHooks]int{ + Prerouting: 0, + Output: 1, + }, + Underflows: [NumHooks]int{ + Prerouting: 0, + Input: HookUnset, + Forward: HookUnset, + Output: 1, + Postrouting: HookUnset, + }, + }, + filterID: Table{ + Rules: []Rule{ + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv6ProtocolNumber}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv6ProtocolNumber}}, + Rule{Target: &AcceptTarget{NetworkProtocol: header.IPv6ProtocolNumber}}, + Rule{Target: &ErrorTarget{NetworkProtocol: header.IPv6ProtocolNumber}}, }, BuiltinChains: [NumHooks]int{ Prerouting: HookUnset, @@ -166,25 +231,20 @@ func EmptyNATTable() Table { // GetTable returns a table by name. func (it *IPTables) GetTable(name string, ipv6 bool) (Table, bool) { - // TODO(gvisor.dev/issue/3549): Enable IPv6. - if ipv6 { - return Table{}, false - } id, ok := nameToID[name] if !ok { return Table{}, false } it.mu.RLock() defer it.mu.RUnlock() - return it.tables[id], true + if ipv6 { + return it.v6Tables[id], true + } + return it.v4Tables[id], true } // ReplaceTable replaces or inserts table by name. func (it *IPTables) ReplaceTable(name string, table Table, ipv6 bool) *tcpip.Error { - // TODO(gvisor.dev/issue/3549): Enable IPv6. - if ipv6 { - return tcpip.ErrInvalidOptionValue - } id, ok := nameToID[name] if !ok { return tcpip.ErrInvalidOptionValue @@ -198,7 +258,11 @@ func (it *IPTables) ReplaceTable(name string, table Table, ipv6 bool) *tcpip.Err it.startReaper(reaperDelay) } it.modified = true - it.tables[id] = table + if ipv6 { + it.v6Tables[id] = table + } else { + it.v4Tables[id] = table + } return nil } @@ -221,8 +285,15 @@ const ( // should continue traversing the network stack and false when it should be // dropped. // +// TODO(gvisor.dev/issue/170): PacketBuffer should hold the GSO and route, from +// which address and nicName can be gathered. Currently, address is only +// needed for prerouting and nicName is only needed for output. +// // Precondition: pkt.NetworkHeader is set. -func (it *IPTables) Check(hook Hook, pkt *PacketBuffer, gso *GSO, r *Route, address tcpip.Address, nicName string) bool { +func (it *IPTables) Check(hook Hook, pkt *PacketBuffer, gso *GSO, r *Route, preroutingAddr tcpip.Address, nicName string) bool { + if pkt.NetworkProtocolNumber != header.IPv4ProtocolNumber && pkt.NetworkProtocolNumber != header.IPv6ProtocolNumber { + return true + } // Many users never configure iptables. Spare them the cost of rule // traversal if rules have never been set. it.mu.RLock() @@ -243,9 +314,14 @@ func (it *IPTables) Check(hook Hook, pkt *PacketBuffer, gso *GSO, r *Route, addr if tableID == natID && pkt.NatDone { continue } - table := it.tables[tableID] + var table Table + if pkt.NetworkProtocolNumber == header.IPv6ProtocolNumber { + table = it.v6Tables[tableID] + } else { + table = it.v4Tables[tableID] + } ruleIdx := table.BuiltinChains[hook] - switch verdict := it.checkChain(hook, pkt, table, ruleIdx, gso, r, address, nicName); verdict { + switch verdict := it.checkChain(hook, pkt, table, ruleIdx, gso, r, preroutingAddr, nicName); verdict { // If the table returns Accept, move on to the next table. case chainAccept: continue @@ -256,7 +332,7 @@ func (it *IPTables) Check(hook Hook, pkt *PacketBuffer, gso *GSO, r *Route, addr // Any Return from a built-in chain means we have to // call the underflow. underflow := table.Rules[table.Underflows[hook]] - switch v, _ := underflow.Target.Action(pkt, &it.connections, hook, gso, r, address); v { + switch v, _ := underflow.Target.Action(pkt, &it.connections, hook, gso, r, preroutingAddr); v { case RuleAccept: continue case RuleDrop: @@ -351,11 +427,11 @@ func (it *IPTables) CheckPackets(hook Hook, pkts PacketBufferList, gso *GSO, r * // Preconditions: // * pkt is a IPv4 packet of at least length header.IPv4MinimumSize. // * pkt.NetworkHeader is not nil. -func (it *IPTables) checkChain(hook Hook, pkt *PacketBuffer, table Table, ruleIdx int, gso *GSO, r *Route, address tcpip.Address, nicName string) chainVerdict { +func (it *IPTables) checkChain(hook Hook, pkt *PacketBuffer, table Table, ruleIdx int, gso *GSO, r *Route, preroutingAddr tcpip.Address, nicName string) chainVerdict { // Start from ruleIdx and walk the list of rules until a rule gives us // a verdict. for ruleIdx < len(table.Rules) { - switch verdict, jumpTo := it.checkRule(hook, pkt, table, ruleIdx, gso, r, address, nicName); verdict { + switch verdict, jumpTo := it.checkRule(hook, pkt, table, ruleIdx, gso, r, preroutingAddr, nicName); verdict { case RuleAccept: return chainAccept @@ -372,7 +448,7 @@ func (it *IPTables) checkChain(hook Hook, pkt *PacketBuffer, table Table, ruleId ruleIdx++ continue } - switch verdict := it.checkChain(hook, pkt, table, jumpTo, gso, r, address, nicName); verdict { + switch verdict := it.checkChain(hook, pkt, table, jumpTo, gso, r, preroutingAddr, nicName); verdict { case chainAccept: return chainAccept case chainDrop: @@ -398,11 +474,11 @@ func (it *IPTables) checkChain(hook Hook, pkt *PacketBuffer, table Table, ruleId // Preconditions: // * pkt is a IPv4 packet of at least length header.IPv4MinimumSize. // * pkt.NetworkHeader is not nil. -func (it *IPTables) checkRule(hook Hook, pkt *PacketBuffer, table Table, ruleIdx int, gso *GSO, r *Route, address tcpip.Address, nicName string) (RuleVerdict, int) { +func (it *IPTables) checkRule(hook Hook, pkt *PacketBuffer, table Table, ruleIdx int, gso *GSO, r *Route, preroutingAddr tcpip.Address, nicName string) (RuleVerdict, int) { rule := table.Rules[ruleIdx] // Check whether the packet matches the IP header filter. - if !rule.Filter.match(header.IPv4(pkt.NetworkHeader().View()), hook, nicName) { + if !rule.Filter.match(pkt, hook, nicName) { // Continue on to the next rule. return RuleJump, ruleIdx + 1 } @@ -421,16 +497,16 @@ func (it *IPTables) checkRule(hook Hook, pkt *PacketBuffer, table Table, ruleIdx } // All the matchers matched, so run the target. - return rule.Target.Action(pkt, &it.connections, hook, gso, r, address) + return rule.Target.Action(pkt, &it.connections, hook, gso, r, preroutingAddr) } // OriginalDst returns the original destination of redirected connections. It // returns an error if the connection doesn't exist or isn't redirected. -func (it *IPTables) OriginalDst(epID TransportEndpointID) (tcpip.Address, uint16, *tcpip.Error) { +func (it *IPTables) OriginalDst(epID TransportEndpointID, netProto tcpip.NetworkProtocolNumber) (tcpip.Address, uint16, *tcpip.Error) { it.mu.RLock() defer it.mu.RUnlock() if !it.modified { return "", 0, tcpip.ErrNotConnected } - return it.connections.originalDst(epID) + return it.connections.originalDst(epID, netProto) } diff --git a/pkg/tcpip/stack/iptables_targets.go b/pkg/tcpip/stack/iptables_targets.go index 5f1b2af64..538c4625d 100644 --- a/pkg/tcpip/stack/iptables_targets.go +++ b/pkg/tcpip/stack/iptables_targets.go @@ -21,78 +21,139 @@ import ( ) // AcceptTarget accepts packets. -type AcceptTarget struct{} +type AcceptTarget struct { + // NetworkProtocol is the network protocol the target is used with. + NetworkProtocol tcpip.NetworkProtocolNumber +} + +// ID implements Target.ID. +func (at *AcceptTarget) ID() TargetID { + return TargetID{ + NetworkProtocol: at.NetworkProtocol, + } +} // Action implements Target.Action. -func (AcceptTarget) Action(*PacketBuffer, *ConnTrack, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { +func (*AcceptTarget) Action(*PacketBuffer, *ConnTrack, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { return RuleAccept, 0 } // DropTarget drops packets. -type DropTarget struct{} +type DropTarget struct { + // NetworkProtocol is the network protocol the target is used with. + NetworkProtocol tcpip.NetworkProtocolNumber +} + +// ID implements Target.ID. +func (dt *DropTarget) ID() TargetID { + return TargetID{ + NetworkProtocol: dt.NetworkProtocol, + } +} // Action implements Target.Action. -func (DropTarget) Action(*PacketBuffer, *ConnTrack, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { +func (*DropTarget) Action(*PacketBuffer, *ConnTrack, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { return RuleDrop, 0 } +// ErrorTargetName is used to mark targets as error targets. Error targets +// shouldn't be reached - an error has occurred if we fall through to one. +const ErrorTargetName = "ERROR" + // ErrorTarget logs an error and drops the packet. It represents a target that // should be unreachable. -type ErrorTarget struct{} +type ErrorTarget struct { + // NetworkProtocol is the network protocol the target is used with. + NetworkProtocol tcpip.NetworkProtocolNumber +} + +// ID implements Target.ID. +func (et *ErrorTarget) ID() TargetID { + return TargetID{ + Name: ErrorTargetName, + NetworkProtocol: et.NetworkProtocol, + } +} // Action implements Target.Action. -func (ErrorTarget) Action(*PacketBuffer, *ConnTrack, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { +func (*ErrorTarget) Action(*PacketBuffer, *ConnTrack, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { log.Debugf("ErrorTarget triggered.") return RuleDrop, 0 } // UserChainTarget marks a rule as the beginning of a user chain. type UserChainTarget struct { + // Name is the chain name. Name string + + // NetworkProtocol is the network protocol the target is used with. + NetworkProtocol tcpip.NetworkProtocolNumber +} + +// ID implements Target.ID. +func (uc *UserChainTarget) ID() TargetID { + return TargetID{ + Name: ErrorTargetName, + NetworkProtocol: uc.NetworkProtocol, + } } // Action implements Target.Action. -func (UserChainTarget) Action(*PacketBuffer, *ConnTrack, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { +func (*UserChainTarget) Action(*PacketBuffer, *ConnTrack, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { panic("UserChainTarget should never be called.") } // ReturnTarget returns from the current chain. If the chain is a built-in, the // hook's underflow should be called. -type ReturnTarget struct{} +type ReturnTarget struct { + // NetworkProtocol is the network protocol the target is used with. + NetworkProtocol tcpip.NetworkProtocolNumber +} + +// ID implements Target.ID. +func (rt *ReturnTarget) ID() TargetID { + return TargetID{ + NetworkProtocol: rt.NetworkProtocol, + } +} // Action implements Target.Action. -func (ReturnTarget) Action(*PacketBuffer, *ConnTrack, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { +func (*ReturnTarget) Action(*PacketBuffer, *ConnTrack, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { return RuleReturn, 0 } +// RedirectTargetName is used to mark targets as redirect targets. Redirect +// targets should be reached for only NAT and Mangle tables. These targets will +// change the destination port/destination IP for packets. +const RedirectTargetName = "REDIRECT" + // RedirectTarget redirects the packet by modifying the destination port/IP. -// Min and Max values for IP and Ports in the struct indicate the range of -// values which can be used to redirect. +// TODO(gvisor.dev/issue/170): Other flags need to be added after we support +// them. type RedirectTarget struct { - // TODO(gvisor.dev/issue/170): Other flags need to be added after - // we support them. - // RangeProtoSpecified flag indicates single port is specified to - // redirect. - RangeProtoSpecified bool + // Addr indicates address used to redirect. + Addr tcpip.Address - // MinIP indicates address used to redirect. - MinIP tcpip.Address + // Port indicates port used to redirect. + Port uint16 - // MaxIP indicates address used to redirect. - MaxIP tcpip.Address - - // MinPort indicates port used to redirect. - MinPort uint16 + // NetworkProtocol is the network protocol the target is used with. + NetworkProtocol tcpip.NetworkProtocolNumber +} - // MaxPort indicates port used to redirect. - MaxPort uint16 +// ID implements Target.ID. +func (rt *RedirectTarget) ID() TargetID { + return TargetID{ + Name: RedirectTargetName, + NetworkProtocol: rt.NetworkProtocol, + } } // Action implements Target.Action. // TODO(gvisor.dev/issue/170): Parse headers without copying. The current // implementation only works for PREROUTING and calls pkt.Clone(), neither // of which should be the case. -func (rt RedirectTarget) Action(pkt *PacketBuffer, ct *ConnTrack, hook Hook, gso *GSO, r *Route, address tcpip.Address) (RuleVerdict, int) { +func (rt *RedirectTarget) Action(pkt *PacketBuffer, ct *ConnTrack, hook Hook, gso *GSO, r *Route, address tcpip.Address) (RuleVerdict, int) { // Packet is already manipulated. if pkt.NatDone { return RuleAccept, 0 @@ -103,34 +164,35 @@ func (rt RedirectTarget) Action(pkt *PacketBuffer, ct *ConnTrack, hook Hook, gso return RuleDrop, 0 } - // Change the address to localhost (127.0.0.1) in Output and - // to primary address of the incoming interface in Prerouting. + // Change the address to localhost (127.0.0.1 or ::1) in Output and to + // the primary address of the incoming interface in Prerouting. switch hook { case Output: - rt.MinIP = tcpip.Address([]byte{127, 0, 0, 1}) - rt.MaxIP = tcpip.Address([]byte{127, 0, 0, 1}) + if pkt.NetworkProtocolNumber == header.IPv4ProtocolNumber { + rt.Addr = tcpip.Address([]byte{127, 0, 0, 1}) + } else { + rt.Addr = header.IPv6Loopback + } case Prerouting: - rt.MinIP = address - rt.MaxIP = address + rt.Addr = address default: panic("redirect target is supported only on output and prerouting hooks") } // TODO(gvisor.dev/issue/170): Check Flags in RedirectTarget if // we need to change dest address (for OUTPUT chain) or ports. - netHeader := header.IPv4(pkt.NetworkHeader().View()) - switch protocol := netHeader.TransportProtocol(); protocol { + switch protocol := pkt.TransportProtocolNumber; protocol { case header.UDPProtocolNumber: udpHeader := header.UDP(pkt.TransportHeader().View()) - udpHeader.SetDestinationPort(rt.MinPort) + udpHeader.SetDestinationPort(rt.Port) // Calculate UDP checksum and set it. if hook == Output { udpHeader.SetChecksum(0) - length := uint16(pkt.Size()) - uint16(netHeader.HeaderLength()) // Only calculate the checksum if offloading isn't supported. if r.Capabilities()&CapabilityTXChecksumOffload == 0 { + length := uint16(pkt.Size()) - uint16(len(pkt.NetworkHeader().View())) xsum := r.PseudoHeaderChecksum(protocol, length) for _, v := range pkt.Data.Views() { xsum = header.Checksum(v, xsum) @@ -139,10 +201,15 @@ func (rt RedirectTarget) Action(pkt *PacketBuffer, ct *ConnTrack, hook Hook, gso udpHeader.SetChecksum(^udpHeader.CalculateChecksum(xsum)) } } - // Change destination address. - netHeader.SetDestinationAddress(rt.MinIP) - netHeader.SetChecksum(0) - netHeader.SetChecksum(^netHeader.CalculateChecksum()) + + pkt.Network().SetDestinationAddress(rt.Addr) + + // After modification, IPv4 packets need a valid checksum. + if pkt.NetworkProtocolNumber == header.IPv4ProtocolNumber { + netHeader := header.IPv4(pkt.NetworkHeader().View()) + netHeader.SetChecksum(0) + netHeader.SetChecksum(^netHeader.CalculateChecksum()) + } pkt.NatDone = true case header.TCPProtocolNumber: if ct == nil { diff --git a/pkg/tcpip/stack/iptables_types.go b/pkg/tcpip/stack/iptables_types.go index fbbd2f50f..7b3f3e88b 100644 --- a/pkg/tcpip/stack/iptables_types.go +++ b/pkg/tcpip/stack/iptables_types.go @@ -15,6 +15,7 @@ package stack import ( + "fmt" "strings" "sync" @@ -81,31 +82,42 @@ const ( // // +stateify savable type IPTables struct { - // mu protects tables, priorities, and modified. + // mu protects v4Tables, v6Tables, and modified. mu sync.RWMutex - - // tables maps tableIDs to tables. Holds builtin tables only, not user - // tables. mu must be locked for accessing. - tables [numTables]Table - - // priorities maps each hook to a list of table names. The order of the - // list is the order in which each table should be visited for that - // hook. mu needs to be locked for accessing. - priorities [NumHooks][]tableID - + // v4Tables and v6tables map tableIDs to tables. They hold builtin + // tables only, not user tables. mu must be locked for accessing. + v4Tables [numTables]Table + v6Tables [numTables]Table // modified is whether tables have been modified at least once. It is // used to elide the iptables performance overhead for workloads that // don't utilize iptables. modified bool + // priorities maps each hook to a list of table names. The order of the + // list is the order in which each table should be visited for that + // hook. It is immutable. + priorities [NumHooks][]tableID + connections ConnTrack - // reaperDone can be signalled to stop the reaper goroutine. + // reaperDone can be signaled to stop the reaper goroutine. reaperDone chan struct{} } -// A Table defines a set of chains and hooks into the network stack. It is -// really just a list of rules. +// A Table defines a set of chains and hooks into the network stack. +// +// It is a list of Rules, entry points (BuiltinChains), and error handlers +// (Underflows). As packets traverse netstack, they hit hooks. When a packet +// hits a hook, iptables compares it to Rules starting from that hook's entry +// point. So if a packet hits the Input hook, we look up the corresponding +// entry point in BuiltinChains and jump to that point. +// +// If the Rule doesn't match the packet, iptables continues to the next Rule. +// If a Rule does match, it can issue a verdict on the packet (e.g. RuleAccept +// or RuleDrop) that causes the packet to stop traversing iptables. It can also +// jump to other rules or perform custom actions based on Rule.Target. +// +// Underflow Rules are invoked when a chain returns without reaching a verdict. // // +stateify savable type Table struct { @@ -148,7 +160,7 @@ type Rule struct { Target Target } -// IPHeaderFilter holds basic IP filtering data common to every rule. +// IPHeaderFilter performs basic IP header matching common to every rule. // // +stateify savable type IPHeaderFilter struct { @@ -196,16 +208,43 @@ type IPHeaderFilter struct { OutputInterfaceInvert bool } -// match returns whether hdr matches the filter. -func (fl IPHeaderFilter) match(hdr header.IPv4, hook Hook, nicName string) bool { - // TODO(gvisor.dev/issue/170): Support other fields of the filter. +// match returns whether pkt matches the filter. +// +// Preconditions: pkt.NetworkHeader is set and is at least of the minimal IPv4 +// or IPv6 header length. +func (fl IPHeaderFilter) match(pkt *PacketBuffer, hook Hook, nicName string) bool { + // Extract header fields. + var ( + // TODO(gvisor.dev/issue/170): Support other filter fields. + transProto tcpip.TransportProtocolNumber + dstAddr tcpip.Address + srcAddr tcpip.Address + ) + switch proto := pkt.NetworkProtocolNumber; proto { + case header.IPv4ProtocolNumber: + hdr := header.IPv4(pkt.NetworkHeader().View()) + transProto = hdr.TransportProtocol() + dstAddr = hdr.DestinationAddress() + srcAddr = hdr.SourceAddress() + + case header.IPv6ProtocolNumber: + hdr := header.IPv6(pkt.NetworkHeader().View()) + transProto = hdr.TransportProtocol() + dstAddr = hdr.DestinationAddress() + srcAddr = hdr.SourceAddress() + + default: + panic(fmt.Sprintf("unknown network protocol with EtherType: %d", proto)) + } + // Check the transport protocol. - if fl.Protocol != 0 && fl.Protocol != hdr.TransportProtocol() { + if fl.CheckProtocol && fl.Protocol != transProto { return false } - // Check the source and destination IPs. - if !filterAddress(hdr.DestinationAddress(), fl.DstMask, fl.Dst, fl.DstInvert) || !filterAddress(hdr.SourceAddress(), fl.SrcMask, fl.Src, fl.SrcInvert) { + // Check the addresses. + if !filterAddress(dstAddr, fl.DstMask, fl.Dst, fl.DstInvert) || + !filterAddress(srcAddr, fl.SrcMask, fl.Src, fl.SrcInvert) { return false } @@ -233,6 +272,18 @@ func (fl IPHeaderFilter) match(hdr header.IPv4, hook Hook, nicName string) bool return true } +// NetworkProtocol returns the protocol (IPv4 or IPv6) on to which the header +// applies. +func (fl IPHeaderFilter) NetworkProtocol() tcpip.NetworkProtocolNumber { + switch len(fl.Src) { + case header.IPv4AddressSize: + return header.IPv4ProtocolNumber + case header.IPv6AddressSize: + return header.IPv6ProtocolNumber + } + panic(fmt.Sprintf("invalid address in IPHeaderFilter: %s", fl.Src)) +} + // filterAddress returns whether addr matches the filter. func filterAddress(addr, mask, filterAddr tcpip.Address, invert bool) bool { matches := true @@ -258,8 +309,23 @@ type Matcher interface { Match(hook Hook, packet *PacketBuffer, interfaceName string) (matches bool, hotdrop bool) } +// A TargetID uniquely identifies a target. +type TargetID struct { + // Name is the target name as stored in the xt_entry_target struct. + Name string + + // NetworkProtocol is the protocol to which the target applies. + NetworkProtocol tcpip.NetworkProtocolNumber + + // Revision is the version of the target. + Revision uint8 +} + // A Target is the interface for taking an action for a packet. type Target interface { + // ID uniquely identifies the Target. + ID() TargetID + // Action takes an action on the packet and returns a verdict on how // traversal should (or should not) continue. If the return value is // Jump, it also returns the index of the rule to jump to. diff --git a/pkg/tcpip/stack/ndp_test.go b/pkg/tcpip/stack/ndp_test.go index 67dc5364f..73a01c2dd 100644 --- a/pkg/tcpip/stack/ndp_test.go +++ b/pkg/tcpip/stack/ndp_test.go @@ -150,10 +150,10 @@ type ndpDNSSLEvent struct { type ndpDHCPv6Event struct { nicID tcpip.NICID - configuration stack.DHCPv6ConfigurationFromNDPRA + configuration ipv6.DHCPv6ConfigurationFromNDPRA } -var _ stack.NDPDispatcher = (*ndpDispatcher)(nil) +var _ ipv6.NDPDispatcher = (*ndpDispatcher)(nil) // ndpDispatcher implements NDPDispatcher so tests can know when various NDP // related events happen for test purposes. @@ -170,7 +170,7 @@ type ndpDispatcher struct { dhcpv6ConfigurationC chan ndpDHCPv6Event } -// Implements stack.NDPDispatcher.OnDuplicateAddressDetectionStatus. +// Implements ipv6.NDPDispatcher.OnDuplicateAddressDetectionStatus. func (n *ndpDispatcher) OnDuplicateAddressDetectionStatus(nicID tcpip.NICID, addr tcpip.Address, resolved bool, err *tcpip.Error) { if n.dadC != nil { n.dadC <- ndpDADEvent{ @@ -182,7 +182,7 @@ func (n *ndpDispatcher) OnDuplicateAddressDetectionStatus(nicID tcpip.NICID, add } } -// Implements stack.NDPDispatcher.OnDefaultRouterDiscovered. +// Implements ipv6.NDPDispatcher.OnDefaultRouterDiscovered. func (n *ndpDispatcher) OnDefaultRouterDiscovered(nicID tcpip.NICID, addr tcpip.Address) bool { if c := n.routerC; c != nil { c <- ndpRouterEvent{ @@ -195,7 +195,7 @@ func (n *ndpDispatcher) OnDefaultRouterDiscovered(nicID tcpip.NICID, addr tcpip. return n.rememberRouter } -// Implements stack.NDPDispatcher.OnDefaultRouterInvalidated. +// Implements ipv6.NDPDispatcher.OnDefaultRouterInvalidated. func (n *ndpDispatcher) OnDefaultRouterInvalidated(nicID tcpip.NICID, addr tcpip.Address) { if c := n.routerC; c != nil { c <- ndpRouterEvent{ @@ -206,7 +206,7 @@ func (n *ndpDispatcher) OnDefaultRouterInvalidated(nicID tcpip.NICID, addr tcpip } } -// Implements stack.NDPDispatcher.OnOnLinkPrefixDiscovered. +// Implements ipv6.NDPDispatcher.OnOnLinkPrefixDiscovered. func (n *ndpDispatcher) OnOnLinkPrefixDiscovered(nicID tcpip.NICID, prefix tcpip.Subnet) bool { if c := n.prefixC; c != nil { c <- ndpPrefixEvent{ @@ -219,7 +219,7 @@ func (n *ndpDispatcher) OnOnLinkPrefixDiscovered(nicID tcpip.NICID, prefix tcpip return n.rememberPrefix } -// Implements stack.NDPDispatcher.OnOnLinkPrefixInvalidated. +// Implements ipv6.NDPDispatcher.OnOnLinkPrefixInvalidated. func (n *ndpDispatcher) OnOnLinkPrefixInvalidated(nicID tcpip.NICID, prefix tcpip.Subnet) { if c := n.prefixC; c != nil { c <- ndpPrefixEvent{ @@ -261,7 +261,7 @@ func (n *ndpDispatcher) OnAutoGenAddressInvalidated(nicID tcpip.NICID, addr tcpi } } -// Implements stack.NDPDispatcher.OnRecursiveDNSServerOption. +// Implements ipv6.NDPDispatcher.OnRecursiveDNSServerOption. func (n *ndpDispatcher) OnRecursiveDNSServerOption(nicID tcpip.NICID, addrs []tcpip.Address, lifetime time.Duration) { if c := n.rdnssC; c != nil { c <- ndpRDNSSEvent{ @@ -274,7 +274,7 @@ func (n *ndpDispatcher) OnRecursiveDNSServerOption(nicID tcpip.NICID, addrs []tc } } -// Implements stack.NDPDispatcher.OnDNSSearchListOption. +// Implements ipv6.NDPDispatcher.OnDNSSearchListOption. func (n *ndpDispatcher) OnDNSSearchListOption(nicID tcpip.NICID, domainNames []string, lifetime time.Duration) { if n.dnsslC != nil { n.dnsslC <- ndpDNSSLEvent{ @@ -285,8 +285,8 @@ func (n *ndpDispatcher) OnDNSSearchListOption(nicID tcpip.NICID, domainNames []s } } -// Implements stack.NDPDispatcher.OnDHCPv6Configuration. -func (n *ndpDispatcher) OnDHCPv6Configuration(nicID tcpip.NICID, configuration stack.DHCPv6ConfigurationFromNDPRA) { +// Implements ipv6.NDPDispatcher.OnDHCPv6Configuration. +func (n *ndpDispatcher) OnDHCPv6Configuration(nicID tcpip.NICID, configuration ipv6.DHCPv6ConfigurationFromNDPRA) { if c := n.dhcpv6ConfigurationC; c != nil { c <- ndpDHCPv6Event{ nicID, @@ -319,13 +319,12 @@ func TestDADDisabled(t *testing.T) { ndpDisp := ndpDispatcher{ dadC: make(chan ndpDADEvent, 1), } - opts := stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPDisp: &ndpDisp, - } - e := channel.New(0, 1280, linkAddr1) - s := stack.New(opts) + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPDisp: &ndpDisp, + })}, + }) if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(%d, _) = %s", nicID, err) } @@ -413,19 +412,21 @@ func TestDADResolve(t *testing.T) { ndpDisp := ndpDispatcher{ dadC: make(chan ndpDADEvent), } - opts := stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPDisp: &ndpDisp, - } - opts.NDPConfigs.RetransmitTimer = test.retransTimer - opts.NDPConfigs.DupAddrDetectTransmits = test.dupAddrDetectTransmits e := channelLinkWithHeaderLength{ Endpoint: channel.New(int(test.dupAddrDetectTransmits), 1280, linkAddr1), headerLength: test.linkHeaderLen, } e.Endpoint.LinkEPCapabilities |= stack.CapabilityResolutionRequired - s := stack.New(opts) + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPDisp: &ndpDisp, + NDPConfigs: ipv6.NDPConfigurations{ + RetransmitTimer: test.retransTimer, + DupAddrDetectTransmits: test.dupAddrDetectTransmits, + }, + })}, + }) if err := s.CreateNIC(nicID, &e); err != nil { t.Fatalf("CreateNIC(%d, _) = %s", nicID, err) } @@ -558,6 +559,26 @@ func TestDADResolve(t *testing.T) { } } +func rxNDPSolicit(e *channel.Endpoint, tgt tcpip.Address) { + hdr := buffer.NewPrependable(header.IPv6MinimumSize + header.ICMPv6NeighborSolicitMinimumSize) + pkt := header.ICMPv6(hdr.Prepend(header.ICMPv6NeighborSolicitMinimumSize)) + pkt.SetType(header.ICMPv6NeighborSolicit) + ns := header.NDPNeighborSolicit(pkt.NDPPayload()) + ns.SetTargetAddress(tgt) + snmc := header.SolicitedNodeAddr(tgt) + pkt.SetChecksum(header.ICMPv6Checksum(pkt, header.IPv6Any, snmc, buffer.VectorisedView{})) + payloadLength := hdr.UsedLength() + ip := header.IPv6(hdr.Prepend(header.IPv6MinimumSize)) + ip.Encode(&header.IPv6Fields{ + PayloadLength: uint16(payloadLength), + NextHeader: uint8(icmp.ProtocolNumber6), + HopLimit: 255, + SrcAddr: header.IPv6Any, + DstAddr: snmc, + }) + e.InjectInbound(header.IPv6ProtocolNumber, stack.NewPacketBuffer(stack.PacketBufferOptions{Data: hdr.View().ToVectorisedView()})) +} + // TestDADFail tests to make sure that the DAD process fails if another node is // detected to be performing DAD on the same address (receive an NS message from // a node doing DAD for the same address), or if another node is detected to own @@ -567,39 +588,19 @@ func TestDADFail(t *testing.T) { tests := []struct { name string - makeBuf func(tgt tcpip.Address) buffer.Prependable + rxPkt func(e *channel.Endpoint, tgt tcpip.Address) getStat func(s tcpip.ICMPv6ReceivedPacketStats) *tcpip.StatCounter }{ { - "RxSolicit", - func(tgt tcpip.Address) buffer.Prependable { - hdr := buffer.NewPrependable(header.IPv6MinimumSize + header.ICMPv6NeighborSolicitMinimumSize) - pkt := header.ICMPv6(hdr.Prepend(header.ICMPv6NeighborSolicitMinimumSize)) - pkt.SetType(header.ICMPv6NeighborSolicit) - ns := header.NDPNeighborSolicit(pkt.NDPPayload()) - ns.SetTargetAddress(tgt) - snmc := header.SolicitedNodeAddr(tgt) - pkt.SetChecksum(header.ICMPv6Checksum(pkt, header.IPv6Any, snmc, buffer.VectorisedView{})) - payloadLength := hdr.UsedLength() - ip := header.IPv6(hdr.Prepend(header.IPv6MinimumSize)) - ip.Encode(&header.IPv6Fields{ - PayloadLength: uint16(payloadLength), - NextHeader: uint8(icmp.ProtocolNumber6), - HopLimit: 255, - SrcAddr: header.IPv6Any, - DstAddr: snmc, - }) - - return hdr - - }, - func(s tcpip.ICMPv6ReceivedPacketStats) *tcpip.StatCounter { + name: "RxSolicit", + rxPkt: rxNDPSolicit, + getStat: func(s tcpip.ICMPv6ReceivedPacketStats) *tcpip.StatCounter { return s.NeighborSolicit }, }, { - "RxAdvert", - func(tgt tcpip.Address) buffer.Prependable { + name: "RxAdvert", + rxPkt: func(e *channel.Endpoint, tgt tcpip.Address) { naSize := header.ICMPv6NeighborAdvertMinimumSize + header.NDPLinkLayerAddressSize hdr := buffer.NewPrependable(header.IPv6MinimumSize + naSize) pkt := header.ICMPv6(hdr.Prepend(naSize)) @@ -621,11 +622,9 @@ func TestDADFail(t *testing.T) { SrcAddr: tgt, DstAddr: header.IPv6AllNodesMulticastAddress, }) - - return hdr - + e.InjectInbound(header.IPv6ProtocolNumber, stack.NewPacketBuffer(stack.PacketBufferOptions{Data: hdr.View().ToVectorisedView()})) }, - func(s tcpip.ICMPv6ReceivedPacketStats) *tcpip.StatCounter { + getStat: func(s tcpip.ICMPv6ReceivedPacketStats) *tcpip.StatCounter { return s.NeighborAdvert }, }, @@ -636,16 +635,16 @@ func TestDADFail(t *testing.T) { ndpDisp := ndpDispatcher{ dadC: make(chan ndpDADEvent, 1), } - ndpConfigs := stack.DefaultNDPConfigurations() - opts := stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: ndpConfigs, - NDPDisp: &ndpDisp, - } - opts.NDPConfigs.RetransmitTimer = time.Second * 2 + ndpConfigs := ipv6.DefaultNDPConfigurations() + ndpConfigs.RetransmitTimer = time.Second * 2 e := channel.New(0, 1280, linkAddr1) - s := stack.New(opts) + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPDisp: &ndpDisp, + NDPConfigs: ndpConfigs, + })}, + }) if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(%d, _) = %s", nicID, err) } @@ -664,13 +663,8 @@ func TestDADFail(t *testing.T) { t.Fatalf("got stack.GetMainNICAddress(%d, %d) = (%s, nil), want = (%s, nil)", nicID, header.IPv6ProtocolNumber, addr, want) } - // Receive a packet to simulate multiple nodes owning or - // attempting to own the same address. - hdr := test.makeBuf(addr1) - pkt := stack.NewPacketBuffer(stack.PacketBufferOptions{ - Data: hdr.View().ToVectorisedView(), - }) - e.InjectInbound(header.IPv6ProtocolNumber, pkt) + // Receive a packet to simulate an address conflict. + test.rxPkt(e, addr1) stat := test.getStat(s.Stats().ICMP.V6PacketsReceived) if got := stat.Value(); got != 1 { @@ -754,18 +748,19 @@ func TestDADStop(t *testing.T) { ndpDisp := ndpDispatcher{ dadC: make(chan ndpDADEvent, 1), } - ndpConfigs := stack.NDPConfigurations{ + + ndpConfigs := ipv6.NDPConfigurations{ RetransmitTimer: time.Second, DupAddrDetectTransmits: 2, } - opts := stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPDisp: &ndpDisp, - NDPConfigs: ndpConfigs, - } e := channel.New(0, 1280, linkAddr1) - s := stack.New(opts) + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPDisp: &ndpDisp, + NDPConfigs: ndpConfigs, + })}, + }) if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(%d, _): %s", nicID, err) } @@ -815,19 +810,6 @@ func TestDADStop(t *testing.T) { } } -// TestSetNDPConfigurationFailsForBadNICID tests to make sure we get an error if -// we attempt to update NDP configurations using an invalid NICID. -func TestSetNDPConfigurationFailsForBadNICID(t *testing.T) { - s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - }) - - // No NIC with ID 1 yet. - if got := s.SetNDPConfigurations(1, stack.NDPConfigurations{}); got != tcpip.ErrUnknownNICID { - t.Fatalf("got s.SetNDPConfigurations = %v, want = %s", got, tcpip.ErrUnknownNICID) - } -} - // TestSetNDPConfigurations tests that we can update and use per-interface NDP // configurations without affecting the default NDP configurations or other // interfaces' configurations. @@ -863,8 +845,9 @@ func TestSetNDPConfigurations(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPDisp: &ndpDisp, + })}, }) expectDADEvent := func(nicID tcpip.NICID, addr tcpip.Address) { @@ -892,12 +875,15 @@ func TestSetNDPConfigurations(t *testing.T) { } // Update the NDP configurations on NIC(1) to use DAD. - configs := stack.NDPConfigurations{ + configs := ipv6.NDPConfigurations{ DupAddrDetectTransmits: test.dupAddrDetectTransmits, RetransmitTimer: test.retransmitTimer, } - if err := s.SetNDPConfigurations(nicID1, configs); err != nil { - t.Fatalf("got SetNDPConfigurations(%d, _) = %s", nicID1, err) + if ipv6Ep, err := s.GetNetworkEndpoint(nicID1, header.IPv6ProtocolNumber); err != nil { + t.Fatalf("s.GetNetworkEndpoint(%d, %d): %s", nicID1, header.IPv6ProtocolNumber, err) + } else { + ndpEP := ipv6Ep.(ipv6.NDPEndpoint) + ndpEP.SetNDPConfigurations(configs) } // Created after updating NIC(1)'s NDP configurations @@ -1113,14 +1099,15 @@ func TestNoRouterDiscovery(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: handle, - DiscoverDefaultRouters: discover, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: handle, + DiscoverDefaultRouters: discover, + }, + NDPDisp: &ndpDisp, + })}, }) - s.SetForwarding(forwarding) + s.SetForwarding(ipv6.ProtocolNumber, forwarding) if err := s.CreateNIC(1, e); err != nil { t.Fatalf("CreateNIC(1) = %s", err) @@ -1151,12 +1138,13 @@ func TestRouterDiscoveryDispatcherNoRemember(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - DiscoverDefaultRouters: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + DiscoverDefaultRouters: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(1, e); err != nil { @@ -1192,12 +1180,13 @@ func TestRouterDiscovery(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - DiscoverDefaultRouters: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + DiscoverDefaultRouters: true, + }, + NDPDisp: &ndpDisp, + })}, }) expectRouterEvent := func(addr tcpip.Address, discovered bool) { @@ -1285,7 +1274,7 @@ func TestRouterDiscovery(t *testing.T) { } // TestRouterDiscoveryMaxRouters tests that only -// stack.MaxDiscoveredDefaultRouters discovered routers are remembered. +// ipv6.MaxDiscoveredDefaultRouters discovered routers are remembered. func TestRouterDiscoveryMaxRouters(t *testing.T) { ndpDisp := ndpDispatcher{ routerC: make(chan ndpRouterEvent, 1), @@ -1293,12 +1282,13 @@ func TestRouterDiscoveryMaxRouters(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - DiscoverDefaultRouters: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + DiscoverDefaultRouters: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(1, e); err != nil { @@ -1306,14 +1296,14 @@ func TestRouterDiscoveryMaxRouters(t *testing.T) { } // Receive an RA from 2 more than the max number of discovered routers. - for i := 1; i <= stack.MaxDiscoveredDefaultRouters+2; i++ { + for i := 1; i <= ipv6.MaxDiscoveredDefaultRouters+2; i++ { linkAddr := []byte{2, 2, 3, 4, 5, 0} linkAddr[5] = byte(i) llAddr := header.LinkLocalAddr(tcpip.LinkAddress(linkAddr)) e.InjectInbound(header.IPv6ProtocolNumber, raBuf(llAddr, 5)) - if i <= stack.MaxDiscoveredDefaultRouters { + if i <= ipv6.MaxDiscoveredDefaultRouters { select { case e := <-ndpDisp.routerC: if diff := checkRouterEvent(e, llAddr, true); diff != "" { @@ -1358,14 +1348,15 @@ func TestNoPrefixDiscovery(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: handle, - DiscoverOnLinkPrefixes: discover, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: handle, + DiscoverOnLinkPrefixes: discover, + }, + NDPDisp: &ndpDisp, + })}, }) - s.SetForwarding(forwarding) + s.SetForwarding(ipv6.ProtocolNumber, forwarding) if err := s.CreateNIC(1, e); err != nil { t.Fatalf("CreateNIC(1) = %s", err) @@ -1399,13 +1390,14 @@ func TestPrefixDiscoveryDispatcherNoRemember(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - DiscoverDefaultRouters: false, - DiscoverOnLinkPrefixes: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + DiscoverDefaultRouters: false, + DiscoverOnLinkPrefixes: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(1, e); err != nil { @@ -1445,12 +1437,13 @@ func TestPrefixDiscovery(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - DiscoverOnLinkPrefixes: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + DiscoverOnLinkPrefixes: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(1, e); err != nil { @@ -1545,12 +1538,13 @@ func TestPrefixDiscoveryWithInfiniteLifetime(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - DiscoverOnLinkPrefixes: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + DiscoverOnLinkPrefixes: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(1, e); err != nil { @@ -1621,33 +1615,34 @@ func TestPrefixDiscoveryWithInfiniteLifetime(t *testing.T) { } // TestPrefixDiscoveryMaxRouters tests that only -// stack.MaxDiscoveredOnLinkPrefixes discovered on-link prefixes are remembered. +// ipv6.MaxDiscoveredOnLinkPrefixes discovered on-link prefixes are remembered. func TestPrefixDiscoveryMaxOnLinkPrefixes(t *testing.T) { ndpDisp := ndpDispatcher{ - prefixC: make(chan ndpPrefixEvent, stack.MaxDiscoveredOnLinkPrefixes+3), + prefixC: make(chan ndpPrefixEvent, ipv6.MaxDiscoveredOnLinkPrefixes+3), rememberPrefix: true, } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - DiscoverDefaultRouters: false, - DiscoverOnLinkPrefixes: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + DiscoverDefaultRouters: false, + DiscoverOnLinkPrefixes: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(1, e); err != nil { t.Fatalf("CreateNIC(1) = %s", err) } - optSer := make(header.NDPOptionsSerializer, stack.MaxDiscoveredOnLinkPrefixes+2) - prefixes := [stack.MaxDiscoveredOnLinkPrefixes + 2]tcpip.Subnet{} + optSer := make(header.NDPOptionsSerializer, ipv6.MaxDiscoveredOnLinkPrefixes+2) + prefixes := [ipv6.MaxDiscoveredOnLinkPrefixes + 2]tcpip.Subnet{} // Receive an RA with 2 more than the max number of discovered on-link // prefixes. - for i := 0; i < stack.MaxDiscoveredOnLinkPrefixes+2; i++ { + for i := 0; i < ipv6.MaxDiscoveredOnLinkPrefixes+2; i++ { prefixAddr := [16]byte{1, 2, 3, 4, 5, 6, 7, 8, 0, 0, 0, 0, 0, 0, 0, 0} prefixAddr[7] = byte(i) prefix := tcpip.AddressWithPrefix{ @@ -1665,8 +1660,8 @@ func TestPrefixDiscoveryMaxOnLinkPrefixes(t *testing.T) { } e.InjectInbound(header.IPv6ProtocolNumber, raBufWithOpts(llAddr1, 0, optSer)) - for i := 0; i < stack.MaxDiscoveredOnLinkPrefixes+2; i++ { - if i < stack.MaxDiscoveredOnLinkPrefixes { + for i := 0; i < ipv6.MaxDiscoveredOnLinkPrefixes+2; i++ { + if i < ipv6.MaxDiscoveredOnLinkPrefixes { select { case e := <-ndpDisp.prefixC: if diff := checkPrefixEvent(e, prefixes[i], true); diff != "" { @@ -1716,14 +1711,15 @@ func TestNoAutoGenAddr(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: handle, - AutoGenGlobalAddresses: autogen, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: handle, + AutoGenGlobalAddresses: autogen, + }, + NDPDisp: &ndpDisp, + })}, }) - s.SetForwarding(forwarding) + s.SetForwarding(ipv6.ProtocolNumber, forwarding) if err := s.CreateNIC(1, e); err != nil { t.Fatalf("CreateNIC(1) = %s", err) @@ -1749,14 +1745,14 @@ func checkAutoGenAddrEvent(e ndpAutoGenAddrEvent, addr tcpip.AddressWithPrefix, // TestAutoGenAddr tests that an address is properly generated and invalidated // when configured to do so. -func TestAutoGenAddr(t *testing.T) { +func TestAutoGenAddr2(t *testing.T) { const newMinVL = 2 newMinVLDuration := newMinVL * time.Second - saved := stack.MinPrefixInformationValidLifetimeForUpdate + saved := ipv6.MinPrefixInformationValidLifetimeForUpdate defer func() { - stack.MinPrefixInformationValidLifetimeForUpdate = saved + ipv6.MinPrefixInformationValidLifetimeForUpdate = saved }() - stack.MinPrefixInformationValidLifetimeForUpdate = newMinVLDuration + ipv6.MinPrefixInformationValidLifetimeForUpdate = newMinVLDuration prefix1, _, addr1 := prefixSubnetAddr(0, linkAddr1) prefix2, _, addr2 := prefixSubnetAddr(1, linkAddr1) @@ -1766,12 +1762,13 @@ func TestAutoGenAddr(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - AutoGenGlobalAddresses: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + AutoGenGlobalAddresses: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(1, e); err != nil { @@ -1876,14 +1873,14 @@ func TestAutoGenTempAddr(t *testing.T) { newMinVLDuration = newMinVL * time.Second ) - savedMinPrefixInformationValidLifetimeForUpdate := stack.MinPrefixInformationValidLifetimeForUpdate - savedMaxDesync := stack.MaxDesyncFactor + savedMinPrefixInformationValidLifetimeForUpdate := ipv6.MinPrefixInformationValidLifetimeForUpdate + savedMaxDesync := ipv6.MaxDesyncFactor defer func() { - stack.MinPrefixInformationValidLifetimeForUpdate = savedMinPrefixInformationValidLifetimeForUpdate - stack.MaxDesyncFactor = savedMaxDesync + ipv6.MinPrefixInformationValidLifetimeForUpdate = savedMinPrefixInformationValidLifetimeForUpdate + ipv6.MaxDesyncFactor = savedMaxDesync }() - stack.MinPrefixInformationValidLifetimeForUpdate = newMinVLDuration - stack.MaxDesyncFactor = time.Nanosecond + ipv6.MinPrefixInformationValidLifetimeForUpdate = newMinVLDuration + ipv6.MaxDesyncFactor = time.Nanosecond prefix1, _, addr1 := prefixSubnetAddr(0, linkAddr1) prefix2, _, addr2 := prefixSubnetAddr(1, linkAddr1) @@ -1931,16 +1928,17 @@ func TestAutoGenTempAddr(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - DupAddrDetectTransmits: test.dupAddrTransmits, - RetransmitTimer: test.retransmitTimer, - HandleRAs: true, - AutoGenGlobalAddresses: true, - AutoGenTempGlobalAddresses: true, - }, - NDPDisp: &ndpDisp, - TempIIDSeed: seed, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + DupAddrDetectTransmits: test.dupAddrTransmits, + RetransmitTimer: test.retransmitTimer, + HandleRAs: true, + AutoGenGlobalAddresses: true, + AutoGenTempGlobalAddresses: true, + }, + NDPDisp: &ndpDisp, + TempIIDSeed: seed, + })}, }) if err := s.CreateNIC(nicID, e); err != nil { @@ -2119,11 +2117,11 @@ func TestAutoGenTempAddr(t *testing.T) { func TestNoAutoGenTempAddrForLinkLocal(t *testing.T) { const nicID = 1 - savedMaxDesyncFactor := stack.MaxDesyncFactor + savedMaxDesyncFactor := ipv6.MaxDesyncFactor defer func() { - stack.MaxDesyncFactor = savedMaxDesyncFactor + ipv6.MaxDesyncFactor = savedMaxDesyncFactor }() - stack.MaxDesyncFactor = time.Nanosecond + ipv6.MaxDesyncFactor = time.Nanosecond tests := []struct { name string @@ -2160,12 +2158,13 @@ func TestNoAutoGenTempAddrForLinkLocal(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - AutoGenTempGlobalAddresses: true, - }, - NDPDisp: &ndpDisp, - AutoGenIPv6LinkLocal: true, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + AutoGenTempGlobalAddresses: true, + }, + NDPDisp: &ndpDisp, + AutoGenIPv6LinkLocal: true, + })}, }) if err := s.CreateNIC(nicID, e); err != nil { @@ -2211,11 +2210,11 @@ func TestNoAutoGenTempAddrWithoutStableAddr(t *testing.T) { retransmitTimer = 2 * time.Second ) - savedMaxDesyncFactor := stack.MaxDesyncFactor + savedMaxDesyncFactor := ipv6.MaxDesyncFactor defer func() { - stack.MaxDesyncFactor = savedMaxDesyncFactor + ipv6.MaxDesyncFactor = savedMaxDesyncFactor }() - stack.MaxDesyncFactor = 0 + ipv6.MaxDesyncFactor = 0 prefix, _, addr := prefixSubnetAddr(0, linkAddr1) var tempIIDHistory [header.IIDSize]byte @@ -2228,15 +2227,16 @@ func TestNoAutoGenTempAddrWithoutStableAddr(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - DupAddrDetectTransmits: dadTransmits, - RetransmitTimer: retransmitTimer, - HandleRAs: true, - AutoGenGlobalAddresses: true, - AutoGenTempGlobalAddresses: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + DupAddrDetectTransmits: dadTransmits, + RetransmitTimer: retransmitTimer, + HandleRAs: true, + AutoGenGlobalAddresses: true, + AutoGenTempGlobalAddresses: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(nicID, e); err != nil { @@ -2294,17 +2294,17 @@ func TestAutoGenTempAddrRegen(t *testing.T) { newMinVLDuration = newMinVL * time.Second ) - savedMaxDesyncFactor := stack.MaxDesyncFactor - savedMinMaxTempAddrPreferredLifetime := stack.MinMaxTempAddrPreferredLifetime - savedMinMaxTempAddrValidLifetime := stack.MinMaxTempAddrValidLifetime + savedMaxDesyncFactor := ipv6.MaxDesyncFactor + savedMinMaxTempAddrPreferredLifetime := ipv6.MinMaxTempAddrPreferredLifetime + savedMinMaxTempAddrValidLifetime := ipv6.MinMaxTempAddrValidLifetime defer func() { - stack.MaxDesyncFactor = savedMaxDesyncFactor - stack.MinMaxTempAddrPreferredLifetime = savedMinMaxTempAddrPreferredLifetime - stack.MinMaxTempAddrValidLifetime = savedMinMaxTempAddrValidLifetime + ipv6.MaxDesyncFactor = savedMaxDesyncFactor + ipv6.MinMaxTempAddrPreferredLifetime = savedMinMaxTempAddrPreferredLifetime + ipv6.MinMaxTempAddrValidLifetime = savedMinMaxTempAddrValidLifetime }() - stack.MaxDesyncFactor = 0 - stack.MinMaxTempAddrPreferredLifetime = newMinVLDuration - stack.MinMaxTempAddrValidLifetime = newMinVLDuration + ipv6.MaxDesyncFactor = 0 + ipv6.MinMaxTempAddrPreferredLifetime = newMinVLDuration + ipv6.MinMaxTempAddrValidLifetime = newMinVLDuration prefix, _, addr := prefixSubnetAddr(0, linkAddr1) var tempIIDHistory [header.IIDSize]byte @@ -2317,16 +2317,17 @@ func TestAutoGenTempAddrRegen(t *testing.T) { autoGenAddrC: make(chan ndpAutoGenAddrEvent, 2), } e := channel.New(0, 1280, linkAddr1) - ndpConfigs := stack.NDPConfigurations{ + ndpConfigs := ipv6.NDPConfigurations{ HandleRAs: true, AutoGenGlobalAddresses: true, AutoGenTempGlobalAddresses: true, RegenAdvanceDuration: newMinVLDuration - regenAfter, } s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: ndpConfigs, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ndpConfigs, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(nicID, e); err != nil { @@ -2382,8 +2383,11 @@ func TestAutoGenTempAddrRegen(t *testing.T) { // Stop generating temporary addresses ndpConfigs.AutoGenTempGlobalAddresses = false - if err := s.SetNDPConfigurations(nicID, ndpConfigs); err != nil { - t.Fatalf("s.SetNDPConfigurations(%d, _): %s", nicID, err) + if ipv6Ep, err := s.GetNetworkEndpoint(nicID, header.IPv6ProtocolNumber); err != nil { + t.Fatalf("s.GetNetworkEndpoint(%d, %d): %s", nicID, header.IPv6ProtocolNumber, err) + } else { + ndpEP := ipv6Ep.(ipv6.NDPEndpoint) + ndpEP.SetNDPConfigurations(ndpConfigs) } // Wait for all the temporary addresses to get invalidated. @@ -2439,17 +2443,17 @@ func TestAutoGenTempAddrRegenJobUpdates(t *testing.T) { newMinVLDuration = newMinVL * time.Second ) - savedMaxDesyncFactor := stack.MaxDesyncFactor - savedMinMaxTempAddrPreferredLifetime := stack.MinMaxTempAddrPreferredLifetime - savedMinMaxTempAddrValidLifetime := stack.MinMaxTempAddrValidLifetime + savedMaxDesyncFactor := ipv6.MaxDesyncFactor + savedMinMaxTempAddrPreferredLifetime := ipv6.MinMaxTempAddrPreferredLifetime + savedMinMaxTempAddrValidLifetime := ipv6.MinMaxTempAddrValidLifetime defer func() { - stack.MaxDesyncFactor = savedMaxDesyncFactor - stack.MinMaxTempAddrPreferredLifetime = savedMinMaxTempAddrPreferredLifetime - stack.MinMaxTempAddrValidLifetime = savedMinMaxTempAddrValidLifetime + ipv6.MaxDesyncFactor = savedMaxDesyncFactor + ipv6.MinMaxTempAddrPreferredLifetime = savedMinMaxTempAddrPreferredLifetime + ipv6.MinMaxTempAddrValidLifetime = savedMinMaxTempAddrValidLifetime }() - stack.MaxDesyncFactor = 0 - stack.MinMaxTempAddrPreferredLifetime = newMinVLDuration - stack.MinMaxTempAddrValidLifetime = newMinVLDuration + ipv6.MaxDesyncFactor = 0 + ipv6.MinMaxTempAddrPreferredLifetime = newMinVLDuration + ipv6.MinMaxTempAddrValidLifetime = newMinVLDuration prefix, _, addr := prefixSubnetAddr(0, linkAddr1) var tempIIDHistory [header.IIDSize]byte @@ -2462,16 +2466,17 @@ func TestAutoGenTempAddrRegenJobUpdates(t *testing.T) { autoGenAddrC: make(chan ndpAutoGenAddrEvent, 2), } e := channel.New(0, 1280, linkAddr1) - ndpConfigs := stack.NDPConfigurations{ + ndpConfigs := ipv6.NDPConfigurations{ HandleRAs: true, AutoGenGlobalAddresses: true, AutoGenTempGlobalAddresses: true, RegenAdvanceDuration: newMinVLDuration - regenAfter, } s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: ndpConfigs, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ndpConfigs, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(nicID, e); err != nil { @@ -2545,9 +2550,12 @@ func TestAutoGenTempAddrRegenJobUpdates(t *testing.T) { // as paased. ndpConfigs.MaxTempAddrValidLifetime = 100 * time.Second ndpConfigs.MaxTempAddrPreferredLifetime = 100 * time.Second - if err := s.SetNDPConfigurations(nicID, ndpConfigs); err != nil { - t.Fatalf("s.SetNDPConfigurations(%d, _): %s", nicID, err) + ipv6Ep, err := s.GetNetworkEndpoint(nicID, header.IPv6ProtocolNumber) + if err != nil { + t.Fatalf("s.GetNetworkEndpoint(%d, %d): %s", nicID, header.IPv6ProtocolNumber, err) } + ndpEP := ipv6Ep.(ipv6.NDPEndpoint) + ndpEP.SetNDPConfigurations(ndpConfigs) e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, 100, 100)) select { case e := <-ndpDisp.autoGenAddrC: @@ -2565,9 +2573,7 @@ func TestAutoGenTempAddrRegenJobUpdates(t *testing.T) { newLifetimes := newMinVLDuration + regenAfter + defaultAsyncNegativeEventTimeout ndpConfigs.MaxTempAddrValidLifetime = newLifetimes ndpConfigs.MaxTempAddrPreferredLifetime = newLifetimes - if err := s.SetNDPConfigurations(nicID, ndpConfigs); err != nil { - t.Fatalf("s.SetNDPConfigurations(%d, _): %s", nicID, err) - } + ndpEP.SetNDPConfigurations(ndpConfigs) e.InjectInbound(header.IPv6ProtocolNumber, raBufWithPI(llAddr2, 0, prefix, true, true, 100, 100)) expectAutoGenAddrEventAsync(tempAddr3, newAddr, regenAfter+defaultAsyncPositiveEventTimeout) } @@ -2655,20 +2661,21 @@ func TestMixedSLAACAddrConflictRegen(t *testing.T) { autoGenAddrC: make(chan ndpAutoGenAddrEvent, 2), } e := channel.New(0, 1280, linkAddr1) - ndpConfigs := stack.NDPConfigurations{ + ndpConfigs := ipv6.NDPConfigurations{ HandleRAs: true, AutoGenGlobalAddresses: true, AutoGenTempGlobalAddresses: test.tempAddrs, AutoGenAddressConflictRetries: 1, } s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, - NDPConfigs: ndpConfigs, - NDPDisp: &ndpDisp, - OpaqueIIDOpts: stack.OpaqueInterfaceIdentifierOptions{ - NICNameFromID: test.nicNameFromID, - }, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ndpConfigs, + NDPDisp: &ndpDisp, + OpaqueIIDOpts: ipv6.OpaqueInterfaceIdentifierOptions{ + NICNameFromID: test.nicNameFromID, + }, + })}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, }) s.SetRouteTable([]tcpip.Route{{ @@ -2739,8 +2746,11 @@ func TestMixedSLAACAddrConflictRegen(t *testing.T) { ndpDisp.dadC = make(chan ndpDADEvent, 2) ndpConfigs.DupAddrDetectTransmits = dupAddrTransmits ndpConfigs.RetransmitTimer = retransmitTimer - if err := s.SetNDPConfigurations(nicID, ndpConfigs); err != nil { - t.Fatalf("s.SetNDPConfigurations(%d, _): %s", nicID, err) + if ipv6Ep, err := s.GetNetworkEndpoint(nicID, header.IPv6ProtocolNumber); err != nil { + t.Fatalf("s.GetNetworkEndpoint(%d, %d): %s", nicID, header.IPv6ProtocolNumber, err) + } else { + ndpEP := ipv6Ep.(ipv6.NDPEndpoint) + ndpEP.SetNDPConfigurations(ndpConfigs) } // Do SLAAC for prefix. @@ -2754,9 +2764,7 @@ func TestMixedSLAACAddrConflictRegen(t *testing.T) { // DAD failure to restart the local generation process. addr := test.addrs[maxSLAACAddrLocalRegenAttempts-1] expectAutoGenAddrAsyncEvent(addr, newAddr) - if err := s.DupTentativeAddrDetected(nicID, addr.Address); err != nil { - t.Fatalf("s.DupTentativeAddrDetected(%d, %s): %s", nicID, addr.Address, err) - } + rxNDPSolicit(e, addr.Address) select { case e := <-ndpDisp.dadC: if diff := checkDADEvent(e, nicID, addr.Address, false, nil); diff != "" { @@ -2794,14 +2802,15 @@ func stackAndNdpDispatcherWithDefaultRoute(t *testing.T, nicID tcpip.NICID, useN } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - AutoGenGlobalAddresses: true, - }, - NDPDisp: ndpDisp, - UseNeighborCache: useNeighborCache, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + AutoGenGlobalAddresses: true, + }, + NDPDisp: ndpDisp, + })}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, + UseNeighborCache: useNeighborCache, }) if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(%d, _) = %s", nicID, err) @@ -3036,11 +3045,11 @@ func TestAutoGenAddrJobDeprecation(t *testing.T) { for _, stackTyp := range stacks { t.Run(stackTyp.name, func(t *testing.T) { - saved := stack.MinPrefixInformationValidLifetimeForUpdate + saved := ipv6.MinPrefixInformationValidLifetimeForUpdate defer func() { - stack.MinPrefixInformationValidLifetimeForUpdate = saved + ipv6.MinPrefixInformationValidLifetimeForUpdate = saved }() - stack.MinPrefixInformationValidLifetimeForUpdate = newMinVLDuration + ipv6.MinPrefixInformationValidLifetimeForUpdate = newMinVLDuration prefix1, _, addr1 := prefixSubnetAddr(0, linkAddr1) prefix2, _, addr2 := prefixSubnetAddr(1, linkAddr1) @@ -3258,12 +3267,12 @@ func TestAutoGenAddrFiniteToInfiniteToFiniteVL(t *testing.T) { const infiniteVLSeconds = 2 const minVLSeconds = 1 savedIL := header.NDPInfiniteLifetime - savedMinVL := stack.MinPrefixInformationValidLifetimeForUpdate + savedMinVL := ipv6.MinPrefixInformationValidLifetimeForUpdate defer func() { - stack.MinPrefixInformationValidLifetimeForUpdate = savedMinVL + ipv6.MinPrefixInformationValidLifetimeForUpdate = savedMinVL header.NDPInfiniteLifetime = savedIL }() - stack.MinPrefixInformationValidLifetimeForUpdate = minVLSeconds * time.Second + ipv6.MinPrefixInformationValidLifetimeForUpdate = minVLSeconds * time.Second header.NDPInfiniteLifetime = infiniteVLSeconds * time.Second prefix, _, addr := prefixSubnetAddr(0, linkAddr1) @@ -3307,12 +3316,13 @@ func TestAutoGenAddrFiniteToInfiniteToFiniteVL(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - AutoGenGlobalAddresses: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + AutoGenGlobalAddresses: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(1, e); err != nil { @@ -3357,11 +3367,11 @@ func TestAutoGenAddrFiniteToInfiniteToFiniteVL(t *testing.T) { func TestAutoGenAddrValidLifetimeUpdates(t *testing.T) { const infiniteVL = 4294967295 const newMinVL = 4 - saved := stack.MinPrefixInformationValidLifetimeForUpdate + saved := ipv6.MinPrefixInformationValidLifetimeForUpdate defer func() { - stack.MinPrefixInformationValidLifetimeForUpdate = saved + ipv6.MinPrefixInformationValidLifetimeForUpdate = saved }() - stack.MinPrefixInformationValidLifetimeForUpdate = newMinVL * time.Second + ipv6.MinPrefixInformationValidLifetimeForUpdate = newMinVL * time.Second prefix, _, addr := prefixSubnetAddr(0, linkAddr1) @@ -3449,12 +3459,13 @@ func TestAutoGenAddrValidLifetimeUpdates(t *testing.T) { } e := channel.New(10, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - AutoGenGlobalAddresses: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + AutoGenGlobalAddresses: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(1, e); err != nil { @@ -3515,12 +3526,13 @@ func TestAutoGenAddrRemoval(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - AutoGenGlobalAddresses: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + AutoGenGlobalAddresses: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(1, e); err != nil { @@ -3700,12 +3712,13 @@ func TestAutoGenAddrStaticConflict(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - AutoGenGlobalAddresses: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + AutoGenGlobalAddresses: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(1, e); err != nil { @@ -3781,18 +3794,19 @@ func TestAutoGenAddrWithOpaqueIID(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - AutoGenGlobalAddresses: true, - }, - NDPDisp: &ndpDisp, - OpaqueIIDOpts: stack.OpaqueInterfaceIdentifierOptions{ - NICNameFromID: func(_ tcpip.NICID, nicName string) string { - return nicName + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + AutoGenGlobalAddresses: true, }, - SecretKey: secretKey, - }, + NDPDisp: &ndpDisp, + OpaqueIIDOpts: ipv6.OpaqueInterfaceIdentifierOptions{ + NICNameFromID: func(_ tcpip.NICID, nicName string) string { + return nicName + }, + SecretKey: secretKey, + }, + })}, }) opts := stack.NICOptions{Name: nicName} if err := s.CreateNICWithOptions(nicID, e, opts); err != nil { @@ -3856,11 +3870,11 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) { const lifetimeSeconds = 10 // Needed for the temporary address sub test. - savedMaxDesync := stack.MaxDesyncFactor + savedMaxDesync := ipv6.MaxDesyncFactor defer func() { - stack.MaxDesyncFactor = savedMaxDesync + ipv6.MaxDesyncFactor = savedMaxDesync }() - stack.MaxDesyncFactor = time.Nanosecond + ipv6.MaxDesyncFactor = time.Nanosecond var secretKeyBuf [header.OpaqueIIDSecretKeyMinBytes]byte secretKey := secretKeyBuf[:] @@ -3938,14 +3952,14 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) { addrTypes := []struct { name string - ndpConfigs stack.NDPConfigurations + ndpConfigs ipv6.NDPConfigurations autoGenLinkLocal bool prepareFn func(t *testing.T, ndpDisp *ndpDispatcher, e *channel.Endpoint, tempIIDHistory []byte) []tcpip.AddressWithPrefix addrGenFn func(dadCounter uint8, tempIIDHistory []byte) tcpip.AddressWithPrefix }{ { name: "Global address", - ndpConfigs: stack.NDPConfigurations{ + ndpConfigs: ipv6.NDPConfigurations{ DupAddrDetectTransmits: dadTransmits, RetransmitTimer: retransmitTimer, HandleRAs: true, @@ -3963,7 +3977,7 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) { }, { name: "LinkLocal address", - ndpConfigs: stack.NDPConfigurations{ + ndpConfigs: ipv6.NDPConfigurations{ DupAddrDetectTransmits: dadTransmits, RetransmitTimer: retransmitTimer, }, @@ -3977,7 +3991,7 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) { }, { name: "Temporary address", - ndpConfigs: stack.NDPConfigurations{ + ndpConfigs: ipv6.NDPConfigurations{ DupAddrDetectTransmits: dadTransmits, RetransmitTimer: retransmitTimer, HandleRAs: true, @@ -4029,16 +4043,17 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) { ndpConfigs := addrType.ndpConfigs ndpConfigs.AutoGenAddressConflictRetries = maxRetries s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - AutoGenIPv6LinkLocal: addrType.autoGenLinkLocal, - NDPConfigs: ndpConfigs, - NDPDisp: &ndpDisp, - OpaqueIIDOpts: stack.OpaqueInterfaceIdentifierOptions{ - NICNameFromID: func(_ tcpip.NICID, nicName string) string { - return nicName + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + AutoGenIPv6LinkLocal: addrType.autoGenLinkLocal, + NDPConfigs: ndpConfigs, + NDPDisp: &ndpDisp, + OpaqueIIDOpts: ipv6.OpaqueInterfaceIdentifierOptions{ + NICNameFromID: func(_ tcpip.NICID, nicName string) string { + return nicName + }, + SecretKey: secretKey, }, - SecretKey: secretKey, - }, + })}, }) opts := stack.NICOptions{Name: nicName} if err := s.CreateNICWithOptions(nicID, e, opts); err != nil { @@ -4059,9 +4074,7 @@ func TestAutoGenAddrInResponseToDADConflicts(t *testing.T) { } // Simulate a DAD conflict. - if err := s.DupTentativeAddrDetected(nicID, addr.Address); err != nil { - t.Fatalf("s.DupTentativeAddrDetected(%d, %s): %s", nicID, addr.Address, err) - } + rxNDPSolicit(e, addr.Address) expectAutoGenAddrEvent(t, &ndpDisp, addr, invalidatedAddr) expectDADEvent(t, &ndpDisp, addr.Address, false) @@ -4119,14 +4132,14 @@ func TestAutoGenAddrWithEUI64IIDNoDADRetries(t *testing.T) { addrTypes := []struct { name string - ndpConfigs stack.NDPConfigurations + ndpConfigs ipv6.NDPConfigurations autoGenLinkLocal bool subnet tcpip.Subnet triggerSLAACFn func(e *channel.Endpoint) }{ { name: "Global address", - ndpConfigs: stack.NDPConfigurations{ + ndpConfigs: ipv6.NDPConfigurations{ DupAddrDetectTransmits: dadTransmits, RetransmitTimer: retransmitTimer, HandleRAs: true, @@ -4142,7 +4155,7 @@ func TestAutoGenAddrWithEUI64IIDNoDADRetries(t *testing.T) { }, { name: "LinkLocal address", - ndpConfigs: stack.NDPConfigurations{ + ndpConfigs: ipv6.NDPConfigurations{ DupAddrDetectTransmits: dadTransmits, RetransmitTimer: retransmitTimer, AutoGenAddressConflictRetries: maxRetries, @@ -4165,10 +4178,11 @@ func TestAutoGenAddrWithEUI64IIDNoDADRetries(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - AutoGenIPv6LinkLocal: addrType.autoGenLinkLocal, - NDPConfigs: addrType.ndpConfigs, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + AutoGenIPv6LinkLocal: addrType.autoGenLinkLocal, + NDPConfigs: addrType.ndpConfigs, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(%d, _) = %s", nicID, err) @@ -4198,9 +4212,7 @@ func TestAutoGenAddrWithEUI64IIDNoDADRetries(t *testing.T) { expectAutoGenAddrEvent(addr, newAddr) // Simulate a DAD conflict. - if err := s.DupTentativeAddrDetected(nicID, addr.Address); err != nil { - t.Fatalf("s.DupTentativeAddrDetected(%d, %s): %s", nicID, addr.Address, err) - } + rxNDPSolicit(e, addr.Address) expectAutoGenAddrEvent(addr, invalidatedAddr) select { case e := <-ndpDisp.dadC: @@ -4250,21 +4262,22 @@ func TestAutoGenAddrContinuesLifetimesAfterRetry(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - DupAddrDetectTransmits: dadTransmits, - RetransmitTimer: retransmitTimer, - HandleRAs: true, - AutoGenGlobalAddresses: true, - AutoGenAddressConflictRetries: maxRetries, - }, - NDPDisp: &ndpDisp, - OpaqueIIDOpts: stack.OpaqueInterfaceIdentifierOptions{ - NICNameFromID: func(_ tcpip.NICID, nicName string) string { - return nicName + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + DupAddrDetectTransmits: dadTransmits, + RetransmitTimer: retransmitTimer, + HandleRAs: true, + AutoGenGlobalAddresses: true, + AutoGenAddressConflictRetries: maxRetries, }, - SecretKey: secretKey, - }, + NDPDisp: &ndpDisp, + OpaqueIIDOpts: ipv6.OpaqueInterfaceIdentifierOptions{ + NICNameFromID: func(_ tcpip.NICID, nicName string) string { + return nicName + }, + SecretKey: secretKey, + }, + })}, }) opts := stack.NICOptions{Name: nicName} if err := s.CreateNICWithOptions(nicID, e, opts); err != nil { @@ -4296,9 +4309,7 @@ func TestAutoGenAddrContinuesLifetimesAfterRetry(t *testing.T) { // Simulate a DAD conflict after some time has passed. time.Sleep(failureTimer) - if err := s.DupTentativeAddrDetected(nicID, addr.Address); err != nil { - t.Fatalf("s.DupTentativeAddrDetected(%d, %s): %s", nicID, addr.Address, err) - } + rxNDPSolicit(e, addr.Address) expectAutoGenAddrEvent(addr, invalidatedAddr) select { case e := <-ndpDisp.dadC: @@ -4459,11 +4470,12 @@ func TestNDPRecursiveDNSServerDispatch(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(1, e); err != nil { t.Fatalf("CreateNIC(1) = %s", err) @@ -4509,11 +4521,12 @@ func TestNDPDNSSearchListDispatch(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(%d, _) = %s", nicID, err) @@ -4640,7 +4653,7 @@ func TestCleanupNDPState(t *testing.T) { name: "Enable forwarding", cleanupFn: func(t *testing.T, s *stack.Stack) { t.Helper() - s.SetForwarding(true) + s.SetForwarding(ipv6.ProtocolNumber, true) }, keepAutoGenLinkLocal: true, maxAutoGenAddrEvents: 4, @@ -4694,15 +4707,16 @@ func TestCleanupNDPState(t *testing.T) { autoGenAddrC: make(chan ndpAutoGenAddrEvent, test.maxAutoGenAddrEvents), } s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - AutoGenIPv6LinkLocal: true, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - DiscoverDefaultRouters: true, - DiscoverOnLinkPrefixes: true, - AutoGenGlobalAddresses: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + AutoGenIPv6LinkLocal: true, + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + DiscoverDefaultRouters: true, + DiscoverOnLinkPrefixes: true, + AutoGenGlobalAddresses: true, + }, + NDPDisp: &ndpDisp, + })}, }) expectRouterEvent := func() (bool, ndpRouterEvent) { @@ -4967,18 +4981,19 @@ func TestDHCPv6ConfigurationFromNDPDA(t *testing.T) { } e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + }, + NDPDisp: &ndpDisp, + })}, }) if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(%d, _) = %s", nicID, err) } - expectDHCPv6Event := func(configuration stack.DHCPv6ConfigurationFromNDPRA) { + expectDHCPv6Event := func(configuration ipv6.DHCPv6ConfigurationFromNDPRA) { t.Helper() select { case e := <-ndpDisp.dhcpv6ConfigurationC: @@ -5002,7 +5017,7 @@ func TestDHCPv6ConfigurationFromNDPDA(t *testing.T) { // Even if the first RA reports no DHCPv6 configurations are available, the // dispatcher should get an event. e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, false, false)) - expectDHCPv6Event(stack.DHCPv6NoConfiguration) + expectDHCPv6Event(ipv6.DHCPv6NoConfiguration) // Receiving the same update again should not result in an event to the // dispatcher. e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, false, false)) @@ -5011,19 +5026,19 @@ func TestDHCPv6ConfigurationFromNDPDA(t *testing.T) { // Receive an RA that updates the DHCPv6 configuration to Other // Configurations. e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, false, true)) - expectDHCPv6Event(stack.DHCPv6OtherConfigurations) + expectDHCPv6Event(ipv6.DHCPv6OtherConfigurations) e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, false, true)) expectNoDHCPv6Event() // Receive an RA that updates the DHCPv6 configuration to Managed Address. e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, true, false)) - expectDHCPv6Event(stack.DHCPv6ManagedAddress) + expectDHCPv6Event(ipv6.DHCPv6ManagedAddress) e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, true, false)) expectNoDHCPv6Event() // Receive an RA that updates the DHCPv6 configuration to none. e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, false, false)) - expectDHCPv6Event(stack.DHCPv6NoConfiguration) + expectDHCPv6Event(ipv6.DHCPv6NoConfiguration) e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, false, false)) expectNoDHCPv6Event() @@ -5031,7 +5046,7 @@ func TestDHCPv6ConfigurationFromNDPDA(t *testing.T) { // // Note, when the M flag is set, the O flag is redundant. e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, true, true)) - expectDHCPv6Event(stack.DHCPv6ManagedAddress) + expectDHCPv6Event(ipv6.DHCPv6ManagedAddress) e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, true, true)) expectNoDHCPv6Event() // Even though the DHCPv6 flags are different, the effective configuration is @@ -5044,7 +5059,7 @@ func TestDHCPv6ConfigurationFromNDPDA(t *testing.T) { // Receive an RA that updates the DHCPv6 configuration to Other // Configurations. e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, false, true)) - expectDHCPv6Event(stack.DHCPv6OtherConfigurations) + expectDHCPv6Event(ipv6.DHCPv6OtherConfigurations) e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, false, true)) expectNoDHCPv6Event() @@ -5059,7 +5074,7 @@ func TestDHCPv6ConfigurationFromNDPDA(t *testing.T) { // Receive an RA that updates the DHCPv6 configuration to Other // Configurations. e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, false, true)) - expectDHCPv6Event(stack.DHCPv6OtherConfigurations) + expectDHCPv6Event(ipv6.DHCPv6OtherConfigurations) e.InjectInbound(header.IPv6ProtocolNumber, raBufWithDHCPv6(llAddr2, false, true)) expectNoDHCPv6Event() } @@ -5217,12 +5232,13 @@ func TestRouterSolicitation(t *testing.T) { } } s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - MaxRtrSolicitations: test.maxRtrSolicit, - RtrSolicitationInterval: test.rtrSolicitInt, - MaxRtrSolicitationDelay: test.maxRtrSolicitDelay, - }, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + MaxRtrSolicitations: test.maxRtrSolicit, + RtrSolicitationInterval: test.rtrSolicitInt, + MaxRtrSolicitationDelay: test.maxRtrSolicitDelay, + }, + })}, }) if err := s.CreateNIC(nicID, &e); err != nil { t.Fatalf("CreateNIC(%d, _) = %s", nicID, err) @@ -5286,11 +5302,11 @@ func TestStopStartSolicitingRouters(t *testing.T) { name: "Enable and disable forwarding", startFn: func(t *testing.T, s *stack.Stack) { t.Helper() - s.SetForwarding(false) + s.SetForwarding(ipv6.ProtocolNumber, false) }, stopFn: func(t *testing.T, s *stack.Stack, _ bool) { t.Helper() - s.SetForwarding(true) + s.SetForwarding(ipv6.ProtocolNumber, true) }, }, @@ -5357,12 +5373,13 @@ func TestStopStartSolicitingRouters(t *testing.T) { checker.NDPRS()) } s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - MaxRtrSolicitations: maxRtrSolicitations, - RtrSolicitationInterval: interval, - MaxRtrSolicitationDelay: delay, - }, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + MaxRtrSolicitations: maxRtrSolicitations, + RtrSolicitationInterval: interval, + MaxRtrSolicitationDelay: delay, + }, + })}, }) if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(%d, _) = %s", nicID, err) diff --git a/pkg/tcpip/stack/neighbor_cache_test.go b/pkg/tcpip/stack/neighbor_cache_test.go index b4fa69e3e..a0b7da5cd 100644 --- a/pkg/tcpip/stack/neighbor_cache_test.go +++ b/pkg/tcpip/stack/neighbor_cache_test.go @@ -30,6 +30,7 @@ import ( "github.com/google/go-cmp/cmp/cmpopts" "gvisor.dev/gvisor/pkg/sleep" "gvisor.dev/gvisor/pkg/tcpip" + "gvisor.dev/gvisor/pkg/tcpip/faketime" ) const ( @@ -239,7 +240,7 @@ type entryEvent struct { func TestNeighborCacheGetConfig(t *testing.T) { nudDisp := testNUDDispatcher{} c := DefaultNUDConfigurations() - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(&nudDisp, c, clock) if got, want := neigh.config(), c; got != want { @@ -257,7 +258,7 @@ func TestNeighborCacheGetConfig(t *testing.T) { func TestNeighborCacheSetConfig(t *testing.T) { nudDisp := testNUDDispatcher{} c := DefaultNUDConfigurations() - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(&nudDisp, c, clock) c.MinRandomFactor = 1 @@ -279,7 +280,7 @@ func TestNeighborCacheSetConfig(t *testing.T) { func TestNeighborCacheEntry(t *testing.T) { c := DefaultNUDConfigurations() nudDisp := testNUDDispatcher{} - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(&nudDisp, c, clock) store := newTestEntryStore() linkRes := &testNeighborResolver{ @@ -298,7 +299,7 @@ func TestNeighborCacheEntry(t *testing.T) { t.Errorf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } - clock.advance(typicalLatency) + clock.Advance(typicalLatency) wantEvents := []testEntryEventInfo{ { @@ -339,7 +340,7 @@ func TestNeighborCacheRemoveEntry(t *testing.T) { config := DefaultNUDConfigurations() nudDisp := testNUDDispatcher{} - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(&nudDisp, config, clock) store := newTestEntryStore() linkRes := &testNeighborResolver{ @@ -358,7 +359,7 @@ func TestNeighborCacheRemoveEntry(t *testing.T) { t.Errorf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } - clock.advance(typicalLatency) + clock.Advance(typicalLatency) wantEvents := []testEntryEventInfo{ { @@ -409,7 +410,7 @@ func TestNeighborCacheRemoveEntry(t *testing.T) { } type testContext struct { - clock *fakeClock + clock *faketime.ManualClock neigh *neighborCache store *testEntryStore linkRes *testNeighborResolver @@ -418,7 +419,7 @@ type testContext struct { func newTestContext(c NUDConfigurations) testContext { nudDisp := &testNUDDispatcher{} - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(nudDisp, c, clock) store := newTestEntryStore() linkRes := &testNeighborResolver{ @@ -454,7 +455,7 @@ func (c *testContext) overflowCache(opts overflowOptions) error { if _, _, err := c.neigh.entry(entry.Addr, entry.LocalAddr, c.linkRes, nil); err != tcpip.ErrWouldBlock { return fmt.Errorf("got c.neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } - c.clock.advance(c.neigh.config().RetransmitTimer) + c.clock.Advance(c.neigh.config().RetransmitTimer) var wantEvents []testEntryEventInfo @@ -567,7 +568,7 @@ func TestNeighborCacheRemoveEntryThenOverflow(t *testing.T) { if err != tcpip.ErrWouldBlock { t.Errorf("got c.neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } - c.clock.advance(c.neigh.config().RetransmitTimer) + c.clock.Advance(c.neigh.config().RetransmitTimer) wantEvents := []testEntryEventInfo{ { EventType: entryTestAdded, @@ -803,7 +804,7 @@ func TestNeighborCacheOverwriteWithStaticEntryThenOverflow(t *testing.T) { if err != tcpip.ErrWouldBlock { t.Errorf("got c.neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } - c.clock.advance(typicalLatency) + c.clock.Advance(typicalLatency) wantEvents := []testEntryEventInfo{ { EventType: entryTestAdded, @@ -876,7 +877,7 @@ func TestNeighborCacheNotifiesWaker(t *testing.T) { config := DefaultNUDConfigurations() nudDisp := testNUDDispatcher{} - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(&nudDisp, config, clock) store := newTestEntryStore() linkRes := &testNeighborResolver{ @@ -902,7 +903,7 @@ func TestNeighborCacheNotifiesWaker(t *testing.T) { if doneCh == nil { t.Fatalf("expected done channel from neigh.entry(%s, %s, _, _)", entry.Addr, entry.LocalAddr) } - clock.advance(typicalLatency) + clock.Advance(typicalLatency) select { case <-doneCh: @@ -944,7 +945,7 @@ func TestNeighborCacheRemoveWaker(t *testing.T) { config := DefaultNUDConfigurations() nudDisp := testNUDDispatcher{} - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(&nudDisp, config, clock) store := newTestEntryStore() linkRes := &testNeighborResolver{ @@ -974,7 +975,7 @@ func TestNeighborCacheRemoveWaker(t *testing.T) { // Remove the waker before the neighbor cache has the opportunity to send a // notification. neigh.removeWaker(entry.Addr, &w) - clock.advance(typicalLatency) + clock.Advance(typicalLatency) select { case <-doneCh: @@ -1073,7 +1074,7 @@ func TestNeighborCacheClear(t *testing.T) { config := DefaultNUDConfigurations() nudDisp := testNUDDispatcher{} - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(&nudDisp, config, clock) store := newTestEntryStore() linkRes := &testNeighborResolver{ @@ -1092,7 +1093,7 @@ func TestNeighborCacheClear(t *testing.T) { if err != tcpip.ErrWouldBlock { t.Errorf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } - clock.advance(typicalLatency) + clock.Advance(typicalLatency) wantEvents := []testEntryEventInfo{ { @@ -1188,7 +1189,7 @@ func TestNeighborCacheClearThenOverflow(t *testing.T) { if err != tcpip.ErrWouldBlock { t.Errorf("got c.neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } - c.clock.advance(typicalLatency) + c.clock.Advance(typicalLatency) wantEvents := []testEntryEventInfo{ { EventType: entryTestAdded, @@ -1249,7 +1250,7 @@ func TestNeighborCacheKeepFrequentlyUsed(t *testing.T) { config.MaxRandomFactor = 1 nudDisp := testNUDDispatcher{} - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(&nudDisp, config, clock) store := newTestEntryStore() linkRes := &testNeighborResolver{ @@ -1277,7 +1278,7 @@ func TestNeighborCacheKeepFrequentlyUsed(t *testing.T) { if err != tcpip.ErrWouldBlock { t.Errorf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } - clock.advance(typicalLatency) + clock.Advance(typicalLatency) select { case <-doneCh: default: @@ -1325,7 +1326,7 @@ func TestNeighborCacheKeepFrequentlyUsed(t *testing.T) { if err != tcpip.ErrWouldBlock { t.Errorf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } - clock.advance(typicalLatency) + clock.Advance(typicalLatency) select { case <-doneCh: default: @@ -1412,7 +1413,7 @@ func TestNeighborCacheConcurrent(t *testing.T) { config := DefaultNUDConfigurations() nudDisp := testNUDDispatcher{} - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(&nudDisp, config, clock) store := newTestEntryStore() linkRes := &testNeighborResolver{ @@ -1440,7 +1441,7 @@ func TestNeighborCacheConcurrent(t *testing.T) { wg.Wait() // Process all the requests for a single entry concurrently - clock.advance(typicalLatency) + clock.Advance(typicalLatency) } // All goroutines add in the same order and add more values than can fit in @@ -1472,7 +1473,7 @@ func TestNeighborCacheReplace(t *testing.T) { config := DefaultNUDConfigurations() nudDisp := testNUDDispatcher{} - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(&nudDisp, config, clock) store := newTestEntryStore() linkRes := &testNeighborResolver{ @@ -1491,7 +1492,7 @@ func TestNeighborCacheReplace(t *testing.T) { if err != tcpip.ErrWouldBlock { t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } - clock.advance(typicalLatency) + clock.Advance(typicalLatency) select { case <-doneCh: default: @@ -1541,7 +1542,7 @@ func TestNeighborCacheReplace(t *testing.T) { if err != tcpip.ErrWouldBlock { t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } - clock.advance(config.DelayFirstProbeTime + typicalLatency) + clock.Advance(config.DelayFirstProbeTime + typicalLatency) select { case <-doneCh: default: @@ -1552,7 +1553,7 @@ func TestNeighborCacheReplace(t *testing.T) { // Verify the entry's new link address { e, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil) - clock.advance(typicalLatency) + clock.Advance(typicalLatency) if err != nil { t.Errorf("unexpected error from neigh.entry(%s, %s, _, nil): %s", entry.Addr, entry.LocalAddr, err) } @@ -1572,7 +1573,7 @@ func TestNeighborCacheResolutionFailed(t *testing.T) { config := DefaultNUDConfigurations() nudDisp := testNUDDispatcher{} - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(&nudDisp, config, clock) store := newTestEntryStore() @@ -1595,7 +1596,7 @@ func TestNeighborCacheResolutionFailed(t *testing.T) { if _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil); err != tcpip.ErrWouldBlock { t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } - clock.advance(typicalLatency) + clock.Advance(typicalLatency) got, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil) if err != nil { t.Fatalf("unexpected error from neigh.entry(%s, %s, _, nil): %s", entry.Addr, entry.LocalAddr, err) @@ -1618,7 +1619,7 @@ func TestNeighborCacheResolutionFailed(t *testing.T) { t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } waitFor := config.DelayFirstProbeTime + typicalLatency*time.Duration(config.MaxMulticastProbes) - clock.advance(waitFor) + clock.Advance(waitFor) if _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil); err != tcpip.ErrNoLinkAddress { t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrNoLinkAddress) } @@ -1636,7 +1637,7 @@ func TestNeighborCacheResolutionTimeout(t *testing.T) { config := DefaultNUDConfigurations() config.RetransmitTimer = time.Millisecond // small enough to cause timeout - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(nil, config, clock) store := newTestEntryStore() linkRes := &testNeighborResolver{ @@ -1654,7 +1655,7 @@ func TestNeighborCacheResolutionTimeout(t *testing.T) { t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrWouldBlock) } waitFor := config.RetransmitTimer * time.Duration(config.MaxMulticastProbes) - clock.advance(waitFor) + clock.Advance(waitFor) if _, _, err := neigh.entry(entry.Addr, entry.LocalAddr, linkRes, nil); err != tcpip.ErrNoLinkAddress { t.Fatalf("got neigh.entry(%s, %s, _, nil) = %v, want = %s", entry.Addr, entry.LocalAddr, err, tcpip.ErrNoLinkAddress) } @@ -1664,7 +1665,7 @@ func TestNeighborCacheResolutionTimeout(t *testing.T) { // resolved immediately and don't send resolution requests. func TestNeighborCacheStaticResolution(t *testing.T) { config := DefaultNUDConfigurations() - clock := newFakeClock() + clock := faketime.NewManualClock() neigh := newTestNeighborCache(nil, config, clock) store := newTestEntryStore() linkRes := &testNeighborResolver{ diff --git a/pkg/tcpip/stack/neighbor_entry.go b/pkg/tcpip/stack/neighbor_entry.go index 0068cacb8..9a72bec79 100644 --- a/pkg/tcpip/stack/neighbor_entry.go +++ b/pkg/tcpip/stack/neighbor_entry.go @@ -21,6 +21,7 @@ import ( "gvisor.dev/gvisor/pkg/sleep" "gvisor.dev/gvisor/pkg/tcpip" + "gvisor.dev/gvisor/pkg/tcpip/header" ) // NeighborEntry describes a neighboring device in the local network. @@ -73,8 +74,7 @@ const ( type neighborEntry struct { neighborEntryEntry - nic *NIC - protocol tcpip.NetworkProtocolNumber + nic *NIC // linkRes provides the functionality to send reachability probes, used in // Neighbor Unreachability Detection. @@ -440,7 +440,7 @@ func (e *neighborEntry) handleConfirmationLocked(linkAddr tcpip.LinkAddress, fla e.notifyWakersLocked() } - if e.isRouter && !flags.IsRouter { + if e.isRouter && !flags.IsRouter && header.IsV6UnicastAddress(e.neigh.Addr) { // "In those cases where the IsRouter flag changes from TRUE to FALSE as // a result of this update, the node MUST remove that router from the // Default Router List and update the Destination Cache entries for all @@ -448,9 +448,17 @@ func (e *neighborEntry) handleConfirmationLocked(linkAddr tcpip.LinkAddress, fla // 7.3.3. This is needed to detect when a node that is used as a router // stops forwarding packets due to being configured as a host." // - RFC 4861 section 7.2.5 - e.nic.mu.Lock() - e.nic.mu.ndp.invalidateDefaultRouter(e.neigh.Addr) - e.nic.mu.Unlock() + // + // TODO(gvisor.dev/issue/4085): Remove the special casing we do for IPv6 + // here. + ep, ok := e.nic.networkEndpoints[header.IPv6ProtocolNumber] + if !ok { + panic(fmt.Sprintf("have a neighbor entry for an IPv6 router but no IPv6 network endpoint")) + } + + if ndpEP, ok := ep.(NDPEndpoint); ok { + ndpEP.InvalidateDefaultRouter(e.neigh.Addr) + } } e.isRouter = flags.IsRouter diff --git a/pkg/tcpip/stack/neighbor_entry_test.go b/pkg/tcpip/stack/neighbor_entry_test.go index b769fb2fa..a265fff0a 100644 --- a/pkg/tcpip/stack/neighbor_entry_test.go +++ b/pkg/tcpip/stack/neighbor_entry_test.go @@ -27,6 +27,8 @@ import ( "github.com/google/go-cmp/cmp/cmpopts" "gvisor.dev/gvisor/pkg/sleep" "gvisor.dev/gvisor/pkg/tcpip" + "gvisor.dev/gvisor/pkg/tcpip/faketime" + "gvisor.dev/gvisor/pkg/tcpip/header" ) const ( @@ -221,8 +223,8 @@ func (r *entryTestLinkResolver) LinkAddressProtocol() tcpip.NetworkProtocolNumbe return entryTestNetNumber } -func entryTestSetup(c NUDConfigurations) (*neighborEntry, *testNUDDispatcher, *entryTestLinkResolver, *fakeClock) { - clock := newFakeClock() +func entryTestSetup(c NUDConfigurations) (*neighborEntry, *testNUDDispatcher, *entryTestLinkResolver, *faketime.ManualClock) { + clock := faketime.NewManualClock() disp := testNUDDispatcher{} nic := NIC{ id: entryTestNICID, @@ -232,18 +234,15 @@ func entryTestSetup(c NUDConfigurations) (*neighborEntry, *testNUDDispatcher, *e nudDisp: &disp, }, } + nic.networkEndpoints = map[tcpip.NetworkProtocolNumber]NetworkEndpoint{ + header.IPv6ProtocolNumber: (&testIPv6Protocol{}).NewEndpoint(&nic, nil, nil, nil), + } rng := rand.New(rand.NewSource(time.Now().UnixNano())) nudState := NewNUDState(c, rng) linkRes := entryTestLinkResolver{} entry := newNeighborEntry(&nic, entryTestAddr1 /* remoteAddr */, entryTestAddr2 /* localAddr */, nudState, &linkRes) - // Stub out ndpState to verify modification of default routers. - nic.mu.ndp = ndpState{ - nic: &nic, - defaultRouters: make(map[tcpip.Address]defaultRouterState), - } - // Stub out the neighbor cache to verify deletion from the cache. nic.neigh = &neighborCache{ nic: &nic, @@ -267,7 +266,7 @@ func TestEntryInitiallyUnknown(t *testing.T) { } e.mu.Unlock() - clock.advance(c.RetransmitTimer) + clock.Advance(c.RetransmitTimer) // No probes should have been sent. linkRes.mu.Lock() @@ -300,7 +299,7 @@ func TestEntryUnknownToUnknownWhenConfirmationWithUnknownAddress(t *testing.T) { } e.mu.Unlock() - clock.advance(time.Hour) + clock.Advance(time.Hour) // No probes should have been sent. linkRes.mu.Lock() @@ -410,7 +409,7 @@ func TestEntryIncompleteToIncompleteDoesNotChangeUpdatedAt(t *testing.T) { updatedAt := e.neigh.UpdatedAt e.mu.Unlock() - clock.advance(c.RetransmitTimer) + clock.Advance(c.RetransmitTimer) // UpdatedAt should remain the same during address resolution. wantProbes := []entryTestProbeInfo{ @@ -439,7 +438,7 @@ func TestEntryIncompleteToIncompleteDoesNotChangeUpdatedAt(t *testing.T) { } e.mu.Unlock() - clock.advance(c.RetransmitTimer) + clock.Advance(c.RetransmitTimer) // UpdatedAt should change after failing address resolution. Timing out after // sending the last probe transitions the entry to Failed. @@ -459,7 +458,7 @@ func TestEntryIncompleteToIncompleteDoesNotChangeUpdatedAt(t *testing.T) { } } - clock.advance(c.RetransmitTimer) + clock.Advance(c.RetransmitTimer) wantEvents := []testEntryEventInfo{ { @@ -748,7 +747,7 @@ func TestEntryIncompleteToFailed(t *testing.T) { e.mu.Unlock() waitFor := c.RetransmitTimer * time.Duration(c.MaxMulticastProbes) - clock.advance(waitFor) + clock.Advance(waitFor) wantProbes := []entryTestProbeInfo{ // The Incomplete-to-Incomplete state transition is tested here by @@ -816,6 +815,8 @@ func TestEntryStaysReachableWhenConfirmationWithRouterFlag(t *testing.T) { c := DefaultNUDConfigurations() e, nudDisp, linkRes, _ := entryTestSetup(c) + ipv6EP := e.nic.networkEndpoints[header.IPv6ProtocolNumber].(*testIPv6Endpoint) + e.mu.Lock() e.handlePacketQueuedLocked() e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{ @@ -829,9 +830,7 @@ func TestEntryStaysReachableWhenConfirmationWithRouterFlag(t *testing.T) { if got, want := e.isRouter, true; got != want { t.Errorf("got e.isRouter = %t, want = %t", got, want) } - e.nic.mu.ndp.defaultRouters[entryTestAddr1] = defaultRouterState{ - invalidationJob: e.nic.stack.newJob(&testLocker{}, func() {}), - } + e.handleConfirmationLocked(entryTestLinkAddr1, ReachabilityConfirmationFlags{ Solicited: false, Override: false, @@ -840,8 +839,8 @@ func TestEntryStaysReachableWhenConfirmationWithRouterFlag(t *testing.T) { if got, want := e.isRouter, false; got != want { t.Errorf("got e.isRouter = %t, want = %t", got, want) } - if _, ok := e.nic.mu.ndp.defaultRouters[entryTestAddr1]; ok { - t.Errorf("unexpected defaultRouter for %s", entryTestAddr1) + if ipv6EP.invalidatedRtr != e.neigh.Addr { + t.Errorf("got ipv6EP.invalidatedRtr = %s, want = %s", ipv6EP.invalidatedRtr, e.neigh.Addr) } e.mu.Unlock() @@ -983,7 +982,7 @@ func TestEntryReachableToStaleWhenTimeout(t *testing.T) { t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff) } - clock.advance(c.BaseReachableTime) + clock.Advance(c.BaseReachableTime) wantEvents := []testEntryEventInfo{ { @@ -1612,7 +1611,7 @@ func TestEntryDelayToReachableWhenUpperLevelConfirmation(t *testing.T) { t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff) } - clock.advance(c.BaseReachableTime) + clock.Advance(c.BaseReachableTime) wantEvents := []testEntryEventInfo{ { @@ -1706,7 +1705,7 @@ func TestEntryDelayToReachableWhenSolicitedOverrideConfirmation(t *testing.T) { t.Fatalf("link address resolver probes mismatch (-got, +want):\n%s", diff) } - clock.advance(c.BaseReachableTime) + clock.Advance(c.BaseReachableTime) wantEvents := []testEntryEventInfo{ { @@ -1989,7 +1988,7 @@ func TestEntryDelayToProbe(t *testing.T) { } e.mu.Unlock() - clock.advance(c.DelayFirstProbeTime) + clock.Advance(c.DelayFirstProbeTime) wantProbes := []entryTestProbeInfo{ // The first probe is caused by the Unknown-to-Incomplete transition. @@ -2069,7 +2068,7 @@ func TestEntryProbeToStaleWhenProbeWithDifferentAddress(t *testing.T) { e.handlePacketQueuedLocked() e.mu.Unlock() - clock.advance(c.DelayFirstProbeTime) + clock.Advance(c.DelayFirstProbeTime) wantProbes := []entryTestProbeInfo{ // The first probe is caused by the Unknown-to-Incomplete transition. @@ -2166,7 +2165,7 @@ func TestEntryProbeToStaleWhenConfirmationWithDifferentAddress(t *testing.T) { e.handlePacketQueuedLocked() e.mu.Unlock() - clock.advance(c.DelayFirstProbeTime) + clock.Advance(c.DelayFirstProbeTime) wantProbes := []entryTestProbeInfo{ // The first probe is caused by the Unknown-to-Incomplete transition. @@ -2267,7 +2266,7 @@ func TestEntryStaysProbeWhenOverrideConfirmationWithSameAddress(t *testing.T) { e.handlePacketQueuedLocked() e.mu.Unlock() - clock.advance(c.DelayFirstProbeTime) + clock.Advance(c.DelayFirstProbeTime) wantProbes := []entryTestProbeInfo{ // The first probe is caused by the Unknown-to-Incomplete transition. @@ -2364,7 +2363,7 @@ func TestEntryUnknownToStaleToProbeToReachable(t *testing.T) { e.handlePacketQueuedLocked() e.mu.Unlock() - clock.advance(c.DelayFirstProbeTime) + clock.Advance(c.DelayFirstProbeTime) wantProbes := []entryTestProbeInfo{ // Probe caused by the Delay-to-Probe transition @@ -2398,7 +2397,7 @@ func TestEntryUnknownToStaleToProbeToReachable(t *testing.T) { } e.mu.Unlock() - clock.advance(c.BaseReachableTime) + clock.Advance(c.BaseReachableTime) wantEvents := []testEntryEventInfo{ { @@ -2463,7 +2462,7 @@ func TestEntryProbeToReachableWhenSolicitedOverrideConfirmation(t *testing.T) { e.handlePacketQueuedLocked() e.mu.Unlock() - clock.advance(c.DelayFirstProbeTime) + clock.Advance(c.DelayFirstProbeTime) wantProbes := []entryTestProbeInfo{ // The first probe is caused by the Unknown-to-Incomplete transition. @@ -2503,7 +2502,7 @@ func TestEntryProbeToReachableWhenSolicitedOverrideConfirmation(t *testing.T) { } e.mu.Unlock() - clock.advance(c.BaseReachableTime) + clock.Advance(c.BaseReachableTime) wantEvents := []testEntryEventInfo{ { @@ -2575,7 +2574,7 @@ func TestEntryProbeToReachableWhenSolicitedConfirmationWithSameAddress(t *testin e.handlePacketQueuedLocked() e.mu.Unlock() - clock.advance(c.DelayFirstProbeTime) + clock.Advance(c.DelayFirstProbeTime) wantProbes := []entryTestProbeInfo{ // The first probe is caused by the Unknown-to-Incomplete transition. @@ -2612,7 +2611,7 @@ func TestEntryProbeToReachableWhenSolicitedConfirmationWithSameAddress(t *testin } e.mu.Unlock() - clock.advance(c.BaseReachableTime) + clock.Advance(c.BaseReachableTime) wantEvents := []testEntryEventInfo{ { @@ -2682,7 +2681,7 @@ func TestEntryProbeToFailed(t *testing.T) { e.mu.Unlock() waitFor := c.DelayFirstProbeTime + c.RetransmitTimer*time.Duration(c.MaxUnicastProbes) - clock.advance(waitFor) + clock.Advance(waitFor) wantProbes := []entryTestProbeInfo{ // The first probe is caused by the Unknown-to-Incomplete transition. @@ -2787,7 +2786,7 @@ func TestEntryFailedGetsDeleted(t *testing.T) { e.mu.Unlock() waitFor := c.DelayFirstProbeTime + c.RetransmitTimer*time.Duration(c.MaxUnicastProbes) + c.UnreachableTime - clock.advance(waitFor) + clock.Advance(waitFor) wantProbes := []entryTestProbeInfo{ // The first probe is caused by the Unknown-to-Incomplete transition. diff --git a/pkg/tcpip/stack/nic.go b/pkg/tcpip/stack/nic.go index 1f1a1426b..6cf54cc89 100644 --- a/pkg/tcpip/stack/nic.go +++ b/pkg/tcpip/stack/nic.go @@ -18,7 +18,6 @@ import ( "fmt" "math/rand" "reflect" - "sort" "sync/atomic" "gvisor.dev/gvisor/pkg/sleep" @@ -28,13 +27,7 @@ import ( "gvisor.dev/gvisor/pkg/tcpip/header" ) -var ipv4BroadcastAddr = tcpip.ProtocolAddress{ - Protocol: header.IPv4ProtocolNumber, - AddressWithPrefix: tcpip.AddressWithPrefix{ - Address: header.IPv4Broadcast, - PrefixLen: 8 * header.IPv4AddressSize, - }, -} +var _ NetworkInterface = (*NIC)(nil) // NIC represents a "network interface card" to which the networking stack is // attached. @@ -45,22 +38,25 @@ type NIC struct { linkEP LinkEndpoint context NICContext - stats NICStats - neigh *neighborCache + stats NICStats + neigh *neighborCache + + // The network endpoints themselves may be modified by calling the interface's + // methods, but the map reference and entries must be constant. networkEndpoints map[tcpip.NetworkProtocolNumber]NetworkEndpoint + // enabled is set to 1 when the NIC is enabled and 0 when it is disabled. + // + // Must be accessed using atomic operations. + enabled uint32 + mu struct { sync.RWMutex - enabled bool spoofing bool promiscuous bool - primary map[tcpip.NetworkProtocolNumber][]*referencedNetworkEndpoint - endpoints map[NetworkEndpointID]*referencedNetworkEndpoint - mcastJoins map[NetworkEndpointID]uint32 // packetEPs is protected by mu, but the contained PacketEndpoint // values are not. packetEPs map[tcpip.NetworkProtocolNumber][]PacketEndpoint - ndp ndpState } } @@ -84,25 +80,6 @@ type DirectionStats struct { Bytes *tcpip.StatCounter } -// PrimaryEndpointBehavior is an enumeration of an endpoint's primacy behavior. -type PrimaryEndpointBehavior int - -const ( - // CanBePrimaryEndpoint indicates the endpoint can be used as a primary - // endpoint for new connections with no local address. This is the - // default when calling NIC.AddAddress. - CanBePrimaryEndpoint PrimaryEndpointBehavior = iota - - // FirstPrimaryEndpoint indicates the endpoint should be the first - // primary endpoint considered. If there are multiple endpoints with - // this behavior, the most recently-added one will be first. - FirstPrimaryEndpoint - - // NeverPrimaryEndpoint indicates the endpoint should never be a - // primary endpoint. - NeverPrimaryEndpoint -) - // newNIC returns a new NIC using the default NDP configurations from stack. func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, ctx NICContext) *NIC { // TODO(b/141011931): Validate a LinkEndpoint (ep) is valid. For @@ -122,19 +99,7 @@ func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, ctx NICC stats: makeNICStats(), networkEndpoints: make(map[tcpip.NetworkProtocolNumber]NetworkEndpoint), } - nic.mu.primary = make(map[tcpip.NetworkProtocolNumber][]*referencedNetworkEndpoint) - nic.mu.endpoints = make(map[NetworkEndpointID]*referencedNetworkEndpoint) - nic.mu.mcastJoins = make(map[NetworkEndpointID]uint32) nic.mu.packetEPs = make(map[tcpip.NetworkProtocolNumber][]PacketEndpoint) - nic.mu.ndp = ndpState{ - nic: nic, - configs: stack.ndpConfigs, - dad: make(map[tcpip.Address]dadState), - defaultRouters: make(map[tcpip.Address]defaultRouterState), - onLinkPrefixes: make(map[tcpip.Subnet]onLinkPrefixState), - slaacPrefixes: make(map[tcpip.Subnet]slaacPrefixState), - } - nic.mu.ndp.initializeTempAddrState() // Check for Neighbor Unreachability Detection support. var nud NUDHandler @@ -162,7 +127,7 @@ func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, ctx NICC for _, netProto := range stack.networkProtocols { netNum := netProto.Number() nic.mu.packetEPs[netNum] = nil - nic.networkEndpoints[netNum] = netProto.NewEndpoint(id, stack, nud, nic, ep, stack) + nic.networkEndpoints[netNum] = netProto.NewEndpoint(nic, stack, nud, nic) } nic.linkEP.Attach(nic) @@ -170,29 +135,32 @@ func newNIC(stack *Stack, id tcpip.NICID, name string, ep LinkEndpoint, ctx NICC return nic } -// enabled returns true if n is enabled. -func (n *NIC) enabled() bool { - n.mu.RLock() - enabled := n.mu.enabled - n.mu.RUnlock() - return enabled +func (n *NIC) getNetworkEndpoint(proto tcpip.NetworkProtocolNumber) NetworkEndpoint { + return n.networkEndpoints[proto] } -// disable disables n. +// Enabled implements NetworkInterface. +func (n *NIC) Enabled() bool { + return atomic.LoadUint32(&n.enabled) == 1 +} + +// setEnabled sets the enabled status for the NIC. // -// It undoes the work done by enable. -func (n *NIC) disable() *tcpip.Error { - n.mu.RLock() - enabled := n.mu.enabled - n.mu.RUnlock() - if !enabled { - return nil +// Returns true if the enabled status was updated. +func (n *NIC) setEnabled(v bool) bool { + if v { + return atomic.SwapUint32(&n.enabled, 1) == 0 } + return atomic.SwapUint32(&n.enabled, 0) == 1 +} +// disable disables n. +// +// It undoes the work done by enable. +func (n *NIC) disable() { n.mu.Lock() - err := n.disableLocked() + n.disableLocked() n.mu.Unlock() - return err } // disableLocked disables n. @@ -200,9 +168,9 @@ func (n *NIC) disable() *tcpip.Error { // It undoes the work done by enable. // // n MUST be locked. -func (n *NIC) disableLocked() *tcpip.Error { - if !n.mu.enabled { - return nil +func (n *NIC) disableLocked() { + if !n.setEnabled(false) { + return } // TODO(gvisor.dev/issue/1491): Should Routes that are currently bound to n be @@ -210,38 +178,9 @@ func (n *NIC) disableLocked() *tcpip.Error { // again, and applications may not know that the underlying NIC was ever // disabled. - if _, ok := n.stack.networkProtocols[header.IPv6ProtocolNumber]; ok { - n.mu.ndp.stopSolicitingRouters() - n.mu.ndp.cleanupState(false /* hostOnly */) - - // Stop DAD for all the unicast IPv6 endpoints that are in the - // permanentTentative state. - for _, r := range n.mu.endpoints { - if addr := r.address(); r.getKind() == permanentTentative && header.IsV6UnicastAddress(addr) { - n.mu.ndp.stopDuplicateAddressDetection(addr) - } - } - - // The NIC may have already left the multicast group. - if err := n.leaveGroupLocked(header.IPv6AllNodesMulticastAddress, false /* force */); err != nil && err != tcpip.ErrBadLocalAddress { - return err - } - } - - if _, ok := n.stack.networkProtocols[header.IPv4ProtocolNumber]; ok { - // The NIC may have already left the multicast group. - if err := n.leaveGroupLocked(header.IPv4AllSystems, false /* force */); err != nil && err != tcpip.ErrBadLocalAddress { - return err - } - - // The address may have already been removed. - if err := n.removePermanentAddressLocked(ipv4BroadcastAddr.AddressWithPrefix.Address); err != nil && err != tcpip.ErrBadLocalAddress { - return err - } + for _, ep := range n.networkEndpoints { + ep.Disable() } - - n.mu.enabled = false - return nil } // enable enables n. @@ -251,162 +190,38 @@ func (n *NIC) disableLocked() *tcpip.Error { // routers if the stack is not operating as a router. If the stack is also // configured to auto-generate a link-local address, one will be generated. func (n *NIC) enable() *tcpip.Error { - n.mu.RLock() - enabled := n.mu.enabled - n.mu.RUnlock() - if enabled { - return nil - } - n.mu.Lock() defer n.mu.Unlock() - if n.mu.enabled { + if !n.setEnabled(true) { return nil } - n.mu.enabled = true - - // Create an endpoint to receive broadcast packets on this interface. - if _, ok := n.stack.networkProtocols[header.IPv4ProtocolNumber]; ok { - if _, err := n.addAddressLocked(ipv4BroadcastAddr, NeverPrimaryEndpoint, permanent, static, false /* deprecated */); err != nil { - return err - } - - // As per RFC 1122 section 3.3.7, all hosts should join the all-hosts - // multicast group. Note, the IANA calls the all-hosts multicast group the - // all-systems multicast group. - if err := n.joinGroupLocked(header.IPv4ProtocolNumber, header.IPv4AllSystems); err != nil { - return err - } - } - - // Join the IPv6 All-Nodes Multicast group if the stack is configured to - // use IPv6. This is required to ensure that this node properly receives - // and responds to the various NDP messages that are destined to the - // all-nodes multicast address. An example is the Neighbor Advertisement - // when we perform Duplicate Address Detection, or Router Advertisement - // when we do Router Discovery. See RFC 4862, section 5.4.2 and RFC 4861 - // section 4.2 for more information. - // - // Also auto-generate an IPv6 link-local address based on the NIC's - // link address if it is configured to do so. Note, each interface is - // required to have IPv6 link-local unicast address, as per RFC 4291 - // section 2.1. - _, ok := n.stack.networkProtocols[header.IPv6ProtocolNumber] - if !ok { - return nil - } - - // Join the All-Nodes multicast group before starting DAD as responses to DAD - // (NDP NS) messages may be sent to the All-Nodes multicast group if the - // source address of the NDP NS is the unspecified address, as per RFC 4861 - // section 7.2.4. - if err := n.joinGroupLocked(header.IPv6ProtocolNumber, header.IPv6AllNodesMulticastAddress); err != nil { - return err - } - - // Perform DAD on the all the unicast IPv6 endpoints that are in the permanent - // state. - // - // Addresses may have aleady completed DAD but in the time since the NIC was - // last enabled, other devices may have acquired the same addresses. - for _, r := range n.mu.endpoints { - addr := r.address() - if k := r.getKind(); (k != permanent && k != permanentTentative) || !header.IsV6UnicastAddress(addr) { - continue - } - - r.setKind(permanentTentative) - if err := n.mu.ndp.startDuplicateAddressDetection(addr, r); err != nil { + for _, ep := range n.networkEndpoints { + if err := ep.Enable(); err != nil { return err } } - // Do not auto-generate an IPv6 link-local address for loopback devices. - if n.stack.autoGenIPv6LinkLocal && !n.isLoopback() { - // The valid and preferred lifetime is infinite for the auto-generated - // link-local address. - n.mu.ndp.doSLAAC(header.IPv6LinkLocalPrefix.Subnet(), header.NDPInfiniteLifetime, header.NDPInfiniteLifetime) - } - - // If we are operating as a router, then do not solicit routers since we - // won't process the RAs anyways. - // - // Routers do not process Router Advertisements (RA) the same way a host - // does. That is, routers do not learn from RAs (e.g. on-link prefixes - // and default routers). Therefore, soliciting RAs from other routers on - // a link is unnecessary for routers. - if !n.stack.forwarding { - n.mu.ndp.startSolicitingRouters() - } - return nil } -// remove detaches NIC from the link endpoint, and marks existing referenced -// network endpoints expired. This guarantees no packets between this NIC and -// the network stack. +// remove detaches NIC from the link endpoint and releases network endpoint +// resources. This guarantees no packets between this NIC and the network +// stack. func (n *NIC) remove() *tcpip.Error { n.mu.Lock() defer n.mu.Unlock() n.disableLocked() - // TODO(b/151378115): come up with a better way to pick an error than the - // first one. - var err *tcpip.Error - - // Forcefully leave multicast groups. - for nid := range n.mu.mcastJoins { - if tempErr := n.leaveGroupLocked(nid.LocalAddress, true /* force */); tempErr != nil && err == nil { - err = tempErr - } - } - - // Remove permanent and permanentTentative addresses, so no packet goes out. - for nid, ref := range n.mu.endpoints { - switch ref.getKind() { - case permanentTentative, permanent: - if tempErr := n.removePermanentAddressLocked(nid.LocalAddress); tempErr != nil && err == nil { - err = tempErr - } - } - } - - // Release any resources the network endpoint may hold. for _, ep := range n.networkEndpoints { ep.Close() } // Detach from link endpoint, so no packet comes in. n.linkEP.Attach(nil) - - return err -} - -// becomeIPv6Router transitions n into an IPv6 router. -// -// When transitioning into an IPv6 router, host-only state (NDP discovered -// routers, discovered on-link prefixes, and auto-generated addresses) will -// be cleaned up/invalidated and NDP router solicitations will be stopped. -func (n *NIC) becomeIPv6Router() { - n.mu.Lock() - defer n.mu.Unlock() - - n.mu.ndp.cleanupState(true /* hostOnly */) - n.mu.ndp.stopSolicitingRouters() -} - -// becomeIPv6Host transitions n into an IPv6 host. -// -// When transitioning into an IPv6 host, NDP router solicitations will be -// started. -func (n *NIC) becomeIPv6Host() { - n.mu.Lock() - defer n.mu.Unlock() - - n.mu.ndp.startSolicitingRouters() + return nil } // setPromiscuousMode enables or disables promiscuous mode. @@ -423,7 +238,8 @@ func (n *NIC) isPromiscuousMode() bool { return rv } -func (n *NIC) isLoopback() bool { +// IsLoopback implements NetworkInterface. +func (n *NIC) IsLoopback() bool { return n.linkEP.Capabilities()&CapabilityLoopback != 0 } @@ -434,206 +250,44 @@ func (n *NIC) setSpoofing(enable bool) { n.mu.Unlock() } -// primaryEndpoint will return the first non-deprecated endpoint if such an -// endpoint exists for the given protocol and remoteAddr. If no non-deprecated -// endpoint exists, the first deprecated endpoint will be returned. -// -// If an IPv6 primary endpoint is requested, Source Address Selection (as -// defined by RFC 6724 section 5) will be performed. -func (n *NIC) primaryEndpoint(protocol tcpip.NetworkProtocolNumber, remoteAddr tcpip.Address) *referencedNetworkEndpoint { - if protocol == header.IPv6ProtocolNumber && len(remoteAddr) != 0 { - return n.primaryIPv6Endpoint(remoteAddr) - } - - n.mu.RLock() - defer n.mu.RUnlock() - - var deprecatedEndpoint *referencedNetworkEndpoint - for _, r := range n.mu.primary[protocol] { - if !r.isValidForOutgoingRLocked() { - continue - } - - if !r.deprecated { - if r.tryIncRef() { - // r is not deprecated, so return it immediately. - // - // If we kept track of a deprecated endpoint, decrement its reference - // count since it was incremented when we decided to keep track of it. - if deprecatedEndpoint != nil { - deprecatedEndpoint.decRefLocked() - deprecatedEndpoint = nil - } - - return r - } - } else if deprecatedEndpoint == nil && r.tryIncRef() { - // We prefer an endpoint that is not deprecated, but we keep track of r in - // case n doesn't have any non-deprecated endpoints. - // - // If we end up finding a more preferred endpoint, r's reference count - // will be decremented when such an endpoint is found. - deprecatedEndpoint = r - } - } - - // n doesn't have any valid non-deprecated endpoints, so return - // deprecatedEndpoint (which may be nil if n doesn't have any valid deprecated - // endpoints either). - return deprecatedEndpoint -} - -// ipv6AddrCandidate is an IPv6 candidate for Source Address Selection (RFC -// 6724 section 5). -type ipv6AddrCandidate struct { - ref *referencedNetworkEndpoint - scope header.IPv6AddressScope -} - -// primaryIPv6Endpoint returns an IPv6 endpoint following Source Address -// Selection (RFC 6724 section 5). -// -// Note, only rules 1-3 and 7 are followed. -// -// remoteAddr must be a valid IPv6 address. -func (n *NIC) primaryIPv6Endpoint(remoteAddr tcpip.Address) *referencedNetworkEndpoint { +// primaryAddress returns an address that can be used to communicate with +// remoteAddr. +func (n *NIC) primaryEndpoint(protocol tcpip.NetworkProtocolNumber, remoteAddr tcpip.Address) AssignableAddressEndpoint { n.mu.RLock() - ref := n.primaryIPv6EndpointRLocked(remoteAddr) + spoofing := n.mu.spoofing n.mu.RUnlock() - return ref -} - -// primaryIPv6EndpointLocked returns an IPv6 endpoint following Source Address -// Selection (RFC 6724 section 5). -// -// Note, only rules 1-3 and 7 are followed. -// -// remoteAddr must be a valid IPv6 address. -// -// n.mu MUST be read locked. -func (n *NIC) primaryIPv6EndpointRLocked(remoteAddr tcpip.Address) *referencedNetworkEndpoint { - primaryAddrs := n.mu.primary[header.IPv6ProtocolNumber] - - if len(primaryAddrs) == 0 { - return nil - } - - // Create a candidate set of available addresses we can potentially use as a - // source address. - cs := make([]ipv6AddrCandidate, 0, len(primaryAddrs)) - for _, r := range primaryAddrs { - // If r is not valid for outgoing connections, it is not a valid endpoint. - if !r.isValidForOutgoingRLocked() { - continue - } - - addr := r.address() - scope, err := header.ScopeForIPv6Address(addr) - if err != nil { - // Should never happen as we got r from the primary IPv6 endpoint list and - // ScopeForIPv6Address only returns an error if addr is not an IPv6 - // address. - panic(fmt.Sprintf("header.ScopeForIPv6Address(%s): %s", addr, err)) - } - - cs = append(cs, ipv6AddrCandidate{ - ref: r, - scope: scope, - }) - } - - remoteScope, err := header.ScopeForIPv6Address(remoteAddr) - if err != nil { - // primaryIPv6Endpoint should never be called with an invalid IPv6 address. - panic(fmt.Sprintf("header.ScopeForIPv6Address(%s): %s", remoteAddr, err)) - } - - // Sort the addresses as per RFC 6724 section 5 rules 1-3. - // - // TODO(b/146021396): Implement rules 4-8 of RFC 6724 section 5. - sort.Slice(cs, func(i, j int) bool { - sa := cs[i] - sb := cs[j] - - // Prefer same address as per RFC 6724 section 5 rule 1. - if sa.ref.address() == remoteAddr { - return true - } - if sb.ref.address() == remoteAddr { - return false - } - - // Prefer appropriate scope as per RFC 6724 section 5 rule 2. - if sa.scope < sb.scope { - return sa.scope >= remoteScope - } else if sb.scope < sa.scope { - return sb.scope < remoteScope - } - - // Avoid deprecated addresses as per RFC 6724 section 5 rule 3. - if saDep, sbDep := sa.ref.deprecated, sb.ref.deprecated; saDep != sbDep { - // If sa is not deprecated, it is preferred over sb. - return sbDep - } - - // Prefer temporary addresses as per RFC 6724 section 5 rule 7. - if saTemp, sbTemp := sa.ref.configType == slaacTemp, sb.ref.configType == slaacTemp; saTemp != sbTemp { - return saTemp - } - - // sa and sb are equal, return the endpoint that is closest to the front of - // the primary endpoint list. - return i < j - }) - - // Return the most preferred address that can have its reference count - // incremented. - for _, c := range cs { - if r := c.ref; r.tryIncRef() { - return r - } - } - - return nil -} - -// hasPermanentAddrLocked returns true if n has a permanent (including currently -// tentative) address, addr. -func (n *NIC) hasPermanentAddrLocked(addr tcpip.Address) bool { - ref, ok := n.mu.endpoints[NetworkEndpointID{addr}] + ep, ok := n.networkEndpoints[protocol] if !ok { - return false + return nil } - kind := ref.getKind() - - return kind == permanent || kind == permanentTentative + return ep.AcquireOutgoingPrimaryAddress(remoteAddr, spoofing) } -type getRefBehaviour int +type getAddressBehaviour int const ( // spoofing indicates that the NIC's spoofing flag should be observed when - // getting a NIC's referenced network endpoint. - spoofing getRefBehaviour = iota + // getting a NIC's address endpoint. + spoofing getAddressBehaviour = iota // promiscuous indicates that the NIC's promiscuous flag should be observed - // when getting a NIC's referenced network endpoint. + // when getting a NIC's address endpoint. promiscuous ) -func (n *NIC) getRef(protocol tcpip.NetworkProtocolNumber, dst tcpip.Address) *referencedNetworkEndpoint { - return n.getRefOrCreateTemp(protocol, dst, CanBePrimaryEndpoint, promiscuous) +func (n *NIC) getAddress(protocol tcpip.NetworkProtocolNumber, dst tcpip.Address) AssignableAddressEndpoint { + return n.getAddressOrCreateTemp(protocol, dst, CanBePrimaryEndpoint, promiscuous) } // findEndpoint finds the endpoint, if any, with the given address. -func (n *NIC) findEndpoint(protocol tcpip.NetworkProtocolNumber, address tcpip.Address, peb PrimaryEndpointBehavior) *referencedNetworkEndpoint { - return n.getRefOrCreateTemp(protocol, address, peb, spoofing) +func (n *NIC) findEndpoint(protocol tcpip.NetworkProtocolNumber, address tcpip.Address, peb PrimaryEndpointBehavior) AssignableAddressEndpoint { + return n.getAddressOrCreateTemp(protocol, address, peb, spoofing) } -// getRefEpOrCreateTemp returns the referenced network endpoint for the given -// protocol and address. +// getAddressEpOrCreateTemp returns the address endpoint for the given protocol +// and address. // // If none exists a temporary one may be created if we are in promiscuous mode // or spoofing. Promiscuous mode will only be checked if promiscuous is true. @@ -641,9 +295,8 @@ func (n *NIC) findEndpoint(protocol tcpip.NetworkProtocolNumber, address tcpip.A // // If the address is the IPv4 broadcast address for an endpoint's network, that // endpoint will be returned. -func (n *NIC) getRefOrCreateTemp(protocol tcpip.NetworkProtocolNumber, address tcpip.Address, peb PrimaryEndpointBehavior, tempRef getRefBehaviour) *referencedNetworkEndpoint { +func (n *NIC) getAddressOrCreateTemp(protocol tcpip.NetworkProtocolNumber, address tcpip.Address, peb PrimaryEndpointBehavior, tempRef getAddressBehaviour) AssignableAddressEndpoint { n.mu.RLock() - var spoofingOrPromiscuous bool switch tempRef { case spoofing: @@ -651,274 +304,54 @@ func (n *NIC) getRefOrCreateTemp(protocol tcpip.NetworkProtocolNumber, address t case promiscuous: spoofingOrPromiscuous = n.mu.promiscuous } - - if ref, ok := n.mu.endpoints[NetworkEndpointID{address}]; ok { - // An endpoint with this id exists, check if it can be used and return it. - if !ref.isAssignedRLocked(spoofingOrPromiscuous) { - n.mu.RUnlock() - return nil - } - - if ref.tryIncRef() { - n.mu.RUnlock() - return ref - } - } - - if protocol == header.IPv4ProtocolNumber { - if ref := n.getIPv4RefForBroadcastOrLoopbackRLocked(address); ref != nil { - n.mu.RUnlock() - return ref - } - } n.mu.RUnlock() - - if !spoofingOrPromiscuous { - return nil - } - - // Try again with the lock in exclusive mode. If we still can't get the - // endpoint, create a new "temporary" endpoint. It will only exist while - // there's a route through it. - n.mu.Lock() - ref := n.getRefOrCreateTempLocked(protocol, address, peb) - n.mu.Unlock() - return ref + return n.getAddressOrCreateTempInner(protocol, address, spoofingOrPromiscuous, peb) } -// getRefForBroadcastOrLoopbackRLocked returns an endpoint whose address is the -// broadcast address for the endpoint's network or an address in the endpoint's -// subnet if the NIC is a loopback interface. This matches linux behaviour. -// -// n.mu MUST be read or write locked. -func (n *NIC) getIPv4RefForBroadcastOrLoopbackRLocked(address tcpip.Address) *referencedNetworkEndpoint { - for _, ref := range n.mu.endpoints { - // Only IPv4 has a notion of broadcast addresses or considers the loopback - // interface bound to an address's whole subnet (on linux). - if ref.protocol != header.IPv4ProtocolNumber { - continue - } - - subnet := ref.addrWithPrefix().Subnet() - if (subnet.IsBroadcast(address) || (n.isLoopback() && subnet.Contains(address))) && ref.isValidForOutgoingRLocked() && ref.tryIncRef() { - return ref - } +// getAddressOrCreateTempInner is like getAddressEpOrCreateTemp except a boolean +// is passed to indicate whether or not we should generate temporary endpoints. +func (n *NIC) getAddressOrCreateTempInner(protocol tcpip.NetworkProtocolNumber, address tcpip.Address, createTemp bool, peb PrimaryEndpointBehavior) AssignableAddressEndpoint { + if ep, ok := n.networkEndpoints[protocol]; ok { + return ep.AcquireAssignedAddress(address, createTemp, peb) } return nil } -/// getRefOrCreateTempLocked returns an existing endpoint for address or creates -/// and returns a temporary endpoint. -// -// If the address is the IPv4 broadcast address for an endpoint's network, that -// endpoint will be returned. -// -// n.mu must be write locked. -func (n *NIC) getRefOrCreateTempLocked(protocol tcpip.NetworkProtocolNumber, address tcpip.Address, peb PrimaryEndpointBehavior) *referencedNetworkEndpoint { - if ref, ok := n.mu.endpoints[NetworkEndpointID{address}]; ok { - // No need to check the type as we are ok with expired endpoints at this - // point. - if ref.tryIncRef() { - return ref - } - // tryIncRef failing means the endpoint is scheduled to be removed once the - // lock is released. Remove it here so we can create a new (temporary) one. - // The removal logic waiting for the lock handles this case. - n.removeEndpointLocked(ref) - } - - if protocol == header.IPv4ProtocolNumber { - if ref := n.getIPv4RefForBroadcastOrLoopbackRLocked(address); ref != nil { - return ref - } - } - - // Add a new temporary endpoint. - netProto, ok := n.stack.networkProtocols[protocol] - if !ok { - return nil - } - ref, _ := n.addAddressLocked(tcpip.ProtocolAddress{ - Protocol: protocol, - AddressWithPrefix: tcpip.AddressWithPrefix{ - Address: address, - PrefixLen: netProto.DefaultPrefixLen(), - }, - }, peb, temporary, static, false) - return ref -} - -// addAddressLocked adds a new protocolAddress to n. -// -// If n already has the address in a non-permanent state, and the kind given is -// permanent, that address will be promoted in place and its properties set to -// the properties provided. Otherwise, it returns tcpip.ErrDuplicateAddress. -func (n *NIC) addAddressLocked(protocolAddress tcpip.ProtocolAddress, peb PrimaryEndpointBehavior, kind networkEndpointKind, configType networkEndpointConfigType, deprecated bool) (*referencedNetworkEndpoint, *tcpip.Error) { - // TODO(b/141022673): Validate IP addresses before adding them. - - // Sanity check. - id := NetworkEndpointID{LocalAddress: protocolAddress.AddressWithPrefix.Address} - if ref, ok := n.mu.endpoints[id]; ok { - // Endpoint already exists. - if kind != permanent { - return nil, tcpip.ErrDuplicateAddress - } - switch ref.getKind() { - case permanentTentative, permanent: - // The NIC already have a permanent endpoint with that address. - return nil, tcpip.ErrDuplicateAddress - case permanentExpired, temporary: - // Promote the endpoint to become permanent and respect the new peb, - // configType and deprecated status. - if ref.tryIncRef() { - // TODO(b/147748385): Perform Duplicate Address Detection when promoting - // an IPv6 endpoint to permanent. - ref.setKind(permanent) - ref.deprecated = deprecated - ref.configType = configType - - refs := n.mu.primary[ref.protocol] - for i, r := range refs { - if r == ref { - switch peb { - case CanBePrimaryEndpoint: - return ref, nil - case FirstPrimaryEndpoint: - if i == 0 { - return ref, nil - } - n.mu.primary[r.protocol] = append(refs[:i], refs[i+1:]...) - case NeverPrimaryEndpoint: - n.mu.primary[r.protocol] = append(refs[:i], refs[i+1:]...) - return ref, nil - } - } - } - - n.insertPrimaryEndpointLocked(ref, peb) - - return ref, nil - } - // tryIncRef failing means the endpoint is scheduled to be removed once - // the lock is released. Remove it here so we can create a new - // (permanent) one. The removal logic waiting for the lock handles this - // case. - n.removeEndpointLocked(ref) - } - } - +// addAddress adds a new address to n, so that it starts accepting packets +// targeted at the given address (and network protocol). +func (n *NIC) addAddress(protocolAddress tcpip.ProtocolAddress, peb PrimaryEndpointBehavior) *tcpip.Error { ep, ok := n.networkEndpoints[protocolAddress.Protocol] if !ok { - return nil, tcpip.ErrUnknownProtocol - } - - isIPv6Unicast := protocolAddress.Protocol == header.IPv6ProtocolNumber && header.IsV6UnicastAddress(protocolAddress.AddressWithPrefix.Address) - - // If the address is an IPv6 address and it is a permanent address, - // mark it as tentative so it goes through the DAD process if the NIC is - // enabled. If the NIC is not enabled, DAD will be started when the NIC is - // enabled. - if isIPv6Unicast && kind == permanent { - kind = permanentTentative - } - - ref := &referencedNetworkEndpoint{ - refs: 1, - addr: protocolAddress.AddressWithPrefix, - ep: ep, - nic: n, - protocol: protocolAddress.Protocol, - kind: kind, - configType: configType, - deprecated: deprecated, + return tcpip.ErrUnknownProtocol } - // Set up resolver if link address resolution exists for this protocol. - if n.linkEP.Capabilities()&CapabilityResolutionRequired != 0 { - if linkRes, ok := n.stack.linkAddrResolvers[protocolAddress.Protocol]; ok { - ref.linkCache = n.stack - ref.linkRes = linkRes - } - } - - // If we are adding an IPv6 unicast address, join the solicited-node - // multicast address. - if isIPv6Unicast { - snmc := header.SolicitedNodeAddr(protocolAddress.AddressWithPrefix.Address) - if err := n.joinGroupLocked(protocolAddress.Protocol, snmc); err != nil { - return nil, err - } + addressEndpoint, err := ep.AddAndAcquirePermanentAddress(protocolAddress.AddressWithPrefix, peb, AddressConfigStatic, false /* deprecated */) + if err == nil { + // We have no need for the address endpoint. + addressEndpoint.DecRef() } - - n.mu.endpoints[id] = ref - - n.insertPrimaryEndpointLocked(ref, peb) - - // If we are adding a tentative IPv6 address, start DAD if the NIC is enabled. - if isIPv6Unicast && kind == permanentTentative && n.mu.enabled { - if err := n.mu.ndp.startDuplicateAddressDetection(protocolAddress.AddressWithPrefix.Address, ref); err != nil { - return nil, err - } - } - - return ref, nil -} - -// AddAddress adds a new address to n, so that it starts accepting packets -// targeted at the given address (and network protocol). -func (n *NIC) AddAddress(protocolAddress tcpip.ProtocolAddress, peb PrimaryEndpointBehavior) *tcpip.Error { - // Add the endpoint. - n.mu.Lock() - _, err := n.addAddressLocked(protocolAddress, peb, permanent, static, false /* deprecated */) - n.mu.Unlock() - return err } -// AllAddresses returns all addresses (primary and non-primary) associated with +// allPermanentAddresses returns all permanent addresses associated with // this NIC. -func (n *NIC) AllAddresses() []tcpip.ProtocolAddress { - n.mu.RLock() - defer n.mu.RUnlock() - - addrs := make([]tcpip.ProtocolAddress, 0, len(n.mu.endpoints)) - for _, ref := range n.mu.endpoints { - // Don't include tentative, expired or temporary endpoints to - // avoid confusion and prevent the caller from using those. - switch ref.getKind() { - case permanentExpired, temporary: - continue +func (n *NIC) allPermanentAddresses() []tcpip.ProtocolAddress { + var addrs []tcpip.ProtocolAddress + for p, ep := range n.networkEndpoints { + for _, a := range ep.PermanentAddresses() { + addrs = append(addrs, tcpip.ProtocolAddress{Protocol: p, AddressWithPrefix: a}) } - - addrs = append(addrs, tcpip.ProtocolAddress{ - Protocol: ref.protocol, - AddressWithPrefix: ref.addrWithPrefix(), - }) } return addrs } -// PrimaryAddresses returns the primary addresses associated with this NIC. -func (n *NIC) PrimaryAddresses() []tcpip.ProtocolAddress { - n.mu.RLock() - defer n.mu.RUnlock() - +// primaryAddresses returns the primary addresses associated with this NIC. +func (n *NIC) primaryAddresses() []tcpip.ProtocolAddress { var addrs []tcpip.ProtocolAddress - for proto, list := range n.mu.primary { - for _, ref := range list { - // Don't include tentative, expired or tempory endpoints - // to avoid confusion and prevent the caller from using - // those. - switch ref.getKind() { - case permanentTentative, permanentExpired, temporary: - continue - } - - addrs = append(addrs, tcpip.ProtocolAddress{ - Protocol: proto, - AddressWithPrefix: ref.addrWithPrefix(), - }) + for p, ep := range n.networkEndpoints { + for _, a := range ep.PrimaryAddresses() { + addrs = append(addrs, tcpip.ProtocolAddress{Protocol: p, AddressWithPrefix: a}) } } return addrs @@ -930,147 +363,25 @@ func (n *NIC) PrimaryAddresses() []tcpip.ProtocolAddress { // address exists. If no non-deprecated address exists, the first deprecated // address will be returned. func (n *NIC) primaryAddress(proto tcpip.NetworkProtocolNumber) tcpip.AddressWithPrefix { - n.mu.RLock() - defer n.mu.RUnlock() - - list, ok := n.mu.primary[proto] + ep, ok := n.networkEndpoints[proto] if !ok { return tcpip.AddressWithPrefix{} } - var deprecatedEndpoint *referencedNetworkEndpoint - for _, ref := range list { - // Don't include tentative, expired or tempory endpoints to avoid confusion - // and prevent the caller from using those. - switch ref.getKind() { - case permanentTentative, permanentExpired, temporary: - continue - } - - if !ref.deprecated { - return ref.addrWithPrefix() - } - - if deprecatedEndpoint == nil { - deprecatedEndpoint = ref - } - } - - if deprecatedEndpoint != nil { - return deprecatedEndpoint.addrWithPrefix() - } - - return tcpip.AddressWithPrefix{} -} - -// insertPrimaryEndpointLocked adds r to n's primary endpoint list as required -// by peb. -// -// n MUST be locked. -func (n *NIC) insertPrimaryEndpointLocked(r *referencedNetworkEndpoint, peb PrimaryEndpointBehavior) { - switch peb { - case CanBePrimaryEndpoint: - n.mu.primary[r.protocol] = append(n.mu.primary[r.protocol], r) - case FirstPrimaryEndpoint: - n.mu.primary[r.protocol] = append([]*referencedNetworkEndpoint{r}, n.mu.primary[r.protocol]...) - } -} - -func (n *NIC) removeEndpointLocked(r *referencedNetworkEndpoint) { - id := NetworkEndpointID{LocalAddress: r.address()} - - // Nothing to do if the reference has already been replaced with a different - // one. This happens in the case where 1) this endpoint's ref count hit zero - // and was waiting (on the lock) to be removed and 2) the same address was - // re-added in the meantime by removing this endpoint from the list and - // adding a new one. - if n.mu.endpoints[id] != r { - return - } - - if r.getKind() == permanent { - panic("Reference count dropped to zero before being removed") - } - - delete(n.mu.endpoints, id) - refs := n.mu.primary[r.protocol] - for i, ref := range refs { - if ref == r { - n.mu.primary[r.protocol] = append(refs[:i], refs[i+1:]...) - refs[len(refs)-1] = nil - break - } - } -} - -func (n *NIC) removeEndpoint(r *referencedNetworkEndpoint) { - n.mu.Lock() - n.removeEndpointLocked(r) - n.mu.Unlock() -} - -func (n *NIC) removePermanentAddressLocked(addr tcpip.Address) *tcpip.Error { - r, ok := n.mu.endpoints[NetworkEndpointID{addr}] - if !ok { - return tcpip.ErrBadLocalAddress - } - - kind := r.getKind() - if kind != permanent && kind != permanentTentative { - return tcpip.ErrBadLocalAddress - } - - switch r.protocol { - case header.IPv6ProtocolNumber: - return n.removePermanentIPv6EndpointLocked(r, true /* allowSLAACInvalidation */) - default: - r.expireLocked() - return nil - } + return ep.MainAddress() } -func (n *NIC) removePermanentIPv6EndpointLocked(r *referencedNetworkEndpoint, allowSLAACInvalidation bool) *tcpip.Error { - addr := r.addrWithPrefix() - - isIPv6Unicast := header.IsV6UnicastAddress(addr.Address) - - if isIPv6Unicast { - n.mu.ndp.stopDuplicateAddressDetection(addr.Address) - - // If we are removing an address generated via SLAAC, cleanup - // its SLAAC resources and notify the integrator. - switch r.configType { - case slaac: - n.mu.ndp.cleanupSLAACAddrResourcesAndNotify(addr, allowSLAACInvalidation) - case slaacTemp: - n.mu.ndp.cleanupTempSLAACAddrResourcesAndNotify(addr, allowSLAACInvalidation) - } - } - - r.expireLocked() - - // At this point the endpoint is deleted. - - // If we are removing an IPv6 unicast address, leave the solicited-node - // multicast address. - // - // We ignore the tcpip.ErrBadLocalAddress error because the solicited-node - // multicast group may be left by user action. - if isIPv6Unicast { - snmc := header.SolicitedNodeAddr(addr.Address) - if err := n.leaveGroupLocked(snmc, false /* force */); err != nil && err != tcpip.ErrBadLocalAddress { +// removeAddress removes an address from n. +func (n *NIC) removeAddress(addr tcpip.Address) *tcpip.Error { + for _, ep := range n.networkEndpoints { + if err := ep.RemovePermanentAddress(addr); err == tcpip.ErrBadLocalAddress { + continue + } else { return err } } - return nil -} - -// RemoveAddress removes an address from n. -func (n *NIC) RemoveAddress(addr tcpip.Address) *tcpip.Error { - n.mu.Lock() - defer n.mu.Unlock() - return n.removePermanentAddressLocked(addr) + return tcpip.ErrBadLocalAddress } func (n *NIC) neighbors() ([]NeighborEntry, *tcpip.Error) { @@ -1121,91 +432,66 @@ func (n *NIC) clearNeighbors() *tcpip.Error { // joinGroup adds a new endpoint for the given multicast address, if none // exists yet. Otherwise it just increments its count. func (n *NIC) joinGroup(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address) *tcpip.Error { - n.mu.Lock() - defer n.mu.Unlock() - - return n.joinGroupLocked(protocol, addr) -} - -// joinGroupLocked adds a new endpoint for the given multicast address, if none -// exists yet. Otherwise it just increments its count. n MUST be locked before -// joinGroupLocked is called. -func (n *NIC) joinGroupLocked(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address) *tcpip.Error { // TODO(b/143102137): When implementing MLD, make sure MLD packets are // not sent unless a valid link-local address is available for use on n // as an MLD packet's source address must be a link-local address as // outlined in RFC 3810 section 5. - id := NetworkEndpointID{addr} - joins := n.mu.mcastJoins[id] - if joins == 0 { - netProto, ok := n.stack.networkProtocols[protocol] - if !ok { - return tcpip.ErrUnknownProtocol - } - if _, err := n.addAddressLocked(tcpip.ProtocolAddress{ - Protocol: protocol, - AddressWithPrefix: tcpip.AddressWithPrefix{ - Address: addr, - PrefixLen: netProto.DefaultPrefixLen(), - }, - }, NeverPrimaryEndpoint, permanent, static, false /* deprecated */); err != nil { - return err - } + ep, ok := n.networkEndpoints[protocol] + if !ok { + return tcpip.ErrNotSupported } - n.mu.mcastJoins[id] = joins + 1 - return nil + + gep, ok := ep.(GroupAddressableEndpoint) + if !ok { + return tcpip.ErrNotSupported + } + + _, err := gep.JoinGroup(addr) + return err } // leaveGroup decrements the count for the given multicast address, and when it // reaches zero removes the endpoint for this address. -func (n *NIC) leaveGroup(addr tcpip.Address) *tcpip.Error { - n.mu.Lock() - defer n.mu.Unlock() - - return n.leaveGroupLocked(addr, false /* force */) -} +func (n *NIC) leaveGroup(protocol tcpip.NetworkProtocolNumber, addr tcpip.Address) *tcpip.Error { + ep, ok := n.networkEndpoints[protocol] + if !ok { + return tcpip.ErrNotSupported + } -// leaveGroupLocked decrements the count for the given multicast address, and -// when it reaches zero removes the endpoint for this address. n MUST be locked -// before leaveGroupLocked is called. -// -// If force is true, then the count for the multicast addres is ignored and the -// endpoint will be removed immediately. -func (n *NIC) leaveGroupLocked(addr tcpip.Address, force bool) *tcpip.Error { - id := NetworkEndpointID{addr} - joins, ok := n.mu.mcastJoins[id] + gep, ok := ep.(GroupAddressableEndpoint) if !ok { - // There are no joins with this address on this NIC. - return tcpip.ErrBadLocalAddress + return tcpip.ErrNotSupported } - joins-- - if force || joins == 0 { - // There are no outstanding joins or we are forced to leave, clean up. - delete(n.mu.mcastJoins, id) - return n.removePermanentAddressLocked(addr) + if _, err := gep.LeaveGroup(addr); err != nil { + return err } - n.mu.mcastJoins[id] = joins return nil } // isInGroup returns true if n has joined the multicast group addr. func (n *NIC) isInGroup(addr tcpip.Address) bool { - n.mu.RLock() - joins := n.mu.mcastJoins[NetworkEndpointID{addr}] - n.mu.RUnlock() + for _, ep := range n.networkEndpoints { + gep, ok := ep.(GroupAddressableEndpoint) + if !ok { + continue + } - return joins != 0 + if gep.IsInGroup(addr) { + return true + } + } + + return false } -func handlePacket(protocol tcpip.NetworkProtocolNumber, dst, src tcpip.Address, localLinkAddr, remotelinkAddr tcpip.LinkAddress, ref *referencedNetworkEndpoint, pkt *PacketBuffer) { - r := makeRoute(protocol, dst, src, localLinkAddr, ref, false /* handleLocal */, false /* multicastLoop */) +func (n *NIC) handlePacket(protocol tcpip.NetworkProtocolNumber, dst, src tcpip.Address, remotelinkAddr tcpip.LinkAddress, addressEndpoint AssignableAddressEndpoint, pkt *PacketBuffer) { + r := makeRoute(protocol, dst, src, n, addressEndpoint, false /* handleLocal */, false /* multicastLoop */) + defer r.Release() r.RemoteLinkAddress = remotelinkAddr - - ref.ep.HandlePacket(&r, pkt) - ref.decRef() + n.getNetworkEndpoint(protocol).HandlePacket(&r, pkt) } // DeliverNetworkPacket finds the appropriate network protocol endpoint and @@ -1216,7 +502,7 @@ func handlePacket(protocol tcpip.NetworkProtocolNumber, dst, src tcpip.Address, // the ownership of the items is not retained by the caller. func (n *NIC) DeliverNetworkPacket(remote, local tcpip.LinkAddress, protocol tcpip.NetworkProtocolNumber, pkt *PacketBuffer) { n.mu.RLock() - enabled := n.mu.enabled + enabled := n.Enabled() // If the NIC is not yet enabled, don't receive any packets. if !enabled { n.mu.RUnlock() @@ -1242,9 +528,9 @@ func (n *NIC) DeliverNetworkPacket(remote, local tcpip.LinkAddress, protocol tcp local = n.linkEP.LinkAddress() } - // Are any packet sockets listening for this network protocol? + // Are any packet type sockets listening for this network protocol? packetEPs := n.mu.packetEPs[protocol] - // Add any other packet sockets that maybe listening for all protocols. + // Add any other packet type sockets that may be listening for all protocols. packetEPs = append(packetEPs, n.mu.packetEPs[header.EthernetProtocolAll]...) n.mu.RUnlock() for _, ep := range packetEPs { @@ -1265,6 +551,7 @@ func (n *NIC) DeliverNetworkPacket(remote, local tcpip.LinkAddress, protocol tcp return } if hasTransportHdr { + pkt.TransportProtocolNumber = transProtoNum // Parse the transport header if present. if state, ok := n.stack.transportProtocols[transProtoNum]; ok { state.proto.Parse(pkt) @@ -1273,29 +560,33 @@ func (n *NIC) DeliverNetworkPacket(remote, local tcpip.LinkAddress, protocol tcp src, dst := netProto.ParseAddresses(pkt.NetworkHeader().View()) - if n.stack.handleLocal && !n.isLoopback() && n.getRef(protocol, src) != nil { - // The source address is one of our own, so we never should have gotten a - // packet like this unless handleLocal is false. Loopback also calls this - // function even though the packets didn't come from the physical interface - // so don't drop those. - n.stack.stats.IP.InvalidSourceAddressesReceived.Increment() - return + if n.stack.handleLocal && !n.IsLoopback() { + if r := n.getAddress(protocol, src); r != nil { + r.DecRef() + + // The source address is one of our own, so we never should have gotten a + // packet like this unless handleLocal is false. Loopback also calls this + // function even though the packets didn't come from the physical interface + // so don't drop those. + n.stack.stats.IP.InvalidSourceAddressesReceived.Increment() + return + } } - // TODO(gvisor.dev/issue/170): Not supporting iptables for IPv6 yet. // Loopback traffic skips the prerouting chain. - if protocol == header.IPv4ProtocolNumber && !n.isLoopback() { + if !n.IsLoopback() { // iptables filtering. ipt := n.stack.IPTables() address := n.primaryAddress(protocol) if ok := ipt.Check(Prerouting, pkt, nil, nil, address.Address, ""); !ok { // iptables is telling us to drop the packet. + n.stack.stats.IP.IPTablesPreroutingDropped.Increment() return } } - if ref := n.getRef(protocol, dst); ref != nil { - handlePacket(protocol, dst, src, n.linkEP.LinkAddress(), remote, ref, pkt) + if addressEndpoint := n.getAddress(protocol, dst); addressEndpoint != nil { + n.handlePacket(protocol, dst, src, remote, addressEndpoint, pkt) return } @@ -1303,7 +594,7 @@ func (n *NIC) DeliverNetworkPacket(remote, local tcpip.LinkAddress, protocol tcp // packet and forward it to the NIC. // // TODO: Should we be forwarding the packet even if promiscuous? - if n.stack.Forwarding() { + if n.stack.Forwarding(protocol) { r, err := n.stack.FindRoute(0, "", dst, protocol, false /* multicastLoop */) if err != nil { n.stack.stats.IP.InvalidDestinationAddressesReceived.Increment() @@ -1311,25 +602,26 @@ func (n *NIC) DeliverNetworkPacket(remote, local tcpip.LinkAddress, protocol tcp } // Found a NIC. - n := r.ref.nic - n.mu.RLock() - ref, ok := n.mu.endpoints[NetworkEndpointID{dst}] - ok = ok && ref.isValidForOutgoingRLocked() && ref.tryIncRef() - n.mu.RUnlock() - if ok { - r.LocalLinkAddress = n.linkEP.LinkAddress() - r.RemoteLinkAddress = remote - r.RemoteAddress = src - // TODO(b/123449044): Update the source NIC as well. - ref.ep.HandlePacket(&r, pkt) - ref.decRef() - r.Release() - return + n := r.nic + if addressEndpoint := n.getAddressOrCreateTempInner(protocol, dst, false, NeverPrimaryEndpoint); addressEndpoint != nil { + if n.isValidForOutgoing(addressEndpoint) { + r.LocalLinkAddress = n.linkEP.LinkAddress() + r.RemoteLinkAddress = remote + r.RemoteAddress = src + // TODO(b/123449044): Update the source NIC as well. + n.getNetworkEndpoint(protocol).HandlePacket(&r, pkt) + addressEndpoint.DecRef() + r.Release() + return + } + + addressEndpoint.DecRef() } // n doesn't have a destination endpoint. // Send the packet out of n. // TODO(b/128629022): move this logic to route.WritePacket. + // TODO(gvisor.dev/issue/1085): According to the RFC, we must decrease the TTL field for ipv4/ipv6. if ch, err := r.Resolve(nil); err != nil { if err == tcpip.ErrWouldBlock { n.stack.forwarder.enqueue(ch, n, &r, protocol, pkt) @@ -1396,11 +688,11 @@ func (n *NIC) forwardPacket(r *Route, protocol tcpip.NetworkProtocolNumber, pkt // DeliverTransportPacket delivers the packets to the appropriate transport // protocol endpoint. -func (n *NIC) DeliverTransportPacket(r *Route, protocol tcpip.TransportProtocolNumber, pkt *PacketBuffer) { +func (n *NIC) DeliverTransportPacket(r *Route, protocol tcpip.TransportProtocolNumber, pkt *PacketBuffer) TransportPacketDisposition { state, ok := n.stack.transportProtocols[protocol] if !ok { n.stack.stats.UnknownProtocolRcvdPackets.Increment() - return + return TransportPacketProtocolUnreachable } transProto := state.proto @@ -1421,41 +713,47 @@ func (n *NIC) DeliverTransportPacket(r *Route, protocol tcpip.TransportProtocolN // we parse it using the minimum size. if _, ok := pkt.TransportHeader().Consume(transProto.MinimumPacketSize()); !ok { n.stack.stats.MalformedRcvdPackets.Increment() - return + // We consider a malformed transport packet handled because there is + // nothing the caller can do. + return TransportPacketHandled } - } else { - // This is either a bad packet or was re-assembled from fragments. - transProto.Parse(pkt) + } else if !transProto.Parse(pkt) { + n.stack.stats.MalformedRcvdPackets.Increment() + return TransportPacketHandled } } - if pkt.TransportHeader().View().Size() < transProto.MinimumPacketSize() { - n.stack.stats.MalformedRcvdPackets.Increment() - return - } - srcPort, dstPort, err := transProto.ParsePorts(pkt.TransportHeader().View()) if err != nil { n.stack.stats.MalformedRcvdPackets.Increment() - return + return TransportPacketHandled } id := TransportEndpointID{dstPort, r.LocalAddress, srcPort, r.RemoteAddress} if n.stack.demux.deliverPacket(r, protocol, pkt, id) { - return + return TransportPacketHandled } // Try to deliver to per-stack default handler. if state.defaultHandler != nil { if state.defaultHandler(r, id, pkt) { - return + return TransportPacketHandled } } - // We could not find an appropriate destination for this packet, so - // deliver it to the global handler. - if !transProto.HandleUnknownDestinationPacket(r, id, pkt) { + // We could not find an appropriate destination for this packet so + // give the protocol specific error handler a chance to handle it. + // If it doesn't handle it then we should do so. + switch res := transProto.HandleUnknownDestinationPacket(r, id, pkt); res { + case UnknownDestinationPacketMalformed: n.stack.stats.MalformedRcvdPackets.Increment() + return TransportPacketHandled + case UnknownDestinationPacketUnhandled: + return TransportPacketDestinationPortUnreachable + case UnknownDestinationPacketHandled: + return TransportPacketHandled + default: + panic(fmt.Sprintf("unrecognized result from HandleUnknownDestinationPacket = %d", res)) } } @@ -1488,96 +786,23 @@ func (n *NIC) DeliverTransportControlPacket(local, remote tcpip.Address, net tcp } } -// ID returns the identifier of n. +// ID implements NetworkInterface. func (n *NIC) ID() tcpip.NICID { return n.id } -// Name returns the name of n. +// Name implements NetworkInterface. func (n *NIC) Name() string { return n.name } -// Stack returns the instance of the Stack that owns this NIC. -func (n *NIC) Stack() *Stack { - return n.stack -} - -// LinkEndpoint returns the link endpoint of n. +// LinkEndpoint implements NetworkInterface. func (n *NIC) LinkEndpoint() LinkEndpoint { return n.linkEP } -// isAddrTentative returns true if addr is tentative on n. -// -// Note that if addr is not associated with n, then this function will return -// false. It will only return true if the address is associated with the NIC -// AND it is tentative. -func (n *NIC) isAddrTentative(addr tcpip.Address) bool { - n.mu.RLock() - defer n.mu.RUnlock() - - ref, ok := n.mu.endpoints[NetworkEndpointID{addr}] - if !ok { - return false - } - - return ref.getKind() == permanentTentative -} - -// dupTentativeAddrDetected attempts to inform n that a tentative addr is a -// duplicate on a link. -// -// dupTentativeAddrDetected will remove the tentative address if it exists. If -// the address was generated via SLAAC, an attempt will be made to generate a -// new address. -func (n *NIC) dupTentativeAddrDetected(addr tcpip.Address) *tcpip.Error { - n.mu.Lock() - defer n.mu.Unlock() - - ref, ok := n.mu.endpoints[NetworkEndpointID{addr}] - if !ok { - return tcpip.ErrBadAddress - } - - if ref.getKind() != permanentTentative { - return tcpip.ErrInvalidEndpointState - } - - // If the address is a SLAAC address, do not invalidate its SLAAC prefix as a - // new address will be generated for it. - if err := n.removePermanentIPv6EndpointLocked(ref, false /* allowSLAACInvalidation */); err != nil { - return err - } - - prefix := ref.addrWithPrefix().Subnet() - - switch ref.configType { - case slaac: - n.mu.ndp.regenerateSLAACAddr(prefix) - case slaacTemp: - // Do not reset the generation attempts counter for the prefix as the - // temporary address is being regenerated in response to a DAD conflict. - n.mu.ndp.regenerateTempSLAACAddr(prefix, false /* resetGenAttempts */) - } - - return nil -} - -// setNDPConfigs sets the NDP configurations for n. -// -// Note, if c contains invalid NDP configuration values, it will be fixed to -// use default values for the erroneous values. -func (n *NIC) setNDPConfigs(c NDPConfigurations) { - c.validate() - - n.mu.Lock() - n.mu.ndp.configs = c - n.mu.Unlock() -} - -// NUDConfigs gets the NUD configurations for n. -func (n *NIC) NUDConfigs() (NUDConfigurations, *tcpip.Error) { +// nudConfigs gets the NUD configurations for n. +func (n *NIC) nudConfigs() (NUDConfigurations, *tcpip.Error) { if n.neigh == nil { return NUDConfigurations{}, tcpip.ErrNotSupported } @@ -1597,49 +822,6 @@ func (n *NIC) setNUDConfigs(c NUDConfigurations) *tcpip.Error { return nil } -// handleNDPRA handles an NDP Router Advertisement message that arrived on n. -func (n *NIC) handleNDPRA(ip tcpip.Address, ra header.NDPRouterAdvert) { - n.mu.Lock() - defer n.mu.Unlock() - - n.mu.ndp.handleRA(ip, ra) -} - -type networkEndpointKind int32 - -const ( - // A permanentTentative endpoint is a permanent address that is not yet - // considered to be fully bound to an interface in the traditional - // sense. That is, the address is associated with a NIC, but packets - // destined to the address MUST NOT be accepted and MUST be silently - // dropped, and the address MUST NOT be used as a source address for - // outgoing packets. For IPv6, addresses will be of this kind until - // NDP's Duplicate Address Detection has resolved, or be deleted if - // the process results in detecting a duplicate address. - permanentTentative networkEndpointKind = iota - - // A permanent endpoint is created by adding a permanent address (vs. a - // temporary one) to the NIC. Its reference count is biased by 1 to avoid - // removal when no route holds a reference to it. It is removed by explicitly - // removing the permanent address from the NIC. - permanent - - // An expired permanent endpoint is a permanent endpoint that had its address - // removed from the NIC, and it is waiting to be removed once no more routes - // hold a reference to it. This is achieved by decreasing its reference count - // by 1. If its address is re-added before the endpoint is removed, its type - // changes back to permanent and its reference count increases by 1 again. - permanentExpired - - // A temporary endpoint is created for spoofing outgoing packets, or when in - // promiscuous mode and accepting incoming packets that don't match any - // permanent endpoint. Its reference count is not biased by 1 and the - // endpoint is removed immediately when no more route holds a reference to - // it. A temporary endpoint can be promoted to permanent if its address - // is added permanently. - temporary -) - func (n *NIC) registerPacketEndpoint(netProto tcpip.NetworkProtocolNumber, ep PacketEndpoint) *tcpip.Error { n.mu.Lock() defer n.mu.Unlock() @@ -1670,153 +852,12 @@ func (n *NIC) unregisterPacketEndpoint(netProto tcpip.NetworkProtocolNumber, ep } } -type networkEndpointConfigType int32 - -const ( - // A statically configured endpoint is an address that was added by - // some user-specified action (adding an explicit address, joining a - // multicast group). - static networkEndpointConfigType = iota - - // A SLAAC configured endpoint is an IPv6 endpoint that was added by - // SLAAC as per RFC 4862 section 5.5.3. - slaac - - // A temporary SLAAC configured endpoint is an IPv6 endpoint that was added by - // SLAAC as per RFC 4941. Temporary SLAAC addresses are short-lived and are - // not expected to be valid (or preferred) forever; hence the term temporary. - slaacTemp -) - -type referencedNetworkEndpoint struct { - ep NetworkEndpoint - addr tcpip.AddressWithPrefix - nic *NIC - protocol tcpip.NetworkProtocolNumber - - // linkCache is set if link address resolution is enabled for this - // protocol. Set to nil otherwise. - linkCache LinkAddressCache - - // linkRes is set if link address resolution is enabled for this protocol. - // Set to nil otherwise. - linkRes LinkAddressResolver - - // refs is counting references held for this endpoint. When refs hits zero it - // triggers the automatic removal of the endpoint from the NIC. - refs int32 - - // networkEndpointKind must only be accessed using {get,set}Kind(). - kind networkEndpointKind - - // configType is the method that was used to configure this endpoint. - // This must never change except during endpoint creation and promotion to - // permanent. - configType networkEndpointConfigType - - // deprecated indicates whether or not the endpoint should be considered - // deprecated. That is, when deprecated is true, other endpoints that are not - // deprecated should be preferred. - deprecated bool -} - -func (r *referencedNetworkEndpoint) address() tcpip.Address { - return r.addr.Address -} - -func (r *referencedNetworkEndpoint) addrWithPrefix() tcpip.AddressWithPrefix { - return r.addr -} - -func (r *referencedNetworkEndpoint) getKind() networkEndpointKind { - return networkEndpointKind(atomic.LoadInt32((*int32)(&r.kind))) -} - -func (r *referencedNetworkEndpoint) setKind(kind networkEndpointKind) { - atomic.StoreInt32((*int32)(&r.kind), int32(kind)) -} - // isValidForOutgoing returns true if the endpoint can be used to send out a // packet. It requires the endpoint to not be marked expired (i.e., its address) // has been removed) unless the NIC is in spoofing mode, or temporary. -func (r *referencedNetworkEndpoint) isValidForOutgoing() bool { - r.nic.mu.RLock() - defer r.nic.mu.RUnlock() - - return r.isValidForOutgoingRLocked() -} - -// isValidForOutgoingRLocked is the same as isValidForOutgoing but requires -// r.nic.mu to be read locked. -func (r *referencedNetworkEndpoint) isValidForOutgoingRLocked() bool { - if !r.nic.mu.enabled { - return false - } - - return r.isAssignedRLocked(r.nic.mu.spoofing) -} - -// isAssignedRLocked returns true if r is considered to be assigned to the NIC. -// -// r.nic.mu must be read locked. -func (r *referencedNetworkEndpoint) isAssignedRLocked(spoofingOrPromiscuous bool) bool { - switch r.getKind() { - case permanentTentative: - return false - case permanentExpired: - return spoofingOrPromiscuous - default: - return true - } -} - -// expireLocked decrements the reference count and marks the permanent endpoint -// as expired. -func (r *referencedNetworkEndpoint) expireLocked() { - r.setKind(permanentExpired) - r.decRefLocked() -} - -// decRef decrements the ref count and cleans up the endpoint once it reaches -// zero. -func (r *referencedNetworkEndpoint) decRef() { - if atomic.AddInt32(&r.refs, -1) == 0 { - r.nic.removeEndpoint(r) - } -} - -// decRefLocked is the same as decRef but assumes that the NIC.mu mutex is -// locked. -func (r *referencedNetworkEndpoint) decRefLocked() { - if atomic.AddInt32(&r.refs, -1) == 0 { - r.nic.removeEndpointLocked(r) - } -} - -// incRef increments the ref count. It must only be called when the caller is -// known to be holding a reference to the endpoint, otherwise tryIncRef should -// be used. -func (r *referencedNetworkEndpoint) incRef() { - atomic.AddInt32(&r.refs, 1) -} - -// tryIncRef attempts to increment the ref count from n to n+1, but only if n is -// not zero. That is, it will increment the count if the endpoint is still -// alive, and do nothing if it has already been clean up. -func (r *referencedNetworkEndpoint) tryIncRef() bool { - for { - v := atomic.LoadInt32(&r.refs) - if v == 0 { - return false - } - - if atomic.CompareAndSwapInt32(&r.refs, v, v+1) { - return true - } - } -} - -// stack returns the Stack instance that owns the underlying endpoint. -func (r *referencedNetworkEndpoint) stack() *Stack { - return r.nic.stack +func (n *NIC) isValidForOutgoing(ep AssignableAddressEndpoint) bool { + n.mu.RLock() + spoofing := n.mu.spoofing + n.mu.RUnlock() + return n.Enabled() && ep.IsAssigned(spoofing) } diff --git a/pkg/tcpip/stack/nic_test.go b/pkg/tcpip/stack/nic_test.go index dd6474297..fdd49b77f 100644 --- a/pkg/tcpip/stack/nic_test.go +++ b/pkg/tcpip/stack/nic_test.go @@ -15,96 +15,40 @@ package stack import ( - "math" "testing" - "time" "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/header" ) -var _ LinkEndpoint = (*testLinkEndpoint)(nil) +var _ AddressableEndpoint = (*testIPv6Endpoint)(nil) +var _ NetworkEndpoint = (*testIPv6Endpoint)(nil) +var _ NDPEndpoint = (*testIPv6Endpoint)(nil) -// A LinkEndpoint that throws away outgoing packets. +// An IPv6 NetworkEndpoint that throws away outgoing packets. // -// We use this instead of the channel endpoint as the channel package depends on +// We use this instead of ipv6.endpoint because the ipv6 package depends on // the stack package which this test lives in, causing a cyclic dependency. -type testLinkEndpoint struct { - dispatcher NetworkDispatcher -} - -// Attach implements LinkEndpoint.Attach. -func (e *testLinkEndpoint) Attach(dispatcher NetworkDispatcher) { - e.dispatcher = dispatcher -} - -// IsAttached implements LinkEndpoint.IsAttached. -func (e *testLinkEndpoint) IsAttached() bool { - return e.dispatcher != nil -} - -// MTU implements LinkEndpoint.MTU. -func (*testLinkEndpoint) MTU() uint32 { - return math.MaxUint16 -} - -// Capabilities implements LinkEndpoint.Capabilities. -func (*testLinkEndpoint) Capabilities() LinkEndpointCapabilities { - return CapabilityResolutionRequired -} +type testIPv6Endpoint struct { + AddressableEndpointState -// MaxHeaderLength implements LinkEndpoint.MaxHeaderLength. -func (*testLinkEndpoint) MaxHeaderLength() uint16 { - return 0 -} + nicID tcpip.NICID + linkEP LinkEndpoint + protocol *testIPv6Protocol -// LinkAddress returns the link address of this endpoint. -func (*testLinkEndpoint) LinkAddress() tcpip.LinkAddress { - return "" + invalidatedRtr tcpip.Address } -// Wait implements LinkEndpoint.Wait. -func (*testLinkEndpoint) Wait() {} - -// WritePacket implements LinkEndpoint.WritePacket. -func (e *testLinkEndpoint) WritePacket(*Route, *GSO, tcpip.NetworkProtocolNumber, *PacketBuffer) *tcpip.Error { +func (*testIPv6Endpoint) Enable() *tcpip.Error { return nil } -// WritePackets implements LinkEndpoint.WritePackets. -func (e *testLinkEndpoint) WritePackets(*Route, *GSO, PacketBufferList, tcpip.NetworkProtocolNumber) (int, *tcpip.Error) { - // Our tests don't use this so we don't support it. - return 0, tcpip.ErrNotSupported -} - -// WriteRawPacket implements LinkEndpoint.WriteRawPacket. -func (e *testLinkEndpoint) WriteRawPacket(buffer.VectorisedView) *tcpip.Error { - // Our tests don't use this so we don't support it. - return tcpip.ErrNotSupported -} - -// ARPHardwareType implements stack.LinkEndpoint.ARPHardwareType. -func (*testLinkEndpoint) ARPHardwareType() header.ARPHardwareType { - panic("not implemented") -} - -// AddHeader implements stack.LinkEndpoint.AddHeader. -func (e *testLinkEndpoint) AddHeader(local, remote tcpip.LinkAddress, protocol tcpip.NetworkProtocolNumber, pkt *PacketBuffer) { - panic("not implemented") +func (*testIPv6Endpoint) Enabled() bool { + return true } -var _ NetworkEndpoint = (*testIPv6Endpoint)(nil) - -// An IPv6 NetworkEndpoint that throws away outgoing packets. -// -// We use this instead of ipv6.endpoint because the ipv6 package depends on -// the stack package which this test lives in, causing a cyclic dependency. -type testIPv6Endpoint struct { - nicID tcpip.NICID - linkEP LinkEndpoint - protocol *testIPv6Protocol -} +func (*testIPv6Endpoint) Disable() {} // DefaultTTL implements NetworkEndpoint.DefaultTTL. func (*testIPv6Endpoint) DefaultTTL() uint8 { @@ -116,11 +60,6 @@ func (e *testIPv6Endpoint) MTU() uint32 { return e.linkEP.MTU() - header.IPv6MinimumSize } -// Capabilities implements NetworkEndpoint.Capabilities. -func (e *testIPv6Endpoint) Capabilities() LinkEndpointCapabilities { - return e.linkEP.Capabilities() -} - // MaxHeaderLength implements NetworkEndpoint.MaxHeaderLength. func (e *testIPv6Endpoint) MaxHeaderLength() uint16 { return e.linkEP.MaxHeaderLength() + header.IPv6MinimumSize @@ -144,23 +83,24 @@ func (*testIPv6Endpoint) WriteHeaderIncludedPacket(*Route, *PacketBuffer) *tcpip return tcpip.ErrNotSupported } -// NICID implements NetworkEndpoint.NICID. -func (e *testIPv6Endpoint) NICID() tcpip.NICID { - return e.nicID -} - // HandlePacket implements NetworkEndpoint.HandlePacket. func (*testIPv6Endpoint) HandlePacket(*Route, *PacketBuffer) { } // Close implements NetworkEndpoint.Close. -func (*testIPv6Endpoint) Close() {} +func (e *testIPv6Endpoint) Close() { + e.AddressableEndpointState.Cleanup() +} // NetworkProtocolNumber implements NetworkEndpoint.NetworkProtocolNumber. func (*testIPv6Endpoint) NetworkProtocolNumber() tcpip.NetworkProtocolNumber { return header.IPv6ProtocolNumber } +func (e *testIPv6Endpoint) InvalidateDefaultRouter(rtr tcpip.Address) { + e.invalidatedRtr = rtr +} + var _ NetworkProtocol = (*testIPv6Protocol)(nil) // An IPv6 NetworkProtocol that supports the bare minimum to make a stack @@ -192,12 +132,14 @@ func (*testIPv6Protocol) ParseAddresses(v buffer.View) (src, dst tcpip.Address) } // NewEndpoint implements NetworkProtocol.NewEndpoint. -func (p *testIPv6Protocol) NewEndpoint(nicID tcpip.NICID, _ LinkAddressCache, _ NUDHandler, _ TransportDispatcher, linkEP LinkEndpoint, _ *Stack) NetworkEndpoint { - return &testIPv6Endpoint{ - nicID: nicID, - linkEP: linkEP, +func (p *testIPv6Protocol) NewEndpoint(nic NetworkInterface, _ LinkAddressCache, _ NUDHandler, _ TransportDispatcher) NetworkEndpoint { + e := &testIPv6Endpoint{ + nicID: nic.ID(), + linkEP: nic.LinkEndpoint(), protocol: p, } + e.AddressableEndpointState.Init(e) + return e } // SetOption implements NetworkProtocol.SetOption. @@ -241,38 +183,6 @@ func (*testIPv6Protocol) ResolveStaticAddress(addr tcpip.Address) (tcpip.LinkAdd return "", false } -// Test the race condition where a NIC is removed and an RS timer fires at the -// same time. -func TestRemoveNICWhileHandlingRSTimer(t *testing.T) { - const ( - nicID = 1 - - maxRtrSolicitations = 5 - ) - - e := testLinkEndpoint{} - s := New(Options{ - NetworkProtocols: []NetworkProtocol{&testIPv6Protocol{}}, - NDPConfigs: NDPConfigurations{ - MaxRtrSolicitations: maxRtrSolicitations, - RtrSolicitationInterval: minimumRtrSolicitationInterval, - }, - }) - - if err := s.CreateNIC(nicID, &e); err != nil { - t.Fatalf("s.CreateNIC(%d, _) = %s", nicID, err) - } - - s.mu.Lock() - // Wait for the router solicitation timer to fire and block trying to obtain - // the stack lock when doing link address resolution. - time.Sleep(minimumRtrSolicitationInterval * 2) - if err := s.removeNICLocked(nicID); err != nil { - t.Fatalf("s.removeNICLocked(%d) = %s", nicID, err) - } - s.mu.Unlock() -} - func TestDisabledRxStatsWhenNICDisabled(t *testing.T) { // When the NIC is disabled, the only field that matters is the stats field. // This test is limited to stats counter checks. diff --git a/pkg/tcpip/stack/nud_test.go b/pkg/tcpip/stack/nud_test.go index 2b97e5972..8cffb9fc6 100644 --- a/pkg/tcpip/stack/nud_test.go +++ b/pkg/tcpip/stack/nud_test.go @@ -60,7 +60,7 @@ func TestSetNUDConfigurationFailsForBadNICID(t *testing.T) { // A neighbor cache is required to store NUDConfigurations. The networking // stack will only allocate neighbor caches if a protocol providing link // address resolution is specified (e.g. ARP or IPv6). - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocol}, UseNeighborCache: true, }) @@ -137,7 +137,7 @@ func TestDefaultNUDConfigurations(t *testing.T) { // A neighbor cache is required to store NUDConfigurations. The networking // stack will only allocate neighbor caches if a protocol providing link // address resolution is specified (e.g. ARP or IPv6). - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocol}, NUDConfigs: stack.DefaultNUDConfigurations(), UseNeighborCache: true, }) @@ -192,7 +192,7 @@ func TestNUDConfigurationsBaseReachableTime(t *testing.T) { // A neighbor cache is required to store NUDConfigurations. The // networking stack will only allocate neighbor caches if a protocol // providing link address resolution is specified (e.g. ARP or IPv6). - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocol}, NUDConfigs: c, UseNeighborCache: true, }) @@ -249,7 +249,7 @@ func TestNUDConfigurationsMinRandomFactor(t *testing.T) { // A neighbor cache is required to store NUDConfigurations. The // networking stack will only allocate neighbor caches if a protocol // providing link address resolution is specified (e.g. ARP or IPv6). - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocol}, NUDConfigs: c, UseNeighborCache: true, }) @@ -329,7 +329,7 @@ func TestNUDConfigurationsMaxRandomFactor(t *testing.T) { // A neighbor cache is required to store NUDConfigurations. The // networking stack will only allocate neighbor caches if a protocol // providing link address resolution is specified (e.g. ARP or IPv6). - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocol}, NUDConfigs: c, UseNeighborCache: true, }) @@ -391,7 +391,7 @@ func TestNUDConfigurationsRetransmitTimer(t *testing.T) { // A neighbor cache is required to store NUDConfigurations. The // networking stack will only allocate neighbor caches if a protocol // providing link address resolution is specified (e.g. ARP or IPv6). - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocol}, NUDConfigs: c, UseNeighborCache: true, }) @@ -443,7 +443,7 @@ func TestNUDConfigurationsDelayFirstProbeTime(t *testing.T) { // A neighbor cache is required to store NUDConfigurations. The // networking stack will only allocate neighbor caches if a protocol // providing link address resolution is specified (e.g. ARP or IPv6). - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocol}, NUDConfigs: c, UseNeighborCache: true, }) @@ -495,7 +495,7 @@ func TestNUDConfigurationsMaxMulticastProbes(t *testing.T) { // A neighbor cache is required to store NUDConfigurations. The // networking stack will only allocate neighbor caches if a protocol // providing link address resolution is specified (e.g. ARP or IPv6). - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocol}, NUDConfigs: c, UseNeighborCache: true, }) @@ -547,7 +547,7 @@ func TestNUDConfigurationsMaxUnicastProbes(t *testing.T) { // A neighbor cache is required to store NUDConfigurations. The // networking stack will only allocate neighbor caches if a protocol // providing link address resolution is specified (e.g. ARP or IPv6). - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocol}, NUDConfigs: c, UseNeighborCache: true, }) @@ -599,7 +599,7 @@ func TestNUDConfigurationsUnreachableTime(t *testing.T) { // A neighbor cache is required to store NUDConfigurations. The // networking stack will only allocate neighbor caches if a protocol // providing link address resolution is specified (e.g. ARP or IPv6). - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocol}, NUDConfigs: c, UseNeighborCache: true, }) diff --git a/pkg/tcpip/stack/packet_buffer.go b/pkg/tcpip/stack/packet_buffer.go index 17b8beebb..105583c49 100644 --- a/pkg/tcpip/stack/packet_buffer.go +++ b/pkg/tcpip/stack/packet_buffer.go @@ -19,6 +19,7 @@ import ( "gvisor.dev/gvisor/pkg/sync" "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" + "gvisor.dev/gvisor/pkg/tcpip/header" ) type headerType int @@ -80,11 +81,17 @@ type PacketBuffer struct { // data are held in the same underlying buffer storage. header buffer.Prependable - // NetworkProtocol is only valid when NetworkHeader is set. + // NetworkProtocolNumber is only valid when NetworkHeader().View().IsEmpty() + // returns false. // TODO(gvisor.dev/issue/3574): Remove the separately passed protocol // numbers in registration APIs that take a PacketBuffer. NetworkProtocolNumber tcpip.NetworkProtocolNumber + // TransportProtocol is only valid if it is non zero. + // TODO(gvisor.dev/issue/3810): This and the network protocol number should + // be moved into the headerinfo. This should resolve the validity issue. + TransportProtocolNumber tcpip.TransportProtocolNumber + // Hash is the transport layer hash of this packet. A value of zero // indicates no valid hash has been set. Hash uint32 @@ -234,20 +241,35 @@ func (pk *PacketBuffer) consume(typ headerType, size int) (v buffer.View, consum // underlying packet payload. func (pk *PacketBuffer) Clone() *PacketBuffer { newPk := &PacketBuffer{ - PacketBufferEntry: pk.PacketBufferEntry, - Data: pk.Data.Clone(nil), - headers: pk.headers, - header: pk.header, - Hash: pk.Hash, - Owner: pk.Owner, - EgressRoute: pk.EgressRoute, - GSOOptions: pk.GSOOptions, - NetworkProtocolNumber: pk.NetworkProtocolNumber, - NatDone: pk.NatDone, + PacketBufferEntry: pk.PacketBufferEntry, + Data: pk.Data.Clone(nil), + headers: pk.headers, + header: pk.header, + Hash: pk.Hash, + Owner: pk.Owner, + EgressRoute: pk.EgressRoute, + GSOOptions: pk.GSOOptions, + NetworkProtocolNumber: pk.NetworkProtocolNumber, + NatDone: pk.NatDone, + TransportProtocolNumber: pk.TransportProtocolNumber, } return newPk } +// Network returns the network header as a header.Network. +// +// Network should only be called when NetworkHeader has been set. +func (pk *PacketBuffer) Network() header.Network { + switch netProto := pk.NetworkProtocolNumber; netProto { + case header.IPv4ProtocolNumber: + return header.IPv4(pk.NetworkHeader().View()) + case header.IPv6ProtocolNumber: + return header.IPv6(pk.NetworkHeader().View()) + default: + panic(fmt.Sprintf("unknown network protocol number %d", netProto)) + } +} + // headerInfo stores metadata about a header in a packet. type headerInfo struct { // buf is the memorized slice for both prepended and consumed header. diff --git a/pkg/tcpip/stack/registration.go b/pkg/tcpip/stack/registration.go index 4fa86a3ac..be9bd8042 100644 --- a/pkg/tcpip/stack/registration.go +++ b/pkg/tcpip/stack/registration.go @@ -15,6 +15,8 @@ package stack import ( + "fmt" + "gvisor.dev/gvisor/pkg/sleep" "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" @@ -125,6 +127,26 @@ type PacketEndpoint interface { HandlePacket(nicID tcpip.NICID, addr tcpip.LinkAddress, netProto tcpip.NetworkProtocolNumber, pkt *PacketBuffer) } +// UnknownDestinationPacketDisposition enumerates the possible return vaues from +// HandleUnknownDestinationPacket(). +type UnknownDestinationPacketDisposition int + +const ( + // UnknownDestinationPacketMalformed denotes that the packet was malformed + // and no further processing should be attempted other than updating + // statistics. + UnknownDestinationPacketMalformed UnknownDestinationPacketDisposition = iota + + // UnknownDestinationPacketUnhandled tells the caller that the packet was + // well formed but that the issue was not handled and the stack should take + // the default action. + UnknownDestinationPacketUnhandled + + // UnknownDestinationPacketHandled tells the caller that it should do + // no further processing. + UnknownDestinationPacketHandled +) + // TransportProtocol is the interface that needs to be implemented by transport // protocols (e.g., tcp, udp) that want to be part of the networking stack. type TransportProtocol interface { @@ -132,10 +154,10 @@ type TransportProtocol interface { Number() tcpip.TransportProtocolNumber // NewEndpoint creates a new endpoint of the transport protocol. - NewEndpoint(stack *Stack, netProto tcpip.NetworkProtocolNumber, waitQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) + NewEndpoint(netProto tcpip.NetworkProtocolNumber, waitQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) // NewRawEndpoint creates a new raw endpoint of the transport protocol. - NewRawEndpoint(stack *Stack, netProto tcpip.NetworkProtocolNumber, waitQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) + NewRawEndpoint(netProto tcpip.NetworkProtocolNumber, waitQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) // MinimumPacketSize returns the minimum valid packet size of this // transport protocol. The stack automatically drops any packets smaller @@ -147,14 +169,12 @@ type TransportProtocol interface { ParsePorts(v buffer.View) (src, dst uint16, err *tcpip.Error) // HandleUnknownDestinationPacket handles packets targeted at this - // protocol but that don't match any existing endpoint. For example, - // it is targeted at a port that have no listeners. - // - // The return value indicates whether the packet was well-formed (for - // stats purposes only). + // protocol that don't match any existing endpoint. For example, + // it is targeted at a port that has no listeners. // - // HandleUnknownDestinationPacket takes ownership of pkt. - HandleUnknownDestinationPacket(r *Route, id TransportEndpointID, pkt *PacketBuffer) bool + // HandleUnknownDestinationPacket takes ownership of pkt if it handles + // the issue. + HandleUnknownDestinationPacket(r *Route, id TransportEndpointID, pkt *PacketBuffer) UnknownDestinationPacketDisposition // SetOption allows enabling/disabling protocol specific features. // SetOption returns an error if the option is not supported or the @@ -179,6 +199,25 @@ type TransportProtocol interface { Parse(pkt *PacketBuffer) (ok bool) } +// TransportPacketDisposition is the result from attempting to deliver a packet +// to the transport layer. +type TransportPacketDisposition int + +const ( + // TransportPacketHandled indicates that a transport packet was handled by the + // transport layer and callers need not take any further action. + TransportPacketHandled TransportPacketDisposition = iota + + // TransportPacketProtocolUnreachable indicates that the transport + // protocol requested in the packet is not supported. + TransportPacketProtocolUnreachable + + // TransportPacketDestinationPortUnreachable indicates that there weren't any + // listeners interested in the packet and the transport protocol has no means + // to notify the sender. + TransportPacketDestinationPortUnreachable +) + // TransportDispatcher contains the methods used by the network stack to deliver // packets to the appropriate transport endpoint after it has been handled by // the network layer. @@ -189,7 +228,7 @@ type TransportDispatcher interface { // pkt.NetworkHeader must be set before calling DeliverTransportPacket. // // DeliverTransportPacket takes ownership of pkt. - DeliverTransportPacket(r *Route, protocol tcpip.TransportProtocolNumber, pkt *PacketBuffer) + DeliverTransportPacket(r *Route, protocol tcpip.TransportProtocolNumber, pkt *PacketBuffer) TransportPacketDisposition // DeliverTransportControlPacket delivers control packets to the // appropriate transport protocol endpoint. @@ -226,9 +265,253 @@ type NetworkHeaderParams struct { TOS uint8 } +// GroupAddressableEndpoint is an endpoint that supports group addressing. +// +// An endpoint is considered to support group addressing when one or more +// endpoints may associate themselves with the same identifier (group address). +type GroupAddressableEndpoint interface { + // JoinGroup joins the spcified group. + // + // Returns true if the group was newly joined. + JoinGroup(group tcpip.Address) (bool, *tcpip.Error) + + // LeaveGroup attempts to leave the specified group. + // + // Returns tcpip.ErrBadLocalAddress if the endpoint has not joined the group. + LeaveGroup(group tcpip.Address) (bool, *tcpip.Error) + + // IsInGroup returns true if the endpoint is a member of the specified group. + IsInGroup(group tcpip.Address) bool +} + +// PrimaryEndpointBehavior is an enumeration of an AddressEndpoint's primary +// behavior. +type PrimaryEndpointBehavior int + +const ( + // CanBePrimaryEndpoint indicates the endpoint can be used as a primary + // endpoint for new connections with no local address. This is the + // default when calling NIC.AddAddress. + CanBePrimaryEndpoint PrimaryEndpointBehavior = iota + + // FirstPrimaryEndpoint indicates the endpoint should be the first + // primary endpoint considered. If there are multiple endpoints with + // this behavior, they are ordered by recency. + FirstPrimaryEndpoint + + // NeverPrimaryEndpoint indicates the endpoint should never be a + // primary endpoint. + NeverPrimaryEndpoint +) + +// AddressConfigType is the method used to add an address. +type AddressConfigType int + +const ( + // AddressConfigStatic is a statically configured address endpoint that was + // added by some user-specified action (adding an explicit address, joining a + // multicast group). + AddressConfigStatic AddressConfigType = iota + + // AddressConfigSlaac is an address endpoint added by SLAAC, as per RFC 4862 + // section 5.5.3. + AddressConfigSlaac + + // AddressConfigSlaacTemp is a temporary address endpoint added by SLAAC as + // per RFC 4941. Temporary SLAAC addresses are short-lived and are not + // to be valid (or preferred) forever; hence the term temporary. + AddressConfigSlaacTemp +) + +// AssignableAddressEndpoint is a reference counted address endpoint that may be +// assigned to a NetworkEndpoint. +type AssignableAddressEndpoint interface { + // AddressWithPrefix returns the endpoint's address. + AddressWithPrefix() tcpip.AddressWithPrefix + + // IsAssigned returns whether or not the endpoint is considered bound + // to its NetworkEndpoint. + IsAssigned(allowExpired bool) bool + + // IncRef increments this endpoint's reference count. + // + // Returns true if it was successfully incremented. If it returns false, then + // the endpoint is considered expired and should no longer be used. + IncRef() bool + + // DecRef decrements this endpoint's reference count. + DecRef() +} + +// AddressEndpoint is an endpoint representing an address assigned to an +// AddressableEndpoint. +type AddressEndpoint interface { + AssignableAddressEndpoint + + // GetKind returns the address kind for this endpoint. + GetKind() AddressKind + + // SetKind sets the address kind for this endpoint. + SetKind(AddressKind) + + // ConfigType returns the method used to add the address. + ConfigType() AddressConfigType + + // Deprecated returns whether or not this endpoint is deprecated. + Deprecated() bool + + // SetDeprecated sets this endpoint's deprecated status. + SetDeprecated(bool) +} + +// AddressKind is the kind of of an address. +// +// See the values of AddressKind for more details. +type AddressKind int + +const ( + // PermanentTentative is a permanent address endpoint that is not yet + // considered to be fully bound to an interface in the traditional + // sense. That is, the address is associated with a NIC, but packets + // destined to the address MUST NOT be accepted and MUST be silently + // dropped, and the address MUST NOT be used as a source address for + // outgoing packets. For IPv6, addresses are of this kind until NDP's + // Duplicate Address Detection (DAD) resolves. If DAD fails, the address + // is removed. + PermanentTentative AddressKind = iota + + // Permanent is a permanent endpoint (vs. a temporary one) assigned to the + // NIC. Its reference count is biased by 1 to avoid removal when no route + // holds a reference to it. It is removed by explicitly removing the address + // from the NIC. + Permanent + + // PermanentExpired is a permanent endpoint that had its address removed from + // the NIC, and it is waiting to be removed once no references to it are held. + // + // If the address is re-added before the endpoint is removed, its type + // changes back to Permanent. + PermanentExpired + + // Temporary is an endpoint, created on a one-off basis to temporarily + // consider the NIC bound an an address that it is not explictiy bound to + // (such as a permanent address). Its reference count must not be biased by 1 + // so that the address is removed immediately when references to it are no + // longer held. + // + // A temporary endpoint may be promoted to permanent if the address is added + // permanently. + Temporary +) + +// IsPermanent returns true if the AddressKind represents a permanent address. +func (k AddressKind) IsPermanent() bool { + switch k { + case Permanent, PermanentTentative: + return true + case Temporary, PermanentExpired: + return false + default: + panic(fmt.Sprintf("unrecognized address kind = %d", k)) + } +} + +// AddressableEndpoint is an endpoint that supports addressing. +// +// An endpoint is considered to support addressing when the endpoint may +// associate itself with an identifier (address). +type AddressableEndpoint interface { + // AddAndAcquirePermanentAddress adds the passed permanent address. + // + // Returns tcpip.ErrDuplicateAddress if the address exists. + // + // Acquires and returns the AddressEndpoint for the added address. + AddAndAcquirePermanentAddress(addr tcpip.AddressWithPrefix, peb PrimaryEndpointBehavior, configType AddressConfigType, deprecated bool) (AddressEndpoint, *tcpip.Error) + + // RemovePermanentAddress removes the passed address if it is a permanent + // address. + // + // Returns tcpip.ErrBadLocalAddress if the endpoint does not have the passed + // permanent address. + RemovePermanentAddress(addr tcpip.Address) *tcpip.Error + + // MainAddress returns the endpoint's primary permanent address. + MainAddress() tcpip.AddressWithPrefix + + // AcquireAssignedAddress returns an address endpoint for the passed address + // that is considered bound to the endpoint, optionally creating a temporary + // endpoint if requested and no existing address exists. + // + // The returned endpoint's reference count is incremented. + // + // Returns nil if the specified address is not local to this endpoint. + AcquireAssignedAddress(localAddr tcpip.Address, allowTemp bool, tempPEB PrimaryEndpointBehavior) AddressEndpoint + + // AcquireOutgoingPrimaryAddress returns a primary address that may be used as + // a source address when sending packets to the passed remote address. + // + // If allowExpired is true, expired addresses may be returned. + // + // The returned endpoint's reference count is incremented. + // + // Returns nil if a primary address is not available. + AcquireOutgoingPrimaryAddress(remoteAddr tcpip.Address, allowExpired bool) AddressEndpoint + + // PrimaryAddresses returns the primary addresses. + PrimaryAddresses() []tcpip.AddressWithPrefix + + // PermanentAddresses returns all the permanent addresses. + PermanentAddresses() []tcpip.AddressWithPrefix +} + +// NDPEndpoint is a network endpoint that supports NDP. +type NDPEndpoint interface { + NetworkEndpoint + + // InvalidateDefaultRouter invalidates a default router discovered through + // NDP. + InvalidateDefaultRouter(tcpip.Address) +} + +// NetworkInterface is a network interface. +type NetworkInterface interface { + // ID returns the interface's ID. + ID() tcpip.NICID + + // IsLoopback returns true if the interface is a loopback interface. + IsLoopback() bool + + // Name returns the name of the interface. + // + // May return an empty string if the interface is not configured with a name. + Name() string + + // Enabled returns true if the interface is enabled. + Enabled() bool + + // LinkEndpoint returns the link endpoint backing the interface. + LinkEndpoint() LinkEndpoint +} + // NetworkEndpoint is the interface that needs to be implemented by endpoints // of network layer protocols (e.g., ipv4, ipv6). type NetworkEndpoint interface { + AddressableEndpoint + + // Enable enables the endpoint. + // + // Must only be called when the stack is in a state that allows the endpoint + // to send and receive packets. + // + // Returns tcpip.ErrNotPermitted if the endpoint cannot be enabled. + Enable() *tcpip.Error + + // Enabled returns true if the endpoint is enabled. + Enabled() bool + + // Disable disables the endpoint. + Disable() + // DefaultTTL is the default time-to-live value (or hop limit, in ipv6) // for this endpoint. DefaultTTL() uint8 @@ -238,10 +521,6 @@ type NetworkEndpoint interface { // minus the network endpoint max header length. MTU() uint32 - // Capabilities returns the set of capabilities supported by the - // underlying link-layer endpoint. - Capabilities() LinkEndpointCapabilities - // MaxHeaderLength returns the maximum size the network (and lower // level layers combined) headers can have. Higher levels use this // information to reserve space in the front of the packets they're @@ -262,9 +541,6 @@ type NetworkEndpoint interface { // header to the given destination address. It takes ownership of pkt. WriteHeaderIncludedPacket(r *Route, pkt *PacketBuffer) *tcpip.Error - // NICID returns the id of the NIC this endpoint belongs to. - NICID() tcpip.NICID - // HandlePacket is called by the link layer when new packets arrive to // this network endpoint. It sets pkt.NetworkHeader. // @@ -279,6 +555,17 @@ type NetworkEndpoint interface { NetworkProtocolNumber() tcpip.NetworkProtocolNumber } +// ForwardingNetworkProtocol is a NetworkProtocol that may forward packets. +type ForwardingNetworkProtocol interface { + NetworkProtocol + + // Forwarding returns the forwarding configuration. + Forwarding() bool + + // SetForwarding sets the forwarding configuration. + SetForwarding(bool) +} + // NetworkProtocol is the interface that needs to be implemented by network // protocols (e.g., ipv4, ipv6) that want to be part of the networking stack. type NetworkProtocol interface { @@ -298,7 +585,7 @@ type NetworkProtocol interface { ParseAddresses(v buffer.View) (src, dst tcpip.Address) // NewEndpoint creates a new endpoint of this protocol. - NewEndpoint(nicID tcpip.NICID, linkAddrCache LinkAddressCache, nud NUDHandler, dispatcher TransportDispatcher, sender LinkEndpoint, st *Stack) NetworkEndpoint + NewEndpoint(nic NetworkInterface, linkAddrCache LinkAddressCache, nud NUDHandler, dispatcher TransportDispatcher) NetworkEndpoint // SetOption allows enabling/disabling protocol specific features. // SetOption returns an error if the option is not supported or the @@ -427,8 +714,8 @@ type LinkEndpoint interface { // Attach attaches the data link layer endpoint to the network-layer // dispatcher of the stack. // - // Attach will be called with a nil dispatcher if the receiver's associated - // NIC is being removed. + // Attach is called with a nil dispatcher when the endpoint's NIC is being + // removed. Attach(dispatcher NetworkDispatcher) // IsAttached returns whether a NetworkDispatcher is attached to the diff --git a/pkg/tcpip/stack/route.go b/pkg/tcpip/stack/route.go index 2cbbf0de8..cc39c9a6a 100644 --- a/pkg/tcpip/stack/route.go +++ b/pkg/tcpip/stack/route.go @@ -42,17 +42,27 @@ type Route struct { // NetProto is the network-layer protocol. NetProto tcpip.NetworkProtocolNumber - // ref a reference to the network endpoint through which the route - // starts. - ref *referencedNetworkEndpoint - // Loop controls where WritePacket should send packets. Loop PacketLooping + + // nic is the NIC the route goes through. + nic *NIC + + // addressEndpoint is the local address this route is associated with. + addressEndpoint AssignableAddressEndpoint + + // linkCache is set if link address resolution is enabled for this protocol on + // the route's NIC. + linkCache LinkAddressCache + + // linkRes is set if link address resolution is enabled for this protocol on + // the route's NIC. + linkRes LinkAddressResolver } // makeRoute initializes a new route. It takes ownership of the provided -// reference to a network endpoint. -func makeRoute(netProto tcpip.NetworkProtocolNumber, localAddr, remoteAddr tcpip.Address, localLinkAddr tcpip.LinkAddress, ref *referencedNetworkEndpoint, handleLocal, multicastLoop bool) Route { +// AssignableAddressEndpoint. +func makeRoute(netProto tcpip.NetworkProtocolNumber, localAddr, remoteAddr tcpip.Address, nic *NIC, addressEndpoint AssignableAddressEndpoint, handleLocal, multicastLoop bool) Route { loop := PacketOut if handleLocal && localAddr != "" && remoteAddr == localAddr { loop = PacketLoop @@ -62,29 +72,40 @@ func makeRoute(netProto tcpip.NetworkProtocolNumber, localAddr, remoteAddr tcpip loop |= PacketLoop } - return Route{ + linkEP := nic.LinkEndpoint() + r := Route{ NetProto: netProto, LocalAddress: localAddr, - LocalLinkAddress: localLinkAddr, + LocalLinkAddress: linkEP.LinkAddress(), RemoteAddress: remoteAddr, - ref: ref, + addressEndpoint: addressEndpoint, + nic: nic, Loop: loop, } + + if nic := r.nic; linkEP.Capabilities()&CapabilityResolutionRequired != 0 { + if linkRes, ok := nic.stack.linkAddrResolvers[r.NetProto]; ok { + r.linkRes = linkRes + r.linkCache = nic.stack + } + } + + return r } // NICID returns the id of the NIC from which this route originates. func (r *Route) NICID() tcpip.NICID { - return r.ref.ep.NICID() + return r.nic.ID() } // MaxHeaderLength forwards the call to the network endpoint's implementation. func (r *Route) MaxHeaderLength() uint16 { - return r.ref.ep.MaxHeaderLength() + return r.nic.getNetworkEndpoint(r.NetProto).MaxHeaderLength() } // Stats returns a mutable copy of current stats. func (r *Route) Stats() tcpip.Stats { - return r.ref.nic.stack.Stats() + return r.nic.stack.Stats() } // PseudoHeaderChecksum forwards the call to the network endpoint's @@ -95,12 +116,12 @@ func (r *Route) PseudoHeaderChecksum(protocol tcpip.TransportProtocolNumber, tot // Capabilities returns the link-layer capabilities of the route. func (r *Route) Capabilities() LinkEndpointCapabilities { - return r.ref.ep.Capabilities() + return r.nic.LinkEndpoint().Capabilities() } // GSOMaxSize returns the maximum GSO packet size. func (r *Route) GSOMaxSize() uint32 { - if gso, ok := r.ref.ep.(GSOEndpoint); ok { + if gso, ok := r.nic.getNetworkEndpoint(r.NetProto).(GSOEndpoint); ok { return gso.GSOMaxSize() } return 0 @@ -138,8 +159,8 @@ func (r *Route) Resolve(waker *sleep.Waker) (<-chan struct{}, *tcpip.Error) { nextAddr = r.RemoteAddress } - if r.ref.nic.neigh != nil { - entry, ch, err := r.ref.nic.neigh.entry(nextAddr, r.LocalAddress, r.ref.linkRes, waker) + if neigh := r.nic.neigh; neigh != nil { + entry, ch, err := neigh.entry(nextAddr, r.LocalAddress, r.linkRes, waker) if err != nil { return ch, err } @@ -147,7 +168,7 @@ func (r *Route) Resolve(waker *sleep.Waker) (<-chan struct{}, *tcpip.Error) { return nil, nil } - linkAddr, ch, err := r.ref.linkCache.GetLinkAddress(r.ref.nic.ID(), nextAddr, r.LocalAddress, r.NetProto, waker) + linkAddr, ch, err := r.linkCache.GetLinkAddress(r.nic.ID(), nextAddr, r.LocalAddress, r.NetProto, waker) if err != nil { return ch, err } @@ -162,12 +183,12 @@ func (r *Route) RemoveWaker(waker *sleep.Waker) { nextAddr = r.RemoteAddress } - if r.ref.nic.neigh != nil { - r.ref.nic.neigh.removeWaker(nextAddr, waker) + if neigh := r.nic.neigh; neigh != nil { + neigh.removeWaker(nextAddr, waker) return } - r.ref.linkCache.RemoveWaker(r.ref.nic.ID(), nextAddr, waker) + r.linkCache.RemoveWaker(r.nic.ID(), nextAddr, waker) } // IsResolutionRequired returns true if Resolve() must be called to resolve @@ -175,104 +196,88 @@ func (r *Route) RemoveWaker(waker *sleep.Waker) { // // The NIC r uses must not be locked. func (r *Route) IsResolutionRequired() bool { - if r.ref.nic.neigh != nil { - return r.ref.isValidForOutgoing() && r.ref.linkRes != nil && r.RemoteLinkAddress == "" + if r.nic.neigh != nil { + return r.nic.isValidForOutgoing(r.addressEndpoint) && r.linkRes != nil && r.RemoteLinkAddress == "" } - return r.ref.isValidForOutgoing() && r.ref.linkCache != nil && r.RemoteLinkAddress == "" + return r.nic.isValidForOutgoing(r.addressEndpoint) && r.linkCache != nil && r.RemoteLinkAddress == "" } // WritePacket writes the packet through the given route. func (r *Route) WritePacket(gso *GSO, params NetworkHeaderParams, pkt *PacketBuffer) *tcpip.Error { - if !r.ref.isValidForOutgoing() { + if !r.nic.isValidForOutgoing(r.addressEndpoint) { return tcpip.ErrInvalidEndpointState } // WritePacket takes ownership of pkt, calculate numBytes first. numBytes := pkt.Size() - err := r.ref.ep.WritePacket(r, gso, params, pkt) - if err != nil { - r.Stats().IP.OutgoingPacketErrors.Increment() - } else { - r.ref.nic.stats.Tx.Packets.Increment() - r.ref.nic.stats.Tx.Bytes.IncrementBy(uint64(numBytes)) + if err := r.nic.getNetworkEndpoint(r.NetProto).WritePacket(r, gso, params, pkt); err != nil { + return err } - return err + + r.nic.stats.Tx.Packets.Increment() + r.nic.stats.Tx.Bytes.IncrementBy(uint64(numBytes)) + return nil } // WritePackets writes a list of n packets through the given route and returns // the number of packets written. func (r *Route) WritePackets(gso *GSO, pkts PacketBufferList, params NetworkHeaderParams) (int, *tcpip.Error) { - if !r.ref.isValidForOutgoing() { + if !r.nic.isValidForOutgoing(r.addressEndpoint) { return 0, tcpip.ErrInvalidEndpointState } - // WritePackets takes ownership of pkt, calculate length first. - numPkts := pkts.Len() - - n, err := r.ref.ep.WritePackets(r, gso, pkts, params) - if err != nil { - r.Stats().IP.OutgoingPacketErrors.IncrementBy(uint64(numPkts - n)) - } - r.ref.nic.stats.Tx.Packets.IncrementBy(uint64(n)) - + n, err := r.nic.getNetworkEndpoint(r.NetProto).WritePackets(r, gso, pkts, params) + r.nic.stats.Tx.Packets.IncrementBy(uint64(n)) writtenBytes := 0 for i, pb := 0, pkts.Front(); i < n && pb != nil; i, pb = i+1, pb.Next() { writtenBytes += pb.Size() } - r.ref.nic.stats.Tx.Bytes.IncrementBy(uint64(writtenBytes)) + r.nic.stats.Tx.Bytes.IncrementBy(uint64(writtenBytes)) return n, err } // WriteHeaderIncludedPacket writes a packet already containing a network // header through the given route. func (r *Route) WriteHeaderIncludedPacket(pkt *PacketBuffer) *tcpip.Error { - if !r.ref.isValidForOutgoing() { + if !r.nic.isValidForOutgoing(r.addressEndpoint) { return tcpip.ErrInvalidEndpointState } // WriteHeaderIncludedPacket takes ownership of pkt, calculate numBytes first. numBytes := pkt.Data.Size() - if err := r.ref.ep.WriteHeaderIncludedPacket(r, pkt); err != nil { - r.Stats().IP.OutgoingPacketErrors.Increment() + if err := r.nic.getNetworkEndpoint(r.NetProto).WriteHeaderIncludedPacket(r, pkt); err != nil { return err } - r.ref.nic.stats.Tx.Packets.Increment() - r.ref.nic.stats.Tx.Bytes.IncrementBy(uint64(numBytes)) + r.nic.stats.Tx.Packets.Increment() + r.nic.stats.Tx.Bytes.IncrementBy(uint64(numBytes)) return nil } // DefaultTTL returns the default TTL of the underlying network endpoint. func (r *Route) DefaultTTL() uint8 { - return r.ref.ep.DefaultTTL() + return r.nic.getNetworkEndpoint(r.NetProto).DefaultTTL() } // MTU returns the MTU of the underlying network endpoint. func (r *Route) MTU() uint32 { - return r.ref.ep.MTU() -} - -// NetworkProtocolNumber returns the NetworkProtocolNumber of the underlying -// network endpoint. -func (r *Route) NetworkProtocolNumber() tcpip.NetworkProtocolNumber { - return r.ref.ep.NetworkProtocolNumber() + return r.nic.getNetworkEndpoint(r.NetProto).MTU() } // Release frees all resources associated with the route. func (r *Route) Release() { - if r.ref != nil { - r.ref.decRef() - r.ref = nil + if r.addressEndpoint != nil { + r.addressEndpoint.DecRef() + r.addressEndpoint = nil } } -// Clone Clone a route such that the original one can be released and the new -// one will remain valid. +// Clone clones the route. func (r *Route) Clone() Route { - if r.ref != nil { - r.ref.incRef() + if r.addressEndpoint != nil { + _ = r.addressEndpoint.IncRef() } return *r } @@ -296,7 +301,7 @@ func (r *Route) MakeLoopedRoute() Route { // Stack returns the instance of the Stack that owns this route. func (r *Route) Stack() *Stack { - return r.ref.stack() + return r.nic.stack } func (r *Route) isV4Broadcast(addr tcpip.Address) bool { @@ -304,7 +309,7 @@ func (r *Route) isV4Broadcast(addr tcpip.Address) bool { return true } - subnet := r.ref.addrWithPrefix().Subnet() + subnet := r.addressEndpoint.AddressWithPrefix().Subnet() return subnet.IsBroadcast(addr) } @@ -330,7 +335,10 @@ func (r *Route) ReverseRoute(src tcpip.Address, dst tcpip.Address) Route { LocalLinkAddress: r.RemoteLinkAddress, RemoteAddress: src, RemoteLinkAddress: r.LocalLinkAddress, - ref: r.ref, Loop: r.Loop, + addressEndpoint: r.addressEndpoint, + nic: r.nic, + linkCache: r.linkCache, + linkRes: r.linkRes, } } diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go index 6a683545d..0bf20c0e1 100644 --- a/pkg/tcpip/stack/stack.go +++ b/pkg/tcpip/stack/stack.go @@ -144,10 +144,7 @@ type TCPReceiverState struct { // PendingBufUsed is the number of bytes pending in the receive // queue. - PendingBufUsed seqnum.Size - - // PendingBufSize is the size of the socket receive buffer. - PendingBufSize seqnum.Size + PendingBufUsed int } // TCPSenderState holds a copy of the internal state of the sender for @@ -366,38 +363,6 @@ func (u *uniqueIDGenerator) UniqueID() uint64 { return atomic.AddUint64((*uint64)(u), 1) } -// NICNameFromID is a function that returns a stable name for the specified NIC, -// even if different NIC IDs are used to refer to the same NIC in different -// program runs. It is used when generating opaque interface identifiers (IIDs). -// If the NIC was created with a name, it will be passed to NICNameFromID. -// -// NICNameFromID SHOULD return unique NIC names so unique opaque IIDs are -// generated for the same prefix on differnt NICs. -type NICNameFromID func(tcpip.NICID, string) string - -// OpaqueInterfaceIdentifierOptions holds the options related to the generation -// of opaque interface indentifiers (IIDs) as defined by RFC 7217. -type OpaqueInterfaceIdentifierOptions struct { - // NICNameFromID is a function that returns a stable name for a specified NIC, - // even if the NIC ID changes over time. - // - // Must be specified to generate the opaque IID. - NICNameFromID NICNameFromID - - // SecretKey is a pseudo-random number used as the secret key when generating - // opaque IIDs as defined by RFC 7217. The key SHOULD be at least - // header.OpaqueIIDSecretKeyMinBytes bytes and MUST follow minimum randomness - // requirements for security as outlined by RFC 4086. SecretKey MUST NOT - // change between program runs, unless explicitly changed. - // - // OpaqueInterfaceIdentifierOptions takes ownership of SecretKey. SecretKey - // MUST NOT be modified after Stack is created. - // - // May be nil, but a nil value is highly discouraged to maintain - // some level of randomness between nodes. - SecretKey []byte -} - // Stack is a networking stack, with all supported protocols, NICs, and route // table. type Stack struct { @@ -415,9 +380,8 @@ type Stack struct { linkAddrCache *linkAddrCache - mu sync.RWMutex - nics map[tcpip.NICID]*NIC - forwarding bool + mu sync.RWMutex + nics map[tcpip.NICID]*NIC // cleanupEndpointsMu protects cleanupEndpoints. cleanupEndpointsMu sync.Mutex @@ -458,9 +422,6 @@ type Stack struct { // TODO(gvisor.dev/issue/940): S/R this field. seed uint32 - // ndpConfigs is the default NDP configurations used by interfaces. - ndpConfigs NDPConfigurations - // nudConfigs is the default NUD configurations used by interfaces. nudConfigs NUDConfigurations @@ -468,15 +429,6 @@ type Stack struct { // by the NIC's neighborCache instead of linkAddrCache. useNeighborCache bool - // autoGenIPv6LinkLocal determines whether or not the stack will attempt - // to auto-generate an IPv6 link-local address for newly enabled non-loopback - // NICs. See the AutoGenIPv6LinkLocal field of Options for more details. - autoGenIPv6LinkLocal bool - - // ndpDisp is the NDP event dispatcher that is used to send the netstack - // integrator NDP related events. - ndpDisp NDPDispatcher - // nudDisp is the NUD event dispatcher that is used to send the netstack // integrator NUD related events. nudDisp NUDDispatcher @@ -484,14 +436,6 @@ type Stack struct { // uniqueIDGenerator is a generator of unique identifiers. uniqueIDGenerator UniqueID - // opaqueIIDOpts hold the options for generating opaque interface identifiers - // (IIDs) as outlined by RFC 7217. - opaqueIIDOpts OpaqueInterfaceIdentifierOptions - - // tempIIDSeed is used to seed the initial temporary interface identifier - // history value used to generate IIDs for temporary SLAAC addresses. - tempIIDSeed []byte - // forwarder holds the packets that wait for their link-address resolutions // to complete, and forwards them when each resolution is done. forwarder *forwardQueue @@ -514,13 +458,25 @@ type UniqueID interface { UniqueID() uint64 } +// NetworkProtocolFactory instantiates a network protocol. +// +// NetworkProtocolFactory must not attempt to modify the stack, it may only +// query the stack. +type NetworkProtocolFactory func(*Stack) NetworkProtocol + +// TransportProtocolFactory instantiates a transport protocol. +// +// TransportProtocolFactory must not attempt to modify the stack, it may only +// query the stack. +type TransportProtocolFactory func(*Stack) TransportProtocol + // Options contains optional Stack configuration. type Options struct { // NetworkProtocols lists the network protocols to enable. - NetworkProtocols []NetworkProtocol + NetworkProtocols []NetworkProtocolFactory // TransportProtocols lists the transport protocols to enable. - TransportProtocols []TransportProtocol + TransportProtocols []TransportProtocolFactory // Clock is an optional clock source used for timestampping packets. // @@ -538,13 +494,6 @@ type Options struct { // UniqueID is an optional generator of unique identifiers. UniqueID UniqueID - // NDPConfigs is the default NDP configurations used by interfaces. - // - // By default, NDPConfigs will have a zero value for its - // DupAddrDetectTransmits field, implying that DAD will not be performed - // before assigning an address to a NIC. - NDPConfigs NDPConfigurations - // NUDConfigs is the default NUD configurations used by interfaces. NUDConfigs NUDConfigurations @@ -555,24 +504,6 @@ type Options struct { // and ClearNeighbors. UseNeighborCache bool - // AutoGenIPv6LinkLocal determines whether or not the stack will attempt to - // auto-generate an IPv6 link-local address for newly enabled non-loopback - // NICs. - // - // Note, setting this to true does not mean that a link-local address - // will be assigned right away, or at all. If Duplicate Address Detection - // is enabled, an address will only be assigned if it successfully resolves. - // If it fails, no further attempt will be made to auto-generate an IPv6 - // link-local address. - // - // The generated link-local address will follow RFC 4291 Appendix A - // guidelines. - AutoGenIPv6LinkLocal bool - - // NDPDisp is the NDP event dispatcher that an integrator can provide to - // receive NDP related events. - NDPDisp NDPDispatcher - // NUDDisp is the NUD event dispatcher that an integrator can provide to // receive NUD related events. NUDDisp NUDDispatcher @@ -581,31 +512,12 @@ type Options struct { // this is non-nil. RawFactory RawFactory - // OpaqueIIDOpts hold the options for generating opaque interface - // identifiers (IIDs) as outlined by RFC 7217. - OpaqueIIDOpts OpaqueInterfaceIdentifierOptions - // RandSource is an optional source to use to generate random // numbers. If omitted it defaults to a Source seeded by the data // returned by rand.Read(). // // RandSource must be thread-safe. RandSource mathrand.Source - - // TempIIDSeed is used to seed the initial temporary interface identifier - // history value used to generate IIDs for temporary SLAAC addresses. - // - // Temporary SLAAC adresses are short-lived addresses which are unpredictable - // and random from the perspective of other nodes on the network. It is - // recommended that the seed be a random byte buffer of at least - // header.IIDSize bytes to make sure that temporary SLAAC addresses are - // sufficiently random. It should follow minimum randomness requirements for - // security as outlined by RFC 4086. - // - // Note: using a nil value, the same seed across netstack program runs, or a - // seed that is too small would reduce randomness and increase predictability, - // defeating the purpose of temporary SLAAC addresses. - TempIIDSeed []byte } // TransportEndpointInfo holds useful information about a transport endpoint @@ -708,36 +620,28 @@ func New(opts Options) *Stack { randSrc = &lockedRandomSource{src: mathrand.NewSource(generateRandInt64())} } - // Make sure opts.NDPConfigs contains valid values only. - opts.NDPConfigs.validate() - opts.NUDConfigs.resetInvalidFields() s := &Stack{ - transportProtocols: make(map[tcpip.TransportProtocolNumber]*transportProtocolState), - networkProtocols: make(map[tcpip.NetworkProtocolNumber]NetworkProtocol), - linkAddrResolvers: make(map[tcpip.NetworkProtocolNumber]LinkAddressResolver), - nics: make(map[tcpip.NICID]*NIC), - cleanupEndpoints: make(map[TransportEndpoint]struct{}), - linkAddrCache: newLinkAddrCache(ageLimit, resolutionTimeout, resolutionAttempts), - PortManager: ports.NewPortManager(), - clock: clock, - stats: opts.Stats.FillIn(), - handleLocal: opts.HandleLocal, - tables: DefaultTables(), - icmpRateLimiter: NewICMPRateLimiter(), - seed: generateRandUint32(), - ndpConfigs: opts.NDPConfigs, - nudConfigs: opts.NUDConfigs, - useNeighborCache: opts.UseNeighborCache, - autoGenIPv6LinkLocal: opts.AutoGenIPv6LinkLocal, - uniqueIDGenerator: opts.UniqueID, - ndpDisp: opts.NDPDisp, - nudDisp: opts.NUDDisp, - opaqueIIDOpts: opts.OpaqueIIDOpts, - tempIIDSeed: opts.TempIIDSeed, - forwarder: newForwardQueue(), - randomGenerator: mathrand.New(randSrc), + transportProtocols: make(map[tcpip.TransportProtocolNumber]*transportProtocolState), + networkProtocols: make(map[tcpip.NetworkProtocolNumber]NetworkProtocol), + linkAddrResolvers: make(map[tcpip.NetworkProtocolNumber]LinkAddressResolver), + nics: make(map[tcpip.NICID]*NIC), + cleanupEndpoints: make(map[TransportEndpoint]struct{}), + linkAddrCache: newLinkAddrCache(ageLimit, resolutionTimeout, resolutionAttempts), + PortManager: ports.NewPortManager(), + clock: clock, + stats: opts.Stats.FillIn(), + handleLocal: opts.HandleLocal, + tables: DefaultTables(), + icmpRateLimiter: NewICMPRateLimiter(), + seed: generateRandUint32(), + nudConfigs: opts.NUDConfigs, + useNeighborCache: opts.UseNeighborCache, + uniqueIDGenerator: opts.UniqueID, + nudDisp: opts.NUDDisp, + forwarder: newForwardQueue(), + randomGenerator: mathrand.New(randSrc), sendBufferSize: SendBufferSizeOption{ Min: MinBufferSize, Default: DefaultBufferSize, @@ -751,7 +655,8 @@ func New(opts Options) *Stack { } // Add specified network protocols. - for _, netProto := range opts.NetworkProtocols { + for _, netProtoFactory := range opts.NetworkProtocols { + netProto := netProtoFactory(s) s.networkProtocols[netProto.Number()] = netProto if r, ok := netProto.(LinkAddressResolver); ok { s.linkAddrResolvers[r.LinkAddressProtocol()] = r @@ -759,7 +664,8 @@ func New(opts Options) *Stack { } // Add specified transport protocols. - for _, transProto := range opts.TransportProtocols { + for _, transProtoFactory := range opts.TransportProtocols { + transProto := transProtoFactory(s) s.transportProtocols[transProto.Number()] = &transportProtocolState{ proto: transProto, } @@ -866,46 +772,37 @@ func (s *Stack) Stats() tcpip.Stats { return s.stats } -// SetForwarding enables or disables the packet forwarding between NICs. -// -// When forwarding becomes enabled, any host-only state on all NICs will be -// cleaned up and if IPv6 is enabled, NDP Router Solicitations will be started. -// When forwarding becomes disabled and if IPv6 is enabled, NDP Router -// Solicitations will be stopped. -func (s *Stack) SetForwarding(enable bool) { - // TODO(igudger, bgeffon): Expose via /proc/sys/net/ipv4/ip_forward. - s.mu.Lock() - defer s.mu.Unlock() +// SetForwarding enables or disables packet forwarding between NICs for the +// passed protocol. +func (s *Stack) SetForwarding(protocolNum tcpip.NetworkProtocolNumber, enable bool) *tcpip.Error { + protocol, ok := s.networkProtocols[protocolNum] + if !ok { + return tcpip.ErrUnknownProtocol + } - // If forwarding status didn't change, do nothing further. - if s.forwarding == enable { - return + forwardingProtocol, ok := protocol.(ForwardingNetworkProtocol) + if !ok { + return tcpip.ErrNotSupported } - s.forwarding = enable + forwardingProtocol.SetForwarding(enable) + return nil +} - // If this stack does not support IPv6, do nothing further. - if _, ok := s.networkProtocols[header.IPv6ProtocolNumber]; !ok { - return +// Forwarding returns true if packet forwarding between NICs is enabled for the +// passed protocol. +func (s *Stack) Forwarding(protocolNum tcpip.NetworkProtocolNumber) bool { + protocol, ok := s.networkProtocols[protocolNum] + if !ok { + return false } - if enable { - for _, nic := range s.nics { - nic.becomeIPv6Router() - } - } else { - for _, nic := range s.nics { - nic.becomeIPv6Host() - } + forwardingProtocol, ok := protocol.(ForwardingNetworkProtocol) + if !ok { + return false } -} -// Forwarding returns if the packet forwarding between NICs is enabled. -func (s *Stack) Forwarding() bool { - // TODO(igudger, bgeffon): Expose via /proc/sys/net/ipv4/ip_forward. - s.mu.RLock() - defer s.mu.RUnlock() - return s.forwarding + return forwardingProtocol.Forwarding() } // SetRouteTable assigns the route table to be used by this stack. It @@ -940,7 +837,7 @@ func (s *Stack) NewEndpoint(transport tcpip.TransportProtocolNumber, network tcp return nil, tcpip.ErrUnknownProtocol } - return t.proto.NewEndpoint(s, network, waiterQueue) + return t.proto.NewEndpoint(network, waiterQueue) } // NewRawEndpoint creates a new raw transport layer endpoint of the given @@ -960,7 +857,7 @@ func (s *Stack) NewRawEndpoint(transport tcpip.TransportProtocolNumber, network return nil, tcpip.ErrUnknownProtocol } - return t.proto.NewRawEndpoint(s, network, waiterQueue) + return t.proto.NewRawEndpoint(network, waiterQueue) } // NewPacketEndpoint creates a new packet endpoint listening for the given @@ -1067,7 +964,8 @@ func (s *Stack) DisableNIC(id tcpip.NICID) *tcpip.Error { return tcpip.ErrUnknownNICID } - return nic.disable() + nic.disable() + return nil } // CheckNIC checks if a NIC is usable. @@ -1080,7 +978,7 @@ func (s *Stack) CheckNIC(id tcpip.NICID) bool { return false } - return nic.enabled() + return nic.Enabled() } // RemoveNIC removes NIC and all related routes from the network stack. @@ -1158,14 +1056,14 @@ func (s *Stack) NICInfo() map[tcpip.NICID]NICInfo { for id, nic := range s.nics { flags := NICStateFlags{ Up: true, // Netstack interfaces are always up. - Running: nic.enabled(), + Running: nic.Enabled(), Promiscuous: nic.isPromiscuousMode(), - Loopback: nic.isLoopback(), + Loopback: nic.IsLoopback(), } nics[id] = NICInfo{ Name: nic.name, LinkAddress: nic.linkEP.LinkAddress(), - ProtocolAddresses: nic.PrimaryAddresses(), + ProtocolAddresses: nic.primaryAddresses(), Flags: flags, MTU: nic.linkEP.MTU(), Stats: nic.stats, @@ -1229,7 +1127,7 @@ func (s *Stack) AddProtocolAddressWithOptions(id tcpip.NICID, protocolAddress tc return tcpip.ErrUnknownNICID } - return nic.AddAddress(protocolAddress, peb) + return nic.addAddress(protocolAddress, peb) } // RemoveAddress removes an existing network-layer address from the specified @@ -1239,7 +1137,7 @@ func (s *Stack) RemoveAddress(id tcpip.NICID, addr tcpip.Address) *tcpip.Error { defer s.mu.RUnlock() if nic, ok := s.nics[id]; ok { - return nic.RemoveAddress(addr) + return nic.removeAddress(addr) } return tcpip.ErrUnknownNICID @@ -1253,7 +1151,7 @@ func (s *Stack) AllAddresses() map[tcpip.NICID][]tcpip.ProtocolAddress { nics := make(map[tcpip.NICID][]tcpip.ProtocolAddress) for id, nic := range s.nics { - nics[id] = nic.AllAddresses() + nics[id] = nic.allPermanentAddresses() } return nics } @@ -1275,7 +1173,7 @@ func (s *Stack) GetMainNICAddress(id tcpip.NICID, protocol tcpip.NetworkProtocol return nic.primaryAddress(protocol), nil } -func (s *Stack) getRefEP(nic *NIC, localAddr, remoteAddr tcpip.Address, netProto tcpip.NetworkProtocolNumber) (ref *referencedNetworkEndpoint) { +func (s *Stack) getAddressEP(nic *NIC, localAddr, remoteAddr tcpip.Address, netProto tcpip.NetworkProtocolNumber) AssignableAddressEndpoint { if len(localAddr) == 0 { return nic.primaryEndpoint(netProto, remoteAddr) } @@ -1292,9 +1190,9 @@ func (s *Stack) FindRoute(id tcpip.NICID, localAddr, remoteAddr tcpip.Address, n isMulticast := header.IsV4MulticastAddress(remoteAddr) || header.IsV6MulticastAddress(remoteAddr) needRoute := !(isLocalBroadcast || isMulticast || header.IsV6LinkLocalAddress(remoteAddr)) if id != 0 && !needRoute { - if nic, ok := s.nics[id]; ok && nic.enabled() { - if ref := s.getRefEP(nic, localAddr, remoteAddr, netProto); ref != nil { - return makeRoute(netProto, ref.address(), remoteAddr, nic.linkEP.LinkAddress(), ref, s.handleLocal && !nic.isLoopback(), multicastLoop && !nic.isLoopback()), nil + if nic, ok := s.nics[id]; ok && nic.Enabled() { + if addressEndpoint := s.getAddressEP(nic, localAddr, remoteAddr, netProto); addressEndpoint != nil { + return makeRoute(netProto, addressEndpoint.AddressWithPrefix().Address, remoteAddr, nic, addressEndpoint, s.handleLocal && !nic.IsLoopback(), multicastLoop && !nic.IsLoopback()), nil } } } else { @@ -1302,20 +1200,20 @@ func (s *Stack) FindRoute(id tcpip.NICID, localAddr, remoteAddr tcpip.Address, n if (id != 0 && id != route.NIC) || (len(remoteAddr) != 0 && !route.Destination.Contains(remoteAddr)) { continue } - if nic, ok := s.nics[route.NIC]; ok && nic.enabled() { - if ref := s.getRefEP(nic, localAddr, remoteAddr, netProto); ref != nil { + if nic, ok := s.nics[route.NIC]; ok && nic.Enabled() { + if addressEndpoint := s.getAddressEP(nic, localAddr, remoteAddr, netProto); addressEndpoint != nil { if len(remoteAddr) == 0 { // If no remote address was provided, then the route // provided will refer to the link local address. - remoteAddr = ref.address() + remoteAddr = addressEndpoint.AddressWithPrefix().Address } - r := makeRoute(netProto, ref.address(), remoteAddr, nic.linkEP.LinkAddress(), ref, s.handleLocal && !nic.isLoopback(), multicastLoop && !nic.isLoopback()) + r := makeRoute(netProto, addressEndpoint.AddressWithPrefix().Address, remoteAddr, nic, addressEndpoint, s.handleLocal && !nic.IsLoopback(), multicastLoop && !nic.IsLoopback()) if len(route.Gateway) > 0 { if needRoute { r.NextHop = route.Gateway } - } else if subnet := ref.addrWithPrefix().Subnet(); subnet.IsBroadcast(remoteAddr) { + } else if subnet := addressEndpoint.AddressWithPrefix().Subnet(); subnet.IsBroadcast(remoteAddr) { r.RemoteLinkAddress = header.EthernetBroadcastAddress } @@ -1353,21 +1251,20 @@ func (s *Stack) CheckLocalAddress(nicID tcpip.NICID, protocol tcpip.NetworkProto return 0 } - ref := nic.findEndpoint(protocol, addr, CanBePrimaryEndpoint) - if ref == nil { + addressEndpoint := nic.findEndpoint(protocol, addr, CanBePrimaryEndpoint) + if addressEndpoint == nil { return 0 } - ref.decRef() + addressEndpoint.DecRef() return nic.id } // Go through all the NICs. for _, nic := range s.nics { - ref := nic.findEndpoint(protocol, addr, CanBePrimaryEndpoint) - if ref != nil { - ref.decRef() + if addressEndpoint := nic.findEndpoint(protocol, addr, CanBePrimaryEndpoint); addressEndpoint != nil { + addressEndpoint.DecRef() return nic.id } } @@ -1836,7 +1733,7 @@ func (s *Stack) LeaveGroup(protocol tcpip.NetworkProtocolNumber, nicID tcpip.NIC defer s.mu.RUnlock() if nic, ok := s.nics[nicID]; ok { - return nic.leaveGroup(multicastAddr) + return nic.leaveGroup(protocol, multicastAddr) } return tcpip.ErrUnknownNICID } @@ -1888,53 +1785,18 @@ func (s *Stack) AllowICMPMessage() bool { return s.icmpRateLimiter.Allow() } -// IsAddrTentative returns true if addr is tentative on the NIC with ID id. -// -// Note that if addr is not associated with a NIC with id ID, then this -// function will return false. It will only return true if the address is -// associated with the NIC AND it is tentative. -func (s *Stack) IsAddrTentative(id tcpip.NICID, addr tcpip.Address) (bool, *tcpip.Error) { - s.mu.RLock() - defer s.mu.RUnlock() - - nic, ok := s.nics[id] - if !ok { - return false, tcpip.ErrUnknownNICID - } - - return nic.isAddrTentative(addr), nil -} - -// DupTentativeAddrDetected attempts to inform the NIC with ID id that a -// tentative addr on it is a duplicate on a link. -func (s *Stack) DupTentativeAddrDetected(id tcpip.NICID, addr tcpip.Address) *tcpip.Error { +// GetNetworkEndpoint returns the NetworkEndpoint with the specified protocol +// number installed on the specified NIC. +func (s *Stack) GetNetworkEndpoint(nicID tcpip.NICID, proto tcpip.NetworkProtocolNumber) (NetworkEndpoint, *tcpip.Error) { s.mu.Lock() defer s.mu.Unlock() - nic, ok := s.nics[id] - if !ok { - return tcpip.ErrUnknownNICID - } - - return nic.dupTentativeAddrDetected(addr) -} - -// SetNDPConfigurations sets the per-interface NDP configurations on the NIC -// with ID id to c. -// -// Note, if c contains invalid NDP configuration values, it will be fixed to -// use default values for the erroneous values. -func (s *Stack) SetNDPConfigurations(id tcpip.NICID, c NDPConfigurations) *tcpip.Error { - s.mu.Lock() - defer s.mu.Unlock() - - nic, ok := s.nics[id] + nic, ok := s.nics[nicID] if !ok { - return tcpip.ErrUnknownNICID + return nil, tcpip.ErrUnknownNICID } - nic.setNDPConfigs(c) - return nil + return nic.getNetworkEndpoint(proto), nil } // NUDConfigurations gets the per-interface NUD configurations. @@ -1947,7 +1809,7 @@ func (s *Stack) NUDConfigurations(id tcpip.NICID) (NUDConfigurations, *tcpip.Err return NUDConfigurations{}, tcpip.ErrUnknownNICID } - return nic.NUDConfigs() + return nic.nudConfigs() } // SetNUDConfigurations sets the per-interface NUD configurations. @@ -1966,22 +1828,6 @@ func (s *Stack) SetNUDConfigurations(id tcpip.NICID, c NUDConfigurations) *tcpip return nic.setNUDConfigs(c) } -// HandleNDPRA provides a NIC with ID id a validated NDP Router Advertisement -// message that it needs to handle. -func (s *Stack) HandleNDPRA(id tcpip.NICID, ip tcpip.Address, ra header.NDPRouterAdvert) *tcpip.Error { - s.mu.Lock() - defer s.mu.Unlock() - - nic, ok := s.nics[id] - if !ok { - return tcpip.ErrUnknownNICID - } - - nic.handleNDPRA(ip, ra) - - return nil -} - // Seed returns a 32 bit value that can be used as a seed value for port // picking, ISN generation etc. // @@ -2023,16 +1869,12 @@ func (s *Stack) FindNetworkEndpoint(netProto tcpip.NetworkProtocolNumber, addres defer s.mu.RUnlock() for _, nic := range s.nics { - id := NetworkEndpointID{address} - - if ref, ok := nic.mu.endpoints[id]; ok { - nic.mu.RLock() - defer nic.mu.RUnlock() - - // An endpoint with this id exists, check if it can be - // used and return it. - return ref.ep, nil + addressEndpoint := nic.getAddressOrCreateTempInner(netProto, address, false /* createTemp */, NeverPrimaryEndpoint) + if addressEndpoint == nil { + continue } + addressEndpoint.DecRef() + return nic.getNetworkEndpoint(netProto), nil } return nil, tcpip.ErrBadAddress } @@ -2049,3 +1891,8 @@ func (s *Stack) FindNICNameFromID(id tcpip.NICID) string { return nic.Name() } + +// NewJob returns a new tcpip.Job using the stack's clock. +func (s *Stack) NewJob(l sync.Locker, f func()) *tcpip.Job { + return tcpip.NewJob(s.clock, l, f) +} diff --git a/pkg/tcpip/stack/stack_test.go b/pkg/tcpip/stack/stack_test.go index 60b54c244..aa20f750b 100644 --- a/pkg/tcpip/stack/stack_test.go +++ b/pkg/tcpip/stack/stack_test.go @@ -29,6 +29,7 @@ import ( "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" "gvisor.dev/gvisor/pkg/rand" + "gvisor.dev/gvisor/pkg/sync" "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/header" @@ -68,18 +69,41 @@ const ( // use the first three: destination address, source address, and transport // protocol. They're all one byte fields to simplify parsing. type fakeNetworkEndpoint struct { + stack.AddressableEndpointState + + mu struct { + sync.RWMutex + + enabled bool + } + nicID tcpip.NICID proto *fakeNetworkProtocol dispatcher stack.TransportDispatcher ep stack.LinkEndpoint } -func (f *fakeNetworkEndpoint) MTU() uint32 { - return f.ep.MTU() - uint32(f.MaxHeaderLength()) +func (f *fakeNetworkEndpoint) Enable() *tcpip.Error { + f.mu.Lock() + defer f.mu.Unlock() + f.mu.enabled = true + return nil } -func (f *fakeNetworkEndpoint) NICID() tcpip.NICID { - return f.nicID +func (f *fakeNetworkEndpoint) Enabled() bool { + f.mu.RLock() + defer f.mu.RUnlock() + return f.mu.enabled +} + +func (f *fakeNetworkEndpoint) Disable() { + f.mu.Lock() + defer f.mu.Unlock() + f.mu.enabled = false +} + +func (f *fakeNetworkEndpoint) MTU() uint32 { + return f.ep.MTU() - uint32(f.MaxHeaderLength()) } func (*fakeNetworkEndpoint) DefaultTTL() uint8 { @@ -118,10 +142,6 @@ func (f *fakeNetworkEndpoint) PseudoHeaderChecksum(protocol tcpip.TransportProto return 0 } -func (f *fakeNetworkEndpoint) Capabilities() stack.LinkEndpointCapabilities { - return f.ep.Capabilities() -} - func (f *fakeNetworkEndpoint) NetworkProtocolNumber() tcpip.NetworkProtocolNumber { return f.proto.Number() } @@ -156,7 +176,9 @@ func (*fakeNetworkEndpoint) WriteHeaderIncludedPacket(r *stack.Route, pkt *stack return tcpip.ErrNotSupported } -func (*fakeNetworkEndpoint) Close() {} +func (f *fakeNetworkEndpoint) Close() { + f.AddressableEndpointState.Cleanup() +} // fakeNetworkProtocol is a network-layer protocol descriptor. It aggregates the // number of packets sent and received via endpoints of this protocol. The index @@ -165,6 +187,11 @@ type fakeNetworkProtocol struct { packetCount [10]int sendPacketCount [10]int defaultTTL uint8 + + mu struct { + sync.RWMutex + forwarding bool + } } func (f *fakeNetworkProtocol) Number() tcpip.NetworkProtocolNumber { @@ -187,13 +214,15 @@ func (*fakeNetworkProtocol) ParseAddresses(v buffer.View) (src, dst tcpip.Addres return tcpip.Address(v[srcAddrOffset : srcAddrOffset+1]), tcpip.Address(v[dstAddrOffset : dstAddrOffset+1]) } -func (f *fakeNetworkProtocol) NewEndpoint(nicID tcpip.NICID, _ stack.LinkAddressCache, _ stack.NUDHandler, dispatcher stack.TransportDispatcher, ep stack.LinkEndpoint, _ *stack.Stack) stack.NetworkEndpoint { - return &fakeNetworkEndpoint{ - nicID: nicID, +func (f *fakeNetworkProtocol) NewEndpoint(nic stack.NetworkInterface, _ stack.LinkAddressCache, _ stack.NUDHandler, dispatcher stack.TransportDispatcher) stack.NetworkEndpoint { + e := &fakeNetworkEndpoint{ + nicID: nic.ID(), proto: f, dispatcher: dispatcher, - ep: ep, + ep: nic.LinkEndpoint(), } + e.AddressableEndpointState.Init(e) + return e } func (f *fakeNetworkProtocol) SetOption(option tcpip.SettableNetworkProtocolOption) *tcpip.Error { @@ -216,13 +245,13 @@ func (f *fakeNetworkProtocol) Option(option tcpip.GettableNetworkProtocolOption) } } -// Close implements TransportProtocol.Close. +// Close implements NetworkProtocol.Close. func (*fakeNetworkProtocol) Close() {} -// Wait implements TransportProtocol.Wait. +// Wait implements NetworkProtocol.Wait. func (*fakeNetworkProtocol) Wait() {} -// Parse implements TransportProtocol.Parse. +// Parse implements NetworkProtocol.Parse. func (*fakeNetworkProtocol) Parse(pkt *stack.PacketBuffer) (tcpip.TransportProtocolNumber, bool, bool) { hdr, ok := pkt.NetworkHeader().Consume(fakeNetHeaderLen) if !ok { @@ -231,7 +260,21 @@ func (*fakeNetworkProtocol) Parse(pkt *stack.PacketBuffer) (tcpip.TransportProto return tcpip.TransportProtocolNumber(hdr[protocolNumberOffset]), true, true } -func fakeNetFactory() stack.NetworkProtocol { +// Forwarding implements stack.ForwardingNetworkProtocol. +func (f *fakeNetworkProtocol) Forwarding() bool { + f.mu.RLock() + defer f.mu.RUnlock() + return f.mu.forwarding +} + +// SetForwarding implements stack.ForwardingNetworkProtocol. +func (f *fakeNetworkProtocol) SetForwarding(v bool) { + f.mu.Lock() + defer f.mu.Unlock() + f.mu.forwarding = v +} + +func fakeNetFactory(*stack.Stack) stack.NetworkProtocol { return &fakeNetworkProtocol{} } @@ -268,7 +311,7 @@ func TestNetworkReceive(t *testing.T) { // addresses attached to it: 1 & 2. ep := channel.New(10, defaultMTU, "") s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) if err := s.CreateNIC(1, ep); err != nil { t.Fatal("CreateNIC failed:", err) @@ -428,7 +471,7 @@ func TestNetworkSend(t *testing.T) { // existing nic. ep := channel.New(10, defaultMTU, "") s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) if err := s.CreateNIC(1, ep); err != nil { t.Fatal("NewNIC failed:", err) @@ -455,7 +498,7 @@ func TestNetworkSendMultiRoute(t *testing.T) { // addresses per nic, the first nic has odd address, the second one has // even addresses. s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep1 := channel.New(10, defaultMTU, "") @@ -555,7 +598,7 @@ func TestAttachToLinkEndpointImmediately(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) e := linkEPWithMockedAttach{ @@ -574,7 +617,7 @@ func TestAttachToLinkEndpointImmediately(t *testing.T) { func TestDisableUnknownNIC(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) if err := s.DisableNIC(1); err != tcpip.ErrUnknownNICID { @@ -586,7 +629,7 @@ func TestDisabledNICsNICInfoAndCheckNIC(t *testing.T) { const nicID = 1 s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) e := loopback.New() @@ -633,7 +676,7 @@ func TestDisabledNICsNICInfoAndCheckNIC(t *testing.T) { func TestRemoveUnknownNIC(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) if err := s.RemoveNIC(1); err != tcpip.ErrUnknownNICID { @@ -645,7 +688,7 @@ func TestRemoveNIC(t *testing.T) { const nicID = 1 s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) e := linkEPWithMockedAttach{ @@ -706,7 +749,7 @@ func TestRouteWithDownNIC(t *testing.T) { setup := func(t *testing.T) (*stack.Stack, *channel.Endpoint, *channel.Endpoint) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep1 := channel.New(1, defaultMTU, "") @@ -872,7 +915,7 @@ func TestRoutes(t *testing.T) { // addresses per nic, the first nic has odd address, the second one has // even addresses. s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep1 := channel.New(10, defaultMTU, "") @@ -952,7 +995,7 @@ func TestAddressRemoval(t *testing.T) { remoteAddr := tcpip.Address("\x02") s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") @@ -999,7 +1042,7 @@ func TestAddressRemovalWithRouteHeld(t *testing.T) { remoteAddr := tcpip.Address("\x02") s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") @@ -1090,7 +1133,7 @@ func TestEndpointExpiration(t *testing.T) { for _, spoofing := range []bool{true, false} { t.Run(fmt.Sprintf("promiscuous=%t spoofing=%t", promiscuous, spoofing), func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") @@ -1248,7 +1291,7 @@ func TestEndpointExpiration(t *testing.T) { func TestPromiscuousMode(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") @@ -1300,7 +1343,7 @@ func TestSpoofingWithAddress(t *testing.T) { dstAddr := tcpip.Address("\x03") s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") @@ -1366,7 +1409,7 @@ func TestSpoofingNoAddress(t *testing.T) { dstAddr := tcpip.Address("\x02") s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") @@ -1429,7 +1472,7 @@ func verifyRoute(gotRoute, wantRoute stack.Route) error { func TestOutgoingBroadcastWithEmptyRouteTable(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") @@ -1472,7 +1515,7 @@ func TestOutgoingBroadcastWithRouteTable(t *testing.T) { // Create a new stack with two NICs. s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") if err := s.CreateNIC(1, ep); err != nil { @@ -1573,7 +1616,7 @@ func TestMulticastOrIPv6LinkLocalNeedsNoRoute(t *testing.T) { } { t.Run(tc.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") @@ -1630,8 +1673,8 @@ func TestMulticastOrIPv6LinkLocalNeedsNoRoute(t *testing.T) { func TestNetworkOption(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, - TransportProtocols: []stack.TransportProtocol{}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, + TransportProtocols: []stack.TransportProtocolFactory{}, }) opt := tcpip.DefaultTTLOption(5) @@ -1657,7 +1700,7 @@ func TestGetMainNICAddressAddPrimaryNonPrimary(t *testing.T) { for never := 0; never < 3; never++ { t.Run(fmt.Sprintf("never=%d", never), func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") if err := s.CreateNIC(1, ep); err != nil { @@ -1724,7 +1767,7 @@ func TestGetMainNICAddressAddPrimaryNonPrimary(t *testing.T) { func TestGetMainNICAddressAddRemove(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") if err := s.CreateNIC(1, ep); err != nil { @@ -1809,7 +1852,7 @@ func verifyAddresses(t *testing.T, expectedAddresses, gotAddresses []tcpip.Proto func TestAddAddress(t *testing.T) { const nicID = 1 s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") if err := s.CreateNIC(nicID, ep); err != nil { @@ -1836,7 +1879,7 @@ func TestAddAddress(t *testing.T) { func TestAddProtocolAddress(t *testing.T) { const nicID = 1 s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") if err := s.CreateNIC(nicID, ep); err != nil { @@ -1870,7 +1913,7 @@ func TestAddProtocolAddress(t *testing.T) { func TestAddAddressWithOptions(t *testing.T) { const nicID = 1 s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") if err := s.CreateNIC(nicID, ep); err != nil { @@ -1901,7 +1944,7 @@ func TestAddAddressWithOptions(t *testing.T) { func TestAddProtocolAddressWithOptions(t *testing.T) { const nicID = 1 s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep := channel.New(10, defaultMTU, "") if err := s.CreateNIC(nicID, ep); err != nil { @@ -2022,7 +2065,7 @@ func TestCreateNICWithOptions(t *testing.T) { func TestNICStats(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep1 := channel.New(10, defaultMTU, "") if err := s.CreateNIC(1, ep1); err != nil { @@ -2089,9 +2132,9 @@ func TestNICForwarding(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) - s.SetForwarding(true) + s.SetForwarding(fakeNetNumber, true) ep1 := channel.New(10, defaultMTU, "") if err := s.CreateNIC(nicID1, ep1); err != nil { @@ -2213,7 +2256,7 @@ func TestNICAutoGenLinkLocalAddr(t *testing.T) { nicName string autoGen bool linkAddr tcpip.LinkAddress - iidOpts stack.OpaqueInterfaceIdentifierOptions + iidOpts ipv6.OpaqueInterfaceIdentifierOptions shouldGen bool expectedAddr tcpip.Address }{ @@ -2229,7 +2272,7 @@ func TestNICAutoGenLinkLocalAddr(t *testing.T) { nicName: "nic1", autoGen: false, linkAddr: linkAddr1, - iidOpts: stack.OpaqueInterfaceIdentifierOptions{ + iidOpts: ipv6.OpaqueInterfaceIdentifierOptions{ NICNameFromID: nicNameFunc, SecretKey: secretKey[:], }, @@ -2274,7 +2317,7 @@ func TestNICAutoGenLinkLocalAddr(t *testing.T) { nicName: "nic1", autoGen: true, linkAddr: linkAddr1, - iidOpts: stack.OpaqueInterfaceIdentifierOptions{ + iidOpts: ipv6.OpaqueInterfaceIdentifierOptions{ NICNameFromID: nicNameFunc, SecretKey: secretKey[:], }, @@ -2286,7 +2329,7 @@ func TestNICAutoGenLinkLocalAddr(t *testing.T) { { name: "OIID Empty MAC and empty nicName", autoGen: true, - iidOpts: stack.OpaqueInterfaceIdentifierOptions{ + iidOpts: ipv6.OpaqueInterfaceIdentifierOptions{ NICNameFromID: nicNameFunc, SecretKey: secretKey[:1], }, @@ -2298,7 +2341,7 @@ func TestNICAutoGenLinkLocalAddr(t *testing.T) { nicName: "test", autoGen: true, linkAddr: "\x01\x02\x03", - iidOpts: stack.OpaqueInterfaceIdentifierOptions{ + iidOpts: ipv6.OpaqueInterfaceIdentifierOptions{ NICNameFromID: nicNameFunc, SecretKey: secretKey[:2], }, @@ -2310,7 +2353,7 @@ func TestNICAutoGenLinkLocalAddr(t *testing.T) { nicName: "test2", autoGen: true, linkAddr: "\x01\x02\x03\x04\x05\x06", - iidOpts: stack.OpaqueInterfaceIdentifierOptions{ + iidOpts: ipv6.OpaqueInterfaceIdentifierOptions{ NICNameFromID: nicNameFunc, SecretKey: secretKey[:3], }, @@ -2322,7 +2365,7 @@ func TestNICAutoGenLinkLocalAddr(t *testing.T) { nicName: "test3", autoGen: true, linkAddr: "\x00\x00\x00\x00\x00\x00", - iidOpts: stack.OpaqueInterfaceIdentifierOptions{ + iidOpts: ipv6.OpaqueInterfaceIdentifierOptions{ NICNameFromID: nicNameFunc, }, shouldGen: true, @@ -2336,10 +2379,11 @@ func TestNICAutoGenLinkLocalAddr(t *testing.T) { autoGenAddrC: make(chan ndpAutoGenAddrEvent, 1), } opts := stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - AutoGenIPv6LinkLocal: test.autoGen, - NDPDisp: &ndpDisp, - OpaqueIIDOpts: test.iidOpts, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + AutoGenIPv6LinkLocal: test.autoGen, + NDPDisp: &ndpDisp, + OpaqueIIDOpts: test.iidOpts, + })}, } e := channel.New(0, 1280, test.linkAddr) @@ -2411,15 +2455,15 @@ func TestNoLinkLocalAutoGenForLoopbackNIC(t *testing.T) { tests := []struct { name string - opaqueIIDOpts stack.OpaqueInterfaceIdentifierOptions + opaqueIIDOpts ipv6.OpaqueInterfaceIdentifierOptions }{ { name: "IID From MAC", - opaqueIIDOpts: stack.OpaqueInterfaceIdentifierOptions{}, + opaqueIIDOpts: ipv6.OpaqueInterfaceIdentifierOptions{}, }, { name: "Opaque IID", - opaqueIIDOpts: stack.OpaqueInterfaceIdentifierOptions{ + opaqueIIDOpts: ipv6.OpaqueInterfaceIdentifierOptions{ NICNameFromID: func(_ tcpip.NICID, nicName string) string { return nicName }, @@ -2430,9 +2474,10 @@ func TestNoLinkLocalAutoGenForLoopbackNIC(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { opts := stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - AutoGenIPv6LinkLocal: true, - OpaqueIIDOpts: test.opaqueIIDOpts, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + AutoGenIPv6LinkLocal: true, + OpaqueIIDOpts: test.opaqueIIDOpts, + })}, } e := loopback.New() @@ -2461,12 +2506,13 @@ func TestNICAutoGenAddrDoesDAD(t *testing.T) { ndpDisp := ndpDispatcher{ dadC: make(chan ndpDADEvent), } - ndpConfigs := stack.DefaultNDPConfigurations() + ndpConfigs := ipv6.DefaultNDPConfigurations() opts := stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: ndpConfigs, - AutoGenIPv6LinkLocal: true, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ndpConfigs, + AutoGenIPv6LinkLocal: true, + NDPDisp: &ndpDisp, + })}, } e := channel.New(int(ndpConfigs.DupAddrDetectTransmits), 1280, linkAddr1) @@ -2522,7 +2568,7 @@ func TestNewPEBOnPromotionToPermanent(t *testing.T) { for _, ps := range pebs { t.Run(fmt.Sprintf("%d-to-%d", pi, ps), func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, }) ep1 := channel.New(10, defaultMTU, "") if err := s.CreateNIC(1, ep1); err != nil { @@ -2813,14 +2859,15 @@ func TestIPv6SourceAddressSelectionScopeAndSameAddress(t *testing.T) { t.Run(test.name, func(t *testing.T) { e := channel.New(0, 1280, linkAddr1) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - HandleRAs: true, - AutoGenGlobalAddresses: true, - AutoGenTempGlobalAddresses: true, - }, - NDPDisp: &ndpDispatcher{}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + HandleRAs: true, + AutoGenGlobalAddresses: true, + AutoGenTempGlobalAddresses: true, + }, + NDPDisp: &ndpDispatcher{}, + })}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, }) if err := s.CreateNIC(nicID, e); err != nil { t.Fatalf("CreateNIC(%d, _) = %s", nicID, err) @@ -2869,7 +2916,7 @@ func TestAddRemoveIPv4BroadcastAddressOnNICEnableDisable(t *testing.T) { e := loopback.New() s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, }) nicOpts := stack.NICOptions{Disabled: true} if err := s.CreateNICWithOptions(nicID, e, nicOpts); err != nil { @@ -2921,7 +2968,7 @@ func TestLeaveIPv6SolicitedNodeAddrBeforeAddrRemoval(t *testing.T) { const nicID = 1 s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocol}, }) e := channel.New(10, 1280, linkAddr1) if err := s.CreateNIC(1, e); err != nil { @@ -2982,7 +3029,7 @@ func TestJoinLeaveMulticastOnNICEnableDisable(t *testing.T) { t.Run(test.name, func(t *testing.T) { e := loopback.New() s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol}, }) nicOpts := stack.NICOptions{Disabled: true} if err := s.CreateNICWithOptions(nicID, e, nicOpts); err != nil { @@ -3059,12 +3106,13 @@ func TestDoDADWhenNICEnabled(t *testing.T) { dadC: make(chan ndpDADEvent), } opts := stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv6.NewProtocol()}, - NDPConfigs: stack.NDPConfigurations{ - DupAddrDetectTransmits: dadTransmits, - RetransmitTimer: retransmitTimer, - }, - NDPDisp: &ndpDisp, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPConfigs: ipv6.NDPConfigurations{ + DupAddrDetectTransmits: dadTransmits, + RetransmitTimer: retransmitTimer, + }, + NDPDisp: &ndpDisp, + })}, } e := channel.New(dadTransmits, 1280, linkAddr1) @@ -3423,7 +3471,7 @@ func TestOutgoingSubnetBroadcast(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol}, }) ep := channel.New(0, defaultMTU, "") if err := s.CreateNIC(nicID1, ep); err != nil { @@ -3461,7 +3509,7 @@ func TestResolveWith(t *testing.T) { ) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), arp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, arp.NewProtocol}, }) ep := channel.New(0, defaultMTU, "") ep.LinkEPCapabilities |= stack.CapabilityResolutionRequired @@ -3499,3 +3547,131 @@ func TestResolveWith(t *testing.T) { t.Fatal("got r.IsResolutionRequired() = true, want = false") } } + +// TestRouteReleaseAfterAddrRemoval tests that releasing a Route after its +// associated address is removed should not cause a panic. +func TestRouteReleaseAfterAddrRemoval(t *testing.T) { + const ( + nicID = 1 + localAddr = tcpip.Address("\x01") + remoteAddr = tcpip.Address("\x02") + ) + + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, + }) + + ep := channel.New(0, defaultMTU, "") + if err := s.CreateNIC(nicID, ep); err != nil { + t.Fatalf("CreateNIC(%d, _): %s", nicID, err) + } + if err := s.AddAddress(nicID, fakeNetNumber, localAddr); err != nil { + t.Fatalf("s.AddAddress(%d, %d, %s): %s", nicID, fakeNetNumber, localAddr, err) + } + { + subnet, err := tcpip.NewSubnet("\x00", "\x00") + if err != nil { + t.Fatal(err) + } + s.SetRouteTable([]tcpip.Route{{Destination: subnet, Gateway: "\x00", NIC: 1}}) + } + + r, err := s.FindRoute(nicID, localAddr, remoteAddr, fakeNetNumber, false /* multicastLoop */) + if err != nil { + t.Fatalf("s.FindRoute(%d, %s, %s, %d, false): %s", nicID, localAddr, remoteAddr, fakeNetNumber, err) + } + // Should not panic. + defer r.Release() + + // Check that removing the same address fails. + if err := s.RemoveAddress(nicID, localAddr); err != nil { + t.Fatalf("s.RemoveAddress(%d, %s): %s", nicID, localAddr, err) + } +} + +func TestGetNetworkEndpoint(t *testing.T) { + const nicID = 1 + + tests := []struct { + name string + protoFactory stack.NetworkProtocolFactory + protoNum tcpip.NetworkProtocolNumber + }{ + { + name: "IPv4", + protoFactory: ipv4.NewProtocol, + protoNum: ipv4.ProtocolNumber, + }, + { + name: "IPv6", + protoFactory: ipv6.NewProtocol, + protoNum: ipv6.ProtocolNumber, + }, + } + + factories := make([]stack.NetworkProtocolFactory, 0, len(tests)) + for _, test := range tests { + factories = append(factories, test.protoFactory) + } + + s := stack.New(stack.Options{ + NetworkProtocols: factories, + }) + + if err := s.CreateNIC(nicID, channel.New(0, defaultMTU, "")); err != nil { + t.Fatalf("CreateNIC(%d, _): %s", nicID, err) + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + ep, err := s.GetNetworkEndpoint(nicID, test.protoNum) + if err != nil { + t.Fatalf("s.GetNetworkEndpoint(%d, %d): %s", nicID, test.protoNum, err) + } + + if got := ep.NetworkProtocolNumber(); got != test.protoNum { + t.Fatalf("got ep.NetworkProtocolNumber() = %d, want = %d", got, test.protoNum) + } + }) + } +} + +func TestGetMainNICAddressWhenNICDisabled(t *testing.T) { + const nicID = 1 + + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, + }) + + if err := s.CreateNIC(nicID, channel.New(0, defaultMTU, "")); err != nil { + t.Fatalf("CreateNIC(%d, _): %s", nicID, err) + } + + protocolAddress := tcpip.ProtocolAddress{ + Protocol: fakeNetNumber, + AddressWithPrefix: tcpip.AddressWithPrefix{ + Address: "\x01", + PrefixLen: 8, + }, + } + if err := s.AddProtocolAddress(nicID, protocolAddress); err != nil { + t.Fatalf("AddProtocolAddress(%d, %#v): %s", nicID, protocolAddress, err) + } + + // Check that we get the right initial address and prefix length. + if gotAddr, err := s.GetMainNICAddress(nicID, fakeNetNumber); err != nil { + t.Fatalf("GetMainNICAddress(%d, %d): %s", nicID, fakeNetNumber, err) + } else if gotAddr != protocolAddress.AddressWithPrefix { + t.Fatalf("got GetMainNICAddress(%d, %d) = %s, want = %s", nicID, fakeNetNumber, gotAddr, protocolAddress.AddressWithPrefix) + } + + // Should still get the address when the NIC is diabled. + if err := s.DisableNIC(nicID); err != nil { + t.Fatalf("DisableNIC(%d): %s", nicID, err) + } + if gotAddr, err := s.GetMainNICAddress(nicID, fakeNetNumber); err != nil { + t.Fatalf("GetMainNICAddress(%d, %d): %s", nicID, fakeNetNumber, err) + } else if gotAddr != protocolAddress.AddressWithPrefix { + t.Fatalf("got GetMainNICAddress(%d, %d) = %s, want = %s", nicID, fakeNetNumber, gotAddr, protocolAddress.AddressWithPrefix) + } +} diff --git a/pkg/tcpip/stack/transport_demuxer.go b/pkg/tcpip/stack/transport_demuxer.go index 0774b5382..35e5b1a2e 100644 --- a/pkg/tcpip/stack/transport_demuxer.go +++ b/pkg/tcpip/stack/transport_demuxer.go @@ -155,7 +155,7 @@ func (epsByNIC *endpointsByNIC) transportEndpoints() []TransportEndpoint { func (epsByNIC *endpointsByNIC) handlePacket(r *Route, id TransportEndpointID, pkt *PacketBuffer) { epsByNIC.mu.RLock() - mpep, ok := epsByNIC.endpoints[r.ref.nic.ID()] + mpep, ok := epsByNIC.endpoints[r.nic.ID()] if !ok { if mpep, ok = epsByNIC.endpoints[0]; !ok { epsByNIC.mu.RUnlock() // Don't use defer for performance reasons. @@ -544,9 +544,11 @@ func (d *transportDemuxer) deliverPacket(r *Route, protocol tcpip.TransportProto return true } - // If the packet is a TCP packet with a non-unicast source or destination - // address, then do nothing further and instruct the caller to do the same. - if protocol == header.TCPProtocolNumber && (!isInboundUnicast(r) || !isOutboundUnicast(r)) { + // If the packet is a TCP packet with a unspecified source or non-unicast + // destination address, then do nothing further and instruct the caller to do + // the same. The network layer handles address validation for specified source + // addresses. + if protocol == header.TCPProtocolNumber && (!isSpecified(r.LocalAddress) || !isSpecified(r.RemoteAddress) || isInboundMulticastOrBroadcast(r)) { // TCP can only be used to communicate between a single source and a // single destination; the addresses must be unicast. r.Stats().TCP.InvalidSegmentsReceived.Increment() @@ -626,7 +628,7 @@ func (d *transportDemuxer) findTransportEndpoint(netProto tcpip.NetworkProtocolN epsByNIC.mu.RLock() eps.mu.RUnlock() - mpep, ok := epsByNIC.endpoints[r.ref.nic.ID()] + mpep, ok := epsByNIC.endpoints[r.nic.ID()] if !ok { if mpep, ok = epsByNIC.endpoints[0]; !ok { epsByNIC.mu.RUnlock() // Don't use defer for performance reasons. @@ -681,10 +683,6 @@ func isInboundMulticastOrBroadcast(r *Route) bool { return r.IsInboundBroadcast() || header.IsV4MulticastAddress(r.LocalAddress) || header.IsV6MulticastAddress(r.LocalAddress) } -func isInboundUnicast(r *Route) bool { - return r.LocalAddress != header.IPv4Any && r.LocalAddress != header.IPv6Any && !isInboundMulticastOrBroadcast(r) -} - -func isOutboundUnicast(r *Route) bool { - return r.RemoteAddress != header.IPv4Any && r.RemoteAddress != header.IPv6Any && !r.IsOutboundBroadcast() && !header.IsV4MulticastAddress(r.RemoteAddress) && !header.IsV6MulticastAddress(r.RemoteAddress) +func isSpecified(addr tcpip.Address) bool { + return addr != header.IPv4Any && addr != header.IPv6Any } diff --git a/pkg/tcpip/stack/transport_demuxer_test.go b/pkg/tcpip/stack/transport_demuxer_test.go index 4d6d62eec..698c8609e 100644 --- a/pkg/tcpip/stack/transport_demuxer_test.go +++ b/pkg/tcpip/stack/transport_demuxer_test.go @@ -51,8 +51,8 @@ type testContext struct { // newDualTestContextMultiNIC creates the testing context and also linkEpIDs NICs. func newDualTestContextMultiNIC(t *testing.T, mtu uint32, linkEpIDs []tcpip.NICID) *testContext { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, }) linkEps := make(map[tcpip.NICID]*channel.Endpoint) for _, linkEpID := range linkEpIDs { @@ -182,8 +182,8 @@ func TestTransportDemuxerRegister(t *testing.T) { } { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, }) var wq waiter.Queue ep, err := s.NewEndpoint(udp.ProtocolNumber, ipv4.ProtocolNumber, &wq) diff --git a/pkg/tcpip/stack/transport_test.go b/pkg/tcpip/stack/transport_test.go index ef3457e32..62ab6d92f 100644 --- a/pkg/tcpip/stack/transport_test.go +++ b/pkg/tcpip/stack/transport_test.go @@ -39,7 +39,7 @@ const ( // use it. type fakeTransportEndpoint struct { stack.TransportEndpointInfo - stack *stack.Stack + proto *fakeTransportProtocol peerAddr tcpip.Address route stack.Route @@ -59,8 +59,8 @@ func (*fakeTransportEndpoint) Stats() tcpip.EndpointStats { func (*fakeTransportEndpoint) SetOwner(owner tcpip.PacketOwner) {} -func newFakeTransportEndpoint(s *stack.Stack, proto *fakeTransportProtocol, netProto tcpip.NetworkProtocolNumber, uniqueID uint64) tcpip.Endpoint { - return &fakeTransportEndpoint{stack: s, TransportEndpointInfo: stack.TransportEndpointInfo{NetProto: netProto}, proto: proto, uniqueID: uniqueID} +func newFakeTransportEndpoint(proto *fakeTransportProtocol, netProto tcpip.NetworkProtocolNumber, uniqueID uint64) tcpip.Endpoint { + return &fakeTransportEndpoint{TransportEndpointInfo: stack.TransportEndpointInfo{NetProto: netProto}, proto: proto, uniqueID: uniqueID} } func (f *fakeTransportEndpoint) Abort() { @@ -143,7 +143,7 @@ func (f *fakeTransportEndpoint) Connect(addr tcpip.FullAddress) *tcpip.Error { f.peerAddr = addr.Addr // Find the route. - r, err := f.stack.FindRoute(addr.NIC, "", addr.Addr, fakeNetNumber, false /* multicastLoop */) + r, err := f.proto.stack.FindRoute(addr.NIC, "", addr.Addr, fakeNetNumber, false /* multicastLoop */) if err != nil { return tcpip.ErrNoRoute } @@ -151,7 +151,7 @@ func (f *fakeTransportEndpoint) Connect(addr tcpip.FullAddress) *tcpip.Error { // Try to register so that we can start receiving packets. f.ID.RemoteAddress = addr.Addr - err = f.stack.RegisterTransportEndpoint(0, []tcpip.NetworkProtocolNumber{fakeNetNumber}, fakeTransNumber, f.ID, f, ports.Flags{}, 0 /* bindToDevice */) + err = f.proto.stack.RegisterTransportEndpoint(0, []tcpip.NetworkProtocolNumber{fakeNetNumber}, fakeTransNumber, f.ID, f, ports.Flags{}, 0 /* bindToDevice */) if err != nil { return err } @@ -190,7 +190,7 @@ func (f *fakeTransportEndpoint) Accept(*tcpip.FullAddress) (tcpip.Endpoint, *wai } func (f *fakeTransportEndpoint) Bind(a tcpip.FullAddress) *tcpip.Error { - if err := f.stack.RegisterTransportEndpoint( + if err := f.proto.stack.RegisterTransportEndpoint( a.NIC, []tcpip.NetworkProtocolNumber{fakeNetNumber}, fakeTransNumber, @@ -218,7 +218,6 @@ func (f *fakeTransportEndpoint) HandlePacket(r *stack.Route, id stack.TransportE f.proto.packetCount++ if f.acceptQueue != nil { f.acceptQueue = append(f.acceptQueue, fakeTransportEndpoint{ - stack: f.stack, TransportEndpointInfo: stack.TransportEndpointInfo{ ID: f.ID, NetProto: f.NetProto, @@ -262,6 +261,8 @@ type fakeTransportProtocolOptions struct { // fakeTransportProtocol is a transport-layer protocol descriptor. It // aggregates the number of packets received via endpoints of this protocol. type fakeTransportProtocol struct { + stack *stack.Stack + packetCount int controlCount int opts fakeTransportProtocolOptions @@ -271,11 +272,11 @@ func (*fakeTransportProtocol) Number() tcpip.TransportProtocolNumber { return fakeTransNumber } -func (f *fakeTransportProtocol) NewEndpoint(stack *stack.Stack, netProto tcpip.NetworkProtocolNumber, _ *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { - return newFakeTransportEndpoint(stack, f, netProto, stack.UniqueID()), nil +func (f *fakeTransportProtocol) NewEndpoint(netProto tcpip.NetworkProtocolNumber, _ *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { + return newFakeTransportEndpoint(f, netProto, f.stack.UniqueID()), nil } -func (*fakeTransportProtocol) NewRawEndpoint(stack *stack.Stack, netProto tcpip.NetworkProtocolNumber, _ *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { +func (*fakeTransportProtocol) NewRawEndpoint(tcpip.NetworkProtocolNumber, *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { return nil, tcpip.ErrUnknownProtocol } @@ -287,8 +288,8 @@ func (*fakeTransportProtocol) ParsePorts(buffer.View) (src, dst uint16, err *tcp return 0, 0, nil } -func (*fakeTransportProtocol) HandleUnknownDestinationPacket(*stack.Route, stack.TransportEndpointID, *stack.PacketBuffer) bool { - return true +func (*fakeTransportProtocol) HandleUnknownDestinationPacket(*stack.Route, stack.TransportEndpointID, *stack.PacketBuffer) stack.UnknownDestinationPacketDisposition { + return stack.UnknownDestinationPacketHandled } func (f *fakeTransportProtocol) SetOption(option tcpip.SettableTransportProtocolOption) *tcpip.Error { @@ -326,15 +327,15 @@ func (*fakeTransportProtocol) Parse(pkt *stack.PacketBuffer) bool { return ok } -func fakeTransFactory() stack.TransportProtocol { - return &fakeTransportProtocol{} +func fakeTransFactory(s *stack.Stack) stack.TransportProtocol { + return &fakeTransportProtocol{stack: s} } func TestTransportReceive(t *testing.T) { linkEP := channel.New(10, defaultMTU, "") s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, - TransportProtocols: []stack.TransportProtocol{fakeTransFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, + TransportProtocols: []stack.TransportProtocolFactory{fakeTransFactory}, }) if err := s.CreateNIC(1, linkEP); err != nil { t.Fatalf("CreateNIC failed: %v", err) @@ -404,8 +405,8 @@ func TestTransportReceive(t *testing.T) { func TestTransportControlReceive(t *testing.T) { linkEP := channel.New(10, defaultMTU, "") s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, - TransportProtocols: []stack.TransportProtocol{fakeTransFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, + TransportProtocols: []stack.TransportProtocolFactory{fakeTransFactory}, }) if err := s.CreateNIC(1, linkEP); err != nil { t.Fatalf("CreateNIC failed: %v", err) @@ -481,8 +482,8 @@ func TestTransportControlReceive(t *testing.T) { func TestTransportSend(t *testing.T) { linkEP := channel.New(10, defaultMTU, "") s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, - TransportProtocols: []stack.TransportProtocol{fakeTransFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, + TransportProtocols: []stack.TransportProtocolFactory{fakeTransFactory}, }) if err := s.CreateNIC(1, linkEP); err != nil { t.Fatalf("CreateNIC failed: %v", err) @@ -527,8 +528,8 @@ func TestTransportSend(t *testing.T) { func TestTransportOptions(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, - TransportProtocols: []stack.TransportProtocol{fakeTransFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, + TransportProtocols: []stack.TransportProtocolFactory{fakeTransFactory}, }) v := tcpip.TCPModerateReceiveBufferOption(true) @@ -546,10 +547,10 @@ func TestTransportOptions(t *testing.T) { func TestTransportForwarding(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{fakeNetFactory()}, - TransportProtocols: []stack.TransportProtocol{fakeTransFactory()}, + NetworkProtocols: []stack.NetworkProtocolFactory{fakeNetFactory}, + TransportProtocols: []stack.TransportProtocolFactory{fakeTransFactory}, }) - s.SetForwarding(true) + s.SetForwarding(fakeNetNumber, true) // TODO(b/123449044): Change this to a channel NIC. ep1 := loopback.New() diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go index b2ddb24ec..c42bb0991 100644 --- a/pkg/tcpip/tcpip.go +++ b/pkg/tcpip/tcpip.go @@ -237,6 +237,14 @@ type Timer interface { // network node. Or, in the case of unix endpoints, it may represent a path. type Address string +// WithPrefix returns the address with a prefix that represents a point subnet. +func (a Address) WithPrefix() AddressWithPrefix { + return AddressWithPrefix{ + Address: a, + PrefixLen: len(a) * 8, + } +} + // AddressMask is a bitmask for an address. type AddressMask string @@ -1474,6 +1482,18 @@ type IPStats struct { // MalformedFragmentsReceived is the total number of IP Fragments that were // dropped due to the fragment failing validation checks. MalformedFragmentsReceived *StatCounter + + // IPTablesPreroutingDropped is the total number of IP packets dropped + // in the Prerouting chain. + IPTablesPreroutingDropped *StatCounter + + // IPTablesInputDropped is the total number of IP packets dropped in + // the Input chain. + IPTablesInputDropped *StatCounter + + // IPTablesOutputDropped is the total number of IP packets dropped in + // the Output chain. + IPTablesOutputDropped *StatCounter } // TCPStats collects TCP-specific stats. @@ -1602,9 +1622,6 @@ type UDPStats struct { // ChecksumErrors is the number of datagrams dropped due to bad checksums. ChecksumErrors *StatCounter - - // InvalidSourceAddress is the number of invalid sourced datagrams dropped. - InvalidSourceAddress *StatCounter } // Stats holds statistics about the networking stack. diff --git a/pkg/tcpip/tests/integration/loopback_test.go b/pkg/tcpip/tests/integration/loopback_test.go index fecbe7ba7..e8caf09ba 100644 --- a/pkg/tcpip/tests/integration/loopback_test.go +++ b/pkg/tcpip/tests/integration/loopback_test.go @@ -16,6 +16,7 @@ package integration_test import ( "testing" + "time" "github.com/google/go-cmp/cmp" "gvisor.dev/gvisor/pkg/tcpip" @@ -29,6 +30,69 @@ import ( "gvisor.dev/gvisor/pkg/waiter" ) +var _ ipv6.NDPDispatcher = (*ndpDispatcher)(nil) + +type ndpDispatcher struct{} + +func (*ndpDispatcher) OnDuplicateAddressDetectionStatus(tcpip.NICID, tcpip.Address, bool, *tcpip.Error) { +} + +func (*ndpDispatcher) OnDefaultRouterDiscovered(tcpip.NICID, tcpip.Address) bool { + return false +} + +func (*ndpDispatcher) OnDefaultRouterInvalidated(tcpip.NICID, tcpip.Address) {} + +func (*ndpDispatcher) OnOnLinkPrefixDiscovered(tcpip.NICID, tcpip.Subnet) bool { + return false +} + +func (*ndpDispatcher) OnOnLinkPrefixInvalidated(tcpip.NICID, tcpip.Subnet) {} + +func (*ndpDispatcher) OnAutoGenAddress(tcpip.NICID, tcpip.AddressWithPrefix) bool { + return true +} + +func (*ndpDispatcher) OnAutoGenAddressDeprecated(tcpip.NICID, tcpip.AddressWithPrefix) {} + +func (*ndpDispatcher) OnAutoGenAddressInvalidated(tcpip.NICID, tcpip.AddressWithPrefix) {} + +func (*ndpDispatcher) OnRecursiveDNSServerOption(tcpip.NICID, []tcpip.Address, time.Duration) {} + +func (*ndpDispatcher) OnDNSSearchListOption(tcpip.NICID, []string, time.Duration) {} + +func (*ndpDispatcher) OnDHCPv6Configuration(tcpip.NICID, ipv6.DHCPv6ConfigurationFromNDPRA) {} + +// TestInitialLoopbackAddresses tests that the loopback interface does not +// auto-generate a link-local address when it is brought up. +func TestInitialLoopbackAddresses(t *testing.T) { + const nicID = 1 + + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocolWithOptions(ipv6.Options{ + NDPDisp: &ndpDispatcher{}, + AutoGenIPv6LinkLocal: true, + OpaqueIIDOpts: ipv6.OpaqueInterfaceIdentifierOptions{ + NICNameFromID: func(nicID tcpip.NICID, nicName string) string { + t.Fatalf("should not attempt to get name for NIC with ID = %d; nicName = %s", nicID, nicName) + return "" + }, + }, + })}, + }) + + if err := s.CreateNIC(nicID, loopback.New()); err != nil { + t.Fatalf("CreateNIC(%d, _): %s", nicID, err) + } + + nicsInfo := s.NICInfo() + if nicInfo, ok := nicsInfo[nicID]; !ok { + t.Fatalf("did not find NIC with ID = %d in s.NICInfo() = %#v", nicID, nicsInfo) + } else if got := len(nicInfo.ProtocolAddresses); got != 0 { + t.Fatalf("got len(nicInfo.ProtocolAddresses) = %d, want = 0; nicInfo.ProtocolAddresses = %#v", got, nicInfo.ProtocolAddresses) + } +} + // TestLoopbackAcceptAllInSubnet tests that a loopback interface considers // itself bound to all addresses in the subnet of an assigned address. func TestLoopbackAcceptAllInSubnet(t *testing.T) { @@ -120,8 +184,8 @@ func TestLoopbackAcceptAllInSubnet(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, }) if err := s.CreateNIC(nicID, loopback.New()); err != nil { t.Fatalf("CreateNIC(%d, _): %s", nicID, err) @@ -203,7 +267,7 @@ func TestLoopbackSubnetLifetimeBoundToAddr(t *testing.T) { otherAddr := tcpip.Address(addrBytes) s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, }) if err := s.CreateNIC(nicID, loopback.New()); err != nil { t.Fatalf("s.CreateNIC(%d, _): %s", nicID, err) diff --git a/pkg/tcpip/tests/integration/multicast_broadcast_test.go b/pkg/tcpip/tests/integration/multicast_broadcast_test.go index 659acbc7a..72d86b5ab 100644 --- a/pkg/tcpip/tests/integration/multicast_broadcast_test.go +++ b/pkg/tcpip/tests/integration/multicast_broadcast_test.go @@ -140,11 +140,9 @@ func TestPingMulticastBroadcast(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - ipv4Proto := ipv4.NewProtocol() - ipv6Proto := ipv6.NewProtocol() s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4Proto, ipv6Proto}, - TransportProtocols: []stack.TransportProtocol{icmp.NewProtocol4(), icmp.NewProtocol6()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{icmp.NewProtocol4, icmp.NewProtocol6}, }) // We only expect a single packet in response to our ICMP Echo Request. e := channel.New(1, defaultMTU, "") @@ -176,18 +174,18 @@ func TestPingMulticastBroadcast(t *testing.T) { var rxICMP func(*channel.Endpoint, tcpip.Address) var expectedSrc tcpip.Address var expectedDst tcpip.Address - var proto stack.NetworkProtocol + var protoNum tcpip.NetworkProtocolNumber switch l := len(test.dstAddr); l { case header.IPv4AddressSize: rxICMP = rxIPv4ICMP expectedSrc = ipv4Addr.Address expectedDst = remoteIPv4Addr - proto = ipv4Proto + protoNum = header.IPv4ProtocolNumber case header.IPv6AddressSize: rxICMP = rxIPv6ICMP expectedSrc = ipv6Addr.Address expectedDst = remoteIPv6Addr - proto = ipv6Proto + protoNum = header.IPv6ProtocolNumber default: t.Fatalf("got unexpected address length = %d bytes", l) } @@ -205,7 +203,7 @@ func TestPingMulticastBroadcast(t *testing.T) { t.Errorf("got pkt.Route.RemoteAddress = %s, want = %s", pkt.Route.RemoteAddress, expectedDst) } - src, dst := proto.ParseAddresses(pkt.Pkt.NetworkHeader().View()) + src, dst := s.NetworkProtocolInstance(protoNum).ParseAddresses(pkt.Pkt.NetworkHeader().View()) if src != expectedSrc { t.Errorf("got pkt source = %s, want = %s", src, expectedSrc) } @@ -380,8 +378,8 @@ func TestIncomingMulticastAndBroadcast(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, }) e := channel.New(0, defaultMTU, "") if err := s.CreateNIC(nicID, e); err != nil { @@ -466,8 +464,8 @@ func TestReuseAddrAndBroadcast(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, }) if err := s.CreateNIC(nicID, loopback.New()); err != nil { t.Fatalf("CreateNIC(%d, _): %s", nicID, err) diff --git a/pkg/tcpip/transport/icmp/endpoint.go b/pkg/tcpip/transport/icmp/endpoint.go index ad71ff3b6..41eb0ca44 100644 --- a/pkg/tcpip/transport/icmp/endpoint.go +++ b/pkg/tcpip/transport/icmp/endpoint.go @@ -74,6 +74,8 @@ type endpoint struct { route stack.Route `state:"manual"` ttl uint8 stats tcpip.TransportEndpointStats `state:"nosave"` + // linger is used for SO_LINGER socket option. + linger tcpip.LingerOption // owner is used to get uid and gid of the packet. owner tcpip.PacketOwner @@ -344,9 +346,14 @@ func (e *endpoint) Peek([][]byte) (int64, tcpip.ControlMessages, *tcpip.Error) { // SetSockOpt sets a socket option. func (e *endpoint) SetSockOpt(opt tcpip.SettableSocketOption) *tcpip.Error { - switch opt.(type) { + switch v := opt.(type) { case *tcpip.SocketDetachFilterOption: return nil + + case *tcpip.LingerOption: + e.mu.Lock() + e.linger = *v + e.mu.Unlock() } return nil } @@ -415,8 +422,17 @@ func (e *endpoint) GetSockOptInt(opt tcpip.SockOptInt) (int, *tcpip.Error) { } // GetSockOpt implements tcpip.Endpoint.GetSockOpt. -func (*endpoint) GetSockOpt(tcpip.GettableSocketOption) *tcpip.Error { - return tcpip.ErrUnknownProtocolOption +func (e *endpoint) GetSockOpt(opt tcpip.GettableSocketOption) *tcpip.Error { + switch o := opt.(type) { + case *tcpip.LingerOption: + e.mu.Lock() + *o = e.linger + e.mu.Unlock() + return nil + + default: + return tcpip.ErrUnknownProtocolOption + } } func send4(r *stack.Route, ident uint16, data buffer.View, ttl uint8, owner tcpip.PacketOwner) *tcpip.Error { @@ -430,6 +446,7 @@ func send4(r *stack.Route, ident uint16, data buffer.View, ttl uint8, owner tcpi pkt.Owner = owner icmpv4 := header.ICMPv4(pkt.TransportHeader().Push(header.ICMPv4MinimumSize)) + pkt.TransportProtocolNumber = header.ICMPv4ProtocolNumber copy(icmpv4, data) // Set the ident to the user-specified port. Sequence number should // already be set by the user. @@ -462,6 +479,7 @@ func send6(r *stack.Route, ident uint16, data buffer.View, ttl uint8) *tcpip.Err }) icmpv6 := header.ICMPv6(pkt.TransportHeader().Push(header.ICMPv6MinimumSize)) + pkt.TransportProtocolNumber = header.ICMPv6ProtocolNumber copy(icmpv6, data) // Set the ident. Sequence number is provided by the user. icmpv6.SetIdent(ident) diff --git a/pkg/tcpip/transport/icmp/protocol.go b/pkg/tcpip/transport/icmp/protocol.go index bb11e4e83..87d510f96 100644 --- a/pkg/tcpip/transport/icmp/protocol.go +++ b/pkg/tcpip/transport/icmp/protocol.go @@ -13,12 +13,7 @@ // limitations under the License. // Package icmp contains the implementation of the ICMP and IPv6-ICMP transport -// protocols for use in ping. To use it in the networking stack, this package -// must be added to the project, and activated on the stack by passing -// icmp.NewProtocol4() and/or icmp.NewProtocol6() as one of the transport -// protocols when calling stack.New(). Then endpoints can be created by passing -// icmp.ProtocolNumber or icmp.ProtocolNumber6 as the transport protocol number -// when calling Stack.NewEndpoint(). +// protocols for use in ping. package icmp import ( @@ -42,6 +37,8 @@ const ( // protocol implements stack.TransportProtocol. type protocol struct { + stack *stack.Stack + number tcpip.TransportProtocolNumber } @@ -62,20 +59,20 @@ func (p *protocol) netProto() tcpip.NetworkProtocolNumber { // NewEndpoint creates a new icmp endpoint. It implements // stack.TransportProtocol.NewEndpoint. -func (p *protocol) NewEndpoint(stack *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { +func (p *protocol) NewEndpoint(netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { if netProto != p.netProto() { return nil, tcpip.ErrUnknownProtocol } - return newEndpoint(stack, netProto, p.number, waiterQueue) + return newEndpoint(p.stack, netProto, p.number, waiterQueue) } // NewRawEndpoint creates a new raw icmp endpoint. It implements // stack.TransportProtocol.NewRawEndpoint. -func (p *protocol) NewRawEndpoint(stack *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { +func (p *protocol) NewRawEndpoint(netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { if netProto != p.netProto() { return nil, tcpip.ErrUnknownProtocol } - return raw.NewEndpoint(stack, netProto, p.number, waiterQueue) + return raw.NewEndpoint(p.stack, netProto, p.number, waiterQueue) } // MinimumPacketSize returns the minimum valid icmp packet size. @@ -104,8 +101,8 @@ func (p *protocol) ParsePorts(v buffer.View) (src, dst uint16, err *tcpip.Error) // HandleUnknownDestinationPacket handles packets targeted at this protocol but // that don't match any existing endpoint. -func (*protocol) HandleUnknownDestinationPacket(*stack.Route, stack.TransportEndpointID, *stack.PacketBuffer) bool { - return true +func (*protocol) HandleUnknownDestinationPacket(*stack.Route, stack.TransportEndpointID, *stack.PacketBuffer) stack.UnknownDestinationPacketDisposition { + return stack.UnknownDestinationPacketHandled } // SetOption implements stack.TransportProtocol.SetOption. @@ -135,11 +132,11 @@ func (*protocol) Parse(pkt *stack.PacketBuffer) bool { } // NewProtocol4 returns an ICMPv4 transport protocol. -func NewProtocol4() stack.TransportProtocol { - return &protocol{ProtocolNumber4} +func NewProtocol4(s *stack.Stack) stack.TransportProtocol { + return &protocol{stack: s, number: ProtocolNumber4} } // NewProtocol6 returns an ICMPv6 transport protocol. -func NewProtocol6() stack.TransportProtocol { - return &protocol{ProtocolNumber6} +func NewProtocol6(s *stack.Stack) stack.TransportProtocol { + return &protocol{stack: s, number: ProtocolNumber6} } diff --git a/pkg/tcpip/transport/packet/endpoint.go b/pkg/tcpip/transport/packet/endpoint.go index 8bd4e5e37..072601d2d 100644 --- a/pkg/tcpip/transport/packet/endpoint.go +++ b/pkg/tcpip/transport/packet/endpoint.go @@ -83,6 +83,8 @@ type endpoint struct { stats tcpip.TransportEndpointStats `state:"nosave"` bound bool boundNIC tcpip.NICID + // linger is used for SO_LINGER socket option. + linger tcpip.LingerOption // lastErrorMu protects lastError. lastErrorMu sync.Mutex `state:"nosave"` @@ -298,10 +300,16 @@ func (ep *endpoint) Readiness(mask waiter.EventMask) waiter.EventMask { // used with SetSockOpt, and this function always returns // tcpip.ErrNotSupported. func (ep *endpoint) SetSockOpt(opt tcpip.SettableSocketOption) *tcpip.Error { - switch opt.(type) { + switch v := opt.(type) { case *tcpip.SocketDetachFilterOption: return nil + case *tcpip.LingerOption: + ep.mu.Lock() + ep.linger = *v + ep.mu.Unlock() + return nil + default: return tcpip.ErrUnknownProtocolOption } @@ -366,8 +374,17 @@ func (ep *endpoint) LastError() *tcpip.Error { } // GetSockOpt implements tcpip.Endpoint.GetSockOpt. -func (*endpoint) GetSockOpt(tcpip.GettableSocketOption) *tcpip.Error { - return tcpip.ErrNotSupported +func (ep *endpoint) GetSockOpt(opt tcpip.GettableSocketOption) *tcpip.Error { + switch o := opt.(type) { + case *tcpip.LingerOption: + ep.mu.Lock() + *o = ep.linger + ep.mu.Unlock() + return nil + + default: + return tcpip.ErrNotSupported + } } // GetSockOptBool implements tcpip.Endpoint.GetSockOptBool. diff --git a/pkg/tcpip/transport/raw/endpoint.go b/pkg/tcpip/transport/raw/endpoint.go index fb03e6047..e37c00523 100644 --- a/pkg/tcpip/transport/raw/endpoint.go +++ b/pkg/tcpip/transport/raw/endpoint.go @@ -84,6 +84,8 @@ type endpoint struct { // Connect(), and is valid only when conneted is true. route stack.Route `state:"manual"` stats tcpip.TransportEndpointStats `state:"nosave"` + // linger is used for SO_LINGER socket option. + linger tcpip.LingerOption // owner is used to get uid and gid of the packet. owner tcpip.PacketOwner @@ -511,10 +513,16 @@ func (e *endpoint) Readiness(mask waiter.EventMask) waiter.EventMask { // SetSockOpt implements tcpip.Endpoint.SetSockOpt. func (e *endpoint) SetSockOpt(opt tcpip.SettableSocketOption) *tcpip.Error { - switch opt.(type) { + switch v := opt.(type) { case *tcpip.SocketDetachFilterOption: return nil + case *tcpip.LingerOption: + e.mu.Lock() + e.linger = *v + e.mu.Unlock() + return nil + default: return tcpip.ErrUnknownProtocolOption } @@ -577,8 +585,17 @@ func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error { } // GetSockOpt implements tcpip.Endpoint.GetSockOpt. -func (*endpoint) GetSockOpt(tcpip.GettableSocketOption) *tcpip.Error { - return tcpip.ErrUnknownProtocolOption +func (e *endpoint) GetSockOpt(opt tcpip.GettableSocketOption) *tcpip.Error { + switch o := opt.(type) { + case *tcpip.LingerOption: + e.mu.Lock() + *o = e.linger + e.mu.Unlock() + return nil + + default: + return tcpip.ErrUnknownProtocolOption + } } // GetSockOptBool implements tcpip.Endpoint.GetSockOptBool. diff --git a/pkg/tcpip/transport/tcp/BUILD b/pkg/tcpip/transport/tcp/BUILD index 234fb95ce..518449602 100644 --- a/pkg/tcpip/transport/tcp/BUILD +++ b/pkg/tcpip/transport/tcp/BUILD @@ -69,6 +69,7 @@ go_library( "//pkg/tcpip/buffer", "//pkg/tcpip/hash/jenkins", "//pkg/tcpip/header", + "//pkg/tcpip/header/parse", "//pkg/tcpip/ports", "//pkg/tcpip/seqnum", "//pkg/tcpip/stack", @@ -93,6 +94,7 @@ go_test( shard_count = 10, deps = [ ":tcp", + "//pkg/rand", "//pkg/sync", "//pkg/tcpip", "//pkg/tcpip/buffer", diff --git a/pkg/tcpip/transport/tcp/connect.go b/pkg/tcpip/transport/tcp/connect.go index 09d53d158..189c01c8f 100644 --- a/pkg/tcpip/transport/tcp/connect.go +++ b/pkg/tcpip/transport/tcp/connect.go @@ -747,6 +747,7 @@ func (e *endpoint) sendTCP(r *stack.Route, tf tcpFields, data buffer.VectorisedV func buildTCPHdr(r *stack.Route, tf tcpFields, pkt *stack.PacketBuffer, gso *stack.GSO) { optLen := len(tf.opts) tcp := header.TCP(pkt.TransportHeader().Push(header.TCPMinimumSize + optLen)) + pkt.TransportProtocolNumber = header.TCPProtocolNumber tcp.Encode(&header.TCPFields{ SrcPort: tf.id.LocalPort, DstPort: tf.id.RemotePort, @@ -803,7 +804,7 @@ func sendTCPBatch(r *stack.Route, tf tcpFields, data buffer.VectorisedView, gso pkt.Owner = owner pkt.EgressRoute = r pkt.GSOOptions = gso - pkt.NetworkProtocolNumber = r.NetworkProtocolNumber() + pkt.NetworkProtocolNumber = r.NetProto data.ReadToVV(&pkt.Data, packetSize) buildTCPHdr(r, tf, pkt, gso) tf.seq = tf.seq.Add(seqnum.Size(packetSize)) @@ -897,7 +898,7 @@ func (e *endpoint) makeOptions(sackBlocks []header.SACKBlock) []byte { // sendRaw sends a TCP segment to the endpoint's peer. func (e *endpoint) sendRaw(data buffer.VectorisedView, flags byte, seq, ack seqnum.Value, rcvWnd seqnum.Size) *tcpip.Error { var sackBlocks []header.SACKBlock - if e.EndpointState() == StateEstablished && e.rcv.pendingBufSize > 0 && (flags&header.TCPFlagAck != 0) { + if e.EndpointState() == StateEstablished && e.rcv.pendingRcvdSegments.Len() > 0 && (flags&header.TCPFlagAck != 0) { sackBlocks = e.sack.Blocks[:e.sack.NumBlocks] } options := e.makeOptions(sackBlocks) @@ -1002,9 +1003,8 @@ func (e *endpoint) transitionToStateEstablishedLocked(h *handshake) { // (indicated by a negative send window scale). e.snd = newSender(e, h.iss, h.ackNum-1, h.sndWnd, h.mss, h.sndWndScale) - rcvBufSize := seqnum.Size(e.receiveBufferSize()) e.rcvListMu.Lock() - e.rcv = newReceiver(e, h.ackNum-1, h.rcvWnd, h.effectiveRcvWndScale(), rcvBufSize) + e.rcv = newReceiver(e, h.ackNum-1, h.rcvWnd, h.effectiveRcvWndScale()) // Bootstrap the auto tuning algorithm. Starting at zero will // result in a really large receive window after the first auto // tuning adjustment. @@ -1135,12 +1135,11 @@ func (e *endpoint) handleSegments(fastPath bool) *tcpip.Error { } cont, err := e.handleSegment(s) + s.decRef() if err != nil { - s.decRef() return err } if !cont { - s.decRef() return nil } } @@ -1220,6 +1219,12 @@ func (e *endpoint) handleSegment(s *segment) (cont bool, err *tcpip.Error) { return true, nil } + // Increase counter if after processing the segment we would potentially + // advertise a zero window. + if crossed, above := e.windowCrossedACKThresholdLocked(-s.segMemSize()); crossed && !above { + e.stats.ReceiveErrors.ZeroRcvWindowState.Increment() + } + // Now check if the received segment has caused us to transition // to a CLOSED state, if yes then terminate processing and do // not invoke the sender. @@ -1232,7 +1237,6 @@ func (e *endpoint) handleSegment(s *segment) (cont bool, err *tcpip.Error) { // or a notification from the protocolMainLoop (caller goroutine). // This means that with this return, the segment dequeue below can // never occur on a closed endpoint. - s.decRef() return false, nil } @@ -1424,10 +1428,6 @@ func (e *endpoint) protocolMainLoop(handshake bool, wakerInitDone chan<- struct{ e.rcv.nonZeroWindow() } - if n¬ifyReceiveWindowChanged != 0 { - e.rcv.pendingBufSize = seqnum.Size(e.receiveBufferSize()) - } - if n¬ifyMTUChanged != 0 { e.sndBufMu.Lock() count := e.packetTooBigCount diff --git a/pkg/tcpip/transport/tcp/dual_stack_test.go b/pkg/tcpip/transport/tcp/dual_stack_test.go index 94207c141..560b4904c 100644 --- a/pkg/tcpip/transport/tcp/dual_stack_test.go +++ b/pkg/tcpip/transport/tcp/dual_stack_test.go @@ -78,8 +78,8 @@ func testV4Connect(t *testing.T, c *context.Context, checkers ...checker.Network ackCheckers := append(checkers, checker.TCP( checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(iss)+1), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(iss)+1), )) checker.IPv4(t, c.GetPacket(), ackCheckers...) @@ -185,8 +185,8 @@ func testV6Connect(t *testing.T, c *context.Context, checkers ...checker.Network ackCheckers := append(checkers, checker.TCP( checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(iss)+1), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(iss)+1), )) checker.IPv6(t, c.GetV6Packet(), ackCheckers...) @@ -283,7 +283,7 @@ func TestV4RefuseOnV6Only(t *testing.T) { checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagRst|header.TCPFlagAck), - checker.AckNum(uint32(irs)+1), + checker.TCPAckNum(uint32(irs)+1), ), ) } @@ -319,7 +319,7 @@ func TestV6RefuseOnBoundToV4Mapped(t *testing.T) { checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagRst|header.TCPFlagAck), - checker.AckNum(uint32(irs)+1), + checker.TCPAckNum(uint32(irs)+1), ), ) } @@ -352,7 +352,7 @@ func testV4Accept(t *testing.T, c *context.Context) { checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagSyn), - checker.AckNum(uint32(irs)+1), + checker.TCPAckNum(uint32(irs)+1), ), ) @@ -492,7 +492,7 @@ func TestV6AcceptOnV6(t *testing.T) { checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagSyn), - checker.AckNum(uint32(irs)+1), + checker.TCPAckNum(uint32(irs)+1), ), ) diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go index faea7f2bb..ae817091a 100644 --- a/pkg/tcpip/transport/tcp/endpoint.go +++ b/pkg/tcpip/transport/tcp/endpoint.go @@ -63,6 +63,17 @@ const ( StateClosing ) +const ( + // rcvAdvWndScale is used to split the available socket buffer into + // application buffer and the window to be advertised to the peer. This is + // currently hard coded to split the available space equally. + rcvAdvWndScale = 1 + + // SegOverheadFactor is used to multiply the value provided by the + // user on a SetSockOpt for setting the socket send/receive buffer sizes. + SegOverheadFactor = 2 +) + // connected returns true when s is one of the states representing an // endpoint connected to a peer. func (s EndpointState) connected() bool { @@ -149,7 +160,6 @@ func (s EndpointState) String() string { // Reasons for notifying the protocol goroutine. const ( notifyNonZeroReceiveWindow = 1 << iota - notifyReceiveWindowChanged notifyClose notifyMTUChanged notifyDrain @@ -384,13 +394,26 @@ type endpoint struct { // to indicate to users that no more data is coming. // // rcvListMu can be taken after the endpoint mu below. - rcvListMu sync.Mutex `state:"nosave"` - rcvList segmentList `state:"wait"` - rcvClosed bool - rcvBufSize int + rcvListMu sync.Mutex `state:"nosave"` + rcvList segmentList `state:"wait"` + rcvClosed bool + // rcvBufSize is the total size of the receive buffer. + rcvBufSize int + // rcvBufUsed is the actual number of payload bytes held in the receive buffer + // not counting any overheads of the segments itself. NOTE: This will always + // be strictly <= rcvMemUsed below. rcvBufUsed int rcvAutoParams rcvBufAutoTuneParams + // rcvMemUsed tracks the total amount of memory in use by received segments + // held in rcvList, pendingRcvdSegments and the segment queue. This is used to + // compute the window and the actual available buffer space. This is distinct + // from rcvBufUsed above which is the actual number of payload bytes held in + // the buffer not including any segment overheads. + // + // rcvMemUsed must be accessed atomically. + rcvMemUsed int32 + // mu protects all endpoint fields unless documented otherwise. mu must // be acquired before interacting with the endpoint fields. mu sync.Mutex `state:"nosave"` @@ -891,7 +914,7 @@ func newEndpoint(s *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQue e.probe = p } - e.segmentQueue.setLimit(MaxUnprocessedSegments) + e.segmentQueue.ep = e e.tsOffset = timeStampOffset() e.acceptCond = sync.NewCond(&e.acceptMu) @@ -904,7 +927,12 @@ func (e *endpoint) Readiness(mask waiter.EventMask) waiter.EventMask { result := waiter.EventMask(0) switch e.EndpointState() { - case StateInitial, StateBound, StateConnecting, StateSynSent, StateSynRecv: + case StateInitial, StateBound: + // This prevents blocking of new sockets which are not + // connected when SO_LINGER is set. + result |= waiter.EventHUp + + case StateConnecting, StateSynSent, StateSynRecv: // Ready for nothing. case StateClose, StateError, StateTimeWait: @@ -1075,6 +1103,8 @@ func (e *endpoint) closeNoShutdownLocked() { e.notifyProtocolGoroutine(notifyClose) } else { e.transitionToStateCloseLocked() + // Notify that the endpoint is closed. + e.waiterQueue.Notify(waiter.EventHUp) } } @@ -1129,10 +1159,16 @@ func (e *endpoint) cleanupLocked() { tcpip.DeleteDanglingEndpoint(e) } +// wndFromSpace returns the window that we can advertise based on the available +// receive buffer space. +func wndFromSpace(space int) int { + return space / (1 << rcvAdvWndScale) +} + // initialReceiveWindow returns the initial receive window to advertise in the // SYN/SYN-ACK. func (e *endpoint) initialReceiveWindow() int { - rcvWnd := e.receiveBufferAvailable() + rcvWnd := wndFromSpace(e.receiveBufferAvailable()) if rcvWnd > math.MaxUint16 { rcvWnd = math.MaxUint16 } @@ -1209,14 +1245,12 @@ func (e *endpoint) ModerateRecvBuf(copied int) { // reject valid data that might already be in flight as the // acceptable window will shrink. if rcvWnd > e.rcvBufSize { - availBefore := e.receiveBufferAvailableLocked() + availBefore := wndFromSpace(e.receiveBufferAvailableLocked()) e.rcvBufSize = rcvWnd - availAfter := e.receiveBufferAvailableLocked() - mask := uint32(notifyReceiveWindowChanged) + availAfter := wndFromSpace(e.receiveBufferAvailableLocked()) if crossed, above := e.windowCrossedACKThresholdLocked(availAfter - availBefore); crossed && above { - mask |= notifyNonZeroReceiveWindow + e.notifyProtocolGoroutine(notifyNonZeroReceiveWindow) } - e.notifyProtocolGoroutine(mask) } // We only update prevCopied when we grow the buffer because in cases @@ -1293,18 +1327,22 @@ func (e *endpoint) readLocked() (buffer.View, *tcpip.Error) { v := views[s.viewToDeliver] s.viewToDeliver++ + var delta int if s.viewToDeliver >= len(views) { e.rcvList.Remove(s) + // We only free up receive buffer space when the segment is released as the + // segment is still holding on to the views even though some views have been + // read out to the user. + delta = s.segMemSize() s.decRef() } e.rcvBufUsed -= len(v) - // If the window was small before this read and if the read freed up // enough buffer space, to either fit an aMSS or half a receive buffer // (whichever smaller), then notify the protocol goroutine to send a // window update. - if crossed, above := e.windowCrossedACKThresholdLocked(len(v)); crossed && above { + if crossed, above := e.windowCrossedACKThresholdLocked(delta); crossed && above { e.notifyProtocolGoroutine(notifyNonZeroReceiveWindow) } @@ -1317,14 +1355,17 @@ func (e *endpoint) readLocked() (buffer.View, *tcpip.Error) { // indicating the reason why it's not writable. // Caller must hold e.mu and e.sndBufMu func (e *endpoint) isEndpointWritableLocked() (int, *tcpip.Error) { - // The endpoint cannot be written to if it's not connected. - if !e.EndpointState().connected() { - switch e.EndpointState() { - case StateError: - return 0, e.HardError - default: - return 0, tcpip.ErrClosedForSend - } + switch s := e.EndpointState(); { + case s == StateError: + return 0, e.HardError + case !s.connecting() && !s.connected(): + return 0, tcpip.ErrClosedForSend + case s.connecting(): + // As per RFC793, page 56, a send request arriving when in connecting + // state, can be queued to be completed after the state becomes + // connected. Return an error code for the caller of endpoint Write to + // try again, until the connection handshake is complete. + return 0, tcpip.ErrWouldBlock } // Check if the connection has already been closed for sends. @@ -1478,11 +1519,11 @@ func (e *endpoint) Peek(vec [][]byte) (int64, tcpip.ControlMessages, *tcpip.Erro } // windowCrossedACKThresholdLocked checks if the receive window to be announced -// now would be under aMSS or under half receive buffer, whichever smaller. This -// is useful as a receive side silly window syndrome prevention mechanism. If -// window grows to reasonable value, we should send ACK to the sender to inform -// the rx space is now large. We also want ensure a series of small read()'s -// won't trigger a flood of spurious tiny ACK's. +// would be under aMSS or under the window derived from half receive buffer, +// whichever smaller. This is useful as a receive side silly window syndrome +// prevention mechanism. If window grows to reasonable value, we should send ACK +// to the sender to inform the rx space is now large. We also want ensure a +// series of small read()'s won't trigger a flood of spurious tiny ACK's. // // For large receive buffers, the threshold is aMSS - once reader reads more // than aMSS we'll send ACK. For tiny receive buffers, the threshold is half of @@ -1493,17 +1534,18 @@ func (e *endpoint) Peek(vec [][]byte) (int64, tcpip.ControlMessages, *tcpip.Erro // // Precondition: e.mu and e.rcvListMu must be held. func (e *endpoint) windowCrossedACKThresholdLocked(deltaBefore int) (crossed bool, above bool) { - newAvail := e.receiveBufferAvailableLocked() + newAvail := wndFromSpace(e.receiveBufferAvailableLocked()) oldAvail := newAvail - deltaBefore if oldAvail < 0 { oldAvail = 0 } - threshold := int(e.amss) - if threshold > e.rcvBufSize/2 { - threshold = e.rcvBufSize / 2 + // rcvBufFraction is the inverse of the fraction of receive buffer size that + // is used to decide if the available buffer space is now above it. + const rcvBufFraction = 2 + if wndThreshold := wndFromSpace(e.rcvBufSize / rcvBufFraction); threshold > wndThreshold { + threshold = wndThreshold } - switch { case oldAvail < threshold && newAvail >= threshold: return true, true @@ -1633,17 +1675,23 @@ func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error { // Make sure the receive buffer size is within the min and max // allowed. var rs tcpip.TCPReceiveBufferSizeRangeOption - if err := e.stack.TransportProtocolOption(ProtocolNumber, &rs); err == nil { + if err := e.stack.TransportProtocolOption(ProtocolNumber, &rs); err != nil { + panic(fmt.Sprintf("e.stack.TransportProtocolOption(%d, %#v) = %s", ProtocolNumber, &rs, err)) + } + + if v > rs.Max { + v = rs.Max + } + + if v < math.MaxInt32/SegOverheadFactor { + v *= SegOverheadFactor if v < rs.Min { v = rs.Min } - if v > rs.Max { - v = rs.Max - } + } else { + v = math.MaxInt32 } - mask := uint32(notifyReceiveWindowChanged) - e.LockUser() e.rcvListMu.Lock() @@ -1657,14 +1705,9 @@ func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error { v = 1 << scale } - // Make sure 2*size doesn't overflow. - if v > math.MaxInt32/2 { - v = math.MaxInt32 / 2 - } - - availBefore := e.receiveBufferAvailableLocked() + availBefore := wndFromSpace(e.receiveBufferAvailableLocked()) e.rcvBufSize = v - availAfter := e.receiveBufferAvailableLocked() + availAfter := wndFromSpace(e.receiveBufferAvailableLocked()) e.rcvAutoParams.disabled = true @@ -1672,24 +1715,31 @@ func (e *endpoint) SetSockOptInt(opt tcpip.SockOptInt, v int) *tcpip.Error { // syndrome prevetion, when our available space grows above aMSS // or half receive buffer, whichever smaller. if crossed, above := e.windowCrossedACKThresholdLocked(availAfter - availBefore); crossed && above { - mask |= notifyNonZeroReceiveWindow + e.notifyProtocolGoroutine(notifyNonZeroReceiveWindow) } e.rcvListMu.Unlock() e.UnlockUser() - e.notifyProtocolGoroutine(mask) case tcpip.SendBufferSizeOption: // Make sure the send buffer size is within the min and max // allowed. var ss tcpip.TCPSendBufferSizeRangeOption - if err := e.stack.TransportProtocolOption(ProtocolNumber, &ss); err == nil { + if err := e.stack.TransportProtocolOption(ProtocolNumber, &ss); err != nil { + panic(fmt.Sprintf("e.stack.TransportProtocolOption(%d, %#v) = %s", ProtocolNumber, &ss, err)) + } + + if v > ss.Max { + v = ss.Max + } + + if v < math.MaxInt32/SegOverheadFactor { + v *= SegOverheadFactor if v < ss.Min { v = ss.Min } - if v > ss.Max { - v = ss.Max - } + } else { + v = math.MaxInt32 } e.sndBufMu.Lock() @@ -2049,7 +2099,7 @@ func (e *endpoint) GetSockOpt(opt tcpip.GettableSocketOption) *tcpip.Error { case *tcpip.OriginalDestinationOption: e.LockUser() ipt := e.stack.IPTables() - addr, port, err := ipt.OriginalDst(e.ID) + addr, port, err := ipt.OriginalDst(e.ID, e.NetProto) e.UnlockUser() if err != nil { return err @@ -2696,13 +2746,8 @@ func (e *endpoint) updateSndBufferUsage(v int) { func (e *endpoint) readyToRead(s *segment) { e.rcvListMu.Lock() if s != nil { + e.rcvBufUsed += s.payloadSize() s.incRef() - e.rcvBufUsed += s.data.Size() - // Increase counter if the receive window falls down below MSS - // or half receive buffer size, whichever smaller. - if crossed, above := e.windowCrossedACKThresholdLocked(-s.data.Size()); crossed && !above { - e.stats.ReceiveErrors.ZeroRcvWindowState.Increment() - } e.rcvList.PushBack(s) } else { e.rcvClosed = true @@ -2717,15 +2762,17 @@ func (e *endpoint) readyToRead(s *segment) { func (e *endpoint) receiveBufferAvailableLocked() int { // We may use more bytes than the buffer size when the receive buffer // shrinks. - if e.rcvBufUsed >= e.rcvBufSize { + memUsed := e.receiveMemUsed() + if memUsed >= e.rcvBufSize { return 0 } - return e.rcvBufSize - e.rcvBufUsed + return e.rcvBufSize - memUsed } // receiveBufferAvailable calculates how many bytes are still available in the -// receive buffer. +// receive buffer based on the actual memory used by all segments held in +// receive buffer/pending and segment queue. func (e *endpoint) receiveBufferAvailable() int { e.rcvListMu.Lock() available := e.receiveBufferAvailableLocked() @@ -2733,14 +2780,35 @@ func (e *endpoint) receiveBufferAvailable() int { return available } +// receiveBufferUsed returns the amount of in-use receive buffer. +func (e *endpoint) receiveBufferUsed() int { + e.rcvListMu.Lock() + used := e.rcvBufUsed + e.rcvListMu.Unlock() + return used +} + +// receiveBufferSize returns the current size of the receive buffer. func (e *endpoint) receiveBufferSize() int { e.rcvListMu.Lock() size := e.rcvBufSize e.rcvListMu.Unlock() - return size } +// receiveMemUsed returns the total memory in use by segments held by this +// endpoint. +func (e *endpoint) receiveMemUsed() int { + return int(atomic.LoadInt32(&e.rcvMemUsed)) +} + +// updateReceiveMemUsed adds the provided delta to e.rcvMemUsed. +func (e *endpoint) updateReceiveMemUsed(delta int) { + atomic.AddInt32(&e.rcvMemUsed, int32(delta)) +} + +// maxReceiveBufferSize returns the stack wide maximum receive buffer size for +// an endpoint. func (e *endpoint) maxReceiveBufferSize() int { var rs tcpip.TCPReceiveBufferSizeRangeOption if err := e.stack.TransportProtocolOption(ProtocolNumber, &rs); err != nil { @@ -2891,7 +2959,6 @@ func (e *endpoint) completeState() stack.TCPEndpointState { RcvAcc: e.rcv.rcvAcc, RcvWndScale: e.rcv.rcvWndScale, PendingBufUsed: e.rcv.pendingBufUsed, - PendingBufSize: e.rcv.pendingBufSize, } // Copy sender state. diff --git a/pkg/tcpip/transport/tcp/endpoint_state.go b/pkg/tcpip/transport/tcp/endpoint_state.go index 41d0050f3..b25431467 100644 --- a/pkg/tcpip/transport/tcp/endpoint_state.go +++ b/pkg/tcpip/transport/tcp/endpoint_state.go @@ -44,7 +44,7 @@ func (e *endpoint) drainSegmentLocked() { // beforeSave is invoked by stateify. func (e *endpoint) beforeSave() { // Stop incoming packets. - e.segmentQueue.setLimit(0) + e.segmentQueue.freeze() e.mu.Lock() defer e.mu.Unlock() @@ -178,7 +178,7 @@ func (e *endpoint) afterLoad() { // Resume implements tcpip.ResumableEndpoint.Resume. func (e *endpoint) Resume(s *stack.Stack) { e.stack = s - e.segmentQueue.setLimit(MaxUnprocessedSegments) + e.segmentQueue.thaw() epState := e.origEndpointState switch epState { case StateInitial, StateBound, StateListen, StateConnecting, StateEstablished: diff --git a/pkg/tcpip/transport/tcp/protocol.go b/pkg/tcpip/transport/tcp/protocol.go index 63ec12be8..5bce73605 100644 --- a/pkg/tcpip/transport/tcp/protocol.go +++ b/pkg/tcpip/transport/tcp/protocol.go @@ -12,12 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Package tcp contains the implementation of the TCP transport protocol. To use -// it in the networking stack, this package must be added to the project, and -// activated on the stack by passing tcp.NewProtocol() as one of the -// transport protocols when calling stack.New(). Then endpoints can be created -// by passing tcp.ProtocolNumber as the transport protocol number when calling -// Stack.NewEndpoint(). +// Package tcp contains the implementation of the TCP transport protocol. package tcp import ( @@ -29,6 +24,7 @@ import ( "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/header" + "gvisor.dev/gvisor/pkg/tcpip/header/parse" "gvisor.dev/gvisor/pkg/tcpip/seqnum" "gvisor.dev/gvisor/pkg/tcpip/stack" "gvisor.dev/gvisor/pkg/tcpip/transport/raw" @@ -137,6 +133,8 @@ func (s *synRcvdCounter) Threshold() uint64 { } type protocol struct { + stack *stack.Stack + mu sync.RWMutex sackEnabled bool recovery tcpip.TCPRecovery @@ -163,14 +161,14 @@ func (*protocol) Number() tcpip.TransportProtocolNumber { } // NewEndpoint creates a new tcp endpoint. -func (p *protocol) NewEndpoint(stack *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { - return newEndpoint(stack, netProto, waiterQueue), nil +func (p *protocol) NewEndpoint(netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { + return newEndpoint(p.stack, netProto, waiterQueue), nil } // NewRawEndpoint creates a new raw TCP endpoint. Raw TCP sockets are currently // unsupported. It implements stack.TransportProtocol.NewRawEndpoint. -func (p *protocol) NewRawEndpoint(stack *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { - return raw.NewEndpoint(stack, netProto, header.TCPProtocolNumber, waiterQueue) +func (p *protocol) NewRawEndpoint(netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { + return raw.NewEndpoint(p.stack, netProto, header.TCPProtocolNumber, waiterQueue) } // MinimumPacketSize returns the minimum valid tcp packet size. @@ -200,21 +198,20 @@ func (p *protocol) QueuePacket(r *stack.Route, ep stack.TransportEndpoint, id st // a reset is sent in response to any incoming segment except another reset. In // particular, SYNs addressed to a non-existent connection are rejected by this // means." -func (*protocol) HandleUnknownDestinationPacket(r *stack.Route, id stack.TransportEndpointID, pkt *stack.PacketBuffer) bool { + +func (*protocol) HandleUnknownDestinationPacket(r *stack.Route, id stack.TransportEndpointID, pkt *stack.PacketBuffer) stack.UnknownDestinationPacketDisposition { s := newSegment(r, id, pkt) defer s.decRef() if !s.parse() || !s.csumValid { - return false + return stack.UnknownDestinationPacketMalformed } - // There's nothing to do if this is already a reset packet. - if s.flagIsSet(header.TCPFlagRst) { - return true + if !s.flagIsSet(header.TCPFlagRst) { + replyWithReset(s, stack.DefaultTOS, s.route.DefaultTTL()) } - replyWithReset(s, stack.DefaultTOS, s.route.DefaultTTL()) - return true + return stack.UnknownDestinationPacketHandled } // replyWithReset replies to the given segment with a reset segment. @@ -506,27 +503,13 @@ func (p *protocol) SynRcvdCounter() *synRcvdCounter { // Parse implements stack.TransportProtocol.Parse. func (*protocol) Parse(pkt *stack.PacketBuffer) bool { - // TCP header is variable length, peek at it first. - hdrLen := header.TCPMinimumSize - hdr, ok := pkt.Data.PullUp(hdrLen) - if !ok { - return false - } - - // If the header has options, pull those up as well. - if offset := int(header.TCP(hdr).DataOffset()); offset > header.TCPMinimumSize && offset <= pkt.Data.Size() { - // TODO(gvisor.dev/issue/2404): Figure out whether to reject this kind of - // packets. - hdrLen = offset - } - - _, ok = pkt.TransportHeader().Consume(hdrLen) - return ok + return parse.TCP(pkt) } // NewProtocol returns a TCP transport protocol. -func NewProtocol() stack.TransportProtocol { +func NewProtocol(s *stack.Stack) stack.TransportProtocol { p := protocol{ + stack: s, sendBufferSize: tcpip.TCPSendBufferSizeRangeOption{ Min: MinBufferSize, Default: DefaultSendBufferSize, diff --git a/pkg/tcpip/transport/tcp/rack.go b/pkg/tcpip/transport/tcp/rack.go index d969ca23a..439932595 100644 --- a/pkg/tcpip/transport/tcp/rack.go +++ b/pkg/tcpip/transport/tcp/rack.go @@ -39,6 +39,9 @@ type rackControl struct { // sequence. fack seqnum.Value + // minRTT is the estimated minimum RTT of the connection. + minRTT time.Duration + // rtt is the RTT of the most recently delivered packet on the // connection (either cumulatively acknowledged or selectively // acknowledged) that was not marked invalid as a possible spurious @@ -48,7 +51,7 @@ type rackControl struct { // Update will update the RACK related fields when an ACK has been received. // See: https://tools.ietf.org/html/draft-ietf-tcpm-rack-08#section-7.2 -func (rc *rackControl) Update(seg *segment, ackSeg *segment, srtt time.Duration, offset uint32) { +func (rc *rackControl) Update(seg *segment, ackSeg *segment, offset uint32) { rtt := time.Now().Sub(seg.xmitTime) // If the ACK is for a retransmitted packet, do not update if it is a @@ -65,12 +68,21 @@ func (rc *rackControl) Update(seg *segment, ackSeg *segment, srtt time.Duration, return } } - if rtt < srtt { + if rtt < rc.minRTT { return } } rc.rtt = rtt + + // The sender can either track a simple global minimum of all RTT + // measurements from the connection, or a windowed min-filtered value + // of recent RTT measurements. This implementation keeps track of the + // simple global minimum of all RTTs for the connection. + if rtt < rc.minRTT || rc.minRTT == 0 { + rc.minRTT = rtt + } + // Update rc.xmitTime and rc.endSequence to the transmit time and // ending sequence number of the packet which has been acknowledged // most recently. diff --git a/pkg/tcpip/transport/tcp/rcv.go b/pkg/tcpip/transport/tcp/rcv.go index cfd43b5e3..48bf196d8 100644 --- a/pkg/tcpip/transport/tcp/rcv.go +++ b/pkg/tcpip/transport/tcp/rcv.go @@ -47,22 +47,24 @@ type receiver struct { closed bool + // pendingRcvdSegments is bounded by the receive buffer size of the + // endpoint. pendingRcvdSegments segmentHeap - pendingBufUsed seqnum.Size - pendingBufSize seqnum.Size + // pendingBufUsed tracks the total number of bytes (including segment + // overhead) currently queued in pendingRcvdSegments. + pendingBufUsed int // Time when the last ack was received. lastRcvdAckTime time.Time `state:".(unixTime)"` } -func newReceiver(ep *endpoint, irs seqnum.Value, rcvWnd seqnum.Size, rcvWndScale uint8, pendingBufSize seqnum.Size) *receiver { +func newReceiver(ep *endpoint, irs seqnum.Value, rcvWnd seqnum.Size, rcvWndScale uint8) *receiver { return &receiver{ ep: ep, rcvNxt: irs + 1, rcvAcc: irs.Add(rcvWnd + 1), rcvWnd: rcvWnd, rcvWndScale: rcvWndScale, - pendingBufSize: pendingBufSize, lastRcvdAckTime: time.Now(), } } @@ -85,15 +87,30 @@ func (r *receiver) acceptable(segSeq seqnum.Value, segLen seqnum.Size) bool { // getSendParams returns the parameters needed by the sender when building // segments to send. func (r *receiver) getSendParams() (rcvNxt seqnum.Value, rcvWnd seqnum.Size) { - // Calculate the window size based on the available buffer space. - receiveBufferAvailable := r.ep.receiveBufferAvailable() - acc := r.rcvNxt.Add(seqnum.Size(receiveBufferAvailable)) - if r.rcvAcc.LessThan(acc) { - r.rcvAcc = acc + avail := wndFromSpace(r.ep.receiveBufferAvailable()) + if avail == 0 { + // We have no space available to accept any data, move to zero window + // state. + r.rcvWnd = 0 + return r.rcvNxt, 0 + } + + acc := r.rcvNxt.Add(seqnum.Size(avail)) + newWnd := r.rcvNxt.Size(acc) + curWnd := r.rcvNxt.Size(r.rcvAcc) + + // Update rcvAcc only if new window is > previously advertised window. We + // should never shrink the acceptable sequence space once it has been + // advertised the peer. If we shrink the acceptable sequence space then we + // would end up dropping bytes that might already be in flight. + if newWnd > curWnd { + r.rcvAcc = r.rcvNxt.Add(newWnd) + } else { + newWnd = curWnd } // Stash away the non-scaled receive window as we use it for measuring // receiver's estimated RTT. - r.rcvWnd = r.rcvNxt.Size(r.rcvAcc) + r.rcvWnd = newWnd return r.rcvNxt, r.rcvWnd >> r.rcvWndScale } @@ -195,7 +212,9 @@ func (r *receiver) consumeSegment(s *segment, segSeq seqnum.Value, segLen seqnum } for i := first; i < len(r.pendingRcvdSegments); i++ { + r.pendingBufUsed -= r.pendingRcvdSegments[i].segMemSize() r.pendingRcvdSegments[i].decRef() + // Note that slice truncation does not allow garbage collection of // truncated items, thus truncated items must be set to nil to avoid // memory leaks. @@ -384,10 +403,16 @@ func (r *receiver) handleRcvdSegment(s *segment) (drop bool, err *tcpip.Error) { // Defer segment processing if it can't be consumed now. if !r.consumeSegment(s, segSeq, segLen) { if segLen > 0 || s.flagIsSet(header.TCPFlagFin) { - // We only store the segment if it's within our buffer - // size limit. - if r.pendingBufUsed < r.pendingBufSize { - r.pendingBufUsed += seqnum.Size(s.segMemSize()) + // We only store the segment if it's within our buffer size limit. + // + // Only use 75% of the receive buffer queue for out-of-order + // segments. This ensures that we always leave some space for the inorder + // segments to arrive allowing pending segments to be processed and + // delivered to the user. + if r.ep.receiveBufferAvailable() > 0 && r.pendingBufUsed < r.ep.receiveBufferSize()>>2 { + r.ep.rcvListMu.Lock() + r.pendingBufUsed += s.segMemSize() + r.ep.rcvListMu.Unlock() s.incRef() heap.Push(&r.pendingRcvdSegments, s) UpdateSACKBlocks(&r.ep.sack, segSeq, segSeq.Add(segLen), r.rcvNxt) @@ -421,7 +446,9 @@ func (r *receiver) handleRcvdSegment(s *segment) (drop bool, err *tcpip.Error) { } heap.Pop(&r.pendingRcvdSegments) - r.pendingBufUsed -= seqnum.Size(s.segMemSize()) + r.ep.rcvListMu.Lock() + r.pendingBufUsed -= s.segMemSize() + r.ep.rcvListMu.Unlock() s.decRef() } return false, nil diff --git a/pkg/tcpip/transport/tcp/segment.go b/pkg/tcpip/transport/tcp/segment.go index 94307d31a..13acaf753 100644 --- a/pkg/tcpip/transport/tcp/segment.go +++ b/pkg/tcpip/transport/tcp/segment.go @@ -15,6 +15,7 @@ package tcp import ( + "fmt" "sync/atomic" "time" @@ -24,6 +25,15 @@ import ( "gvisor.dev/gvisor/pkg/tcpip/stack" ) +// queueFlags are used to indicate which queue of an endpoint a particular segment +// belongs to. This is used to track memory accounting correctly. +type queueFlags uint8 + +const ( + recvQ queueFlags = 1 << iota + sendQ +) + // segment represents a TCP segment. It holds the payload and parsed TCP segment // information, and can be added to intrusive lists. // segment is mostly immutable, the only field allowed to change is viewToDeliver. @@ -32,6 +42,8 @@ import ( type segment struct { segmentEntry refCnt int32 + ep *endpoint + qFlags queueFlags id stack.TransportEndpointID `state:"manual"` route stack.Route `state:"manual"` data buffer.VectorisedView `state:".(buffer.VectorisedView)"` @@ -100,6 +112,8 @@ func (s *segment) clone() *segment { rcvdTime: s.rcvdTime, xmitTime: s.xmitTime, xmitCount: s.xmitCount, + ep: s.ep, + qFlags: s.qFlags, } t.data = s.data.Clone(t.views[:]) return t @@ -115,8 +129,34 @@ func (s *segment) flagsAreSet(flags uint8) bool { return s.flags&flags == flags } +// setOwner sets the owning endpoint for this segment. Its required +// to be called to ensure memory accounting for receive/send buffer +// queues is done properly. +func (s *segment) setOwner(ep *endpoint, qFlags queueFlags) { + switch qFlags { + case recvQ: + ep.updateReceiveMemUsed(s.segMemSize()) + case sendQ: + // no memory account for sendQ yet. + default: + panic(fmt.Sprintf("unexpected queue flag %b", qFlags)) + } + s.ep = ep + s.qFlags = qFlags +} + func (s *segment) decRef() { if atomic.AddInt32(&s.refCnt, -1) == 0 { + if s.ep != nil { + switch s.qFlags { + case recvQ: + s.ep.updateReceiveMemUsed(-s.segMemSize()) + case sendQ: + // no memory accounting for sendQ yet. + default: + panic(fmt.Sprintf("unexpected queue flag %b set for segment", s.qFlags)) + } + } s.route.Release() } } @@ -138,6 +178,11 @@ func (s *segment) logicalLen() seqnum.Size { return l } +// payloadSize is the size of s.data. +func (s *segment) payloadSize() int { + return s.data.Size() +} + // segMemSize is the amount of memory used to hold the segment data and // the associated metadata. func (s *segment) segMemSize() int { diff --git a/pkg/tcpip/transport/tcp/segment_queue.go b/pkg/tcpip/transport/tcp/segment_queue.go index 48a257137..54545a1b1 100644 --- a/pkg/tcpip/transport/tcp/segment_queue.go +++ b/pkg/tcpip/transport/tcp/segment_queue.go @@ -22,16 +22,16 @@ import ( // // +stateify savable type segmentQueue struct { - mu sync.Mutex `state:"nosave"` - list segmentList `state:"wait"` - limit int - used int + mu sync.Mutex `state:"nosave"` + list segmentList `state:"wait"` + ep *endpoint + frozen bool } // emptyLocked determines if the queue is empty. // Preconditions: q.mu must be held. func (q *segmentQueue) emptyLocked() bool { - return q.used == 0 + return q.list.Empty() } // empty determines if the queue is empty. @@ -43,14 +43,6 @@ func (q *segmentQueue) empty() bool { return r } -// setLimit updates the limit. No segments are immediately dropped in case the -// queue becomes full due to the new limit. -func (q *segmentQueue) setLimit(limit int) { - q.mu.Lock() - q.limit = limit - q.mu.Unlock() -} - // enqueue adds the given segment to the queue. // // Returns true when the segment is successfully added to the queue, in which @@ -58,15 +50,23 @@ func (q *segmentQueue) setLimit(limit int) { // false if the queue is full, in which case ownership is retained by the // caller. func (q *segmentQueue) enqueue(s *segment) bool { + // q.ep.receiveBufferParams() must be called without holding q.mu to + // avoid lock order inversion. + bufSz := q.ep.receiveBufferSize() + used := q.ep.receiveMemUsed() q.mu.Lock() - r := q.used < q.limit - if r { + // Allow zero sized segments (ACK/FIN/RSTs etc even if the segment queue + // is currently full). + allow := (used <= bufSz || s.payloadSize() == 0) && !q.frozen + + if allow { q.list.PushBack(s) - q.used++ + // Set the owner now that the endpoint owns the segment. + s.setOwner(q.ep, recvQ) } q.mu.Unlock() - return r + return allow } // dequeue removes and returns the next segment from queue, if one exists. @@ -77,9 +77,25 @@ func (q *segmentQueue) dequeue() *segment { s := q.list.Front() if s != nil { q.list.Remove(s) - q.used-- } q.mu.Unlock() return s } + +// freeze prevents any more segments from being added to the queue. i.e all +// future segmentQueue.enqueue will return false and not add the segment to the +// queue till the queue is unfroze with a corresponding segmentQueue.thaw call. +func (q *segmentQueue) freeze() { + q.mu.Lock() + q.frozen = true + q.mu.Unlock() +} + +// thaw unfreezes a previously frozen queue using segmentQueue.freeze() and +// allows new segments to be queued again. +func (q *segmentQueue) thaw() { + q.mu.Lock() + q.frozen = false + q.mu.Unlock() +} diff --git a/pkg/tcpip/transport/tcp/snd.go b/pkg/tcpip/transport/tcp/snd.go index c55589c45..4c9a86cda 100644 --- a/pkg/tcpip/transport/tcp/snd.go +++ b/pkg/tcpip/transport/tcp/snd.go @@ -1365,9 +1365,6 @@ func (s *sender) handleRcvdSegment(rcvdSeg *segment) { ackLeft := acked originalOutstanding := s.outstanding - s.rtt.Lock() - srtt := s.rtt.srtt - s.rtt.Unlock() for ackLeft > 0 { // We use logicalLen here because we can have FIN // segments (which are always at the end of list) that @@ -1389,7 +1386,7 @@ func (s *sender) handleRcvdSegment(rcvdSeg *segment) { // Update the RACK fields if SACK is enabled. if s.ep.sackPermitted { - s.rc.Update(seg, rcvdSeg, srtt, s.ep.tsOffset) + s.rc.Update(seg, rcvdSeg, s.ep.tsOffset) } s.writeList.Remove(seg) diff --git a/pkg/tcpip/transport/tcp/tcp_test.go b/pkg/tcpip/transport/tcp/tcp_test.go index b1e5f1b24..5b504d0d1 100644 --- a/pkg/tcpip/transport/tcp/tcp_test.go +++ b/pkg/tcpip/transport/tcp/tcp_test.go @@ -21,6 +21,7 @@ import ( "testing" "time" + "gvisor.dev/gvisor/pkg/rand" "gvisor.dev/gvisor/pkg/sync" "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" @@ -240,6 +241,38 @@ func TestTCPResetsSentIncrement(t *testing.T) { } } +// TestTCPResetsSentNoICMP confirms that we don't get an ICMP +// DstUnreachable packet when we try send a packet which is not part +// of an active session. +func TestTCPResetsSentNoICMP(t *testing.T) { + c := context.New(t, defaultMTU) + defer c.Cleanup() + stats := c.Stack().Stats() + + // Send a SYN request for a closed port. This should elicit an RST + // but NOT an ICMPv4 DstUnreachable packet. + iss := seqnum.Value(789) + c.SendPacket(nil, &context.Headers{ + SrcPort: context.TestPort, + DstPort: context.StackPort, + Flags: header.TCPFlagSyn, + SeqNum: iss, + }) + + // Receive whatever comes back. + b := c.GetPacket() + ipHdr := header.IPv4(b) + if got, want := ipHdr.Protocol(), uint8(header.TCPProtocolNumber); got != want { + t.Errorf("unexpected protocol, got = %d, want = %d", got, want) + } + + // Read outgoing ICMP stats and check no ICMP DstUnreachable was recorded. + sent := stats.ICMP.V4PacketsSent + if got, want := sent.DstUnreachable.Value(), uint64(0); got != want { + t.Errorf("got ICMP DstUnreachable.Value() = %d, want = %d", got, want) + } +} + // TestTCPResetSentForACKWhenNotUsingSynCookies checks that the stack generates // a RST if an ACK is received on the listening socket for which there is no // active handshake in progress and we are not using SYN cookies. @@ -317,8 +350,8 @@ func TestTCPResetSentForACKWhenNotUsingSynCookies(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+1)), - checker.AckNum(uint32(iss)+1), + checker.TCPSeqNum(uint32(c.IRS+1)), + checker.TCPAckNum(uint32(iss)+1), checker.TCPFlags(header.TCPFlagFin|header.TCPFlagAck))) finHeaders := &context.Headers{ SrcPort: context.TestPort, @@ -348,8 +381,8 @@ func TestTCPResetSentForACKWhenNotUsingSynCookies(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+1)), - checker.AckNum(0), + checker.TCPSeqNum(uint32(c.IRS+1)), + checker.TCPAckNum(0), checker.TCPFlags(header.TCPFlagRst))) } @@ -447,8 +480,8 @@ func TestConnectResetAfterClose(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagFin), ), ) @@ -489,8 +522,8 @@ func TestConnectResetAfterClose(t *testing.T) { // RST is always generated with sndNxt which if the FIN // has been sent will be 1 higher than the sequence number // of the FIN itself. - checker.SeqNum(uint32(c.IRS)+2), - checker.AckNum(0), + checker.TCPSeqNum(uint32(c.IRS)+2), + checker.TCPAckNum(0), checker.TCPFlags(header.TCPFlagRst), ), ) @@ -529,8 +562,8 @@ func TestCurrentConnectedIncrement(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagFin), ), ) @@ -565,8 +598,8 @@ func TestCurrentConnectedIncrement(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+2), - checker.AckNum(791), + checker.TCPSeqNum(uint32(c.IRS)+2), + checker.TCPAckNum(791), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -612,8 +645,8 @@ func TestClosingWithEnqueuedSegments(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(791), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(791), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -633,8 +666,8 @@ func TestClosingWithEnqueuedSegments(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(791), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(791), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagFin), ), ) @@ -693,8 +726,8 @@ func TestClosingWithEnqueuedSegments(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+2), - checker.AckNum(0), + checker.TCPSeqNum(uint32(c.IRS)+2), + checker.TCPAckNum(0), checker.TCPFlags(header.TCPFlagRst), ), ) @@ -745,8 +778,8 @@ func TestSimpleReceive(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+len(data))), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(790+len(data))), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -998,7 +1031,7 @@ func TestSendRstOnListenerRxSynAckV4(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagRst), - checker.SeqNum(200))) + checker.TCPSeqNum(200))) } func TestSendRstOnListenerRxSynAckV6(t *testing.T) { @@ -1026,7 +1059,7 @@ func TestSendRstOnListenerRxSynAckV6(t *testing.T) { checker.IPv6(t, c.GetV6Packet(), checker.TCP( checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagRst), - checker.SeqNum(200))) + checker.TCPSeqNum(200))) } // TestTCPAckBeforeAcceptV4 tests that once the 3-way handshake is complete, @@ -1063,8 +1096,8 @@ func TestTCPAckBeforeAcceptV4(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck), - checker.SeqNum(uint32(iss+1)), - checker.AckNum(uint32(irs+5)))) + checker.TCPSeqNum(uint32(iss+1)), + checker.TCPAckNum(uint32(irs+5)))) } // TestTCPAckBeforeAcceptV6 tests that once the 3-way handshake is complete, @@ -1101,8 +1134,8 @@ func TestTCPAckBeforeAcceptV6(t *testing.T) { checker.IPv6(t, c.GetV6Packet(), checker.TCP( checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck), - checker.SeqNum(uint32(iss+1)), - checker.AckNum(uint32(irs+5)))) + checker.TCPSeqNum(uint32(iss+1)), + checker.TCPAckNum(uint32(irs+5)))) } func TestSendRstOnListenerRxAckV4(t *testing.T) { @@ -1130,7 +1163,7 @@ func TestSendRstOnListenerRxAckV4(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagRst), - checker.SeqNum(200))) + checker.TCPSeqNum(200))) } func TestSendRstOnListenerRxAckV6(t *testing.T) { @@ -1158,7 +1191,7 @@ func TestSendRstOnListenerRxAckV6(t *testing.T) { checker.IPv6(t, c.GetV6Packet(), checker.TCP( checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagRst), - checker.SeqNum(200))) + checker.TCPSeqNum(200))) } // TestListenShutdown tests for the listening endpoint replying with RST @@ -1274,8 +1307,8 @@ func TestTOSV4(t *testing.T) { checker.PayloadLen(len(data)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), // Acknum is initial sequence number + 1 + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), // Acknum is initial sequence number + 1 checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), checker.TOS(tos, 0), @@ -1323,8 +1356,8 @@ func TestTrafficClassV6(t *testing.T) { checker.PayloadLen(len(data)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), checker.TOS(tos, 0), @@ -1514,8 +1547,8 @@ func TestOutOfOrderReceive(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -1565,8 +1598,8 @@ func TestOutOfOrderReceive(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+len(data))), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(790+len(data))), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -1576,8 +1609,8 @@ func TestOutOfOrderFlood(t *testing.T) { c := context.New(t, defaultMTU) defer c.Cleanup() - // Create a new connection with initial window size of 10. - c.CreateConnected(789, 30000, 10) + rcvBufSz := math.MaxUint16 + c.CreateConnected(789, 30000, rcvBufSz) if _, _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { t.Fatalf("got c.EP.Read(nil) = %s, want = %s", err, tcpip.ErrWouldBlock) @@ -1598,8 +1631,8 @@ func TestOutOfOrderFlood(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -1619,8 +1652,8 @@ func TestOutOfOrderFlood(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -1639,8 +1672,8 @@ func TestOutOfOrderFlood(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(793), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(793), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -1681,8 +1714,8 @@ func TestRstOnCloseWithUnreadData(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+len(data))), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(790+len(data))), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -1696,7 +1729,7 @@ func TestRstOnCloseWithUnreadData(t *testing.T) { checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagRst), // We shouldn't consume a sequence number on RST. - checker.SeqNum(uint32(c.IRS)+1), + checker.TCPSeqNum(uint32(c.IRS)+1), )) // The RST puts the endpoint into an error state. if got, want := tcp.EndpointState(c.EP.State()), tcp.StateError; got != want { @@ -1750,8 +1783,8 @@ func TestRstOnCloseWithUnreadDataFinConvertRst(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+len(data))), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(790+len(data))), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -1764,7 +1797,7 @@ func TestRstOnCloseWithUnreadDataFinConvertRst(t *testing.T) { checker.TCP( checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagFin), - checker.SeqNum(uint32(c.IRS)+1), + checker.TCPSeqNum(uint32(c.IRS)+1), )) if got, want := tcp.EndpointState(c.EP.State()), tcp.StateFinWait1; got != want { @@ -1783,7 +1816,7 @@ func TestRstOnCloseWithUnreadDataFinConvertRst(t *testing.T) { // RST is always generated with sndNxt which if the FIN // has been sent will be 1 higher than the sequence // number of the FIN itself. - checker.SeqNum(uint32(c.IRS)+2), + checker.TCPSeqNum(uint32(c.IRS)+2), )) // The RST puts the endpoint into an error state. if got, want := tcp.EndpointState(c.EP.State()), tcp.StateError; got != want { @@ -1829,7 +1862,8 @@ func TestFullWindowReceive(t *testing.T) { c := context.New(t, defaultMTU) defer c.Cleanup() - c.CreateConnected(789, 30000, 10) + const rcvBufSz = 10 + c.CreateConnected(789, 30000, rcvBufSz) we, ch := waiter.NewChannelEntry(nil) c.WQ.EventRegister(&we, waiter.EventIn) @@ -1840,8 +1874,13 @@ func TestFullWindowReceive(t *testing.T) { t.Fatalf("Read failed: %s", err) } - // Fill up the window. - data := []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10} + // Fill up the window w/ tcp.SegOverheadFactor*rcvBufSz as netstack multiplies + // the provided buffer value by tcp.SegOverheadFactor to calculate the actual + // receive buffer size. + data := make([]byte, tcp.SegOverheadFactor*rcvBufSz) + for i := range data { + data[i] = byte(i % 255) + } c.SendPacket(data, &context.Headers{ SrcPort: context.TestPort, DstPort: c.Port, @@ -1862,10 +1901,10 @@ func TestFullWindowReceive(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+len(data))), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(790+len(data))), checker.TCPFlags(header.TCPFlagAck), - checker.Window(0), + checker.TCPWindow(0), ), ) @@ -1888,10 +1927,10 @@ func TestFullWindowReceive(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+len(data))), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(790+len(data))), checker.TCPFlags(header.TCPFlagAck), - checker.Window(10), + checker.TCPWindow(10), ), ) } @@ -1900,12 +1939,15 @@ func TestNoWindowShrinking(t *testing.T) { c := context.New(t, defaultMTU) defer c.Cleanup() - // Start off with a window size of 10, then shrink it to 5. - c.CreateConnected(789, 30000, 10) - - if err := c.EP.SetSockOptInt(tcpip.ReceiveBufferSizeOption, 5); err != nil { - t.Fatalf("SetSockOptInt(ReceiveBufferSizeOption, 5) failed: %s", err) - } + // Start off with a certain receive buffer then cut it in half and verify that + // the right edge of the window does not shrink. + // NOTE: Netstack doubles the value specified here. + rcvBufSize := 65536 + iss := seqnum.Value(789) + // Enable window scaling with a scale of zero from our end. + c.CreateConnectedWithRawOptions(iss, 30000, rcvBufSize, []byte{ + header.TCPOptionWS, 3, 0, header.TCPOptionNOP, + }) we, ch := waiter.NewChannelEntry(nil) c.WQ.EventRegister(&we, waiter.EventIn) @@ -1914,14 +1956,15 @@ func TestNoWindowShrinking(t *testing.T) { if _, _, err := c.EP.Read(nil); err != tcpip.ErrWouldBlock { t.Fatalf("got c.EP.Read(nil) = %s, want = %s", err, tcpip.ErrWouldBlock) } - - // Send 3 bytes, check that the peer acknowledges them. - data := []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10} - c.SendPacket(data[:3], &context.Headers{ + // Send a 1 byte payload so that we can record the current receive window. + // Send a payload of half the size of rcvBufSize. + seqNum := iss.Add(1) + payload := []byte{1} + c.SendPacket(payload, &context.Headers{ SrcPort: context.TestPort, DstPort: c.Port, Flags: header.TCPFlagAck, - SeqNum: 790, + SeqNum: seqNum, AckNum: c.IRS.Add(1), RcvWnd: 30000, }) @@ -1933,46 +1976,93 @@ func TestNoWindowShrinking(t *testing.T) { t.Fatalf("Timed out waiting for data to arrive") } - // Check that data is acknowledged, and that window doesn't go to zero - // just yet because it was previously set to 10. It must go to 7 now. - checker.IPv4(t, c.GetPacket(), + // Read the 1 byte payload we just sent. + v, _, err := c.EP.Read(nil) + if err != nil { + t.Fatalf("Read failed: %s", err) + } + if got, want := payload, v; !bytes.Equal(got, want) { + t.Fatalf("got data: %v, want: %v", got, want) + } + + seqNum = seqNum.Add(1) + // Verify that the ACK does not shrink the window. + pkt := c.GetPacket() + checker.IPv4(t, pkt, checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(793), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(seqNum)), checker.TCPFlags(header.TCPFlagAck), - checker.Window(7), ), ) + // Stash the initial window. + initialWnd := header.TCP(header.IPv4(pkt).Payload()).WindowSize() << c.RcvdWindowScale + initialLastAcceptableSeq := seqNum.Add(seqnum.Size(initialWnd)) + // Now shrink the receive buffer to half its original size. + if err := c.EP.SetSockOptInt(tcpip.ReceiveBufferSizeOption, rcvBufSize/2); err != nil { + t.Fatalf("SetSockOptInt(ReceiveBufferSizeOption, 5) failed: %s", err) + } - // Send 7 more bytes, check that the window fills up. - c.SendPacket(data[3:], &context.Headers{ + data := generateRandomPayload(t, rcvBufSize) + // Send a payload of half the size of rcvBufSize. + c.SendPacket(data[:rcvBufSize/2], &context.Headers{ SrcPort: context.TestPort, DstPort: c.Port, Flags: header.TCPFlagAck, - SeqNum: 793, + SeqNum: seqNum, AckNum: c.IRS.Add(1), RcvWnd: 30000, }) + seqNum = seqNum.Add(seqnum.Size(rcvBufSize / 2)) - select { - case <-ch: - case <-time.After(5 * time.Second): - t.Fatalf("Timed out waiting for data to arrive") + // Verify that the ACK does not shrink the window. + pkt = c.GetPacket() + checker.IPv4(t, pkt, + checker.TCP( + checker.DstPort(context.TestPort), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(seqNum)), + checker.TCPFlags(header.TCPFlagAck), + ), + ) + newWnd := header.TCP(header.IPv4(pkt).Payload()).WindowSize() << c.RcvdWindowScale + newLastAcceptableSeq := seqNum.Add(seqnum.Size(newWnd)) + if newLastAcceptableSeq.LessThan(initialLastAcceptableSeq) { + t.Fatalf("receive window shrunk unexpectedly got: %d, want >= %d", newLastAcceptableSeq, initialLastAcceptableSeq) } + // Send another payload of half the size of rcvBufSize. This should fill up the + // socket receive buffer and we should see a zero window. + c.SendPacket(data[rcvBufSize/2:], &context.Headers{ + SrcPort: context.TestPort, + DstPort: c.Port, + Flags: header.TCPFlagAck, + SeqNum: seqNum, + AckNum: c.IRS.Add(1), + RcvWnd: 30000, + }) + seqNum = seqNum.Add(seqnum.Size(rcvBufSize / 2)) + checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+len(data))), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(seqNum)), checker.TCPFlags(header.TCPFlagAck), - checker.Window(0), + checker.TCPWindow(0), ), ) + // Wait for receive to be notified. + select { + case <-ch: + case <-time.After(5 * time.Second): + t.Fatalf("Timed out waiting for data to arrive") + } + // Receive data and check it. - read := make([]byte, 0, 10) + read := make([]byte, 0, rcvBufSize) for len(read) < len(data) { v, _, err := c.EP.Read(nil) if err != nil { @@ -1986,15 +2076,15 @@ func TestNoWindowShrinking(t *testing.T) { t.Fatalf("got data = %v, want = %v", read, data) } - // Check that we get an ACK for the newly non-zero window, which is the - // new size. + // Check that we get an ACK for the newly non-zero window, which is the new + // receive buffer size we set after the connection was established. checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+len(data))), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(seqNum)), checker.TCPFlags(header.TCPFlagAck), - checker.Window(5), + checker.TCPWindow(uint16(rcvBufSize/2)>>c.RcvdWindowScale), ), ) } @@ -2019,8 +2109,8 @@ func TestSimpleSend(t *testing.T) { checker.PayloadLen(len(data)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2061,8 +2151,8 @@ func TestZeroWindowSend(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2083,8 +2173,8 @@ func TestZeroWindowSend(t *testing.T) { checker.PayloadLen(len(data)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2123,16 +2213,16 @@ func TestScaledWindowConnect(t *testing.T) { t.Fatalf("Write failed: %s", err) } - // Check that data is received, and that advertised window is 0xbfff, + // Check that data is received, and that advertised window is 0x5fff, // that is, that it is scaled. b := c.GetPacket() checker.IPv4(t, b, checker.PayloadLen(len(data)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), - checker.Window(0xbfff), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), + checker.TCPWindow(0x5fff), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2162,9 +2252,9 @@ func TestNonScaledWindowConnect(t *testing.T) { checker.PayloadLen(len(data)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), - checker.Window(0xffff), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), + checker.TCPWindow(0xffff), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2198,7 +2288,8 @@ func TestScaledWindowAccept(t *testing.T) { } // Do 3-way handshake. - c.PassiveConnectWithOptions(100, 2, header.TCPSynOptions{MSS: defaultIPv4MSS}) + // wndScale expected is 3 as 65535 * 3 * 2 < 65535 * 2^3 but > 65535 *2 *2 + c.PassiveConnectWithOptions(100, 3 /* wndScale */, header.TCPSynOptions{MSS: defaultIPv4MSS}) // Try to accept the connection. we, ch := waiter.NewChannelEntry(nil) @@ -2228,16 +2319,16 @@ func TestScaledWindowAccept(t *testing.T) { t.Fatalf("Write failed: %s", err) } - // Check that data is received, and that advertised window is 0xbfff, + // Check that data is received, and that advertised window is 0x5fff, // that is, that it is scaled. b := c.GetPacket() checker.IPv4(t, b, checker.PayloadLen(len(data)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), - checker.Window(0xbfff), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), + checker.TCPWindow(0x5fff), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2309,9 +2400,9 @@ func TestNonScaledWindowAccept(t *testing.T) { checker.PayloadLen(len(data)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), - checker.Window(0xffff), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), + checker.TCPWindow(0xffff), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2324,18 +2415,19 @@ func TestZeroScaledWindowReceive(t *testing.T) { c := context.New(t, defaultMTU) defer c.Cleanup() - // Set the window size such that a window scale of 4 will be used. - const wnd = 65535 * 10 - const ws = uint32(4) - c.CreateConnectedWithRawOptions(789, 30000, wnd, []byte{ + // Set the buffer size such that a window scale of 5 will be used. + const bufSz = 65535 * 10 + const ws = uint32(5) + c.CreateConnectedWithRawOptions(789, 30000, bufSz, []byte{ header.TCPOptionWS, 3, 0, header.TCPOptionNOP, }) // Write chunks of 50000 bytes. - remain := wnd + remain := 0 sent := 0 data := make([]byte, 50000) - for remain > len(data) { + // Keep writing till the window drops below len(data). + for { c.SendPacket(data, &context.Headers{ SrcPort: context.TestPort, DstPort: c.Port, @@ -2345,21 +2437,25 @@ func TestZeroScaledWindowReceive(t *testing.T) { RcvWnd: 30000, }) sent += len(data) - remain -= len(data) - checker.IPv4(t, c.GetPacket(), + pkt := c.GetPacket() + checker.IPv4(t, pkt, checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+sent)), - checker.Window(uint16(remain>>ws)), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(790+sent)), checker.TCPFlags(header.TCPFlagAck), ), ) + // Don't reduce window to zero here. + if wnd := int(header.TCP(header.IPv4(pkt).Payload()).WindowSize()); wnd<<ws < len(data) { + remain = wnd << ws + break + } } // Make the window non-zero, but the scaled window zero. - if remain >= 16 { + for remain >= 16 { data = data[:remain-15] c.SendPacket(data, &context.Headers{ SrcPort: context.TestPort, @@ -2370,22 +2466,35 @@ func TestZeroScaledWindowReceive(t *testing.T) { RcvWnd: 30000, }) sent += len(data) - remain -= len(data) - checker.IPv4(t, c.GetPacket(), + pkt := c.GetPacket() + checker.IPv4(t, pkt, checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+sent)), - checker.Window(0), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(790+sent)), checker.TCPFlags(header.TCPFlagAck), ), ) + // Since the receive buffer is split between window advertisement and + // application data buffer the window does not always reflect the space + // available and actual space available can be a bit more than what is + // advertised in the window. + wnd := int(header.TCP(header.IPv4(pkt).Payload()).WindowSize()) + if wnd == 0 { + break + } + remain = wnd << ws } - // Read at least 1MSS of data. An ack should be sent in response to that. + // Read at least 2MSS of data. An ack should be sent in response to that. + // Since buffer space is now split in half between window and application + // data we need to read more than 1 MSS(65536) of data for a non-zero window + // update to be sent. For 1MSS worth of window to be available we need to + // read at least 128KB. Since our segments above were 50KB each it means + // we need to read at 3 packets. sz := 0 - for sz < defaultMTU { + for sz < defaultMTU*2 { v, _, err := c.EP.Read(nil) if err != nil { t.Fatalf("Read failed: %s", err) @@ -2397,9 +2506,9 @@ func TestZeroScaledWindowReceive(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+sent)), - checker.Window(uint16(sz>>ws)), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(790+sent)), + checker.TCPWindowGreaterThanEq(uint16(defaultMTU>>ws)), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -2466,8 +2575,8 @@ func TestSegmentMerging(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize+1), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+uint32(i)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+uint32(i)+1), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2489,8 +2598,8 @@ func TestSegmentMerging(t *testing.T) { checker.PayloadLen(len(allData)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+11), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+11), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2537,8 +2646,8 @@ func TestDelay(t *testing.T) { checker.PayloadLen(len(want)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(seq)), - checker.AckNum(790), + checker.TCPSeqNum(uint32(seq)), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2584,8 +2693,8 @@ func TestUndelay(t *testing.T) { checker.PayloadLen(len(allData[0])+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(seq)), - checker.AckNum(790), + checker.TCPSeqNum(uint32(seq)), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2607,8 +2716,8 @@ func TestUndelay(t *testing.T) { checker.PayloadLen(len(allData[1])+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(seq)), - checker.AckNum(790), + checker.TCPSeqNum(uint32(seq)), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2669,8 +2778,8 @@ func TestMSSNotDelayed(t *testing.T) { checker.PayloadLen(len(data)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(seq)), - checker.AckNum(790), + checker.TCPSeqNum(uint32(seq)), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2721,8 +2830,8 @@ func testBrokenUpWrite(t *testing.T, c *context.Context, maxPayload int) { checker.IPv4(t, b, checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1+uint32(bytesReceived)), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1+uint32(bytesReceived)), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -2964,7 +3073,7 @@ func TestSynOptionsOnActiveConnect(t *testing.T) { // Set the buffer size to a deterministic size so that we can check the // window scaling option. const rcvBufferSize = 0x20000 - const wndScale = 2 + const wndScale = 3 if err := c.EP.SetSockOptInt(tcpip.ReceiveBufferSizeOption, rcvBufferSize); err != nil { t.Fatalf("SetSockOptInt(ReceiveBufferSizeOption, %d) failed failed: %s", rcvBufferSize, err) } @@ -2999,7 +3108,7 @@ func TestSynOptionsOnActiveConnect(t *testing.T) { checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagSyn), checker.SrcPort(tcpHdr.SourcePort()), - checker.SeqNum(tcpHdr.SequenceNumber()), + checker.TCPSeqNum(tcpHdr.SequenceNumber()), checker.TCPSynOptions(header.TCPSynOptions{MSS: mss, WS: wndScale}), ), ) @@ -3020,8 +3129,8 @@ func TestSynOptionsOnActiveConnect(t *testing.T) { checker.TCP( checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(iss)+1), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(iss)+1), ), ) @@ -3314,8 +3423,8 @@ func TestFinImmediately(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagFin), ), ) @@ -3335,8 +3444,8 @@ func TestFinImmediately(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+2), - checker.AckNum(791), + checker.TCPSeqNum(uint32(c.IRS)+2), + checker.TCPAckNum(791), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -3357,8 +3466,8 @@ func TestFinRetransmit(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagFin), ), ) @@ -3368,8 +3477,8 @@ func TestFinRetransmit(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagFin), ), ) @@ -3389,8 +3498,8 @@ func TestFinRetransmit(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+2), - checker.AckNum(791), + checker.TCPSeqNum(uint32(c.IRS)+2), + checker.TCPAckNum(791), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -3413,8 +3522,8 @@ func TestFinWithNoPendingData(t *testing.T) { checker.PayloadLen(len(view)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(790), + checker.TCPSeqNum(next), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -3438,8 +3547,8 @@ func TestFinWithNoPendingData(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(790), + checker.TCPSeqNum(next), + checker.TCPAckNum(790), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagFin), ), ) @@ -3460,8 +3569,8 @@ func TestFinWithNoPendingData(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(791), + checker.TCPSeqNum(next), + checker.TCPAckNum(791), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -3488,8 +3597,8 @@ func TestFinWithPendingDataCwndFull(t *testing.T) { checker.PayloadLen(len(view)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(790), + checker.TCPSeqNum(next), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -3507,8 +3616,8 @@ func TestFinWithPendingDataCwndFull(t *testing.T) { checker.PayloadLen(len(view)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -3527,8 +3636,8 @@ func TestFinWithPendingDataCwndFull(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(790), + checker.TCPSeqNum(next), + checker.TCPAckNum(790), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagFin), ), ) @@ -3548,8 +3657,8 @@ func TestFinWithPendingDataCwndFull(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(791), + checker.TCPSeqNum(next), + checker.TCPAckNum(791), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -3572,8 +3681,8 @@ func TestFinWithPendingData(t *testing.T) { checker.PayloadLen(len(view)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(790), + checker.TCPSeqNum(next), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -3597,8 +3706,8 @@ func TestFinWithPendingData(t *testing.T) { checker.PayloadLen(len(view)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(790), + checker.TCPSeqNum(next), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -3613,8 +3722,8 @@ func TestFinWithPendingData(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(790), + checker.TCPSeqNum(next), + checker.TCPAckNum(790), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagFin), ), ) @@ -3634,8 +3743,8 @@ func TestFinWithPendingData(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(791), + checker.TCPSeqNum(next), + checker.TCPAckNum(791), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -3659,8 +3768,8 @@ func TestFinWithPartialAck(t *testing.T) { checker.PayloadLen(len(view)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(790), + checker.TCPSeqNum(next), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -3680,8 +3789,8 @@ func TestFinWithPartialAck(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(791), + checker.TCPSeqNum(next), + checker.TCPAckNum(791), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -3695,8 +3804,8 @@ func TestFinWithPartialAck(t *testing.T) { checker.PayloadLen(len(view)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(791), + checker.TCPSeqNum(next), + checker.TCPAckNum(791), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -3711,8 +3820,8 @@ func TestFinWithPartialAck(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(791), + checker.TCPSeqNum(next), + checker.TCPAckNum(791), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagFin), ), ) @@ -3803,8 +3912,8 @@ func scaledSendWindow(t *testing.T, scale uint8) { checker.PayloadLen((1<<scale)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -3942,7 +4051,7 @@ func TestReceivedSegmentQueuing(t *testing.T) { checker.IPv4(t, b, checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), + checker.TCPSeqNum(uint32(c.IRS)+1), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -3993,8 +4102,8 @@ func TestReadAfterClosedState(t *testing.T) { checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagFin), ), ) @@ -4018,8 +4127,8 @@ func TestReadAfterClosedState(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+2), - checker.AckNum(uint32(791+len(data))), + checker.TCPSeqNum(uint32(c.IRS)+2), + checker.TCPAckNum(uint32(791+len(data))), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -4191,8 +4300,8 @@ func checkSendBufferSize(t *testing.T, ep tcpip.Endpoint, v int) { func TestDefaultBufferSizes(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{tcp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{tcp.NewProtocol}, }) // Check the default values. @@ -4254,8 +4363,8 @@ func TestDefaultBufferSizes(t *testing.T) { func TestMinMaxBufferSizes(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{tcp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{tcp.NewProtocol}, }) // Check the default values. @@ -4280,14 +4389,14 @@ func TestMinMaxBufferSizes(t *testing.T) { } } - // Set values below the min. - if err := ep.SetSockOptInt(tcpip.ReceiveBufferSizeOption, 199); err != nil { + // Set values below the min/2. + if err := ep.SetSockOptInt(tcpip.ReceiveBufferSizeOption, 99); err != nil { t.Fatalf("SetSockOptInt(ReceiveBufferSizeOption, 199) failed: %s", err) } checkRecvBufferSize(t, ep, 200) - if err := ep.SetSockOptInt(tcpip.SendBufferSizeOption, 299); err != nil { + if err := ep.SetSockOptInt(tcpip.SendBufferSizeOption, 149); err != nil { t.Fatalf("SetSockOptInt(SendBufferSizeOption, 299) failed: %s", err) } @@ -4298,19 +4407,21 @@ func TestMinMaxBufferSizes(t *testing.T) { t.Fatalf("SetSockOptInt(ReceiveBufferSizeOption) failed: %s", err) } - checkRecvBufferSize(t, ep, tcp.DefaultReceiveBufferSize*20) + // Values above max are capped at max and then doubled. + checkRecvBufferSize(t, ep, tcp.DefaultReceiveBufferSize*20*2) if err := ep.SetSockOptInt(tcpip.SendBufferSizeOption, 1+tcp.DefaultSendBufferSize*30); err != nil { t.Fatalf("SetSockOptInt(SendBufferSizeOption) failed: %s", err) } - checkSendBufferSize(t, ep, tcp.DefaultSendBufferSize*30) + // Values above max are capped at max and then doubled. + checkSendBufferSize(t, ep, tcp.DefaultSendBufferSize*30*2) } func TestBindToDeviceOption(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{tcp.NewProtocol()}}) + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{tcp.NewProtocol}}) ep, err := s.NewEndpoint(tcp.ProtocolNumber, ipv4.ProtocolNumber, &waiter.Queue{}) if err != nil { @@ -4359,11 +4470,11 @@ func TestBindToDeviceOption(t *testing.T) { func makeStack() (*stack.Stack, *tcpip.Error) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ - ipv4.NewProtocol(), - ipv6.NewProtocol(), + NetworkProtocols: []stack.NetworkProtocolFactory{ + ipv4.NewProtocol, + ipv6.NewProtocol, }, - TransportProtocols: []stack.TransportProtocol{tcp.NewProtocol()}, + TransportProtocols: []stack.TransportProtocolFactory{tcp.NewProtocol}, }) id := loopback.New() @@ -4646,8 +4757,8 @@ func TestPathMTUDiscovery(t *testing.T) { checker.PayloadLen(size+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(seqNum), - checker.AckNum(790), + checker.TCPSeqNum(seqNum), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -4898,8 +5009,8 @@ func TestKeepalive(t *testing.T) { checker.IPv4(t, b, checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)), - checker.AckNum(uint32(790)), + checker.TCPSeqNum(uint32(c.IRS)), + checker.TCPAckNum(uint32(790)), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -4932,8 +5043,8 @@ func TestKeepalive(t *testing.T) { checker.PayloadLen(len(view)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(790), + checker.TCPSeqNum(next), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -4944,8 +5055,8 @@ func TestKeepalive(t *testing.T) { checker.PayloadLen(len(view)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(790), + checker.TCPSeqNum(next), + checker.TCPAckNum(790), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagPsh), ), ) @@ -4970,8 +5081,8 @@ func TestKeepalive(t *testing.T) { checker.IPv4(t, b, checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(next-1)), - checker.AckNum(uint32(790)), + checker.TCPSeqNum(uint32(next-1)), + checker.TCPAckNum(uint32(790)), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -4997,8 +5108,8 @@ func TestKeepalive(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(next)), - checker.AckNum(uint32(0)), + checker.TCPSeqNum(uint32(next)), + checker.TCPAckNum(uint32(0)), checker.TCPFlags(header.TCPFlagRst), ), ) @@ -5038,7 +5149,7 @@ func executeHandshake(t *testing.T, c *context.Context, srcPort uint16, synCooki checker.SrcPort(context.StackPort), checker.DstPort(srcPort), checker.TCPFlags(header.TCPFlagAck | header.TCPFlagSyn), - checker.AckNum(uint32(irs) + 1), + checker.TCPAckNum(uint32(irs) + 1), } if synCookieInUse { @@ -5082,7 +5193,7 @@ func executeV6Handshake(t *testing.T, c *context.Context, srcPort uint16, synCoo checker.SrcPort(context.StackPort), checker.DstPort(srcPort), checker.TCPFlags(header.TCPFlagAck | header.TCPFlagSyn), - checker.AckNum(uint32(irs) + 1), + checker.TCPAckNum(uint32(irs) + 1), } if synCookieInUse { @@ -5316,7 +5427,7 @@ func TestListenNoAcceptNonUnicastV4(t *testing.T) { checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagSyn), - checker.AckNum(uint32(irs)+1))) + checker.TCPAckNum(uint32(irs)+1))) }) } } @@ -5324,8 +5435,8 @@ func TestListenNoAcceptNonUnicastV4(t *testing.T) { // TestListenNoAcceptMulticastBroadcastV6 makes sure that TCP segments with a // non unicast IPv6 address are not accepted. func TestListenNoAcceptNonUnicastV6(t *testing.T) { - multicastAddr := tcpip.Address("\xff\x0e\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x01") - otherMulticastAddr := tcpip.Address("\xff\x0e\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x02") + multicastAddr := tcpip.Address("\xff\x0e\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x01") + otherMulticastAddr := tcpip.Address("\xff\x0e\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x02") tests := []struct { name string @@ -5416,7 +5527,7 @@ func TestListenNoAcceptNonUnicastV6(t *testing.T) { checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagSyn), - checker.AckNum(uint32(irs)+1))) + checker.TCPAckNum(uint32(irs)+1))) }) } } @@ -5464,7 +5575,7 @@ func TestListenSynRcvdQueueFull(t *testing.T) { checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck | header.TCPFlagSyn), - checker.AckNum(uint32(irs) + 1), + checker.TCPAckNum(uint32(irs) + 1), } checker.IPv4(t, b, checker.TCP(tcpCheckers...)) @@ -5642,7 +5753,7 @@ func TestSynRcvdBadSeqNumber(t *testing.T) { checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck | header.TCPFlagSyn), - checker.AckNum(uint32(irs) + 1), + checker.TCPAckNum(uint32(irs) + 1), } checker.IPv4(t, b, checker.TCP(tcpCheckers...)) @@ -5663,8 +5774,8 @@ func TestSynRcvdBadSeqNumber(t *testing.T) { checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck), - checker.AckNum(uint32(irs) + 1), - checker.SeqNum(uint32(iss + 1)), + checker.TCPAckNum(uint32(irs) + 1), + checker.TCPSeqNum(uint32(iss + 1)), } checker.IPv4(t, b, checker.TCP(tcpCheckers...)) @@ -5962,16 +6073,14 @@ func TestReceiveBufferAutoTuningApplicationLimited(t *testing.T) { time.Sleep(latency) rawEP.SendPacketWithTS([]byte{1}, tsVal) - // Verify that the ACK has the expected window. - wantRcvWnd := receiveBufferSize - wantRcvWnd = (wantRcvWnd >> uint32(c.WindowScale)) - rawEP.VerifyACKRcvWnd(uint16(wantRcvWnd - 1)) + pkt := rawEP.VerifyAndReturnACKWithTS(tsVal) + rcvWnd := header.TCP(header.IPv4(pkt).Payload()).WindowSize() time.Sleep(25 * time.Millisecond) // Allocate a large enough payload for the test. - b := make([]byte, int(receiveBufferSize)*2) - offset := 0 - payloadSize := receiveBufferSize - 1 + payloadSize := receiveBufferSize * 2 + b := make([]byte, int(payloadSize)) + worker := (c.EP).(interface { StopWork() ResumeWork() @@ -5980,11 +6089,15 @@ func TestReceiveBufferAutoTuningApplicationLimited(t *testing.T) { // Stop the worker goroutine. worker.StopWork() - start := offset - end := offset + payloadSize + start := 0 + end := payloadSize / 2 packetsSent := 0 for ; start < end; start += mss { - rawEP.SendPacketWithTS(b[start:start+mss], tsVal) + packetEnd := start + mss + if start+mss > end { + packetEnd = end + } + rawEP.SendPacketWithTS(b[start:packetEnd], tsVal) packetsSent++ } @@ -5992,29 +6105,20 @@ func TestReceiveBufferAutoTuningApplicationLimited(t *testing.T) { // are waiting to be read. worker.ResumeWork() - // Since we read no bytes the window should goto zero till the - // application reads some of the data. - // Discard all intermediate acks except the last one. - if packetsSent > 100 { - for i := 0; i < (packetsSent / 100); i++ { - _ = c.GetPacket() - } + // Since we sent almost the full receive buffer worth of data (some may have + // been dropped due to segment overheads), we should get a zero window back. + pkt = c.GetPacket() + tcpHdr := header.TCP(header.IPv4(pkt).Payload()) + gotRcvWnd := tcpHdr.WindowSize() + wantAckNum := tcpHdr.AckNumber() + if got, want := int(gotRcvWnd), 0; got != want { + t.Fatalf("got rcvWnd: %d, want: %d", got, want) } - rawEP.VerifyACKRcvWnd(0) time.Sleep(25 * time.Millisecond) - // Verify that sending more data when window is closed is dropped and - // not acked. + // Verify that sending more data when receiveBuffer is exhausted. rawEP.SendPacketWithTS(b[start:start+mss], tsVal) - // Verify that the stack sends us back an ACK with the sequence number - // of the last packet sent indicating it was dropped. - p := c.GetPacket() - checker.IPv4(t, p, checker.TCP( - checker.AckNum(uint32(rawEP.NextSeqNum)-uint32(mss)), - checker.Window(0), - )) - // Now read all the data from the endpoint and verify that advertised // window increases to the full available buffer size. for { @@ -6027,23 +6131,26 @@ func TestReceiveBufferAutoTuningApplicationLimited(t *testing.T) { // Verify that we receive a non-zero window update ACK. When running // under thread santizer this test can end up sending more than 1 // ack, 1 for the non-zero window - p = c.GetPacket() + p := c.GetPacket() checker.IPv4(t, p, checker.TCP( - checker.AckNum(uint32(rawEP.NextSeqNum)-uint32(mss)), + checker.TCPAckNum(uint32(wantAckNum)), func(t *testing.T, h header.Transport) { tcp, ok := h.(header.TCP) if !ok { return } - if w := tcp.WindowSize(); w == 0 || w > uint16(wantRcvWnd) { - t.Errorf("expected a non-zero window: got %d, want <= wantRcvWnd", w) + // We use 10% here as the error margin upwards as the initial window we + // got was afer 1 segment was already in the receive buffer queue. + tolerance := 1.1 + if w := tcp.WindowSize(); w == 0 || w > uint16(float64(rcvWnd)*tolerance) { + t.Errorf("expected a non-zero window: got %d, want <= %d", w, uint16(float64(rcvWnd)*tolerance)) } }, )) } -// This test verifies that the auto tuning does not grow the receive buffer if -// the application is not reading the data actively. +// This test verifies that the advertised window is auto-tuned up as the +// application is reading the data that is being received. func TestReceiveBufferAutoTuning(t *testing.T) { const mtu = 1500 const mss = mtu - header.IPv4MinimumSize - header.TCPMinimumSize @@ -6053,9 +6160,6 @@ func TestReceiveBufferAutoTuning(t *testing.T) { // Enable Auto-tuning. stk := c.Stack() - // Set lower limits for auto-tuning tests. This is required because the - // test stops the worker which can cause packets to be dropped because - // the segment queue holding unprocessed packets is limited to 300. const receiveBufferSize = 80 << 10 // 80KB. const maxReceiveBufferSize = receiveBufferSize * 10 { @@ -6077,8 +6181,12 @@ func TestReceiveBufferAutoTuning(t *testing.T) { c.WindowScale = uint8(tcp.FindWndScale(maxReceiveBufferSize)) rawEP := c.CreateConnectedWithOptions(header.TCPSynOptions{TS: true, WS: 4}) - - wantRcvWnd := receiveBufferSize + tsVal := uint32(rawEP.TSVal) + rawEP.NextSeqNum-- + rawEP.SendPacketWithTS(nil, tsVal) + rawEP.NextSeqNum++ + pkt := rawEP.VerifyAndReturnACKWithTS(tsVal) + curRcvWnd := int(header.TCP(header.IPv4(pkt).Payload()).WindowSize()) << c.WindowScale scaleRcvWnd := func(rcvWnd int) uint16 { return uint16(rcvWnd >> uint16(c.WindowScale)) } @@ -6095,14 +6203,8 @@ func TestReceiveBufferAutoTuning(t *testing.T) { StopWork() ResumeWork() }) - tsVal := rawEP.TSVal - // We are going to do our own computation of what the moderated receive - // buffer should be based on sent/copied data per RTT and verify that - // the advertised window by the stack matches our calculations. - prevCopied := 0 - done := false latency := 1 * time.Millisecond - for i := 0; !done; i++ { + for i := 0; i < 5; i++ { tsVal++ // Stop the worker goroutine. @@ -6124,15 +6226,20 @@ func TestReceiveBufferAutoTuning(t *testing.T) { // Give 1ms for the worker to process the packets. time.Sleep(1 * time.Millisecond) - // Verify that the advertised window on the ACK is reduced by - // the total bytes sent. - expectedWnd := wantRcvWnd - totalSent - if packetsSent > 100 { - for i := 0; i < (packetsSent / 100); i++ { - _ = c.GetPacket() + lastACK := c.GetPacket() + // Discard any intermediate ACKs and only check the last ACK we get in a + // short time period of few ms. + for { + time.Sleep(1 * time.Millisecond) + pkt := c.GetPacketNonBlocking() + if pkt == nil { + break } + lastACK = pkt + } + if got, want := int(header.TCP(header.IPv4(lastACK).Payload()).WindowSize()), int(scaleRcvWnd(curRcvWnd)); got > want { + t.Fatalf("advertised window got: %d, want <= %d", got, want) } - rawEP.VerifyACKRcvWnd(scaleRcvWnd(expectedWnd)) // Now read all the data from the endpoint and invoke the // moderation API to allow for receive buffer auto-tuning @@ -6157,35 +6264,20 @@ func TestReceiveBufferAutoTuning(t *testing.T) { rawEP.NextSeqNum-- rawEP.SendPacketWithTS(nil, tsVal) rawEP.NextSeqNum++ - if i == 0 { // In the first iteration the receiver based RTT is not // yet known as a result the moderation code should not // increase the advertised window. - rawEP.VerifyACKRcvWnd(scaleRcvWnd(wantRcvWnd)) - prevCopied = totalCopied + rawEP.VerifyACKRcvWnd(scaleRcvWnd(curRcvWnd)) } else { - rttCopied := totalCopied - if i == 1 { - // The moderation code accumulates copied bytes till - // RTT is established. So add in the bytes sent in - // the first iteration to the total bytes for this - // RTT. - rttCopied += prevCopied - // Now reset it to the initial value used by the - // auto tuning logic. - prevCopied = tcp.InitialCwnd * mss * 2 - } - newWnd := rttCopied<<1 + 16*mss - grow := (newWnd * (rttCopied - prevCopied)) / prevCopied - newWnd += (grow << 1) - if newWnd > maxReceiveBufferSize { - newWnd = maxReceiveBufferSize - done = true + pkt := c.GetPacket() + curRcvWnd = int(header.TCP(header.IPv4(pkt).Payload()).WindowSize()) << c.WindowScale + // If thew new current window is close maxReceiveBufferSize then terminate + // the loop. This can happen before all iterations are done due to timing + // differences when running the test. + if int(float64(curRcvWnd)*1.1) > maxReceiveBufferSize/2 { + break } - rawEP.VerifyACKRcvWnd(scaleRcvWnd(newWnd)) - wantRcvWnd = newWnd - prevCopied = rttCopied // Increase the latency after first two iterations to // establish a low RTT value in the receiver since it // only tracks the lowest value. This ensures that when @@ -6198,6 +6290,12 @@ func TestReceiveBufferAutoTuning(t *testing.T) { offset += payloadSize payloadSize *= 2 } + // Check that at the end of our iterations the receive window grew close to the maximum + // permissible size of maxReceiveBufferSize/2 + if got, want := int(float64(curRcvWnd)*1.1), maxReceiveBufferSize/2; got < want { + t.Fatalf("unexpected rcvWnd got: %d, want > %d", got, want) + } + } func TestDelayEnabled(t *testing.T) { @@ -6349,8 +6447,8 @@ func TestTCPTimeWaitRSTIgnored(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+1)), - checker.AckNum(uint32(iss)+1), + checker.TCPSeqNum(uint32(c.IRS+1)), + checker.TCPAckNum(uint32(iss)+1), checker.TCPFlags(header.TCPFlagFin|header.TCPFlagAck))) finHeaders := &context.Headers{ @@ -6367,8 +6465,8 @@ func TestTCPTimeWaitRSTIgnored(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+2)), - checker.AckNum(uint32(iss)+2), + checker.TCPSeqNum(uint32(c.IRS+2)), + checker.TCPAckNum(uint32(iss)+2), checker.TCPFlags(header.TCPFlagAck))) // Now send a RST and this should be ignored and not @@ -6396,8 +6494,8 @@ func TestTCPTimeWaitRSTIgnored(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+2)), - checker.AckNum(uint32(iss)+2), + checker.TCPSeqNum(uint32(c.IRS+2)), + checker.TCPAckNum(uint32(iss)+2), checker.TCPFlags(header.TCPFlagAck))) } @@ -6468,8 +6566,8 @@ func TestTCPTimeWaitOutOfOrder(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+1)), - checker.AckNum(uint32(iss)+1), + checker.TCPSeqNum(uint32(c.IRS+1)), + checker.TCPAckNum(uint32(iss)+1), checker.TCPFlags(header.TCPFlagFin|header.TCPFlagAck))) finHeaders := &context.Headers{ @@ -6486,8 +6584,8 @@ func TestTCPTimeWaitOutOfOrder(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+2)), - checker.AckNum(uint32(iss)+2), + checker.TCPSeqNum(uint32(c.IRS+2)), + checker.TCPAckNum(uint32(iss)+2), checker.TCPFlags(header.TCPFlagAck))) // Out of order ACK should generate an immediate ACK in @@ -6503,8 +6601,8 @@ func TestTCPTimeWaitOutOfOrder(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+2)), - checker.AckNum(uint32(iss)+2), + checker.TCPSeqNum(uint32(c.IRS+2)), + checker.TCPAckNum(uint32(iss)+2), checker.TCPFlags(header.TCPFlagAck))) } @@ -6575,8 +6673,8 @@ func TestTCPTimeWaitNewSyn(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+1)), - checker.AckNum(uint32(iss)+1), + checker.TCPSeqNum(uint32(c.IRS+1)), + checker.TCPAckNum(uint32(iss)+1), checker.TCPFlags(header.TCPFlagFin|header.TCPFlagAck))) finHeaders := &context.Headers{ @@ -6593,8 +6691,8 @@ func TestTCPTimeWaitNewSyn(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+2)), - checker.AckNum(uint32(iss)+2), + checker.TCPSeqNum(uint32(c.IRS+2)), + checker.TCPAckNum(uint32(iss)+2), checker.TCPFlags(header.TCPFlagAck))) // Send a SYN request w/ sequence number lower than @@ -6732,8 +6830,8 @@ func TestTCPTimeWaitDuplicateFINExtendsTimeWait(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+1)), - checker.AckNum(uint32(iss)+1), + checker.TCPSeqNum(uint32(c.IRS+1)), + checker.TCPAckNum(uint32(iss)+1), checker.TCPFlags(header.TCPFlagFin|header.TCPFlagAck))) finHeaders := &context.Headers{ @@ -6750,8 +6848,8 @@ func TestTCPTimeWaitDuplicateFINExtendsTimeWait(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+2)), - checker.AckNum(uint32(iss)+2), + checker.TCPSeqNum(uint32(c.IRS+2)), + checker.TCPAckNum(uint32(iss)+2), checker.TCPFlags(header.TCPFlagAck))) time.Sleep(2 * time.Second) @@ -6765,8 +6863,8 @@ func TestTCPTimeWaitDuplicateFINExtendsTimeWait(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+2)), - checker.AckNum(uint32(iss)+2), + checker.TCPSeqNum(uint32(c.IRS+2)), + checker.TCPAckNum(uint32(iss)+2), checker.TCPFlags(header.TCPFlagAck))) // Sleep for 4 seconds so at this point we are 1 second past the @@ -6794,8 +6892,8 @@ func TestTCPTimeWaitDuplicateFINExtendsTimeWait(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(ackHeaders.AckNum)), - checker.AckNum(0), + checker.TCPSeqNum(uint32(ackHeaders.AckNum)), + checker.TCPAckNum(0), checker.TCPFlags(header.TCPFlagRst))) if got := c.Stack().Stats().TCP.EstablishedClosed.Value(); got != want { @@ -6894,8 +6992,8 @@ func TestTCPCloseWithData(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+1)), - checker.AckNum(uint32(iss)+2), + checker.TCPSeqNum(uint32(c.IRS+1)), + checker.TCPAckNum(uint32(iss)+2), checker.TCPFlags(header.TCPFlagAck))) // Now write a few bytes and then close the endpoint. @@ -6913,8 +7011,8 @@ func TestTCPCloseWithData(t *testing.T) { checker.PayloadLen(len(data)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(iss)+2), // Acknum is initial sequence number + 1 + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(iss)+2), // Acknum is initial sequence number + 1 checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -6928,8 +7026,8 @@ func TestTCPCloseWithData(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+1)+uint32(len(data))), - checker.AckNum(uint32(iss+2)), + checker.TCPSeqNum(uint32(c.IRS+1)+uint32(len(data))), + checker.TCPAckNum(uint32(iss+2)), checker.TCPFlags(header.TCPFlagFin|header.TCPFlagAck))) // First send a partial ACK. @@ -6974,8 +7072,8 @@ func TestTCPCloseWithData(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), - checker.SeqNum(uint32(ackHeaders.AckNum)), - checker.AckNum(0), + checker.TCPSeqNum(uint32(ackHeaders.AckNum)), + checker.TCPAckNum(0), checker.TCPFlags(header.TCPFlagRst))) } @@ -7011,8 +7109,8 @@ func TestTCPUserTimeout(t *testing.T) { checker.PayloadLen(len(view)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(next), - checker.AckNum(790), + checker.TCPSeqNum(next), + checker.TCPAckNum(790), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -7046,8 +7144,8 @@ func TestTCPUserTimeout(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(next)), - checker.AckNum(uint32(0)), + checker.TCPSeqNum(uint32(next)), + checker.TCPAckNum(uint32(0)), checker.TCPFlags(header.TCPFlagRst), ), ) @@ -7108,8 +7206,8 @@ func TestKeepaliveWithUserTimeout(t *testing.T) { checker.IPv4(t, b, checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)), - checker.AckNum(uint32(790)), + checker.TCPSeqNum(uint32(c.IRS)), + checker.TCPAckNum(uint32(790)), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -7134,8 +7232,8 @@ func TestKeepaliveWithUserTimeout(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS+1)), - checker.AckNum(uint32(0)), + checker.TCPSeqNum(uint32(c.IRS+1)), + checker.TCPAckNum(uint32(0)), checker.TCPFlags(header.TCPFlagRst), ), ) @@ -7151,9 +7249,9 @@ func TestKeepaliveWithUserTimeout(t *testing.T) { } } -func TestIncreaseWindowOnReceive(t *testing.T) { +func TestIncreaseWindowOnRead(t *testing.T) { // This test ensures that the endpoint sends an ack, - // after recv() when the window grows to more than 1 MSS. + // after read() when the window grows by more than 1 MSS. c := context.New(t, defaultMTU) defer c.Cleanup() @@ -7162,10 +7260,9 @@ func TestIncreaseWindowOnReceive(t *testing.T) { // Write chunks of ~30000 bytes. It's important that two // payloads make it equal or longer than MSS. - remain := rcvBuf + remain := rcvBuf * 2 sent := 0 data := make([]byte, defaultMTU/2) - lastWnd := uint16(0) for remain > len(data) { c.SendPacket(data, &context.Headers{ @@ -7178,46 +7275,43 @@ func TestIncreaseWindowOnReceive(t *testing.T) { }) sent += len(data) remain -= len(data) - - lastWnd = uint16(remain) - if remain > 0xffff { - lastWnd = 0xffff - } - checker.IPv4(t, c.GetPacket(), + pkt := c.GetPacket() + checker.IPv4(t, pkt, checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+sent)), - checker.Window(lastWnd), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(790+sent)), checker.TCPFlags(header.TCPFlagAck), ), ) + // Break once the window drops below defaultMTU/2 + if wnd := header.TCP(header.IPv4(pkt).Payload()).WindowSize(); wnd < defaultMTU/2 { + break + } } - if lastWnd == 0xffff || lastWnd == 0 { - t.Fatalf("expected small, non-zero window: %d", lastWnd) - } - - // We now have < 1 MSS in the buffer space. Read the data! An - // ack should be sent in response to that. The window was not - // zero, but it grew to larger than MSS. - if _, _, err := c.EP.Read(nil); err != nil { - t.Fatalf("Read failed: %s", err) - } - - if _, _, err := c.EP.Read(nil); err != nil { - t.Fatalf("Read failed: %s", err) + // We now have < 1 MSS in the buffer space. Read at least > 2 MSS + // worth of data as receive buffer space + read := 0 + // defaultMTU is a good enough estimate for the MSS used for this + // connection. + for read < defaultMTU*2 { + v, _, err := c.EP.Read(nil) + if err != nil { + t.Fatalf("Read failed: %s", err) + } + read += len(v) } - // After reading two packets, we surely crossed MSS. See the ack: + // After reading > MSS worth of data, we surely crossed MSS. See the ack: checker.IPv4(t, c.GetPacket(), checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+sent)), - checker.Window(uint16(0xffff)), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(790+sent)), + checker.TCPWindow(uint16(0xffff)), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -7234,10 +7328,9 @@ func TestIncreaseWindowOnBufferResize(t *testing.T) { // Write chunks of ~30000 bytes. It's important that two // payloads make it equal or longer than MSS. - remain := rcvBuf + remain := rcvBuf * 2 sent := 0 data := make([]byte, defaultMTU/2) - lastWnd := uint16(0) for remain > len(data) { c.SendPacket(data, &context.Headers{ @@ -7251,38 +7344,29 @@ func TestIncreaseWindowOnBufferResize(t *testing.T) { sent += len(data) remain -= len(data) - lastWnd = uint16(remain) - if remain > 0xffff { - lastWnd = 0xffff - } checker.IPv4(t, c.GetPacket(), checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+sent)), - checker.Window(lastWnd), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(790+sent)), + checker.TCPWindowLessThanEq(0xffff), checker.TCPFlags(header.TCPFlagAck), ), ) } - if lastWnd == 0xffff || lastWnd == 0 { - t.Fatalf("expected small, non-zero window: %d", lastWnd) - } - // Increasing the buffer from should generate an ACK, // since window grew from small value to larger equal MSS c.EP.SetSockOptInt(tcpip.ReceiveBufferSizeOption, rcvBuf*2) - // After reading two packets, we surely crossed MSS. See the ack: checker.IPv4(t, c.GetPacket(), checker.PayloadLen(header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(790+sent)), - checker.Window(uint16(0xffff)), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(790+sent)), + checker.TCPWindow(uint16(0xffff)), checker.TCPFlags(header.TCPFlagAck), ), ) @@ -7327,8 +7411,8 @@ func TestTCPDeferAccept(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck), - checker.SeqNum(uint32(iss+1)), - checker.AckNum(uint32(irs+5)))) + checker.TCPSeqNum(uint32(iss+1)), + checker.TCPAckNum(uint32(irs+5)))) // Give a bit of time for the socket to be delivered to the accept queue. time.Sleep(50 * time.Millisecond) @@ -7342,8 +7426,8 @@ func TestTCPDeferAccept(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagRst|header.TCPFlagAck), - checker.SeqNum(uint32(iss+1)), - checker.AckNum(uint32(irs+5)))) + checker.TCPSeqNum(uint32(iss+1)), + checker.TCPAckNum(uint32(irs+5)))) } func TestTCPDeferAcceptTimeout(t *testing.T) { @@ -7380,7 +7464,7 @@ func TestTCPDeferAcceptTimeout(t *testing.T) { checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck|header.TCPFlagSyn), - checker.AckNum(uint32(irs)+1))) + checker.TCPAckNum(uint32(irs)+1))) // Send data. This should result in an acceptable endpoint. c.SendPacket([]byte{1, 2, 3, 4}, &context.Headers{ @@ -7396,8 +7480,8 @@ func TestTCPDeferAcceptTimeout(t *testing.T) { checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck), - checker.SeqNum(uint32(iss+1)), - checker.AckNum(uint32(irs+5)))) + checker.TCPSeqNum(uint32(iss+1)), + checker.TCPAckNum(uint32(irs+5)))) // Give sometime for the endpoint to be delivered to the accept queue. time.Sleep(50 * time.Millisecond) @@ -7412,8 +7496,8 @@ func TestTCPDeferAcceptTimeout(t *testing.T) { checker.SrcPort(context.StackPort), checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagRst|header.TCPFlagAck), - checker.SeqNum(uint32(iss+1)), - checker.AckNum(uint32(irs+5)))) + checker.TCPSeqNum(uint32(iss+1)), + checker.TCPAckNum(uint32(irs+5)))) } func TestResetDuringClose(t *testing.T) { @@ -7438,8 +7522,8 @@ func TestResetDuringClose(t *testing.T) { checker.IPv4(t, c.GetPacket(), checker.TCP( checker.DstPort(context.TestPort), checker.TCPFlags(header.TCPFlagAck), - checker.SeqNum(uint32(irs.Add(1))), - checker.AckNum(uint32(iss.Add(5))))) + checker.TCPSeqNum(uint32(irs.Add(1))), + checker.TCPAckNum(uint32(iss.Add(5))))) // Close in a separate goroutine so that we can trigger // a race with the RST we send below. This should not @@ -7520,3 +7604,14 @@ func TestSetStackTimeWaitReuse(t *testing.T) { } } } + +// generateRandomPayload generates a random byte slice of the specified length +// causing a fatal test failure if it is unable to do so. +func generateRandomPayload(t *testing.T, n int) []byte { + t.Helper() + buf := make([]byte, n) + if _, err := rand.Read(buf); err != nil { + t.Fatalf("rand.Read(buf) failed: %s", err) + } + return buf +} diff --git a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go index 44593ed98..0f9ed06cd 100644 --- a/pkg/tcpip/transport/tcp/tcp_timestamp_test.go +++ b/pkg/tcpip/transport/tcp/tcp_timestamp_test.go @@ -159,9 +159,9 @@ func timeStampEnabledAccept(t *testing.T, cookieEnabled bool, wndScale int, wndS checker.PayloadLen(len(data)+header.TCPMinimumSize+12), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), - checker.Window(wndSize), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), + checker.TCPWindow(wndSize), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), checker.TCPTimestampChecker(true, 0, tsVal+1), ), @@ -181,7 +181,8 @@ func TestTimeStampEnabledAccept(t *testing.T) { wndSize uint16 }{ {true, -1, 0xffff}, // When cookie is used window scaling is disabled. - {false, 5, 0x8000}, // DefaultReceiveBufferSize is 1MB >> 5. + // DefaultReceiveBufferSize is 1MB >> 5. Advertised window will be 1/2 of that. + {false, 5, 0x4000}, } for _, tc := range testCases { timeStampEnabledAccept(t, tc.cookieEnabled, tc.wndScale, tc.wndSize) @@ -219,9 +220,9 @@ func timeStampDisabledAccept(t *testing.T, cookieEnabled bool, wndScale int, wnd checker.PayloadLen(len(data)+header.TCPMinimumSize), checker.TCP( checker.DstPort(context.TestPort), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(790), - checker.Window(wndSize), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(790), + checker.TCPWindow(wndSize), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), checker.TCPTimestampChecker(false, 0, 0), ), @@ -237,7 +238,9 @@ func TestTimeStampDisabledAccept(t *testing.T) { wndSize uint16 }{ {true, -1, 0xffff}, // When cookie is used window scaling is disabled. - {false, 5, 0x8000}, // DefaultReceiveBufferSize is 1MB >> 5. + // DefaultReceiveBufferSize is 1MB >> 5. Advertised window will be half of + // that. + {false, 5, 0x4000}, } for _, tc := range testCases { timeStampDisabledAccept(t, tc.cookieEnabled, tc.wndScale, tc.wndSize) diff --git a/pkg/tcpip/transport/tcp/testing/context/context.go b/pkg/tcpip/transport/tcp/testing/context/context.go index 85e8c1c75..faf51ef95 100644 --- a/pkg/tcpip/transport/tcp/testing/context/context.go +++ b/pkg/tcpip/transport/tcp/testing/context/context.go @@ -145,14 +145,18 @@ type Context struct { // WindowScale is the expected window scale in SYN packets sent by // the stack. WindowScale uint8 + + // RcvdWindowScale is the actual window scale sent by the stack in + // SYN/SYN-ACK. + RcvdWindowScale uint8 } // New allocates and initializes a test context containing a new // stack and a link-layer endpoint. func New(t *testing.T, mtu uint32) *Context { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{tcp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{tcp.NewProtocol}, }) const sendBufferSize = 1 << 20 // 1 MiB @@ -261,18 +265,17 @@ func (c *Context) CheckNoPacket(errMsg string) { c.CheckNoPacketTimeout(errMsg, 1*time.Second) } -// GetPacket reads a packet from the link layer endpoint and verifies +// GetPacketWithTimeout reads a packet from the link layer endpoint and verifies // that it is an IPv4 packet with the expected source and destination -// addresses. It will fail with an error if no packet is received for -// 2 seconds. -func (c *Context) GetPacket() []byte { +// addresses. If no packet is received in the specified timeout it will return +// nil. +func (c *Context) GetPacketWithTimeout(timeout time.Duration) []byte { c.t.Helper() - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + ctx, cancel := context.WithTimeout(context.Background(), timeout) defer cancel() p, ok := c.linkEP.ReadContext(ctx) if !ok { - c.t.Fatalf("Packet wasn't written out") return nil } @@ -280,6 +283,14 @@ func (c *Context) GetPacket() []byte { c.t.Fatalf("Bad network protocol: got %v, wanted %v", p.Proto, ipv4.ProtocolNumber) } + // Just check that the stack set the transport protocol number for outbound + // TCP messages. + // TODO(gvisor.dev/issues/3810): Remove when protocol numbers are part + // of the headerinfo. + if p.Pkt.TransportProtocolNumber != tcp.ProtocolNumber { + c.t.Fatalf("got p.Pkt.TransportProtocolNumber = %d, want = %d", p.Pkt.TransportProtocolNumber, tcp.ProtocolNumber) + } + vv := buffer.NewVectorisedView(p.Pkt.Size(), p.Pkt.Views()) b := vv.ToView() @@ -291,6 +302,21 @@ func (c *Context) GetPacket() []byte { return b } +// GetPacket reads a packet from the link layer endpoint and verifies +// that it is an IPv4 packet with the expected source and destination +// addresses. +func (c *Context) GetPacket() []byte { + c.t.Helper() + + p := c.GetPacketWithTimeout(5 * time.Second) + if p == nil { + c.t.Fatalf("Packet wasn't written out") + return nil + } + + return p +} + // GetPacketNonBlocking reads a packet from the link layer endpoint // and verifies that it is an IPv4 packet with the expected source // and destination address. If no packet is available it will return @@ -307,6 +333,14 @@ func (c *Context) GetPacketNonBlocking() []byte { c.t.Fatalf("Bad network protocol: got %v, wanted %v", p.Proto, ipv4.ProtocolNumber) } + // Just check that the stack set the transport protocol number for outbound + // TCP messages. + // TODO(gvisor.dev/issues/3810): Remove when protocol numbers are part + // of the headerinfo. + if p.Pkt.TransportProtocolNumber != tcp.ProtocolNumber { + c.t.Fatalf("got p.Pkt.TransportProtocolNumber = %d, want = %d", p.Pkt.TransportProtocolNumber, tcp.ProtocolNumber) + } + vv := buffer.NewVectorisedView(p.Pkt.Size(), p.Pkt.Views()) b := vv.ToView() @@ -470,8 +504,8 @@ func (c *Context) ReceiveAndCheckPacketWithOptions(data []byte, offset, size, op checker.PayloadLen(size+header.TCPMinimumSize+optlen), checker.TCP( checker.DstPort(TestPort), - checker.SeqNum(uint32(c.IRS.Add(seqnum.Size(1+offset)))), - checker.AckNum(uint32(seqnum.Value(testInitialSequenceNumber).Add(1))), + checker.TCPSeqNum(uint32(c.IRS.Add(seqnum.Size(1+offset)))), + checker.TCPAckNum(uint32(seqnum.Value(testInitialSequenceNumber).Add(1))), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -497,8 +531,8 @@ func (c *Context) ReceiveNonBlockingAndCheckPacket(data []byte, offset, size int checker.PayloadLen(size+header.TCPMinimumSize), checker.TCP( checker.DstPort(TestPort), - checker.SeqNum(uint32(c.IRS.Add(seqnum.Size(1+offset)))), - checker.AckNum(uint32(seqnum.Value(testInitialSequenceNumber).Add(1))), + checker.TCPSeqNum(uint32(c.IRS.Add(seqnum.Size(1+offset)))), + checker.TCPAckNum(uint32(seqnum.Value(testInitialSequenceNumber).Add(1))), checker.TCPFlagsMatch(header.TCPFlagAck, ^uint8(header.TCPFlagPsh)), ), ) @@ -636,6 +670,7 @@ func (c *Context) Connect(iss seqnum.Value, rcvWnd seqnum.Size, options []byte) } tcpHdr := header.TCP(header.IPv4(b).Payload()) + synOpts := header.ParseSynOptions(tcpHdr.Options(), false /* isAck */) c.IRS = seqnum.Value(tcpHdr.SequenceNumber()) c.SendPacket(nil, &Headers{ @@ -653,8 +688,8 @@ func (c *Context) Connect(iss seqnum.Value, rcvWnd seqnum.Size, options []byte) checker.TCP( checker.DstPort(TestPort), checker.TCPFlags(header.TCPFlagAck), - checker.SeqNum(uint32(c.IRS)+1), - checker.AckNum(uint32(iss)+1), + checker.TCPSeqNum(uint32(c.IRS)+1), + checker.TCPAckNum(uint32(iss)+1), ), ) @@ -671,6 +706,7 @@ func (c *Context) Connect(iss seqnum.Value, rcvWnd seqnum.Size, options []byte) c.t.Fatalf("Unexpected endpoint state: want %v, got %v", want, got) } + c.RcvdWindowScale = uint8(synOpts.WS) c.Port = tcpHdr.SourcePort() } @@ -742,17 +778,18 @@ func (r *RawEndpoint) SendPacket(payload []byte, opts []byte) { r.NextSeqNum = r.NextSeqNum.Add(seqnum.Size(len(payload))) } -// VerifyACKWithTS verifies that the tsEcr field in the ack matches the provided -// tsVal. -func (r *RawEndpoint) VerifyACKWithTS(tsVal uint32) { +// VerifyAndReturnACKWithTS verifies that the tsEcr field int he ACK matches +// the provided tsVal as well as returns the original packet. +func (r *RawEndpoint) VerifyAndReturnACKWithTS(tsVal uint32) []byte { + r.C.t.Helper() // Read ACK and verify that tsEcr of ACK packet is [1,2,3,4] ackPacket := r.C.GetPacket() checker.IPv4(r.C.t, ackPacket, checker.TCP( checker.DstPort(r.SrcPort), checker.TCPFlags(header.TCPFlagAck), - checker.SeqNum(uint32(r.AckNum)), - checker.AckNum(uint32(r.NextSeqNum)), + checker.TCPSeqNum(uint32(r.AckNum)), + checker.TCPAckNum(uint32(r.NextSeqNum)), checker.TCPTimestampChecker(true, 0, tsVal), ), ) @@ -760,19 +797,28 @@ func (r *RawEndpoint) VerifyACKWithTS(tsVal uint32) { tcpSeg := header.TCP(header.IPv4(ackPacket).Payload()) opts := tcpSeg.ParsedOptions() r.RecentTS = opts.TSVal + return ackPacket +} + +// VerifyACKWithTS verifies that the tsEcr field in the ack matches the provided +// tsVal. +func (r *RawEndpoint) VerifyACKWithTS(tsVal uint32) { + r.C.t.Helper() + _ = r.VerifyAndReturnACKWithTS(tsVal) } // VerifyACKRcvWnd verifies that the window advertised by the incoming ACK // matches the provided rcvWnd. func (r *RawEndpoint) VerifyACKRcvWnd(rcvWnd uint16) { + r.C.t.Helper() ackPacket := r.C.GetPacket() checker.IPv4(r.C.t, ackPacket, checker.TCP( checker.DstPort(r.SrcPort), checker.TCPFlags(header.TCPFlagAck), - checker.SeqNum(uint32(r.AckNum)), - checker.AckNum(uint32(r.NextSeqNum)), - checker.Window(rcvWnd), + checker.TCPSeqNum(uint32(r.AckNum)), + checker.TCPAckNum(uint32(r.NextSeqNum)), + checker.TCPWindow(rcvWnd), ), ) } @@ -791,8 +837,8 @@ func (r *RawEndpoint) VerifyACKHasSACK(sackBlocks []header.SACKBlock) { checker.TCP( checker.DstPort(r.SrcPort), checker.TCPFlags(header.TCPFlagAck), - checker.SeqNum(uint32(r.AckNum)), - checker.AckNum(uint32(r.NextSeqNum)), + checker.TCPSeqNum(uint32(r.AckNum)), + checker.TCPAckNum(uint32(r.NextSeqNum)), checker.TCPSACKBlockChecker(sackBlocks), ), ) @@ -884,8 +930,8 @@ func (c *Context) CreateConnectedWithOptions(wantOptions header.TCPSynOptions) * tcpCheckers := []checker.TransportChecker{ checker.DstPort(TestPort), checker.TCPFlags(header.TCPFlagAck), - checker.SeqNum(uint32(c.IRS) + 1), - checker.AckNum(uint32(iss) + 1), + checker.TCPSeqNum(uint32(c.IRS) + 1), + checker.TCPAckNum(uint32(iss) + 1), } // Verify that tsEcr of ACK packet is wantOptions.TSVal if the @@ -920,7 +966,7 @@ func (c *Context) CreateConnectedWithOptions(wantOptions header.TCPSynOptions) * // Mark in context that timestamp option is enabled for this endpoint. c.TimeStampEnabled = true - + c.RcvdWindowScale = uint8(synOptions.WS) return &RawEndpoint{ C: c, SrcPort: tcpSeg.DestinationPort(), @@ -1013,6 +1059,7 @@ func (c *Context) PassiveConnect(maxPayload, wndScale int, synOptions header.TCP // value of the window scaling option to be sent in the SYN. If synOptions.WS > // 0 then we send the WindowScale option. func (c *Context) PassiveConnectWithOptions(maxPayload, wndScale int, synOptions header.TCPSynOptions) *RawEndpoint { + c.t.Helper() opts := make([]byte, header.TCPOptionsMaximumSize) offset := 0 offset += header.EncodeMSSOption(uint32(maxPayload), opts) @@ -1051,13 +1098,14 @@ func (c *Context) PassiveConnectWithOptions(maxPayload, wndScale int, synOptions // are present. b := c.GetPacket() tcp := header.TCP(header.IPv4(b).Payload()) + rcvdSynOptions := header.ParseSynOptions(tcp.Options(), true /* isAck */) c.IRS = seqnum.Value(tcp.SequenceNumber()) tcpCheckers := []checker.TransportChecker{ checker.SrcPort(StackPort), checker.DstPort(TestPort), checker.TCPFlags(header.TCPFlagAck | header.TCPFlagSyn), - checker.AckNum(uint32(iss) + 1), + checker.TCPAckNum(uint32(iss) + 1), checker.TCPSynOptions(header.TCPSynOptions{MSS: synOptions.MSS, WS: wndScale, SACKPermitted: synOptions.SACKPermitted && c.SACKEnabled()}), } @@ -1100,6 +1148,7 @@ func (c *Context) PassiveConnectWithOptions(maxPayload, wndScale int, synOptions // Send ACK. c.SendPacket(nil, ackHeaders) + c.RcvdWindowScale = uint8(rcvdSynOptions.WS) c.Port = StackPort return &RawEndpoint{ diff --git a/pkg/tcpip/transport/udp/BUILD b/pkg/tcpip/transport/udp/BUILD index b5d2d0ba6..c78549424 100644 --- a/pkg/tcpip/transport/udp/BUILD +++ b/pkg/tcpip/transport/udp/BUILD @@ -32,6 +32,7 @@ go_library( "//pkg/tcpip", "//pkg/tcpip/buffer", "//pkg/tcpip/header", + "//pkg/tcpip/header/parse", "//pkg/tcpip/ports", "//pkg/tcpip/stack", "//pkg/tcpip/transport/raw", diff --git a/pkg/tcpip/transport/udp/endpoint.go b/pkg/tcpip/transport/udp/endpoint.go index b572c39db..d57ed5d79 100644 --- a/pkg/tcpip/transport/udp/endpoint.go +++ b/pkg/tcpip/transport/udp/endpoint.go @@ -154,6 +154,9 @@ type endpoint struct { // owner is used to get uid and gid of the packet. owner tcpip.PacketOwner + + // linger is used for SO_LINGER socket option. + linger tcpip.LingerOption } // +stateify savable @@ -810,6 +813,11 @@ func (e *endpoint) SetSockOpt(opt tcpip.SettableSocketOption) *tcpip.Error { case *tcpip.SocketDetachFilterOption: return nil + + case *tcpip.LingerOption: + e.mu.Lock() + e.linger = *v + e.mu.Unlock() } return nil } @@ -966,6 +974,11 @@ func (e *endpoint) GetSockOpt(opt tcpip.GettableSocketOption) *tcpip.Error { *o = tcpip.BindToDeviceOption(e.bindToDevice) e.mu.RUnlock() + case *tcpip.LingerOption: + e.mu.RLock() + *o = e.linger + e.mu.RUnlock() + default: return tcpip.ErrUnknownProtocolOption } @@ -983,6 +996,7 @@ func sendUDP(r *stack.Route, data buffer.VectorisedView, localPort, remotePort u // Initialize the UDP header. udp := header.UDP(pkt.TransportHeader().Push(header.UDPMinimumSize)) + pkt.TransportProtocolNumber = ProtocolNumber length := uint16(pkt.Size()) udp.Encode(&header.UDPFields{ @@ -1383,15 +1397,6 @@ func (e *endpoint) HandlePacket(r *stack.Route, id stack.TransportEndpointID, pk return } - // Never receive from a multicast address. - if header.IsV4MulticastAddress(id.RemoteAddress) || - header.IsV6MulticastAddress(id.RemoteAddress) { - e.stack.Stats().UDP.InvalidSourceAddress.Increment() - e.stack.Stats().IP.InvalidSourceAddressesReceived.Increment() - e.stats.ReceiveErrors.MalformedPacketsReceived.Increment() - return - } - if !verifyChecksum(r, hdr, pkt) { // Checksum Error. e.stack.Stats().UDP.ChecksumErrors.Increment() diff --git a/pkg/tcpip/transport/udp/protocol.go b/pkg/tcpip/transport/udp/protocol.go index 3f87e8057..da5b1deb2 100644 --- a/pkg/tcpip/transport/udp/protocol.go +++ b/pkg/tcpip/transport/udp/protocol.go @@ -12,18 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Package udp contains the implementation of the UDP transport protocol. To use -// it in the networking stack, this package must be added to the project, and -// activated on the stack by passing udp.NewProtocol() as one of the -// transport protocols when calling stack.New(). Then endpoints can be created -// by passing udp.ProtocolNumber as the transport protocol number when calling -// Stack.NewEndpoint(). +// Package udp contains the implementation of the UDP transport protocol. package udp import ( "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/buffer" "gvisor.dev/gvisor/pkg/tcpip/header" + "gvisor.dev/gvisor/pkg/tcpip/header/parse" "gvisor.dev/gvisor/pkg/tcpip/stack" "gvisor.dev/gvisor/pkg/tcpip/transport/raw" "gvisor.dev/gvisor/pkg/waiter" @@ -49,6 +45,7 @@ const ( ) type protocol struct { + stack *stack.Stack } // Number returns the udp protocol number. @@ -57,14 +54,14 @@ func (*protocol) Number() tcpip.TransportProtocolNumber { } // NewEndpoint creates a new udp endpoint. -func (*protocol) NewEndpoint(stack *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { - return newEndpoint(stack, netProto, waiterQueue), nil +func (p *protocol) NewEndpoint(netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { + return newEndpoint(p.stack, netProto, waiterQueue), nil } // NewRawEndpoint creates a new raw UDP endpoint. It implements // stack.TransportProtocol.NewRawEndpoint. -func (p *protocol) NewRawEndpoint(stack *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { - return raw.NewEndpoint(stack, netProto, header.UDPProtocolNumber, waiterQueue) +func (p *protocol) NewRawEndpoint(netProto tcpip.NetworkProtocolNumber, waiterQueue *waiter.Queue) (tcpip.Endpoint, *tcpip.Error) { + return raw.NewEndpoint(p.stack, netProto, header.UDPProtocolNumber, waiterQueue) } // MinimumPacketSize returns the minimum valid udp packet size. @@ -79,126 +76,21 @@ func (*protocol) ParsePorts(v buffer.View) (src, dst uint16, err *tcpip.Error) { return h.SourcePort(), h.DestinationPort(), nil } -// HandleUnknownDestinationPacket handles packets targeted at this protocol but -// that don't match any existing endpoint. -func (p *protocol) HandleUnknownDestinationPacket(r *stack.Route, id stack.TransportEndpointID, pkt *stack.PacketBuffer) bool { +// HandleUnknownDestinationPacket handles packets that are targeted at this +// protocol but don't match any existing endpoint. +func (p *protocol) HandleUnknownDestinationPacket(r *stack.Route, id stack.TransportEndpointID, pkt *stack.PacketBuffer) stack.UnknownDestinationPacketDisposition { hdr := header.UDP(pkt.TransportHeader().View()) if int(hdr.Length()) > pkt.Data.Size()+header.UDPMinimumSize { - // Malformed packet. r.Stack().Stats().UDP.MalformedPacketsReceived.Increment() - return true + return stack.UnknownDestinationPacketMalformed } if !verifyChecksum(r, hdr, pkt) { - // Checksum Error. r.Stack().Stats().UDP.ChecksumErrors.Increment() - return true + return stack.UnknownDestinationPacketMalformed } - // Only send ICMP error if the address is not a multicast/broadcast - // v4/v6 address or the source is not the unspecified address. - // - // See: point e) in https://tools.ietf.org/html/rfc4443#section-2.4 - if id.LocalAddress == header.IPv4Broadcast || header.IsV4MulticastAddress(id.LocalAddress) || header.IsV6MulticastAddress(id.LocalAddress) || id.RemoteAddress == header.IPv6Any || id.RemoteAddress == header.IPv4Any { - return true - } - - // As per RFC: 1122 Section 3.2.2.1 A host SHOULD generate Destination - // Unreachable messages with code: - // - // 2 (Protocol Unreachable), when the designated transport protocol - // is not supported; or - // - // 3 (Port Unreachable), when the designated transport protocol - // (e.g., UDP) is unable to demultiplex the datagram but has no - // protocol mechanism to inform the sender. - switch len(id.LocalAddress) { - case header.IPv4AddressSize: - if !r.Stack().AllowICMPMessage() { - r.Stack().Stats().ICMP.V4PacketsSent.RateLimited.Increment() - return true - } - // As per RFC 1812 Section 4.3.2.3 - // - // ICMP datagram SHOULD contain as much of the original - // datagram as possible without the length of the ICMP - // datagram exceeding 576 bytes - // - // NOTE: The above RFC referenced is different from the original - // recommendation in RFC 1122 where it mentioned that at least 8 - // bytes of the payload must be included. Today linux and other - // systems implement the] RFC1812 definition and not the original - // RFC 1122 requirement. - mtu := int(r.MTU()) - if mtu > header.IPv4MinimumProcessableDatagramSize { - mtu = header.IPv4MinimumProcessableDatagramSize - } - headerLen := int(r.MaxHeaderLength()) + header.ICMPv4MinimumSize - available := int(mtu) - headerLen - payloadLen := pkt.NetworkHeader().View().Size() + pkt.TransportHeader().View().Size() + pkt.Data.Size() - if payloadLen > available { - payloadLen = available - } - - // The buffers used by pkt may be used elsewhere in the system. - // For example, a raw or packet socket may use what UDP - // considers an unreachable destination. Thus we deep copy pkt - // to prevent multiple ownership and SR errors. - newHeader := append(buffer.View(nil), pkt.NetworkHeader().View()...) - newHeader = append(newHeader, pkt.TransportHeader().View()...) - payload := newHeader.ToVectorisedView() - payload.AppendView(pkt.Data.ToView()) - payload.CapLength(payloadLen) - - icmpPkt := stack.NewPacketBuffer(stack.PacketBufferOptions{ - ReserveHeaderBytes: headerLen, - Data: payload, - }) - icmpHdr := header.ICMPv4(icmpPkt.TransportHeader().Push(header.ICMPv4MinimumSize)) - icmpHdr.SetType(header.ICMPv4DstUnreachable) - icmpHdr.SetCode(header.ICMPv4PortUnreachable) - icmpHdr.SetChecksum(header.ICMPv4Checksum(icmpHdr, icmpPkt.Data)) - r.WritePacket(nil /* gso */, stack.NetworkHeaderParams{Protocol: header.ICMPv4ProtocolNumber, TTL: r.DefaultTTL(), TOS: stack.DefaultTOS}, icmpPkt) - - case header.IPv6AddressSize: - if !r.Stack().AllowICMPMessage() { - r.Stack().Stats().ICMP.V6PacketsSent.RateLimited.Increment() - return true - } - - // As per RFC 4443 section 2.4 - // - // (c) Every ICMPv6 error message (type < 128) MUST include - // as much of the IPv6 offending (invoking) packet (the - // packet that caused the error) as possible without making - // the error message packet exceed the minimum IPv6 MTU - // [IPv6]. - mtu := int(r.MTU()) - if mtu > header.IPv6MinimumMTU { - mtu = header.IPv6MinimumMTU - } - headerLen := int(r.MaxHeaderLength()) + header.ICMPv6DstUnreachableMinimumSize - available := int(mtu) - headerLen - network, transport := pkt.NetworkHeader().View(), pkt.TransportHeader().View() - payloadLen := len(network) + len(transport) + pkt.Data.Size() - if payloadLen > available { - payloadLen = available - } - payload := buffer.NewVectorisedView(len(network)+len(transport), []buffer.View{network, transport}) - payload.Append(pkt.Data) - payload.CapLength(payloadLen) - - icmpPkt := stack.NewPacketBuffer(stack.PacketBufferOptions{ - ReserveHeaderBytes: headerLen, - Data: payload, - }) - icmpHdr := header.ICMPv6(icmpPkt.TransportHeader().Push(header.ICMPv6DstUnreachableMinimumSize)) - icmpHdr.SetType(header.ICMPv6DstUnreachable) - icmpHdr.SetCode(header.ICMPv6PortUnreachable) - icmpHdr.SetChecksum(header.ICMPv6Checksum(icmpHdr, r.LocalAddress, r.RemoteAddress, icmpPkt.Data)) - r.WritePacket(nil /* gso */, stack.NetworkHeaderParams{Protocol: header.ICMPv6ProtocolNumber, TTL: r.DefaultTTL(), TOS: stack.DefaultTOS}, icmpPkt) - } - return true + return stack.UnknownDestinationPacketUnhandled } // SetOption implements stack.TransportProtocol.SetOption. @@ -219,11 +111,10 @@ func (*protocol) Wait() {} // Parse implements stack.TransportProtocol.Parse. func (*protocol) Parse(pkt *stack.PacketBuffer) bool { - _, ok := pkt.TransportHeader().Consume(header.UDPMinimumSize) - return ok + return parse.UDP(pkt) } // NewProtocol returns a UDP transport protocol. -func NewProtocol() stack.TransportProtocol { - return &protocol{} +func NewProtocol(s *stack.Stack) stack.TransportProtocol { + return &protocol{stack: s} } diff --git a/pkg/tcpip/transport/udp/udp_test.go b/pkg/tcpip/transport/udp/udp_test.go index d5881d183..4e14a5fc5 100644 --- a/pkg/tcpip/transport/udp/udp_test.go +++ b/pkg/tcpip/transport/udp/udp_test.go @@ -294,8 +294,8 @@ type testContext struct { func newDualTestContext(t *testing.T, mtu uint32) *testContext { t.Helper() return newDualTestContextWithOptions(t, mtu, stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, }) } @@ -388,6 +388,10 @@ func (c *testContext) getPacketAndVerify(flow testFlow, checkers ...checker.Netw c.t.Fatalf("Bad network protocol: got %v, wanted %v", p.Proto, flow.netProto()) } + if got, want := p.Pkt.TransportProtocolNumber, header.UDPProtocolNumber; got != want { + c.t.Errorf("got p.Pkt.TransportProtocolNumber = %d, want = %d", got, want) + } + vv := buffer.NewVectorisedView(p.Pkt.Size(), p.Pkt.Views()) b := vv.ToView() @@ -528,8 +532,8 @@ func newMinPayload(minSize int) []byte { func TestBindToDeviceOption(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}}) + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}}) ep, err := s.NewEndpoint(udp.ProtocolNumber, ipv4.ProtocolNumber, &waiter.Queue{}) if err != nil { @@ -803,8 +807,8 @@ func TestV4ReadSelfSource(t *testing.T) { } { t.Run(tt.name, func(t *testing.T) { c := newDualTestContextWithOptions(t, defaultMTU, stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, HandleLocal: tt.handleLocal, }) defer c.cleanup() @@ -924,42 +928,6 @@ func TestReadFromMulticast(t *testing.T) { } } -// TestReadFromMulticaststats checks that a discarded packet -// that that was sent with multicast SOURCE address increments -// the correct counters and that a regular packet does not. -func TestReadFromMulticastStats(t *testing.T) { - t.Helper() - for _, flow := range []testFlow{reverseMulticast4, reverseMulticast6, unicastV4} { - t.Run(fmt.Sprintf("flow:%s", flow), func(t *testing.T) { - c := newDualTestContext(t, defaultMTU) - defer c.cleanup() - - c.createEndpointForFlow(flow) - - if err := c.ep.Bind(tcpip.FullAddress{Port: stackPort}); err != nil { - t.Fatalf("Bind failed: %s", err) - } - - payload := newPayload() - c.injectPacket(flow, payload, false) - - var want uint64 = 0 - if flow.isReverseMulticast() { - want = 1 - } - if got := c.s.Stats().IP.InvalidSourceAddressesReceived.Value(); got != want { - t.Errorf("got stats.IP.InvalidSourceAddressesReceived.Value() = %d, want = %d", got, want) - } - if got := c.s.Stats().UDP.InvalidSourceAddress.Value(); got != want { - t.Errorf("got stats.UDP.InvalidSourceAddress.Value() = %d, want = %d", got, want) - } - if got := c.ep.Stats().(*tcpip.TransportEndpointStats).ReceiveErrors.MalformedPacketsReceived.Value(); got != want { - t.Errorf("got EP Stats.ReceiveErrors.MalformedPacketsReceived stats = %d, want = %d", got, want) - } - }) - } -} - // TestV4ReadBroadcastOnBoundToWildcard checks that an endpoint can bind to ANY // and receive broadcast and unicast data. func TestV4ReadBroadcastOnBoundToWildcard(t *testing.T) { @@ -1462,6 +1430,30 @@ func TestNoChecksum(t *testing.T) { } } +var _ stack.NetworkInterface = (*testInterface)(nil) + +type testInterface struct{} + +func (*testInterface) ID() tcpip.NICID { + return 0 +} + +func (*testInterface) IsLoopback() bool { + return false +} + +func (*testInterface) Name() string { + return "" +} + +func (*testInterface) Enabled() bool { + return true +} + +func (*testInterface) LinkEndpoint() stack.LinkEndpoint { + return nil +} + func TestTTL(t *testing.T) { for _, flow := range []testFlow{unicastV4, unicastV4in6, unicastV6, unicastV6Only, multicastV4, multicastV4in6, multicastV6, broadcast, broadcastIn6} { t.Run(fmt.Sprintf("flow:%s", flow), func(t *testing.T) { @@ -1479,16 +1471,19 @@ func TestTTL(t *testing.T) { if flow.isMulticast() { wantTTL = multicastTTL } else { - var p stack.NetworkProtocol + var p stack.NetworkProtocolFactory + var n tcpip.NetworkProtocolNumber if flow.isV4() { - p = ipv4.NewProtocol() + p = ipv4.NewProtocol + n = ipv4.ProtocolNumber } else { - p = ipv6.NewProtocol() + p = ipv6.NewProtocol + n = ipv6.ProtocolNumber } - ep := p.NewEndpoint(0, nil, nil, nil, nil, stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, - })) + s := stack.New(stack.Options{ + NetworkProtocols: []stack.NetworkProtocolFactory{p}, + }) + ep := s.NetworkProtocolInstance(n).NewEndpoint(&testInterface{}, nil, nil, nil) wantTTL = ep.DefaultTTL() ep.Close() } @@ -1512,18 +1507,6 @@ func TestSetTTL(t *testing.T) { c.t.Fatalf("SetSockOptInt(TTLOption, %d) failed: %s", wantTTL, err) } - var p stack.NetworkProtocol - if flow.isV4() { - p = ipv4.NewProtocol() - } else { - p = ipv6.NewProtocol() - } - ep := p.NewEndpoint(0, nil, nil, nil, nil, stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, - })) - ep.Close() - testWrite(c, flow, checker.TTL(wantTTL)) }) } @@ -2353,9 +2336,8 @@ func TestOutgoingSubnetBroadcast(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { s := stack.New(stack.Options{ - NetworkProtocols: []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol()}, - - TransportProtocols: []stack.TransportProtocol{udp.NewProtocol()}, + NetworkProtocols: []stack.NetworkProtocolFactory{ipv4.NewProtocol, ipv6.NewProtocol}, + TransportProtocols: []stack.TransportProtocolFactory{udp.NewProtocol}, }) e := channel.New(0, defaultMTU, "") if err := s.CreateNIC(nicID1, e); err != nil { |