diff options
Diffstat (limited to 'pkg/tcpip/stack')
-rw-r--r-- | pkg/tcpip/stack/BUILD | 2 | ||||
-rw-r--r-- | pkg/tcpip/stack/conntrack.go | 480 | ||||
-rw-r--r-- | pkg/tcpip/stack/iptables.go | 51 | ||||
-rw-r--r-- | pkg/tcpip/stack/iptables_targets.go | 115 | ||||
-rw-r--r-- | pkg/tcpip/stack/iptables_types.go | 4 | ||||
-rw-r--r-- | pkg/tcpip/stack/nic.go | 4 | ||||
-rw-r--r-- | pkg/tcpip/stack/packet_buffer.go | 4 | ||||
-rw-r--r-- | pkg/tcpip/stack/route.go | 13 | ||||
-rw-r--r-- | pkg/tcpip/stack/stack.go | 19 |
9 files changed, 623 insertions, 69 deletions
diff --git a/pkg/tcpip/stack/BUILD b/pkg/tcpip/stack/BUILD index 5e963a4af..f71073207 100644 --- a/pkg/tcpip/stack/BUILD +++ b/pkg/tcpip/stack/BUILD @@ -30,6 +30,7 @@ go_template_instance( go_library( name = "stack", srcs = [ + "conntrack.go", "dhcpv6configurationfromndpra_string.go", "forwarder.go", "icmp_rate_limit.go", @@ -62,6 +63,7 @@ go_library( "//pkg/tcpip/header", "//pkg/tcpip/ports", "//pkg/tcpip/seqnum", + "//pkg/tcpip/transport/tcpconntrack", "//pkg/waiter", "@org_golang_x_time//rate:go_default_library", ], diff --git a/pkg/tcpip/stack/conntrack.go b/pkg/tcpip/stack/conntrack.go new file mode 100644 index 000000000..7d1ede1f2 --- /dev/null +++ b/pkg/tcpip/stack/conntrack.go @@ -0,0 +1,480 @@ +// 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 ( + "encoding/binary" + "sync" + "time" + + "gvisor.dev/gvisor/pkg/tcpip" + "gvisor.dev/gvisor/pkg/tcpip/buffer" + "gvisor.dev/gvisor/pkg/tcpip/hash/jenkins" + "gvisor.dev/gvisor/pkg/tcpip/header" + "gvisor.dev/gvisor/pkg/tcpip/transport/tcpconntrack" +) + +// Connection tracking is used to track and manipulate packets for NAT rules. +// The connection is created for a packet if it does not exist. Every connection +// contains two tuples (original and reply). The tuples are manipulated if there +// is a matching NAT rule. The packet is modified by looking at the tuples in the +// Prerouting and Output hooks. + +// Direction of the tuple. +type ctDirection int + +const ( + dirOriginal ctDirection = iota + dirReply +) + +// Status of connection. +// TODO(gvisor.dev/issue/170): Add other states of connection. +type connStatus int + +const ( + connNew connStatus = iota + connEstablished +) + +// Manipulation type for the connection. +type manipType int + +const ( + manipDstPrerouting manipType = iota + manipDstOutput +) + +// connTrackMutable is the manipulatable part of the tuple. +type connTrackMutable struct { + // addr is source address of the tuple. + addr tcpip.Address + + // port is source port of the tuple. + port uint16 + + // protocol is network layer protocol. + protocol tcpip.NetworkProtocolNumber +} + +// connTrackImmutable is the non-manipulatable part of the tuple. +type connTrackImmutable struct { + // addr is destination address of the tuple. + addr tcpip.Address + + // direction is direction (original or reply) of the tuple. + direction ctDirection + + // port is destination port of the tuple. + port uint16 + + // protocol is transport layer protocol. + protocol tcpip.TransportProtocolNumber +} + +// connTrackTuple represents the tuple which is created from the +// packet. +type connTrackTuple struct { + // dst is non-manipulatable part of the tuple. + dst connTrackImmutable + + // src is manipulatable part of the tuple. + src connTrackMutable +} + +// connTrackTupleHolder is the container of tuple and connection. +type ConnTrackTupleHolder struct { + // conn is pointer to the connection tracking entry. + conn *connTrack + + // tuple is original or reply tuple. + tuple connTrackTuple +} + +// connTrack is the connection. +type connTrack struct { + // originalTupleHolder contains tuple in original direction. + originalTupleHolder ConnTrackTupleHolder + + // replyTupleHolder contains tuple in reply direction. + replyTupleHolder ConnTrackTupleHolder + + // status indicates connection is new or established. + status connStatus + + // timeout indicates the time connection should be active. + timeout time.Duration + + // manip indicates if the packet should be manipulated. + manip manipType + + // tcb is TCB control block. It is used to keep track of states + // of tcp connection. + tcb tcpconntrack.TCB + + // tcbHook indicates if the packet is inbound or outbound to + // update the state of tcb. + tcbHook Hook +} + +// ConnTrackTable contains a map of all existing connections created for +// NAT rules. +type ConnTrackTable struct { + // connMu protects connTrackTable. + connMu sync.RWMutex + + // connTrackTable maintains a map of tuples needed for connection tracking + // for iptables NAT rules. The key for the map is an integer calculated + // using seed, source address, destination address, source port and + // destination port. + CtMap map[uint32]ConnTrackTupleHolder + + // seed is a one-time random value initialized at stack startup + // and is used in calculation of hash key for connection tracking + // table. + Seed uint32 +} + +// parseHeaders sets headers in the packet. +func parseHeaders(pkt *PacketBuffer) { + newPkt := pkt.Clone() + + // Set network header. + hdr, ok := newPkt.Data.PullUp(header.IPv4MinimumSize) + if !ok { + return + } + netHeader := header.IPv4(hdr) + newPkt.NetworkHeader = hdr + length := int(netHeader.HeaderLength()) + + // TODO(gvisor.dev/issue/170): Need to support for other + // protocols as well. + // Set transport header. + switch protocol := netHeader.TransportProtocol(); protocol { + case header.UDPProtocolNumber: + if newPkt.TransportHeader == nil { + h, ok := newPkt.Data.PullUp(length + header.UDPMinimumSize) + if !ok { + return + } + newPkt.TransportHeader = buffer.View(header.UDP(h[length:])) + } + case header.TCPProtocolNumber: + if newPkt.TransportHeader == nil { + h, ok := newPkt.Data.PullUp(length + header.TCPMinimumSize) + if !ok { + return + } + newPkt.TransportHeader = buffer.View(header.TCP(h[length:])) + } + } + pkt.NetworkHeader = newPkt.NetworkHeader + pkt.TransportHeader = newPkt.TransportHeader +} + +// packetToTuple converts packet to a tuple in original direction. +func packetToTuple(pkt PacketBuffer, hook Hook) (connTrackTuple, *tcpip.Error) { + var tuple connTrackTuple + + netHeader := header.IPv4(pkt.NetworkHeader) + // TODO(gvisor.dev/issue/170): Need to support for other + // protocols as well. + if netHeader == nil || netHeader.TransportProtocol() != header.TCPProtocolNumber { + return tuple, tcpip.ErrUnknownProtocol + } + tcpHeader := header.TCP(pkt.TransportHeader) + if tcpHeader == nil { + return tuple, tcpip.ErrUnknownProtocol + } + + tuple.src.addr = netHeader.SourceAddress() + tuple.src.port = tcpHeader.SourcePort() + tuple.src.protocol = header.IPv4ProtocolNumber + + tuple.dst.addr = netHeader.DestinationAddress() + tuple.dst.port = tcpHeader.DestinationPort() + tuple.dst.protocol = netHeader.TransportProtocol() + + return tuple, nil +} + +// getReplyTuple creates reply tuple for the given tuple. +func getReplyTuple(tuple connTrackTuple) connTrackTuple { + var replyTuple connTrackTuple + replyTuple.src.addr = tuple.dst.addr + replyTuple.src.port = tuple.dst.port + replyTuple.src.protocol = tuple.src.protocol + replyTuple.dst.addr = tuple.src.addr + replyTuple.dst.port = tuple.src.port + replyTuple.dst.protocol = tuple.dst.protocol + replyTuple.dst.direction = dirReply + + return replyTuple +} + +// makeNewConn creates new connection. +func makeNewConn(tuple, replyTuple connTrackTuple) connTrack { + var conn connTrack + conn.status = connNew + conn.originalTupleHolder.tuple = tuple + conn.originalTupleHolder.conn = &conn + conn.replyTupleHolder.tuple = replyTuple + conn.replyTupleHolder.conn = &conn + + return conn +} + +// getTupleHash returns hash of the tuple. The fields used for +// generating hash are seed (generated once for stack), source address, +// destination address, source port and destination ports. +func (ct *ConnTrackTable) getTupleHash(tuple connTrackTuple) uint32 { + h := jenkins.Sum32(ct.Seed) + h.Write([]byte(tuple.src.addr)) + h.Write([]byte(tuple.dst.addr)) + portBuf := make([]byte, 2) + binary.LittleEndian.PutUint16(portBuf, tuple.src.port) + h.Write([]byte(portBuf)) + binary.LittleEndian.PutUint16(portBuf, tuple.dst.port) + h.Write([]byte(portBuf)) + + return h.Sum32() +} + +// connTrackForPacket returns connTrack for packet. +// TODO(gvisor.dev/issue/170): Only TCP packets are supported. Need to support other +// transport protocols. +func (ct *ConnTrackTable) connTrackForPacket(pkt *PacketBuffer, hook Hook, createConn bool) (*connTrack, ctDirection) { + if hook == Prerouting { + // Headers will not be set in Prerouting. + // TODO(gvisor.dev/issue/170): Change this after parsing headers + // code is added. + parseHeaders(pkt) + } + + var dir ctDirection + tuple, err := packetToTuple(*pkt, hook) + if err != nil { + return nil, dir + } + + ct.connMu.Lock() + defer ct.connMu.Unlock() + + connTrackTable := ct.CtMap + hash := ct.getTupleHash(tuple) + + var conn *connTrack + switch createConn { + case true: + // If connection does not exist for the hash, create a new + // connection. + replyTuple := getReplyTuple(tuple) + replyHash := ct.getTupleHash(replyTuple) + newConn := makeNewConn(tuple, replyTuple) + conn = &newConn + + // Add tupleHolders to the map. + // TODO(gvisor.dev/issue/170): Need to support collisions using linked list. + ct.CtMap[hash] = conn.originalTupleHolder + ct.CtMap[replyHash] = conn.replyTupleHolder + default: + tupleHolder, ok := connTrackTable[hash] + if !ok { + return nil, dir + } + + // If this is the reply of new connection, set the connection + // status as ESTABLISHED. + conn = tupleHolder.conn + if conn.status == connNew && tupleHolder.tuple.dst.direction == dirReply { + conn.status = connEstablished + } + if tupleHolder.conn == nil { + panic("tupleHolder has null connection tracking entry") + } + + dir = tupleHolder.tuple.dst.direction + } + return conn, dir +} + +// SetNatInfo will manipulate the tuples according to iptables NAT rules. +func (ct *ConnTrackTable) SetNatInfo(pkt *PacketBuffer, rt RedirectTarget, hook Hook) { + // Get the connection. Connection is always created before this + // function is called. + conn, _ := ct.connTrackForPacket(pkt, hook, false) + if conn == nil { + panic("connection should be created to manipulate tuples.") + } + replyTuple := conn.replyTupleHolder.tuple + replyHash := ct.getTupleHash(replyTuple) + + // TODO(gvisor.dev/issue/170): Support only redirect of ports. Need to + // support changing of address for Prerouting. + + // Change the port as per the iptables rule. This tuple will be used + // to manipulate the packet in HandlePacket. + conn.replyTupleHolder.tuple.src.addr = rt.MinIP + conn.replyTupleHolder.tuple.src.port = rt.MinPort + newHash := ct.getTupleHash(conn.replyTupleHolder.tuple) + + // Add the changed tuple to the map. + ct.connMu.Lock() + defer ct.connMu.Unlock() + ct.CtMap[newHash] = conn.replyTupleHolder + if hook == Output { + conn.replyTupleHolder.conn.manip = manipDstOutput + } + + // Delete the old tuple. + delete(ct.CtMap, replyHash) +} + +// handlePacketPrerouting manipulates ports for packets in Prerouting hook. +// TODO(gvisor.dev/issue/170): Change address for Prerouting hook.. +func handlePacketPrerouting(pkt *PacketBuffer, conn *connTrack, dir ctDirection) { + netHeader := header.IPv4(pkt.NetworkHeader) + tcpHeader := header.TCP(pkt.TransportHeader) + + // For prerouting redirection, packets going in the original direction + // have their destinations modified and replies have their sources + // modified. + switch dir { + case dirOriginal: + port := conn.replyTupleHolder.tuple.src.port + tcpHeader.SetDestinationPort(port) + netHeader.SetDestinationAddress(conn.replyTupleHolder.tuple.src.addr) + case dirReply: + port := conn.originalTupleHolder.tuple.dst.port + tcpHeader.SetSourcePort(port) + netHeader.SetSourceAddress(conn.originalTupleHolder.tuple.dst.addr) + } + + netHeader.SetChecksum(0) + netHeader.SetChecksum(^netHeader.CalculateChecksum()) +} + +// handlePacketOutput manipulates ports for packets in Output hook. +func handlePacketOutput(pkt *PacketBuffer, conn *connTrack, gso *GSO, r *Route, dir ctDirection) { + netHeader := header.IPv4(pkt.NetworkHeader) + tcpHeader := header.TCP(pkt.TransportHeader) + + // For output redirection, packets going in the original direction + // have their destinations modified and replies have their sources + // modified. For prerouting redirection, we only reach this point + // when replying, so packet sources are modified. + if conn.manip == manipDstOutput && dir == dirOriginal { + port := conn.replyTupleHolder.tuple.src.port + tcpHeader.SetDestinationPort(port) + netHeader.SetDestinationAddress(conn.replyTupleHolder.tuple.src.addr) + } else { + port := conn.originalTupleHolder.tuple.dst.port + tcpHeader.SetSourcePort(port) + netHeader.SetSourceAddress(conn.originalTupleHolder.tuple.dst.addr) + } + + // Calculate the TCP checksum and set it. + tcpHeader.SetChecksum(0) + hdr := &pkt.Header + length := uint16(pkt.Data.Size()+hdr.UsedLength()) - uint16(netHeader.HeaderLength()) + xsum := r.PseudoHeaderChecksum(header.TCPProtocolNumber, length) + if gso != nil && gso.NeedsCsum { + tcpHeader.SetChecksum(xsum) + } else if r.Capabilities()&CapabilityTXChecksumOffload == 0 { + xsum = header.ChecksumVVWithOffset(pkt.Data, xsum, int(tcpHeader.DataOffset()), pkt.Data.Size()) + tcpHeader.SetChecksum(^tcpHeader.CalculateChecksum(xsum)) + } + + netHeader.SetChecksum(0) + netHeader.SetChecksum(^netHeader.CalculateChecksum()) +} + +// HandlePacket will manipulate the port and address of the packet if the +// connection exists. +func (ct *ConnTrackTable) HandlePacket(pkt *PacketBuffer, hook Hook, gso *GSO, r *Route) { + if pkt.NatDone { + return + } + + if hook != Prerouting && hook != Output { + return + } + + conn, dir := ct.connTrackForPacket(pkt, hook, false) + // Connection or Rule not found for the packet. + if conn == nil { + return + } + + netHeader := header.IPv4(pkt.NetworkHeader) + // TODO(gvisor.dev/issue/170): Need to support for other transport + // protocols as well. + if netHeader == nil || netHeader.TransportProtocol() != header.TCPProtocolNumber { + return + } + + tcpHeader := header.TCP(pkt.TransportHeader) + if tcpHeader == nil { + return + } + + switch hook { + case Prerouting: + handlePacketPrerouting(pkt, conn, dir) + case Output: + handlePacketOutput(pkt, conn, gso, r, dir) + } + pkt.NatDone = true + + // Update the state of tcb. + // TODO(gvisor.dev/issue/170): Add support in tcpcontrack to handle + // other tcp states. + var st tcpconntrack.Result + if conn.tcb.IsEmpty() { + conn.tcb.Init(tcpHeader) + conn.tcbHook = hook + } else { + switch hook { + case conn.tcbHook: + st = conn.tcb.UpdateStateOutbound(tcpHeader) + default: + st = conn.tcb.UpdateStateInbound(tcpHeader) + } + } + + // Delete conntrack if tcp connection is closed. + if st == tcpconntrack.ResultClosedByPeer || st == tcpconntrack.ResultClosedBySelf || st == tcpconntrack.ResultReset { + ct.deleteConnTrack(conn) + } +} + +// deleteConnTrack deletes the connection. +func (ct *ConnTrackTable) deleteConnTrack(conn *connTrack) { + if conn == nil { + return + } + + tuple := conn.originalTupleHolder.tuple + hash := ct.getTupleHash(tuple) + replyTuple := conn.replyTupleHolder.tuple + replyHash := ct.getTupleHash(replyTuple) + + ct.connMu.Lock() + defer ct.connMu.Unlock() + + delete(ct.CtMap, hash) + delete(ct.CtMap, replyHash) +} diff --git a/pkg/tcpip/stack/iptables.go b/pkg/tcpip/stack/iptables.go index 6b91159d4..7c3c47d50 100644 --- a/pkg/tcpip/stack/iptables.go +++ b/pkg/tcpip/stack/iptables.go @@ -17,6 +17,7 @@ package stack import ( "fmt" + "gvisor.dev/gvisor/pkg/tcpip" "gvisor.dev/gvisor/pkg/tcpip/header" ) @@ -110,6 +111,10 @@ func DefaultTables() IPTables { Prerouting: []string{TablenameMangle, TablenameNat}, Output: []string{TablenameMangle, TablenameNat, TablenameFilter}, }, + connections: ConnTrackTable{ + CtMap: make(map[uint32]ConnTrackTupleHolder), + Seed: generateRandUint32(), + }, } } @@ -173,12 +178,16 @@ const ( // dropped. // // Precondition: pkt.NetworkHeader is set. -func (it *IPTables) Check(hook Hook, pkt PacketBuffer) bool { +func (it *IPTables) Check(hook Hook, pkt *PacketBuffer, gso *GSO, r *Route, address tcpip.Address) bool { + // Packets are manipulated only if connection and matching + // NAT rule exists. + it.connections.HandlePacket(pkt, hook, gso, r) + // Go through each table containing the hook. for _, tablename := range it.Priorities[hook] { table := it.Tables[tablename] ruleIdx := table.BuiltinChains[hook] - switch verdict := it.checkChain(hook, pkt, table, ruleIdx); verdict { + switch verdict := it.checkChain(hook, pkt, table, ruleIdx, gso, r, address); verdict { // If the table returns Accept, move on to the next table. case chainAccept: continue @@ -189,7 +198,7 @@ func (it *IPTables) Check(hook Hook, pkt PacketBuffer) bool { // 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); v { + switch v, _ := underflow.Target.Action(pkt, &it.connections, hook, gso, r, address); v { case RuleAccept: continue case RuleDrop: @@ -219,26 +228,34 @@ func (it *IPTables) Check(hook Hook, pkt PacketBuffer) bool { // // NOTE: unlike the Check API the returned map contains packets that should be // dropped. -func (it *IPTables) CheckPackets(hook Hook, pkts PacketBufferList) (drop map[*PacketBuffer]struct{}) { +func (it *IPTables) CheckPackets(hook Hook, pkts PacketBufferList, gso *GSO, r *Route) (drop map[*PacketBuffer]struct{}, natPkts map[*PacketBuffer]struct{}) { for pkt := pkts.Front(); pkt != nil; pkt = pkt.Next() { - if ok := it.Check(hook, *pkt); !ok { - if drop == nil { - drop = make(map[*PacketBuffer]struct{}) + if !pkt.NatDone { + if ok := it.Check(hook, pkt, gso, r, ""); !ok { + if drop == nil { + drop = make(map[*PacketBuffer]struct{}) + } + drop[pkt] = struct{}{} + } + if pkt.NatDone { + if natPkts == nil { + natPkts = make(map[*PacketBuffer]struct{}) + } + natPkts[pkt] = struct{}{} } - drop[pkt] = struct{}{} } } - return drop + return drop, natPkts } // Precondition: pkt is a IPv4 packet of at least length header.IPv4MinimumSize. -// TODO(gvisor.dev/issue/170): pk.NetworkHeader will always be set as a +// TODO(gvisor.dev/issue/170): pkt.NetworkHeader will always be set as a // precondition. -func (it *IPTables) checkChain(hook Hook, pkt PacketBuffer, table Table, ruleIdx int) chainVerdict { +func (it *IPTables) checkChain(hook Hook, pkt *PacketBuffer, table Table, ruleIdx int, gso *GSO, r *Route, address tcpip.Address) 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); verdict { + switch verdict, jumpTo := it.checkRule(hook, pkt, table, ruleIdx, gso, r, address); verdict { case RuleAccept: return chainAccept @@ -255,7 +272,7 @@ func (it *IPTables) checkChain(hook Hook, pkt PacketBuffer, table Table, ruleIdx ruleIdx++ continue } - switch verdict := it.checkChain(hook, pkt, table, jumpTo); verdict { + switch verdict := it.checkChain(hook, pkt, table, jumpTo, gso, r, address); verdict { case chainAccept: return chainAccept case chainDrop: @@ -279,9 +296,9 @@ func (it *IPTables) checkChain(hook Hook, pkt PacketBuffer, table Table, ruleIdx } // Precondition: pkt is a IPv4 packet of at least length header.IPv4MinimumSize. -// TODO(gvisor.dev/issue/170): pk.NetworkHeader will always be set as a +// TODO(gvisor.dev/issue/170): pkt.NetworkHeader will always be set as a // precondition. -func (it *IPTables) checkRule(hook Hook, pkt PacketBuffer, table Table, ruleIdx int) (RuleVerdict, int) { +func (it *IPTables) checkRule(hook Hook, pkt *PacketBuffer, table Table, ruleIdx int, gso *GSO, r *Route, address tcpip.Address) (RuleVerdict, int) { rule := table.Rules[ruleIdx] // If pkt.NetworkHeader hasn't been set yet, it will be contained in @@ -304,7 +321,7 @@ func (it *IPTables) checkRule(hook Hook, pkt PacketBuffer, table Table, ruleIdx // Go through each rule matcher. If they all match, run // the rule target. for _, matcher := range rule.Matchers { - matches, hotdrop := matcher.Match(hook, pkt, "") + matches, hotdrop := matcher.Match(hook, *pkt, "") if hotdrop { return RuleDrop, 0 } @@ -315,7 +332,7 @@ func (it *IPTables) checkRule(hook Hook, pkt PacketBuffer, table Table, ruleIdx } // All the matchers matched, so run the target. - return rule.Target.Action(pkt) + return rule.Target.Action(pkt, &it.connections, hook, gso, r, address) } func filterMatch(filter IPHeaderFilter, hdr header.IPv4) bool { diff --git a/pkg/tcpip/stack/iptables_targets.go b/pkg/tcpip/stack/iptables_targets.go index 8be61f4b1..36cc6275d 100644 --- a/pkg/tcpip/stack/iptables_targets.go +++ b/pkg/tcpip/stack/iptables_targets.go @@ -24,7 +24,7 @@ import ( type AcceptTarget struct{} // Action implements Target.Action. -func (AcceptTarget) Action(packet PacketBuffer) (RuleVerdict, int) { +func (AcceptTarget) Action(*PacketBuffer, *ConnTrackTable, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { return RuleAccept, 0 } @@ -32,7 +32,7 @@ func (AcceptTarget) Action(packet PacketBuffer) (RuleVerdict, int) { type DropTarget struct{} // Action implements Target.Action. -func (DropTarget) Action(packet PacketBuffer) (RuleVerdict, int) { +func (DropTarget) Action(*PacketBuffer, *ConnTrackTable, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { return RuleDrop, 0 } @@ -41,7 +41,7 @@ func (DropTarget) Action(packet PacketBuffer) (RuleVerdict, int) { type ErrorTarget struct{} // Action implements Target.Action. -func (ErrorTarget) Action(packet PacketBuffer) (RuleVerdict, int) { +func (ErrorTarget) Action(*PacketBuffer, *ConnTrackTable, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { log.Debugf("ErrorTarget triggered.") return RuleDrop, 0 } @@ -52,7 +52,7 @@ type UserChainTarget struct { } // Action implements Target.Action. -func (UserChainTarget) Action(PacketBuffer) (RuleVerdict, int) { +func (UserChainTarget) Action(*PacketBuffer, *ConnTrackTable, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { panic("UserChainTarget should never be called.") } @@ -61,7 +61,7 @@ func (UserChainTarget) Action(PacketBuffer) (RuleVerdict, int) { type ReturnTarget struct{} // Action implements Target.Action. -func (ReturnTarget) Action(PacketBuffer) (RuleVerdict, int) { +func (ReturnTarget) Action(*PacketBuffer, *ConnTrackTable, Hook, *GSO, *Route, tcpip.Address) (RuleVerdict, int) { return RuleReturn, 0 } @@ -75,16 +75,16 @@ type RedirectTarget struct { // redirect. RangeProtoSpecified bool - // Min address used to redirect. + // MinIP indicates address used to redirect. MinIP tcpip.Address - // Max address used to redirect. + // MaxIP indicates address used to redirect. MaxIP tcpip.Address - // Min port used to redirect. + // MinPort indicates port used to redirect. MinPort uint16 - // Max port used to redirect. + // MaxPort indicates port used to redirect. MaxPort uint16 } @@ -92,61 +92,76 @@ type RedirectTarget struct { // 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) (RuleVerdict, int) { - newPkt := pkt.Clone() +func (rt RedirectTarget) Action(pkt *PacketBuffer, ct *ConnTrackTable, hook Hook, gso *GSO, r *Route, address tcpip.Address) (RuleVerdict, int) { + // Packet is already manipulated. + if pkt.NatDone { + return RuleAccept, 0 + } // Set network header. - headerView, ok := newPkt.Data.PullUp(header.IPv4MinimumSize) - if !ok { - return RuleDrop, 0 + if hook == Prerouting { + parseHeaders(pkt) } - netHeader := header.IPv4(headerView) - newPkt.NetworkHeader = headerView - hlen := int(netHeader.HeaderLength()) - tlen := int(netHeader.TotalLength()) - newPkt.Data.TrimFront(hlen) - newPkt.Data.CapLength(tlen - hlen) + // Drop the packet if network and transport header are not set. + if pkt.NetworkHeader == nil || pkt.TransportHeader == nil { + return RuleDrop, 0 + } - // TODO(gvisor.dev/issue/170): Change destination address to - // loopback or interface address on which the packet was - // received. + // Change the address to localhost (127.0.0.1) in Output and + // to 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}) + case Prerouting: + rt.MinIP = address + rt.MaxIP = 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) switch protocol := netHeader.TransportProtocol(); protocol { case header.UDPProtocolNumber: - var udpHeader header.UDP - if newPkt.TransportHeader != nil { - udpHeader = header.UDP(newPkt.TransportHeader) - } else { - if pkt.Data.Size() < header.UDPMinimumSize { - return RuleDrop, 0 - } - hdr, ok := newPkt.Data.PullUp(header.UDPMinimumSize) - if !ok { - return RuleDrop, 0 + udpHeader := header.UDP(pkt.TransportHeader) + udpHeader.SetDestinationPort(rt.MinPort) + + // Calculate UDP checksum and set it. + if hook == Output { + udpHeader.SetChecksum(0) + hdr := &pkt.Header + length := uint16(pkt.Data.Size()+hdr.UsedLength()) - uint16(netHeader.HeaderLength()) + + // Only calculate the checksum if offloading isn't supported. + if r.Capabilities()&CapabilityTXChecksumOffload == 0 { + xsum := r.PseudoHeaderChecksum(protocol, length) + for _, v := range pkt.Data.Views() { + xsum = header.Checksum(v, xsum) + } + udpHeader.SetChecksum(0) + udpHeader.SetChecksum(^udpHeader.CalculateChecksum(xsum)) } - udpHeader = header.UDP(hdr) } - udpHeader.SetDestinationPort(rt.MinPort) + // Change destination address. + netHeader.SetDestinationAddress(rt.MinIP) + netHeader.SetChecksum(0) + netHeader.SetChecksum(^netHeader.CalculateChecksum()) + pkt.NatDone = true case header.TCPProtocolNumber: - var tcpHeader header.TCP - if newPkt.TransportHeader != nil { - tcpHeader = header.TCP(newPkt.TransportHeader) - } else { - if pkt.Data.Size() < header.TCPMinimumSize { - return RuleDrop, 0 - } - hdr, ok := newPkt.Data.PullUp(header.TCPMinimumSize) - if !ok { - return RuleDrop, 0 - } - tcpHeader = header.TCP(hdr) + if ct == nil { + return RuleAccept, 0 + } + + // Set up conection for matching NAT rule. + // Only the first packet of the connection comes here. + // Other packets will be manipulated in connection tracking. + if conn, _ := ct.connTrackForPacket(pkt, hook, true); conn != nil { + ct.SetNatInfo(pkt, rt, hook) + ct.HandlePacket(pkt, hook, gso, r) } - // TODO(gvisor.dev/issue/170): Need to recompute checksum - // and implement nat connection tracking to support TCP. - tcpHeader.SetDestinationPort(rt.MinPort) default: return RuleDrop, 0 } diff --git a/pkg/tcpip/stack/iptables_types.go b/pkg/tcpip/stack/iptables_types.go index 2ffb55f2a..1bb0ba1bd 100644 --- a/pkg/tcpip/stack/iptables_types.go +++ b/pkg/tcpip/stack/iptables_types.go @@ -82,6 +82,8 @@ type IPTables struct { // list is the order in which each table should be visited for that // hook. Priorities map[Hook][]string + + connections ConnTrackTable } // A Table defines a set of chains and hooks into the network stack. It is @@ -176,5 +178,5 @@ type Target interface { // 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. - Action(packet PacketBuffer) (RuleVerdict, int) + Action(packet *PacketBuffer, connections *ConnTrackTable, hook Hook, gso *GSO, r *Route, address tcpip.Address) (RuleVerdict, int) } diff --git a/pkg/tcpip/stack/nic.go b/pkg/tcpip/stack/nic.go index 7b54919bb..8f4c1fe42 100644 --- a/pkg/tcpip/stack/nic.go +++ b/pkg/tcpip/stack/nic.go @@ -1230,8 +1230,10 @@ func (n *NIC) DeliverNetworkPacket(linkEP LinkEndpoint, remote, local tcpip.Link // TODO(gvisor.dev/issue/170): Not supporting iptables for IPv6 yet. if protocol == header.IPv4ProtocolNumber { + // iptables filtering. ipt := n.stack.IPTables() - if ok := ipt.Check(Prerouting, pkt); !ok { + address := n.primaryAddress(protocol) + if ok := ipt.Check(Prerouting, &pkt, nil, nil, address.Address); !ok { // iptables is telling us to drop the packet. return } diff --git a/pkg/tcpip/stack/packet_buffer.go b/pkg/tcpip/stack/packet_buffer.go index 9ff80ab24..926df4d7b 100644 --- a/pkg/tcpip/stack/packet_buffer.go +++ b/pkg/tcpip/stack/packet_buffer.go @@ -72,6 +72,10 @@ type PacketBuffer struct { EgressRoute *Route GSOOptions *GSO NetworkProtocolNumber tcpip.NetworkProtocolNumber + + // NatDone indicates if the packet has been manipulated as per NAT + // iptables rule. + NatDone bool } // Clone makes a copy of pk. It clones the Data field, which creates a new diff --git a/pkg/tcpip/stack/route.go b/pkg/tcpip/stack/route.go index 53148dc03..150297ab9 100644 --- a/pkg/tcpip/stack/route.go +++ b/pkg/tcpip/stack/route.go @@ -261,3 +261,16 @@ func (r *Route) MakeLoopedRoute() Route { func (r *Route) Stack() *Stack { return r.ref.stack() } + +// ReverseRoute returns new route with given source and destination address. +func (r *Route) ReverseRoute(src tcpip.Address, dst tcpip.Address) Route { + return Route{ + NetProto: r.NetProto, + LocalAddress: dst, + LocalLinkAddress: r.RemoteLinkAddress, + RemoteAddress: src, + RemoteLinkAddress: r.LocalLinkAddress, + ref: r.ref, + Loop: r.Loop, + } +} diff --git a/pkg/tcpip/stack/stack.go b/pkg/tcpip/stack/stack.go index 4a2dc3dc6..e33fae4eb 100644 --- a/pkg/tcpip/stack/stack.go +++ b/pkg/tcpip/stack/stack.go @@ -1885,3 +1885,22 @@ func generateRandInt64() int64 { } return v } + +// FindNetworkEndpoint returns the network endpoint for the given address. +func (s *Stack) FindNetworkEndpoint(netProto tcpip.NetworkProtocolNumber, address tcpip.Address) (NetworkEndpoint, *tcpip.Error) { + s.mu.Lock() + defer s.mu.Unlock() + + 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 + } + } + return nil, tcpip.ErrBadAddress +} |