summaryrefslogtreecommitdiffhomepage
path: root/pkg/tcpip
diff options
context:
space:
mode:
authorgVisor bot <gvisor-bot@google.com>2021-01-14 00:18:26 +0000
committergVisor bot <gvisor-bot@google.com>2021-01-14 00:18:26 +0000
commitfc9aec0925d6ff6bdbc8a0c0866f6199ea054d9f (patch)
tree1b71096a33e99d24f588ac1066797fdc7f0146c4 /pkg/tcpip
parentcfa5eea33e3e8a2f8e7cac622ab046df43ccf0a1 (diff)
parent25b5ec7135a6de80674ac1ad4d2289c29e156f42 (diff)
Merge release-20201216.0-105-g25b5ec713 (automated)
Diffstat (limited to 'pkg/tcpip')
-rw-r--r--pkg/tcpip/adapters/gonet/gonet.go29
-rw-r--r--pkg/tcpip/network/ipv4/ipv4.go16
-rw-r--r--pkg/tcpip/network/ipv6/ipv6.go13
-rw-r--r--pkg/tcpip/stack/packet_buffer.go11
-rw-r--r--pkg/tcpip/stack/pending_packets.go8
-rw-r--r--pkg/tcpip/stack/registration.go21
-rw-r--r--pkg/tcpip/tcpip.go12
-rw-r--r--pkg/tcpip/transport/icmp/endpoint.go37
-rw-r--r--pkg/tcpip/transport/packet/endpoint.go4
-rw-r--r--pkg/tcpip/transport/raw/endpoint.go52
-rw-r--r--pkg/tcpip/transport/tcp/accept.go2
-rw-r--r--pkg/tcpip/transport/tcp/connect.go57
-rw-r--r--pkg/tcpip/transport/tcp/endpoint.go13
-rw-r--r--pkg/tcpip/transport/tcp/protocol.go1
-rw-r--r--pkg/tcpip/transport/tcp/segment.go23
-rw-r--r--pkg/tcpip/transport/tcp/tcp_state_autogen.go67
-rw-r--r--pkg/tcpip/transport/udp/endpoint.go45
-rw-r--r--pkg/tcpip/transport/udp/forwarder.go1
18 files changed, 159 insertions, 253 deletions
diff --git a/pkg/tcpip/adapters/gonet/gonet.go b/pkg/tcpip/adapters/gonet/gonet.go
index 7193f56ad..85a0b8b90 100644
--- a/pkg/tcpip/adapters/gonet/gonet.go
+++ b/pkg/tcpip/adapters/gonet/gonet.go
@@ -397,22 +397,9 @@ func (c *TCPConn) Write(b []byte) (int, error) {
}
var n int64
- var resCh <-chan struct{}
- n, resCh, err = c.ep.Write(tcpip.SlicePayload(v), tcpip.WriteOptions{})
+ n, err = c.ep.Write(tcpip.SlicePayload(v), tcpip.WriteOptions{})
nbytes += int(n)
v.TrimFront(int(n))
-
- if resCh != nil {
- select {
- case <-deadline:
- return nbytes, c.newOpError("write", &timeoutError{})
- case <-resCh:
- }
-
- n, _, err = c.ep.Write(tcpip.SlicePayload(v), tcpip.WriteOptions{})
- nbytes += int(n)
- v.TrimFront(int(n))
- }
}
if err == nil {
@@ -666,17 +653,7 @@ func (c *UDPConn) WriteTo(b []byte, addr net.Addr) (int, error) {
v := buffer.NewView(len(b))
copy(v, b)
- n, resCh, err := c.ep.Write(tcpip.SlicePayload(v), wopts)
- if resCh != nil {
- select {
- case <-deadline:
- return int(n), c.newRemoteOpError("write", addr, &timeoutError{})
- case <-resCh:
- }
-
- n, _, err = c.ep.Write(tcpip.SlicePayload(v), wopts)
- }
-
+ n, err := c.ep.Write(tcpip.SlicePayload(v), wopts)
if err == tcpip.ErrWouldBlock {
// Create wait queue entry that notifies a channel.
waitEntry, notifyCh := waiter.NewChannelEntry(nil)
@@ -689,7 +666,7 @@ func (c *UDPConn) WriteTo(b []byte, addr net.Addr) (int, error) {
case <-notifyCh:
}
- n, _, err = c.ep.Write(tcpip.SlicePayload(v), wopts)
+ n, err = c.ep.Write(tcpip.SlicePayload(v), wopts)
if err != tcpip.ErrWouldBlock {
break
}
diff --git a/pkg/tcpip/network/ipv4/ipv4.go b/pkg/tcpip/network/ipv4/ipv4.go
index e9ff70d04..cc045c7a9 100644
--- a/pkg/tcpip/network/ipv4/ipv4.go
+++ b/pkg/tcpip/network/ipv4/ipv4.go
@@ -64,6 +64,7 @@ const (
var ipv4BroadcastAddr = header.IPv4Broadcast.WithPrefix()
+var _ stack.LinkResolvableNetworkEndpoint = (*endpoint)(nil)
var _ stack.GroupAddressableEndpoint = (*endpoint)(nil)
var _ stack.AddressableEndpoint = (*endpoint)(nil)
var _ stack.NetworkEndpoint = (*endpoint)(nil)
@@ -87,6 +88,21 @@ type endpoint struct {
}
}
+// HandleLinkResolutionFailure implements stack.LinkResolvableNetworkEndpoint.
+func (e *endpoint) HandleLinkResolutionFailure(pkt *stack.PacketBuffer) {
+ // handleControl expects the entire offending packet to be in the packet
+ // buffer's data field.
+ pkt = stack.NewPacketBuffer(stack.PacketBufferOptions{
+ Data: buffer.NewVectorisedView(pkt.Size(), pkt.Views()),
+ })
+ pkt.NICID = e.nic.ID()
+ pkt.NetworkProtocolNumber = ProtocolNumber
+ // Use the same control type as an ICMPv4 destination host unreachable error
+ // since the host is considered unreachable if we cannot resolve the link
+ // address to the next hop.
+ e.handleControl(stack.ControlNoRoute, 0, pkt)
+}
+
// NewEndpoint creates a new ipv4 endpoint.
func (p *protocol) NewEndpoint(nic stack.NetworkInterface, _ stack.LinkAddressCache, _ stack.NUDHandler, dispatcher stack.TransportDispatcher) stack.NetworkEndpoint {
e := &endpoint{
diff --git a/pkg/tcpip/network/ipv6/ipv6.go b/pkg/tcpip/network/ipv6/ipv6.go
index f2018d073..2f82c3d5f 100644
--- a/pkg/tcpip/network/ipv6/ipv6.go
+++ b/pkg/tcpip/network/ipv6/ipv6.go
@@ -163,6 +163,7 @@ func getLabel(addr tcpip.Address) uint8 {
panic(fmt.Sprintf("should have a label for address = %s", addr))
}
+var _ stack.LinkResolvableNetworkEndpoint = (*endpoint)(nil)
var _ stack.GroupAddressableEndpoint = (*endpoint)(nil)
var _ stack.AddressableEndpoint = (*endpoint)(nil)
var _ stack.NetworkEndpoint = (*endpoint)(nil)
@@ -224,6 +225,18 @@ type OpaqueInterfaceIdentifierOptions struct {
SecretKey []byte
}
+// HandleLinkResolutionFailure implements stack.LinkResolvableNetworkEndpoint.
+func (e *endpoint) HandleLinkResolutionFailure(pkt *stack.PacketBuffer) {
+ // handleControl expects the entire offending packet to be in the packet
+ // buffer's data field.
+ pkt = stack.NewPacketBuffer(stack.PacketBufferOptions{
+ Data: buffer.NewVectorisedView(pkt.Size(), pkt.Views()),
+ })
+ pkt.NICID = e.nic.ID()
+ pkt.NetworkProtocolNumber = ProtocolNumber
+ e.handleControl(stack.ControlAddressUnreachable, 0, pkt)
+}
+
// onAddressAssignedLocked handles an address being assigned.
//
// Precondition: e.mu must be exclusively locked.
diff --git a/pkg/tcpip/stack/packet_buffer.go b/pkg/tcpip/stack/packet_buffer.go
index 664cc6fa0..5f216ca21 100644
--- a/pkg/tcpip/stack/packet_buffer.go
+++ b/pkg/tcpip/stack/packet_buffer.go
@@ -268,17 +268,6 @@ func (pk *PacketBuffer) Clone() *PacketBuffer {
}
}
-// SourceLinkAddress returns the source link address of the packet.
-func (pk *PacketBuffer) SourceLinkAddress() tcpip.LinkAddress {
- link := pk.LinkHeader().View()
-
- if link.IsEmpty() {
- return ""
- }
-
- return header.Ethernet(link).SourceAddress()
-}
-
// Network returns the network header as a header.Network.
//
// Network should only be called when NetworkHeader has been set.
diff --git a/pkg/tcpip/stack/pending_packets.go b/pkg/tcpip/stack/pending_packets.go
index 4a3adcf33..bded8814e 100644
--- a/pkg/tcpip/stack/pending_packets.go
+++ b/pkg/tcpip/stack/pending_packets.go
@@ -101,10 +101,12 @@ func (f *packetsPendingLinkResolution) enqueue(ch <-chan struct{}, r *Route, pro
}
for _, p := range packets {
- if cancelled {
- p.route.Stats().IP.OutgoingPacketErrors.Increment()
- } else if p.route.IsResolutionRequired() {
+ if cancelled || p.route.IsResolutionRequired() {
p.route.Stats().IP.OutgoingPacketErrors.Increment()
+
+ if linkResolvableEP, ok := p.route.outgoingNIC.getNetworkEndpoint(p.route.NetProto).(LinkResolvableNetworkEndpoint); ok {
+ linkResolvableEP.HandleLinkResolutionFailure(pkt)
+ }
} else {
p.route.outgoingNIC.writePacket(p.route, nil /* gso */, p.proto, p.pkt)
}
diff --git a/pkg/tcpip/stack/registration.go b/pkg/tcpip/stack/registration.go
index 4795208b4..924790779 100644
--- a/pkg/tcpip/stack/registration.go
+++ b/pkg/tcpip/stack/registration.go
@@ -55,7 +55,19 @@ type ControlType int
// The following are the allowed values for ControlType values.
// TODO(http://gvisor.dev/issue/3210): Support time exceeded messages.
const (
- ControlNetworkUnreachable ControlType = iota
+ // ControlAddressUnreachable indicates that an IPv6 packet did not reach its
+ // destination as the destination address was unreachable.
+ //
+ // This maps to the ICMPv6 Destination Ureachable Code 3 error; see
+ // RFC 4443 section 3.1 for more details.
+ ControlAddressUnreachable ControlType = iota
+ ControlNetworkUnreachable
+ // ControlNoRoute indicates that an IPv4 packet did not reach its destination
+ // because the destination host was unreachable.
+ //
+ // This maps to the ICMPv4 Destination Ureachable Code 1 error; see
+ // RFC 791's Destination Unreachable Message section (page 4) for more
+ // details.
ControlNoRoute
ControlPacketTooBig
ControlPortUnreachable
@@ -503,6 +515,13 @@ type NetworkInterface interface {
WritePacketToRemote(tcpip.LinkAddress, *GSO, tcpip.NetworkProtocolNumber, *PacketBuffer) *tcpip.Error
}
+// LinkResolvableNetworkEndpoint handles link resolution events.
+type LinkResolvableNetworkEndpoint interface {
+ // HandleLinkResolutionFailure is called when link resolution prevents the
+ // argument from having been sent.
+ HandleLinkResolutionFailure(*PacketBuffer)
+}
+
// NetworkEndpoint is the interface that needs to be implemented by endpoints
// of network layer protocols (e.g., ipv4, ipv6).
type NetworkEndpoint interface {
diff --git a/pkg/tcpip/tcpip.go b/pkg/tcpip/tcpip.go
index f798056c0..002ddaf67 100644
--- a/pkg/tcpip/tcpip.go
+++ b/pkg/tcpip/tcpip.go
@@ -104,7 +104,6 @@ var (
ErrConnectionAborted = &Error{msg: "connection aborted"}
ErrNoSuchFile = &Error{msg: "no such file"}
ErrInvalidOptionValue = &Error{msg: "invalid option value specified"}
- ErrNoLinkAddress = &Error{msg: "no remote link address"}
ErrBadAddress = &Error{msg: "bad address"}
ErrNetworkUnreachable = &Error{msg: "network is unreachable"}
ErrMessageTooLong = &Error{msg: "message too long"}
@@ -154,7 +153,6 @@ func StringToError(s string) *Error {
ErrConnectionAborted,
ErrNoSuchFile,
ErrInvalidOptionValue,
- ErrNoLinkAddress,
ErrBadAddress,
ErrNetworkUnreachable,
ErrMessageTooLong,
@@ -640,12 +638,7 @@ type Endpoint interface {
// stream (TCP) Endpoints may return partial writes, and even then only
// in the case where writing additional data would block. Other Endpoints
// will either write the entire message or return an error.
- //
- // For UDP and Ping sockets if address resolution is required,
- // ErrNoLinkAddress and a notification channel is returned for the caller to
- // block. Channel is closed once address resolution is complete (success or
- // not). The channel is only non-nil in this case.
- Write(Payloader, WriteOptions) (int64, <-chan struct{}, *Error)
+ Write(Payloader, WriteOptions) (int64, *Error)
// Connect connects the endpoint to its peer. Specifying a NIC is
// optional.
@@ -1784,9 +1777,6 @@ type SendErrors struct {
// NoRoute is the number of times we failed to resolve IP route.
NoRoute StatCounter
-
- // NoLinkAddr is the number of times we failed to resolve ARP.
- NoLinkAddr StatCounter
}
// ReadErrors collects segment read errors from an endpoint read call.
diff --git a/pkg/tcpip/transport/icmp/endpoint.go b/pkg/tcpip/transport/icmp/endpoint.go
index c32fe5c4f..87277fbd3 100644
--- a/pkg/tcpip/transport/icmp/endpoint.go
+++ b/pkg/tcpip/transport/icmp/endpoint.go
@@ -236,8 +236,8 @@ func (e *endpoint) prepareForWrite(to *tcpip.FullAddress) (retry bool, err *tcpi
// Write writes data to the endpoint's peer. This method does not block
// if the data cannot be written.
-func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-chan struct{}, *tcpip.Error) {
- n, ch, err := e.write(p, opts)
+func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, *tcpip.Error) {
+ n, err := e.write(p, opts)
switch err {
case nil:
e.stats.PacketsSent.Increment()
@@ -247,8 +247,6 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
e.stats.WriteErrors.WriteClosed.Increment()
case tcpip.ErrInvalidEndpointState:
e.stats.WriteErrors.InvalidEndpointState.Increment()
- case tcpip.ErrNoLinkAddress:
- e.stats.SendErrors.NoLinkAddr.Increment()
case tcpip.ErrNoRoute, tcpip.ErrBroadcastDisabled, tcpip.ErrNetworkUnreachable:
// Errors indicating any problem with IP routing of the packet.
e.stats.SendErrors.NoRoute.Increment()
@@ -256,13 +254,13 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
// For all other errors when writing to the network layer.
e.stats.SendErrors.SendToNetworkFailed.Increment()
}
- return n, ch, err
+ return n, err
}
-func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-chan struct{}, *tcpip.Error) {
+func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, *tcpip.Error) {
// MSG_MORE is unimplemented. (This also means that MSG_EOR is a no-op.)
if opts.More {
- return 0, nil, tcpip.ErrInvalidOptionValue
+ return 0, tcpip.ErrInvalidOptionValue
}
to := opts.To
@@ -272,14 +270,14 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
// If we've shutdown with SHUT_WR we are in an invalid state for sending.
if e.shutdownFlags&tcpip.ShutdownWrite != 0 {
- return 0, nil, tcpip.ErrClosedForSend
+ return 0, tcpip.ErrClosedForSend
}
// Prepare for write.
for {
retry, err := e.prepareForWrite(to)
if err != nil {
- return 0, nil, err
+ return 0, err
}
if !retry {
@@ -294,7 +292,7 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
nicID := to.NIC
if e.BindNICID != 0 {
if nicID != 0 && nicID != e.BindNICID {
- return 0, nil, tcpip.ErrNoRoute
+ return 0, tcpip.ErrNoRoute
}
nicID = e.BindNICID
@@ -302,31 +300,22 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
dst, netProto, err := e.checkV4MappedLocked(*to)
if err != nil {
- return 0, nil, err
+ return 0, err
}
// Find the endpoint.
r, err := e.stack.FindRoute(nicID, e.BindAddr, dst.Addr, netProto, false /* multicastLoop */)
if err != nil {
- return 0, nil, err
+ return 0, err
}
defer r.Release()
route = r
}
- if route.IsResolutionRequired() {
- if ch, err := route.Resolve(nil); err != nil {
- if err == tcpip.ErrWouldBlock {
- return 0, ch, tcpip.ErrNoLinkAddress
- }
- return 0, nil, err
- }
- }
-
v, err := p.FullPayload()
if err != nil {
- return 0, nil, err
+ return 0, err
}
switch e.NetProto {
@@ -338,10 +327,10 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
}
if err != nil {
- return 0, nil, err
+ return 0, err
}
- return int64(len(v)), nil, nil
+ return int64(len(v)), nil
}
// SetSockOpt sets a socket option.
diff --git a/pkg/tcpip/transport/packet/endpoint.go b/pkg/tcpip/transport/packet/endpoint.go
index 3ab060751..c3b3b8d34 100644
--- a/pkg/tcpip/transport/packet/endpoint.go
+++ b/pkg/tcpip/transport/packet/endpoint.go
@@ -207,9 +207,9 @@ func (ep *endpoint) Read(dst io.Writer, count int, opts tcpip.ReadOptions) (tcpi
return res, nil
}
-func (*endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-chan struct{}, *tcpip.Error) {
+func (*endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, *tcpip.Error) {
// TODO(gvisor.dev/issue/173): Implement.
- return 0, nil, tcpip.ErrInvalidOptionValue
+ return 0, tcpip.ErrInvalidOptionValue
}
// Disconnect implements tcpip.Endpoint.Disconnect. Packet sockets cannot be
diff --git a/pkg/tcpip/transport/raw/endpoint.go b/pkg/tcpip/transport/raw/endpoint.go
index dd260535f..425bcf3ee 100644
--- a/pkg/tcpip/transport/raw/endpoint.go
+++ b/pkg/tcpip/transport/raw/endpoint.go
@@ -234,20 +234,20 @@ func (e *endpoint) Read(dst io.Writer, count int, opts tcpip.ReadOptions) (tcpip
}
// Write implements tcpip.Endpoint.Write.
-func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-chan struct{}, *tcpip.Error) {
+func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, *tcpip.Error) {
// We can create, but not write to, unassociated IPv6 endpoints.
if !e.associated && e.TransportEndpointInfo.NetProto == header.IPv6ProtocolNumber {
- return 0, nil, tcpip.ErrInvalidOptionValue
+ return 0, tcpip.ErrInvalidOptionValue
}
if opts.To != nil {
// Raw sockets do not support sending to a IPv4 address on a IPv6 endpoint.
if e.TransportEndpointInfo.NetProto == header.IPv6ProtocolNumber && len(opts.To.Addr) != header.IPv6AddressSize {
- return 0, nil, tcpip.ErrInvalidOptionValue
+ return 0, tcpip.ErrInvalidOptionValue
}
}
- n, ch, err := e.write(p, opts)
+ n, err := e.write(p, opts)
switch err {
case nil:
e.stats.PacketsSent.Increment()
@@ -257,8 +257,6 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
e.stats.WriteErrors.WriteClosed.Increment()
case tcpip.ErrInvalidEndpointState:
e.stats.WriteErrors.InvalidEndpointState.Increment()
- case tcpip.ErrNoLinkAddress:
- e.stats.SendErrors.NoLinkAddr.Increment()
case tcpip.ErrNoRoute, tcpip.ErrBroadcastDisabled, tcpip.ErrNetworkUnreachable:
// Errors indicating any problem with IP routing of the packet.
e.stats.SendErrors.NoRoute.Increment()
@@ -266,25 +264,25 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
// For all other errors when writing to the network layer.
e.stats.SendErrors.SendToNetworkFailed.Increment()
}
- return n, ch, err
+ return n, err
}
-func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-chan struct{}, *tcpip.Error) {
+func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, *tcpip.Error) {
// MSG_MORE is unimplemented. This also means that MSG_EOR is a no-op.
if opts.More {
- return 0, nil, tcpip.ErrInvalidOptionValue
+ return 0, tcpip.ErrInvalidOptionValue
}
e.mu.RLock()
defer e.mu.RUnlock()
if e.closed {
- return 0, nil, tcpip.ErrInvalidEndpointState
+ return 0, tcpip.ErrInvalidEndpointState
}
payloadBytes, err := p.FullPayload()
if err != nil {
- return 0, nil, err
+ return 0, err
}
// If this is an unassociated socket and callee provided a nonzero
@@ -292,7 +290,7 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
if e.ops.GetHeaderIncluded() {
ip := header.IPv4(payloadBytes)
if !ip.IsValid(len(payloadBytes)) {
- return 0, nil, tcpip.ErrInvalidOptionValue
+ return 0, tcpip.ErrInvalidOptionValue
}
dstAddr := ip.DestinationAddress()
// Update dstAddr with the address in the IP header, unless
@@ -313,7 +311,7 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
// If the user doesn't specify a destination, they should have
// connected to another address.
if !e.connected {
- return 0, nil, tcpip.ErrDestinationRequired
+ return 0, tcpip.ErrDestinationRequired
}
return e.finishWrite(payloadBytes, e.route)
@@ -323,42 +321,30 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
// goes through a different NIC than the endpoint was bound to.
nic := opts.To.NIC
if e.bound && nic != 0 && nic != e.BindNICID {
- return 0, nil, tcpip.ErrNoRoute
+ return 0, tcpip.ErrNoRoute
}
// Find the route to the destination. If BindAddress is 0,
// FindRoute will choose an appropriate source address.
route, err := e.stack.FindRoute(nic, e.BindAddr, opts.To.Addr, e.NetProto, false)
if err != nil {
- return 0, nil, err
+ return 0, err
}
- n, ch, err := e.finishWrite(payloadBytes, route)
+ n, err := e.finishWrite(payloadBytes, route)
route.Release()
- return n, ch, err
+ return n, err
}
// finishWrite writes the payload to a route. It resolves the route if
// necessary. It's really just a helper to make defer unnecessary in Write.
-func (e *endpoint) finishWrite(payloadBytes []byte, route *stack.Route) (int64, <-chan struct{}, *tcpip.Error) {
- // We may need to resolve the route (match a link layer address to the
- // network address). If that requires blocking (e.g. to use ARP),
- // return a channel on which the caller can wait.
- if route.IsResolutionRequired() {
- if ch, err := route.Resolve(nil); err != nil {
- if err == tcpip.ErrWouldBlock {
- return 0, ch, tcpip.ErrNoLinkAddress
- }
- return 0, nil, err
- }
- }
-
+func (e *endpoint) finishWrite(payloadBytes []byte, route *stack.Route) (int64, *tcpip.Error) {
if e.ops.GetHeaderIncluded() {
pkt := stack.NewPacketBuffer(stack.PacketBufferOptions{
Data: buffer.View(payloadBytes).ToVectorisedView(),
})
if err := route.WriteHeaderIncludedPacket(pkt); err != nil {
- return 0, nil, err
+ return 0, err
}
} else {
pkt := stack.NewPacketBuffer(stack.PacketBufferOptions{
@@ -371,11 +357,11 @@ func (e *endpoint) finishWrite(payloadBytes []byte, route *stack.Route) (int64,
TTL: route.DefaultTTL(),
TOS: stack.DefaultTOS,
}, pkt); err != nil {
- return 0, nil, err
+ return 0, err
}
}
- return int64(len(payloadBytes)), nil, nil
+ return int64(len(payloadBytes)), nil
}
// Disconnect implements tcpip.Endpoint.Disconnect.
diff --git a/pkg/tcpip/transport/tcp/accept.go b/pkg/tcpip/transport/tcp/accept.go
index 2d96a65bd..9e8872fc9 100644
--- a/pkg/tcpip/transport/tcp/accept.go
+++ b/pkg/tcpip/transport/tcp/accept.go
@@ -210,7 +210,6 @@ func (l *listenContext) createConnectingEndpoint(s *segment, iss seqnum.Value, i
if err != nil {
return nil, err
}
- route.ResolveWith(s.remoteLinkAddr)
n := newEndpoint(l.stack, netProto, queue)
n.ops.SetV6Only(l.v6Only)
@@ -573,7 +572,6 @@ func (e *endpoint) handleListenSegment(ctx *listenContext, s *segment) *tcpip.Er
return err
}
defer route.Release()
- route.ResolveWith(s.remoteLinkAddr)
// Send SYN without window scaling because we currently
// don't encode this information in the cookie.
diff --git a/pkg/tcpip/transport/tcp/connect.go b/pkg/tcpip/transport/tcp/connect.go
index a00ef97c6..f45d26a87 100644
--- a/pkg/tcpip/transport/tcp/connect.go
+++ b/pkg/tcpip/transport/tcp/connect.go
@@ -460,66 +460,9 @@ func (h *handshake) processSegments() *tcpip.Error {
return nil
}
-func (h *handshake) resolveRoute() *tcpip.Error {
- // Set up the wakers.
- var s sleep.Sleeper
- resolutionWaker := &sleep.Waker{}
- s.AddWaker(resolutionWaker, wakerForResolution)
- s.AddWaker(&h.ep.notificationWaker, wakerForNotification)
- defer s.Done()
-
- // Initial action is to resolve route.
- index := wakerForResolution
- attemptedResolution := false
- for {
- switch index {
- case wakerForResolution:
- if _, err := h.ep.route.Resolve(resolutionWaker.Assert); err != tcpip.ErrWouldBlock {
- if err != nil {
- h.ep.stats.SendErrors.NoRoute.Increment()
- }
- // Either success (err == nil) or failure.
- return err
- }
- if attemptedResolution {
- h.ep.stats.SendErrors.NoLinkAddr.Increment()
- return tcpip.ErrNoLinkAddress
- }
- attemptedResolution = true
- // Resolution not completed. Keep trying...
-
- case wakerForNotification:
- n := h.ep.fetchNotifications()
- if n&notifyClose != 0 {
- return tcpip.ErrAborted
- }
- if n&notifyDrain != 0 {
- close(h.ep.drainDone)
- h.ep.mu.Unlock()
- <-h.ep.undrain
- h.ep.mu.Lock()
- }
- if n&notifyError != 0 {
- return h.ep.lastErrorLocked()
- }
- }
-
- // Wait for notification.
- h.ep.mu.Unlock()
- index, _ = s.Fetch(true /* block */)
- h.ep.mu.Lock()
- }
-}
-
// start resolves the route if necessary and sends the first
// SYN/SYN-ACK.
func (h *handshake) start() *tcpip.Error {
- if h.ep.route.IsResolutionRequired() {
- if err := h.resolveRoute(); err != nil {
- return err
- }
- }
-
h.startTime = time.Now()
h.ep.amss = calculateAdvertisedMSS(h.ep.userMSS, h.ep.route)
var sackEnabled tcpip.TCPSACKEnabled
diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go
index 25b180fa5..ddbed7e46 100644
--- a/pkg/tcpip/transport/tcp/endpoint.go
+++ b/pkg/tcpip/transport/tcp/endpoint.go
@@ -1507,7 +1507,7 @@ func (e *endpoint) isEndpointWritableLocked() (int, *tcpip.Error) {
}
// Write writes data to the endpoint's peer.
-func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-chan struct{}, *tcpip.Error) {
+func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, *tcpip.Error) {
// Linux completely ignores any address passed to sendto(2) for TCP sockets
// (without the MSG_FASTOPEN flag). Corking is unimplemented, so opts.More
// and opts.EndOfRecord are also ignored.
@@ -1520,7 +1520,7 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
e.sndBufMu.Unlock()
e.UnlockUser()
e.stats.WriteErrors.WriteClosed.Increment()
- return 0, nil, err
+ return 0, err
}
// We can release locks while copying data.
@@ -1541,7 +1541,7 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
e.sndBufMu.Unlock()
e.UnlockUser()
}
- return 0, nil, perr
+ return 0, perr
}
if !opts.Atomic {
@@ -1555,7 +1555,7 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
e.sndBufMu.Unlock()
e.UnlockUser()
e.stats.WriteErrors.WriteClosed.Increment()
- return 0, nil, err
+ return 0, err
}
// Discard any excess data copied in due to avail being reduced due
@@ -1575,7 +1575,7 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
// Do the work inline.
e.handleWrite()
e.UnlockUser()
- return int64(len(v)), nil, nil
+ return int64(len(v)), nil
}
// selectWindowLocked returns the new window without checking for shrinking or scaling
@@ -2779,6 +2779,9 @@ func (e *endpoint) HandleControlPacket(typ stack.ControlType, extra uint32, pkt
case stack.ControlNoRoute:
e.onICMPError(tcpip.ErrNoRoute, byte(header.ICMPv4DstUnreachable), byte(header.ICMPv4HostUnreachable), extra, pkt)
+ case stack.ControlAddressUnreachable:
+ e.onICMPError(tcpip.ErrNoRoute, byte(header.ICMPv6DstUnreachable), byte(header.ICMPv6AddressUnreachable), extra, pkt)
+
case stack.ControlNetworkUnreachable:
e.onICMPError(tcpip.ErrNetworkUnreachable, byte(header.ICMPv6DstUnreachable), byte(header.ICMPv6NetworkUnreachable), extra, pkt)
}
diff --git a/pkg/tcpip/transport/tcp/protocol.go b/pkg/tcpip/transport/tcp/protocol.go
index c9e194f82..1720370c9 100644
--- a/pkg/tcpip/transport/tcp/protocol.go
+++ b/pkg/tcpip/transport/tcp/protocol.go
@@ -222,7 +222,6 @@ func replyWithReset(stack *stack.Stack, s *segment, tos, ttl uint8) *tcpip.Error
return err
}
defer route.Release()
- route.ResolveWith(s.remoteLinkAddr)
// Get the seqnum from the packet if the ack flag is set.
seq := seqnum.Value(0)
diff --git a/pkg/tcpip/transport/tcp/segment.go b/pkg/tcpip/transport/tcp/segment.go
index c5a6d2fba..7cca4def5 100644
--- a/pkg/tcpip/transport/tcp/segment.go
+++ b/pkg/tcpip/transport/tcp/segment.go
@@ -49,11 +49,10 @@ type segment struct {
// TODO(gvisor.dev/issue/4417): Hold a stack.PacketBuffer instead of
// individual members for link/network packet info.
- srcAddr tcpip.Address
- dstAddr tcpip.Address
- netProto tcpip.NetworkProtocolNumber
- nicID tcpip.NICID
- remoteLinkAddr tcpip.LinkAddress
+ srcAddr tcpip.Address
+ dstAddr tcpip.Address
+ netProto tcpip.NetworkProtocolNumber
+ nicID tcpip.NICID
data buffer.VectorisedView `state:".(buffer.VectorisedView)"`
@@ -89,13 +88,12 @@ type segment struct {
func newIncomingSegment(id stack.TransportEndpointID, pkt *stack.PacketBuffer) *segment {
netHdr := pkt.Network()
s := &segment{
- refCnt: 1,
- id: id,
- srcAddr: netHdr.SourceAddress(),
- dstAddr: netHdr.DestinationAddress(),
- netProto: pkt.NetworkProtocolNumber,
- nicID: pkt.NICID,
- remoteLinkAddr: pkt.SourceLinkAddress(),
+ refCnt: 1,
+ id: id,
+ srcAddr: netHdr.SourceAddress(),
+ dstAddr: netHdr.DestinationAddress(),
+ netProto: pkt.NetworkProtocolNumber,
+ nicID: pkt.NICID,
}
s.data = pkt.Data.Clone(s.views[:])
s.hdr = header.TCP(pkt.TransportHeader().View())
@@ -128,7 +126,6 @@ func (s *segment) clone() *segment {
window: s.window,
netProto: s.netProto,
nicID: s.nicID,
- remoteLinkAddr: s.remoteLinkAddr,
rcvdTime: s.rcvdTime,
xmitTime: s.xmitTime,
xmitCount: s.xmitCount,
diff --git a/pkg/tcpip/transport/tcp/tcp_state_autogen.go b/pkg/tcpip/transport/tcp/tcp_state_autogen.go
index 251909dae..6131d151a 100644
--- a/pkg/tcpip/transport/tcp/tcp_state_autogen.go
+++ b/pkg/tcpip/transport/tcp/tcp_state_autogen.go
@@ -600,7 +600,6 @@ func (s *segment) StateFields() []string {
"dstAddr",
"netProto",
"nicID",
- "remoteLinkAddr",
"data",
"hdr",
"sequenceNumber",
@@ -625,13 +624,13 @@ func (s *segment) beforeSave() {}
func (s *segment) StateSave(stateSinkObject state.Sink) {
s.beforeSave()
var dataValue buffer.VectorisedView = s.saveData()
- stateSinkObject.SaveValue(9, dataValue)
+ stateSinkObject.SaveValue(8, dataValue)
var optionsValue []byte = s.saveOptions()
- stateSinkObject.SaveValue(18, optionsValue)
+ stateSinkObject.SaveValue(17, optionsValue)
var rcvdTimeValue unixTime = s.saveRcvdTime()
- stateSinkObject.SaveValue(20, rcvdTimeValue)
+ stateSinkObject.SaveValue(19, rcvdTimeValue)
var xmitTimeValue unixTime = s.saveXmitTime()
- stateSinkObject.SaveValue(21, xmitTimeValue)
+ stateSinkObject.SaveValue(20, xmitTimeValue)
stateSinkObject.Save(0, &s.segmentEntry)
stateSinkObject.Save(1, &s.refCnt)
stateSinkObject.Save(2, &s.ep)
@@ -640,19 +639,18 @@ func (s *segment) StateSave(stateSinkObject state.Sink) {
stateSinkObject.Save(5, &s.dstAddr)
stateSinkObject.Save(6, &s.netProto)
stateSinkObject.Save(7, &s.nicID)
- stateSinkObject.Save(8, &s.remoteLinkAddr)
- stateSinkObject.Save(10, &s.hdr)
- stateSinkObject.Save(11, &s.sequenceNumber)
- stateSinkObject.Save(12, &s.ackNumber)
- stateSinkObject.Save(13, &s.flags)
- stateSinkObject.Save(14, &s.window)
- stateSinkObject.Save(15, &s.csum)
- stateSinkObject.Save(16, &s.csumValid)
- stateSinkObject.Save(17, &s.parsedOptions)
- stateSinkObject.Save(19, &s.hasNewSACKInfo)
- stateSinkObject.Save(22, &s.xmitCount)
- stateSinkObject.Save(23, &s.acked)
- stateSinkObject.Save(24, &s.dataMemSize)
+ stateSinkObject.Save(9, &s.hdr)
+ stateSinkObject.Save(10, &s.sequenceNumber)
+ stateSinkObject.Save(11, &s.ackNumber)
+ stateSinkObject.Save(12, &s.flags)
+ stateSinkObject.Save(13, &s.window)
+ stateSinkObject.Save(14, &s.csum)
+ stateSinkObject.Save(15, &s.csumValid)
+ stateSinkObject.Save(16, &s.parsedOptions)
+ stateSinkObject.Save(18, &s.hasNewSACKInfo)
+ stateSinkObject.Save(21, &s.xmitCount)
+ stateSinkObject.Save(22, &s.acked)
+ stateSinkObject.Save(23, &s.dataMemSize)
}
func (s *segment) afterLoad() {}
@@ -666,23 +664,22 @@ func (s *segment) StateLoad(stateSourceObject state.Source) {
stateSourceObject.Load(5, &s.dstAddr)
stateSourceObject.Load(6, &s.netProto)
stateSourceObject.Load(7, &s.nicID)
- stateSourceObject.Load(8, &s.remoteLinkAddr)
- stateSourceObject.Load(10, &s.hdr)
- stateSourceObject.Load(11, &s.sequenceNumber)
- stateSourceObject.Load(12, &s.ackNumber)
- stateSourceObject.Load(13, &s.flags)
- stateSourceObject.Load(14, &s.window)
- stateSourceObject.Load(15, &s.csum)
- stateSourceObject.Load(16, &s.csumValid)
- stateSourceObject.Load(17, &s.parsedOptions)
- stateSourceObject.Load(19, &s.hasNewSACKInfo)
- stateSourceObject.Load(22, &s.xmitCount)
- stateSourceObject.Load(23, &s.acked)
- stateSourceObject.Load(24, &s.dataMemSize)
- stateSourceObject.LoadValue(9, new(buffer.VectorisedView), func(y interface{}) { s.loadData(y.(buffer.VectorisedView)) })
- stateSourceObject.LoadValue(18, new([]byte), func(y interface{}) { s.loadOptions(y.([]byte)) })
- stateSourceObject.LoadValue(20, new(unixTime), func(y interface{}) { s.loadRcvdTime(y.(unixTime)) })
- stateSourceObject.LoadValue(21, new(unixTime), func(y interface{}) { s.loadXmitTime(y.(unixTime)) })
+ stateSourceObject.Load(9, &s.hdr)
+ stateSourceObject.Load(10, &s.sequenceNumber)
+ stateSourceObject.Load(11, &s.ackNumber)
+ stateSourceObject.Load(12, &s.flags)
+ stateSourceObject.Load(13, &s.window)
+ stateSourceObject.Load(14, &s.csum)
+ stateSourceObject.Load(15, &s.csumValid)
+ stateSourceObject.Load(16, &s.parsedOptions)
+ stateSourceObject.Load(18, &s.hasNewSACKInfo)
+ stateSourceObject.Load(21, &s.xmitCount)
+ stateSourceObject.Load(22, &s.acked)
+ stateSourceObject.Load(23, &s.dataMemSize)
+ stateSourceObject.LoadValue(8, new(buffer.VectorisedView), func(y interface{}) { s.loadData(y.(buffer.VectorisedView)) })
+ stateSourceObject.LoadValue(17, new([]byte), func(y interface{}) { s.loadOptions(y.([]byte)) })
+ stateSourceObject.LoadValue(19, new(unixTime), func(y interface{}) { s.loadRcvdTime(y.(unixTime)) })
+ stateSourceObject.LoadValue(20, new(unixTime), func(y interface{}) { s.loadXmitTime(y.(unixTime)) })
}
func (q *segmentQueue) StateTypeName() string {
diff --git a/pkg/tcpip/transport/udp/endpoint.go b/pkg/tcpip/transport/udp/endpoint.go
index 5d87f3a7e..520a0ac9d 100644
--- a/pkg/tcpip/transport/udp/endpoint.go
+++ b/pkg/tcpip/transport/udp/endpoint.go
@@ -417,8 +417,8 @@ func (e *endpoint) connectRoute(nicID tcpip.NICID, addr tcpip.FullAddress, netPr
// Write writes data to the endpoint's peer. This method does not block
// if the data cannot be written.
-func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-chan struct{}, *tcpip.Error) {
- n, ch, err := e.write(p, opts)
+func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, *tcpip.Error) {
+ n, err := e.write(p, opts)
switch err {
case nil:
e.stats.PacketsSent.Increment()
@@ -428,8 +428,6 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
e.stats.WriteErrors.WriteClosed.Increment()
case tcpip.ErrInvalidEndpointState:
e.stats.WriteErrors.InvalidEndpointState.Increment()
- case tcpip.ErrNoLinkAddress:
- e.stats.SendErrors.NoLinkAddr.Increment()
case tcpip.ErrNoRoute, tcpip.ErrBroadcastDisabled, tcpip.ErrNetworkUnreachable:
// Errors indicating any problem with IP routing of the packet.
e.stats.SendErrors.NoRoute.Increment()
@@ -437,17 +435,17 @@ func (e *endpoint) Write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
// For all other errors when writing to the network layer.
e.stats.SendErrors.SendToNetworkFailed.Increment()
}
- return n, ch, err
+ return n, err
}
-func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-chan struct{}, *tcpip.Error) {
+func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, *tcpip.Error) {
if err := e.LastError(); err != nil {
- return 0, nil, err
+ return 0, err
}
// MSG_MORE is unimplemented. (This also means that MSG_EOR is a no-op.)
if opts.More {
- return 0, nil, tcpip.ErrInvalidOptionValue
+ return 0, tcpip.ErrInvalidOptionValue
}
to := opts.To
@@ -463,14 +461,14 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
// If we've shutdown with SHUT_WR we are in an invalid state for sending.
if e.shutdownFlags&tcpip.ShutdownWrite != 0 {
- return 0, nil, tcpip.ErrClosedForSend
+ return 0, tcpip.ErrClosedForSend
}
// Prepare for write.
for {
retry, err := e.prepareForWrite(to)
if err != nil {
- return 0, nil, err
+ return 0, err
}
if !retry {
@@ -486,7 +484,7 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
nicID := to.NIC
if e.BindNICID != 0 {
if nicID != 0 && nicID != e.BindNICID {
- return 0, nil, tcpip.ErrNoRoute
+ return 0, tcpip.ErrNoRoute
}
nicID = e.BindNICID
@@ -494,17 +492,17 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
if to.Port == 0 {
// Port 0 is an invalid port to send to.
- return 0, nil, tcpip.ErrInvalidEndpointState
+ return 0, tcpip.ErrInvalidEndpointState
}
dst, netProto, err := e.checkV4MappedLocked(*to)
if err != nil {
- return 0, nil, err
+ return 0, err
}
r, _, err := e.connectRoute(nicID, dst, netProto)
if err != nil {
- return 0, nil, err
+ return 0, err
}
defer r.Release()
@@ -513,21 +511,12 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
}
if !e.ops.GetBroadcast() && route.IsOutboundBroadcast() {
- return 0, nil, tcpip.ErrBroadcastDisabled
- }
-
- if route.IsResolutionRequired() {
- if ch, err := route.Resolve(nil); err != nil {
- if err == tcpip.ErrWouldBlock {
- return 0, ch, tcpip.ErrNoLinkAddress
- }
- return 0, nil, err
- }
+ return 0, tcpip.ErrBroadcastDisabled
}
v, err := p.FullPayload()
if err != nil {
- return 0, nil, err
+ return 0, err
}
if len(v) > header.UDPMaximumPacketSize {
// Payload can't possibly fit in a packet.
@@ -545,7 +534,7 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
v,
)
}
- return 0, nil, tcpip.ErrMessageTooLong
+ return 0, tcpip.ErrMessageTooLong
}
ttl := e.ttl
@@ -575,9 +564,9 @@ func (e *endpoint) write(p tcpip.Payloader, opts tcpip.WriteOptions) (int64, <-c
// See: https://golang.org/pkg/sync/#RWMutex for details on why recursive read
// locking is prohibited.
if err := sendUDP(route, buffer.View(v).ToVectorisedView(), localPort, dstPort, ttl, useDefaultTTL, sendTOS, owner, noChecksum); err != nil {
- return 0, nil, err
+ return 0, err
}
- return int64(len(v)), nil, nil
+ return int64(len(v)), nil
}
// OnReuseAddressSet implements tcpip.SocketOptionsHandler.OnReuseAddressSet.
diff --git a/pkg/tcpip/transport/udp/forwarder.go b/pkg/tcpip/transport/udp/forwarder.go
index d7fc21f11..49e673d58 100644
--- a/pkg/tcpip/transport/udp/forwarder.go
+++ b/pkg/tcpip/transport/udp/forwarder.go
@@ -75,7 +75,6 @@ func (r *ForwarderRequest) CreateEndpoint(queue *waiter.Queue) (tcpip.Endpoint,
if err != nil {
return nil, err
}
- route.ResolveWith(r.pkt.SourceLinkAddress())
ep := newEndpoint(r.stack, r.pkt.NetworkProtocolNumber, queue)
if err := r.stack.RegisterTransportEndpoint(r.pkt.NICID, []tcpip.NetworkProtocolNumber{r.pkt.NetworkProtocolNumber}, ProtocolNumber, r.id, ep, ep.portFlags, tcpip.NICID(ep.ops.GetBindToDevice())); err != nil {