summaryrefslogtreecommitdiffhomepage
path: root/pkg
diff options
context:
space:
mode:
authorgVisor bot <gvisor-bot@google.com>2020-03-24 22:38:36 +0000
committergVisor bot <gvisor-bot@google.com>2020-03-24 22:38:36 +0000
commit758608224b51a3e8f3d79c98bc6862027625220e (patch)
tree7a2ff29fb4baac30f9805074f012f85a93597db0 /pkg
parentc918c96893792eade0c6ec60bf3ad3aaae6eb438 (diff)
parentc8eeedcc1d6b1ee25532ae630a7efd7aa4656bdc (diff)
Merge release-20200219.0-222-gc8eeedc (automated)
Diffstat (limited to 'pkg')
-rw-r--r--pkg/tcpip/link/fdbased/endpoint.go10
-rwxr-xr-xpkg/tcpip/stack/packet_buffer.go5
-rwxr-xr-xpkg/tcpip/stack/packet_buffer_state.go1
-rwxr-xr-xpkg/tcpip/stack/rand.go40
-rw-r--r--pkg/tcpip/stack/stack.go44
-rwxr-xr-xpkg/tcpip/stack/stack_state_autogen.go2
-rw-r--r--pkg/tcpip/transport/tcp/accept.go20
-rw-r--r--pkg/tcpip/transport/tcp/connect.go138
-rw-r--r--pkg/tcpip/transport/tcp/endpoint.go5
-rw-r--r--pkg/tcpip/transport/tcp/protocol.go10
-rwxr-xr-xpkg/tcpip/transport/tcp/tcp_state_autogen.go2
11 files changed, 228 insertions, 49 deletions
diff --git a/pkg/tcpip/link/fdbased/endpoint.go b/pkg/tcpip/link/fdbased/endpoint.go
index 235e647ff..3b3b6909b 100644
--- a/pkg/tcpip/link/fdbased/endpoint.go
+++ b/pkg/tcpip/link/fdbased/endpoint.go
@@ -405,6 +405,7 @@ func (e *endpoint) WritePacket(r *stack.Route, gso *stack.GSO, protocol tcpip.Ne
eth.Encode(ethHdr)
}
+ fd := e.fds[pkt.Hash%uint32(len(e.fds))]
if e.Capabilities()&stack.CapabilityHardwareGSO != 0 {
vnetHdr := virtioNetHdr{}
if gso != nil {
@@ -428,14 +429,14 @@ func (e *endpoint) WritePacket(r *stack.Route, gso *stack.GSO, protocol tcpip.Ne
}
vnetHdrBuf := vnetHdrToByteSlice(&vnetHdr)
- return rawfile.NonBlockingWrite3(e.fds[0], vnetHdrBuf, pkt.Header.View(), pkt.Data.ToView())
+ return rawfile.NonBlockingWrite3(fd, vnetHdrBuf, pkt.Header.View(), pkt.Data.ToView())
}
if pkt.Data.Size() == 0 {
- return rawfile.NonBlockingWrite(e.fds[0], pkt.Header.View())
+ return rawfile.NonBlockingWrite(fd, pkt.Header.View())
}
- return rawfile.NonBlockingWrite3(e.fds[0], pkt.Header.View(), pkt.Data.ToView(), nil)
+ return rawfile.NonBlockingWrite3(fd, pkt.Header.View(), pkt.Data.ToView(), nil)
}
// WritePackets writes outbound packets to the file descriptor. If it is not
@@ -551,7 +552,8 @@ func (e *endpoint) WritePackets(r *stack.Route, gso *stack.GSO, pkts []stack.Pac
packets := 0
for packets < n {
- sent, err := rawfile.NonBlockingSendMMsg(e.fds[0], mmsgHdrs)
+ fd := e.fds[pkts[packets].Hash%uint32(len(e.fds))]
+ sent, err := rawfile.NonBlockingSendMMsg(fd, mmsgHdrs)
if err != nil {
return packets, err
}
diff --git a/pkg/tcpip/stack/packet_buffer.go b/pkg/tcpip/stack/packet_buffer.go
index 1850fa8c3..9505a4e92 100755
--- a/pkg/tcpip/stack/packet_buffer.go
+++ b/pkg/tcpip/stack/packet_buffer.go
@@ -10,6 +10,7 @@
// 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 "gvisor.dev/gvisor/pkg/tcpip/buffer"
@@ -54,6 +55,10 @@ type PacketBuffer struct {
LinkHeader buffer.View
NetworkHeader buffer.View
TransportHeader buffer.View
+
+ // Hash is the transport layer hash of this packet. A value of zero
+ // indicates no valid hash has been set.
+ Hash uint32
}
// Clone makes a copy of pk. It clones the Data field, which creates a new
diff --git a/pkg/tcpip/stack/packet_buffer_state.go b/pkg/tcpip/stack/packet_buffer_state.go
index 76602549e..0c6b7924c 100755
--- a/pkg/tcpip/stack/packet_buffer_state.go
+++ b/pkg/tcpip/stack/packet_buffer_state.go
@@ -11,6 +11,7 @@
// 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 "gvisor.dev/gvisor/pkg/tcpip/buffer"
diff --git a/pkg/tcpip/stack/rand.go b/pkg/tcpip/stack/rand.go
new file mode 100755
index 000000000..421fb5c15
--- /dev/null
+++ b/pkg/tcpip/stack/rand.go
@@ -0,0 +1,40 @@
+// 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 (
+ mathrand "math/rand"
+
+ "gvisor.dev/gvisor/pkg/sync"
+)
+
+// lockedRandomSource provides a threadsafe rand.Source.
+type lockedRandomSource struct {
+ mu sync.Mutex
+ src mathrand.Source
+}
+
+func (r *lockedRandomSource) Int63() (n int64) {
+ r.mu.Lock()
+ n = r.src.Int63()
+ r.mu.Unlock()
+ return n
+}
+
+func (r *lockedRandomSource) Seed(seed int64) {
+ r.mu.Lock()
+ r.src.Seed(seed)
+ r.mu.Unlock()
+}
diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go
index a9584d636..41398a1b6 100644
--- a/pkg/tcpip/stack/stack.go
+++ b/pkg/tcpip/stack/stack.go
@@ -20,7 +20,9 @@
package stack
import (
+ "bytes"
"encoding/binary"
+ mathrand "math/rand"
"sync/atomic"
"time"
@@ -465,6 +467,10 @@ type Stack struct {
// forwarder holds the packets that wait for their link-address resolutions
// to complete, and forwards them when each resolution is done.
forwarder *forwardQueue
+
+ // randomGenerator is an injectable pseudo random generator that can be
+ // used when a random number is required.
+ randomGenerator *mathrand.Rand
}
// UniqueID is an abstract generator of unique identifiers.
@@ -525,9 +531,16 @@ 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 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
}
// TransportEndpointInfo holds useful information about a transport endpoint
@@ -623,6 +636,13 @@ func New(opts Options) *Stack {
opts.UniqueID = new(uniqueIDGenerator)
}
+ randSrc := opts.RandSource
+ if randSrc == nil {
+ // Source provided by mathrand.NewSource is not thread-safe so
+ // we wrap it in a simple thread-safe version.
+ randSrc = &lockedRandomSource{src: mathrand.NewSource(generateRandInt64())}
+ }
+
// Make sure opts.NDPConfigs contains valid values only.
opts.NDPConfigs.validate()
@@ -645,6 +665,7 @@ func New(opts Options) *Stack {
ndpDisp: opts.NDPDisp,
opaqueIIDOpts: opts.OpaqueIIDOpts,
forwarder: newForwardQueue(),
+ randomGenerator: mathrand.New(randSrc),
}
// Add specified network protocols.
@@ -1818,6 +1839,12 @@ func (s *Stack) Seed() uint32 {
return s.seed
}
+// Rand returns a reference to a pseudo random generator that can be used
+// to generate random numbers as required.
+func (s *Stack) Rand() *mathrand.Rand {
+ return s.randomGenerator
+}
+
func generateRandUint32() uint32 {
b := make([]byte, 4)
if _, err := rand.Read(b); err != nil {
@@ -1825,3 +1852,16 @@ func generateRandUint32() uint32 {
}
return binary.LittleEndian.Uint32(b)
}
+
+func generateRandInt64() int64 {
+ b := make([]byte, 8)
+ if _, err := rand.Read(b); err != nil {
+ panic(err)
+ }
+ buf := bytes.NewReader(b)
+ var v int64
+ if err := binary.Read(buf, binary.LittleEndian, &v); err != nil {
+ panic(err)
+ }
+ return v
+}
diff --git a/pkg/tcpip/stack/stack_state_autogen.go b/pkg/tcpip/stack/stack_state_autogen.go
index 7b279cd4b..61c3fdec2 100755
--- a/pkg/tcpip/stack/stack_state_autogen.go
+++ b/pkg/tcpip/stack/stack_state_autogen.go
@@ -41,6 +41,7 @@ func (x *PacketBuffer) save(m state.Map) {
m.Save("LinkHeader", &x.LinkHeader)
m.Save("NetworkHeader", &x.NetworkHeader)
m.Save("TransportHeader", &x.TransportHeader)
+ m.Save("Hash", &x.Hash)
}
func (x *PacketBuffer) afterLoad() {}
@@ -52,6 +53,7 @@ func (x *PacketBuffer) load(m state.Map) {
m.Load("LinkHeader", &x.LinkHeader)
m.Load("NetworkHeader", &x.NetworkHeader)
m.Load("TransportHeader", &x.TransportHeader)
+ m.Load("Hash", &x.Hash)
}
func (x *TransportEndpointID) beforeSave() {}
diff --git a/pkg/tcpip/transport/tcp/accept.go b/pkg/tcpip/transport/tcp/accept.go
index 3f80995f3..b4c4c8ab1 100644
--- a/pkg/tcpip/transport/tcp/accept.go
+++ b/pkg/tcpip/transport/tcp/accept.go
@@ -445,7 +445,15 @@ func (e *endpoint) handleListenSegment(ctx *listenContext, s *segment) {
// RFC 793 section 3.4 page 35 (figure 12) outlines that a RST
// must be sent in response to a SYN-ACK while in the listen
// state to prevent completing a handshake from an old SYN.
- e.sendTCP(&s.route, s.id, buffer.VectorisedView{}, e.ttl, e.sendTOS, header.TCPFlagRst, s.ackNumber, 0, 0, nil, nil)
+ e.sendTCP(&s.route, tcpFields{
+ id: s.id,
+ ttl: e.ttl,
+ tos: e.sendTOS,
+ flags: header.TCPFlagRst,
+ seq: s.ackNumber,
+ ack: 0,
+ rcvWnd: 0,
+ }, buffer.VectorisedView{}, nil)
return
}
@@ -493,7 +501,15 @@ func (e *endpoint) handleListenSegment(ctx *listenContext, s *segment) {
TSEcr: opts.TSVal,
MSS: mssForRoute(&s.route),
}
- e.sendSynTCP(&s.route, s.id, e.ttl, e.sendTOS, header.TCPFlagSyn|header.TCPFlagAck, cookie, s.sequenceNumber+1, ctx.rcvWnd, synOpts)
+ e.sendSynTCP(&s.route, tcpFields{
+ id: s.id,
+ ttl: e.ttl,
+ tos: e.sendTOS,
+ flags: header.TCPFlagSyn | header.TCPFlagAck,
+ seq: cookie,
+ ack: s.sequenceNumber + 1,
+ rcvWnd: ctx.rcvWnd,
+ }, synOpts)
e.stack.Stats().TCP.ListenOverflowSynCookieSent.Increment()
}
diff --git a/pkg/tcpip/transport/tcp/connect.go b/pkg/tcpip/transport/tcp/connect.go
index 79552fc61..1d245c2c6 100644
--- a/pkg/tcpip/transport/tcp/connect.go
+++ b/pkg/tcpip/transport/tcp/connect.go
@@ -308,7 +308,15 @@ func (h *handshake) synSentState(s *segment) *tcpip.Error {
if ttl == 0 {
ttl = s.route.DefaultTTL()
}
- h.ep.sendSynTCP(&s.route, h.ep.ID, ttl, h.ep.sendTOS, h.flags, h.iss, h.ackNum, h.rcvWnd, synOpts)
+ h.ep.sendSynTCP(&s.route, tcpFields{
+ id: h.ep.ID,
+ ttl: ttl,
+ tos: h.ep.sendTOS,
+ flags: h.flags,
+ seq: h.iss,
+ ack: h.ackNum,
+ rcvWnd: h.rcvWnd,
+ }, synOpts)
return nil
}
@@ -361,7 +369,15 @@ func (h *handshake) synRcvdState(s *segment) *tcpip.Error {
SACKPermitted: h.ep.sackPermitted,
MSS: h.ep.amss,
}
- h.ep.sendSynTCP(&s.route, h.ep.ID, h.ep.ttl, h.ep.sendTOS, h.flags, h.iss, h.ackNum, h.rcvWnd, synOpts)
+ h.ep.sendSynTCP(&s.route, tcpFields{
+ id: h.ep.ID,
+ ttl: h.ep.ttl,
+ tos: h.ep.sendTOS,
+ flags: h.flags,
+ seq: h.iss,
+ ack: h.ackNum,
+ rcvWnd: h.rcvWnd,
+ }, synOpts)
return nil
}
@@ -550,7 +566,16 @@ func (h *handshake) execute() *tcpip.Error {
synOpts.WS = -1
}
}
- h.ep.sendSynTCP(&h.ep.route, h.ep.ID, h.ep.ttl, h.ep.sendTOS, h.flags, h.iss, h.ackNum, h.rcvWnd, synOpts)
+
+ h.ep.sendSynTCP(&h.ep.route, tcpFields{
+ id: h.ep.ID,
+ ttl: h.ep.ttl,
+ tos: h.ep.sendTOS,
+ flags: h.flags,
+ seq: h.iss,
+ ack: h.ackNum,
+ rcvWnd: h.rcvWnd,
+ }, synOpts)
for h.state != handshakeCompleted {
h.ep.mu.Unlock()
@@ -573,7 +598,15 @@ func (h *handshake) execute() *tcpip.Error {
// the connection with another ACK or data (as ACKs are never
// retransmitted on their own).
if h.active || !h.acked || h.deferAccept != 0 && time.Since(h.startTime) > h.deferAccept {
- h.ep.sendSynTCP(&h.ep.route, h.ep.ID, h.ep.ttl, h.ep.sendTOS, h.flags, h.iss, h.ackNum, h.rcvWnd, synOpts)
+ h.ep.sendSynTCP(&h.ep.route, tcpFields{
+ id: h.ep.ID,
+ ttl: h.ep.ttl,
+ tos: h.ep.sendTOS,
+ flags: h.flags,
+ seq: h.iss,
+ ack: h.ackNum,
+ rcvWnd: h.rcvWnd,
+ }, synOpts)
}
case wakerForNotification:
@@ -686,18 +719,33 @@ func makeSynOptions(opts header.TCPSynOptions) []byte {
return options[:offset]
}
-func (e *endpoint) sendSynTCP(r *stack.Route, id stack.TransportEndpointID, ttl, tos uint8, flags byte, seq, ack seqnum.Value, rcvWnd seqnum.Size, opts header.TCPSynOptions) *tcpip.Error {
- options := makeSynOptions(opts)
+// tcpFields is a struct to carry different parameters required by the
+// send*TCP variant functions below.
+type tcpFields struct {
+ id stack.TransportEndpointID
+ ttl uint8
+ tos uint8
+ flags byte
+ seq seqnum.Value
+ ack seqnum.Value
+ rcvWnd seqnum.Size
+ opts []byte
+ txHash uint32
+}
+
+func (e *endpoint) sendSynTCP(r *stack.Route, tf tcpFields, opts header.TCPSynOptions) *tcpip.Error {
+ tf.opts = makeSynOptions(opts)
// We ignore SYN send errors and let the callers re-attempt send.
- if err := e.sendTCP(r, id, buffer.VectorisedView{}, ttl, tos, flags, seq, ack, rcvWnd, options, nil); err != nil {
+ if err := e.sendTCP(r, tf, buffer.VectorisedView{}, nil); err != nil {
e.stats.SendErrors.SynSendToNetworkFailed.Increment()
}
- putOptions(options)
+ putOptions(tf.opts)
return nil
}
-func (e *endpoint) sendTCP(r *stack.Route, id stack.TransportEndpointID, data buffer.VectorisedView, ttl, tos uint8, flags byte, seq, ack seqnum.Value, rcvWnd seqnum.Size, opts []byte, gso *stack.GSO) *tcpip.Error {
- if err := sendTCP(r, id, data, ttl, tos, flags, seq, ack, rcvWnd, opts, gso); err != nil {
+func (e *endpoint) sendTCP(r *stack.Route, tf tcpFields, data buffer.VectorisedView, gso *stack.GSO) *tcpip.Error {
+ tf.txHash = e.txHash
+ if err := sendTCP(r, tf, data, gso); err != nil {
e.stats.SendErrors.SegmentSendToNetworkFailed.Increment()
return err
}
@@ -705,8 +753,8 @@ func (e *endpoint) sendTCP(r *stack.Route, id stack.TransportEndpointID, data bu
return nil
}
-func buildTCPHdr(r *stack.Route, id stack.TransportEndpointID, pkt *stack.PacketBuffer, flags byte, seq, ack seqnum.Value, rcvWnd seqnum.Size, opts []byte, gso *stack.GSO) {
- optLen := len(opts)
+func buildTCPHdr(r *stack.Route, tf tcpFields, pkt *stack.PacketBuffer, gso *stack.GSO) {
+ optLen := len(tf.opts)
hdr := &pkt.Header
packetSize := pkt.DataSize
off := pkt.DataOffset
@@ -714,15 +762,15 @@ func buildTCPHdr(r *stack.Route, id stack.TransportEndpointID, pkt *stack.Packet
tcp := header.TCP(hdr.Prepend(header.TCPMinimumSize + optLen))
pkt.TransportHeader = buffer.View(tcp)
tcp.Encode(&header.TCPFields{
- SrcPort: id.LocalPort,
- DstPort: id.RemotePort,
- SeqNum: uint32(seq),
- AckNum: uint32(ack),
+ SrcPort: tf.id.LocalPort,
+ DstPort: tf.id.RemotePort,
+ SeqNum: uint32(tf.seq),
+ AckNum: uint32(tf.ack),
DataOffset: uint8(header.TCPMinimumSize + optLen),
- Flags: flags,
- WindowSize: uint16(rcvWnd),
+ Flags: tf.flags,
+ WindowSize: uint16(tf.rcvWnd),
})
- copy(tcp[header.TCPMinimumSize:], opts)
+ copy(tcp[header.TCPMinimumSize:], tf.opts)
length := uint16(hdr.UsedLength() + packetSize)
xsum := r.PseudoHeaderChecksum(ProtocolNumber, length)
@@ -737,13 +785,12 @@ func buildTCPHdr(r *stack.Route, id stack.TransportEndpointID, pkt *stack.Packet
xsum = header.ChecksumVVWithOffset(pkt.Data, xsum, off, packetSize)
tcp.SetChecksum(^tcp.CalculateChecksum(xsum))
}
-
}
-func sendTCPBatch(r *stack.Route, id stack.TransportEndpointID, data buffer.VectorisedView, ttl, tos uint8, flags byte, seq, ack seqnum.Value, rcvWnd seqnum.Size, opts []byte, gso *stack.GSO) *tcpip.Error {
- optLen := len(opts)
- if rcvWnd > 0xffff {
- rcvWnd = 0xffff
+func sendTCPBatch(r *stack.Route, tf tcpFields, data buffer.VectorisedView, gso *stack.GSO) *tcpip.Error {
+ optLen := len(tf.opts)
+ if tf.rcvWnd > 0xffff {
+ tf.rcvWnd = 0xffff
}
mss := int(gso.MSS)
@@ -768,14 +815,15 @@ func sendTCPBatch(r *stack.Route, id stack.TransportEndpointID, data buffer.Vect
pkts[i].DataOffset = off
pkts[i].DataSize = packetSize
pkts[i].Data = data
- buildTCPHdr(r, id, &pkts[i], flags, seq, ack, rcvWnd, opts, gso)
+ pkts[i].Hash = tf.txHash
+ buildTCPHdr(r, tf, &pkts[i], gso)
off += packetSize
- seq = seq.Add(seqnum.Size(packetSize))
+ tf.seq = tf.seq.Add(seqnum.Size(packetSize))
}
- if ttl == 0 {
- ttl = r.DefaultTTL()
+ if tf.ttl == 0 {
+ tf.ttl = r.DefaultTTL()
}
- sent, err := r.WritePackets(gso, pkts, stack.NetworkHeaderParams{Protocol: ProtocolNumber, TTL: ttl, TOS: tos})
+ sent, err := r.WritePackets(gso, pkts, stack.NetworkHeaderParams{Protocol: ProtocolNumber, TTL: tf.ttl, TOS: tf.tos})
if err != nil {
r.Stats().TCP.SegmentSendErrors.IncrementBy(uint64(n - sent))
}
@@ -785,14 +833,14 @@ func sendTCPBatch(r *stack.Route, id stack.TransportEndpointID, data buffer.Vect
// sendTCP sends a TCP segment with the provided options via the provided
// network endpoint and under the provided identity.
-func sendTCP(r *stack.Route, id stack.TransportEndpointID, data buffer.VectorisedView, ttl, tos uint8, flags byte, seq, ack seqnum.Value, rcvWnd seqnum.Size, opts []byte, gso *stack.GSO) *tcpip.Error {
- optLen := len(opts)
- if rcvWnd > 0xffff {
- rcvWnd = 0xffff
+func sendTCP(r *stack.Route, tf tcpFields, data buffer.VectorisedView, gso *stack.GSO) *tcpip.Error {
+ optLen := len(tf.opts)
+ if tf.rcvWnd > 0xffff {
+ tf.rcvWnd = 0xffff
}
if r.Loop&stack.PacketLoop == 0 && gso != nil && gso.Type == stack.GSOSW && int(gso.MSS) < data.Size() {
- return sendTCPBatch(r, id, data, ttl, tos, flags, seq, ack, rcvWnd, opts, gso)
+ return sendTCPBatch(r, tf, data, gso)
}
pkt := stack.PacketBuffer{
@@ -800,18 +848,19 @@ func sendTCP(r *stack.Route, id stack.TransportEndpointID, data buffer.Vectorise
DataOffset: 0,
DataSize: data.Size(),
Data: data,
+ Hash: tf.txHash,
}
- buildTCPHdr(r, id, &pkt, flags, seq, ack, rcvWnd, opts, gso)
+ buildTCPHdr(r, tf, &pkt, gso)
- if ttl == 0 {
- ttl = r.DefaultTTL()
+ if tf.ttl == 0 {
+ tf.ttl = r.DefaultTTL()
}
- if err := r.WritePacket(gso, stack.NetworkHeaderParams{Protocol: ProtocolNumber, TTL: ttl, TOS: tos}, pkt); err != nil {
+ if err := r.WritePacket(gso, stack.NetworkHeaderParams{Protocol: ProtocolNumber, TTL: tf.ttl, TOS: tf.tos}, pkt); err != nil {
r.Stats().TCP.SegmentSendErrors.Increment()
return err
}
r.Stats().TCP.SegmentsSent.Increment()
- if (flags & header.TCPFlagRst) != 0 {
+ if (tf.flags & header.TCPFlagRst) != 0 {
r.Stats().TCP.ResetsSent.Increment()
}
return nil
@@ -863,7 +912,16 @@ func (e *endpoint) sendRaw(data buffer.VectorisedView, flags byte, seq, ack seqn
sackBlocks = e.sack.Blocks[:e.sack.NumBlocks]
}
options := e.makeOptions(sackBlocks)
- err := e.sendTCP(&e.route, e.ID, data, e.ttl, e.sendTOS, flags, seq, ack, rcvWnd, options, e.gso)
+ err := e.sendTCP(&e.route, tcpFields{
+ id: e.ID,
+ ttl: e.ttl,
+ tos: e.sendTOS,
+ flags: flags,
+ seq: seq,
+ ack: ack,
+ rcvWnd: rcvWnd,
+ opts: options,
+ }, data, e.gso)
putOptions(options)
return err
}
diff --git a/pkg/tcpip/transport/tcp/endpoint.go b/pkg/tcpip/transport/tcp/endpoint.go
index 594efaa11..b6e571361 100644
--- a/pkg/tcpip/transport/tcp/endpoint.go
+++ b/pkg/tcpip/transport/tcp/endpoint.go
@@ -581,6 +581,10 @@ type endpoint struct {
// endpoint and at this point the endpoint is only around
// to complete the TCP shutdown.
closed bool
+
+ // txHash is the transport layer hash to be set on outbound packets
+ // emitted by this endpoint.
+ txHash uint32
}
// UniqueID implements stack.TransportEndpoint.UniqueID.
@@ -771,6 +775,7 @@ func newEndpoint(s *stack.Stack, netProto tcpip.NetworkProtocolNumber, waiterQue
count: 9,
},
uniqueID: s.UniqueID(),
+ txHash: s.Rand().Uint32(),
}
var ss SendBufferSizeOption
diff --git a/pkg/tcpip/transport/tcp/protocol.go b/pkg/tcpip/transport/tcp/protocol.go
index 57985b85d..1377107ca 100644
--- a/pkg/tcpip/transport/tcp/protocol.go
+++ b/pkg/tcpip/transport/tcp/protocol.go
@@ -191,7 +191,15 @@ func replyWithReset(s *segment) {
flags |= header.TCPFlagAck
ack = s.sequenceNumber.Add(s.logicalLen())
}
- sendTCP(&s.route, s.id, buffer.VectorisedView{}, s.route.DefaultTTL(), stack.DefaultTOS, flags, seq, ack, 0 /* rcvWnd */, nil /* options */, nil /* gso */)
+ sendTCP(&s.route, tcpFields{
+ id: s.id,
+ ttl: s.route.DefaultTTL(),
+ tos: stack.DefaultTOS,
+ flags: flags,
+ seq: seq,
+ ack: ack,
+ rcvWnd: 0,
+ }, buffer.VectorisedView{}, nil /* gso */)
}
// SetOption implements stack.TransportProtocol.SetOption.
diff --git a/pkg/tcpip/transport/tcp/tcp_state_autogen.go b/pkg/tcpip/transport/tcp/tcp_state_autogen.go
index 253582533..885575e5e 100755
--- a/pkg/tcpip/transport/tcp/tcp_state_autogen.go
+++ b/pkg/tcpip/transport/tcp/tcp_state_autogen.go
@@ -150,6 +150,7 @@ func (x *endpoint) save(m state.Map) {
m.Save("gso", &x.gso)
m.Save("tcpLingerTimeout", &x.tcpLingerTimeout)
m.Save("closed", &x.closed)
+ m.Save("txHash", &x.txHash)
}
func (x *endpoint) load(m state.Map) {
@@ -206,6 +207,7 @@ func (x *endpoint) load(m state.Map) {
m.Load("gso", &x.gso)
m.Load("tcpLingerTimeout", &x.tcpLingerTimeout)
m.Load("closed", &x.closed)
+ m.Load("txHash", &x.txHash)
m.LoadValue("lastError", new(string), func(y interface{}) { x.loadLastError(y.(string)) })
m.LoadValue("state", new(EndpointState), func(y interface{}) { x.loadState(y.(EndpointState)) })
m.LoadValue("acceptedChan", new([]*endpoint), func(y interface{}) { x.loadAcceptedChan(y.([]*endpoint)) })