Merge pull request #1683 from kevinGC:ipt-udp-matchers

PiperOrigin-RevId: 293243342
This commit is contained in:
gVisor bot 2020-02-04 16:20:16 -08:00
commit b29aeebaf6
9 changed files with 431 additions and 86 deletions

View File

@ -198,6 +198,13 @@ type XTEntryMatch struct {
// SizeOfXTEntryMatch is the size of an XTEntryMatch.
const SizeOfXTEntryMatch = 32
// KernelXTEntryMatch is identical to XTEntryMatch, but contains
// variable-length Data field.
type KernelXTEntryMatch struct {
XTEntryMatch
Data []byte
}
// XTEntryTarget holds a target for a rule. For example, it can specify that
// packets matching the rule should DROP, ACCEPT, or use an extension target.
// iptables-extension(8) has a list of possible targets.
@ -340,3 +347,43 @@ func goString(cstring []byte) string {
}
return string(cstring)
}
// XTUDP holds data for matching UDP packets. It corresponds to struct xt_udp
// in include/uapi/linux/netfilter/xt_tcpudp.h.
type XTUDP struct {
// SourcePortStart is the inclusive start of the range of source ports
// to which the matcher applies.
SourcePortStart uint16
// SourcePortEnd is the inclusive end of the range of source ports to
// which the matcher applies.
SourcePortEnd uint16
// DestinationPortStart is the inclusive start of the destination port
// range to which the matcher applies.
DestinationPortStart uint16
// DestinationPortEnd is the inclusive end of the destination port
// range to which the matcher applies.
DestinationPortEnd uint16
// InverseFlags flips the meaning of certain fields. See the
// TX_UDP_INV_* flags.
InverseFlags uint8
_ uint8
}
// SizeOfXTUDP is the size of an XTUDP.
const SizeOfXTUDP = 10
// Flags in XTUDP.InverseFlags. Corresponding constants are in
// include/uapi/linux/netfilter/xt_tcpudp.h.
const (
// Invert the meaning of SourcePortStart/End.
XT_UDP_INV_SRCPT = 0x01
// Invert the meaning of DestinationPortStart/End.
XT_UDP_INV_DSTPT = 0x02
// Enable all flags.
XT_UDP_INV_MASK = 0x03
)

View File

@ -34,9 +34,16 @@ import (
// shouldn't be reached - an error has occurred if we fall through to one.
const errorTargetName = "ERROR"
// metadata is opaque to netstack. It holds data that we need to translate
// between Linux's and netstack's iptables representations.
// TODO(gvisor.dev/issue/170): This might be removable.
const (
matcherNameUDP = "udp"
)
// Metadata is used to verify that we are correctly serializing and
// deserializing iptables into structs consumable by the iptables tool. We save
// a metadata struct when the tables are written, and when they are read out we
// verify that certain fields are the same.
//
// metadata is used by this serialization/deserializing code, not netstack.
type metadata struct {
HookEntry [linux.NF_INET_NUMHOOKS]uint32
Underflow [linux.NF_INET_NUMHOOKS]uint32
@ -44,6 +51,11 @@ type metadata struct {
Size uint32
}
// nflog logs messages related to the writing and reading of iptables.
func nflog(format string, args ...interface{}) {
log.Infof("netfilter: "+format, args...)
}
// GetInfo returns information about iptables.
func GetInfo(t *kernel.Task, stack *stack.Stack, outPtr usermem.Addr) (linux.IPTGetinfo, *syserr.Error) {
// Read in the struct and table name.
@ -72,6 +84,8 @@ func GetInfo(t *kernel.Task, stack *stack.Stack, outPtr usermem.Addr) (linux.IPT
info.NumEntries = metadata.NumEntries
info.Size = metadata.Size
nflog("returning info: %+v", info)
return info, nil
}
@ -80,21 +94,26 @@ func GetEntries(t *kernel.Task, stack *stack.Stack, outPtr usermem.Addr, outLen
// Read in the struct and table name.
var userEntries linux.IPTGetEntries
if _, err := t.CopyIn(outPtr, &userEntries); err != nil {
log.Warningf("netfilter: couldn't copy in entries %q", userEntries.Name)
return linux.KernelIPTGetEntries{}, syserr.FromError(err)
}
// Find the appropriate table.
table, err := findTable(stack, userEntries.Name)
if err != nil {
log.Warningf("netfilter: couldn't find table %q", userEntries.Name)
return linux.KernelIPTGetEntries{}, err
}
// Convert netstack's iptables rules to something that the iptables
// tool can understand.
entries, _, err := convertNetstackToBinary(userEntries.Name.String(), table)
entries, meta, err := convertNetstackToBinary(userEntries.Name.String(), table)
if err != nil {
return linux.KernelIPTGetEntries{}, err
}
if meta != table.Metadata().(metadata) {
panic(fmt.Sprintf("Table %q metadata changed between writing and reading. Was saved as %+v, but is now %+v", userEntries.Name.String(), table.Metadata().(metadata), meta))
}
if binary.Size(entries) > uintptr(outLen) {
log.Warningf("Insufficient GetEntries output size: %d", uintptr(outLen))
return linux.KernelIPTGetEntries{}, syserr.ErrInvalidArgument
@ -148,15 +167,19 @@ func convertNetstackToBinary(tablename string, table iptables.Table) (linux.Kern
copy(entries.Name[:], tablename)
for ruleIdx, rule := range table.Rules {
nflog("convert to binary: current offset: %d", entries.Size)
// Is this a chain entry point?
for hook, hookRuleIdx := range table.BuiltinChains {
if hookRuleIdx == ruleIdx {
nflog("convert to binary: found hook %d at offset %d", hook, entries.Size)
meta.HookEntry[hook] = entries.Size
}
}
// Is this a chain underflow point?
for underflow, underflowRuleIdx := range table.Underflows {
if underflowRuleIdx == ruleIdx {
nflog("convert to binary: found underflow %d at offset %d", underflow, entries.Size)
meta.Underflow[underflow] = entries.Size
}
}
@ -176,6 +199,10 @@ func convertNetstackToBinary(tablename string, table iptables.Table) (linux.Kern
// Serialize the matcher and add it to the
// entry.
serialized := marshalMatcher(matcher)
nflog("convert to binary: matcher serialized as: %v", serialized)
if len(serialized)%8 != 0 {
panic(fmt.Sprintf("matcher %T is not 64-bit aligned", matcher))
}
entry.Elems = append(entry.Elems, serialized...)
entry.NextOffset += uint16(len(serialized))
entry.TargetOffset += uint16(len(serialized))
@ -183,27 +210,64 @@ func convertNetstackToBinary(tablename string, table iptables.Table) (linux.Kern
// Serialize and append the target.
serialized := marshalTarget(rule.Target)
if len(serialized)%8 != 0 {
panic(fmt.Sprintf("target %T is not 64-bit aligned", rule.Target))
}
entry.Elems = append(entry.Elems, serialized...)
entry.NextOffset += uint16(len(serialized))
nflog("convert to binary: adding entry: %+v", entry)
entries.Size += uint32(entry.NextOffset)
entries.Entrytable = append(entries.Entrytable, entry)
meta.NumEntries++
}
nflog("convert to binary: finished with an marshalled size of %d", meta.Size)
meta.Size = entries.Size
return entries, meta, nil
}
func marshalMatcher(matcher iptables.Matcher) []byte {
switch matcher.(type) {
switch m := matcher.(type) {
case *iptables.UDPMatcher:
return marshalUDPMatcher(m)
default:
// TODO(gvisor.dev/issue/170): We don't support any matchers
// yet, so any call to marshalMatcher will panic.
// TODO(gvisor.dev/issue/170): Support other matchers.
panic(fmt.Errorf("unknown matcher of type %T", matcher))
}
}
func marshalUDPMatcher(matcher *iptables.UDPMatcher) []byte {
nflog("convert to binary: marshalling UDP matcher: %+v", matcher)
// We have to pad this struct size to a multiple of 8 bytes.
size := alignUp(linux.SizeOfXTEntryMatch+linux.SizeOfXTUDP, 8)
linuxMatcher := linux.KernelXTEntryMatch{
XTEntryMatch: linux.XTEntryMatch{
MatchSize: uint16(size),
},
Data: make([]byte, 0, linux.SizeOfXTUDP),
}
copy(linuxMatcher.Name[:], matcherNameUDP)
xtudp := linux.XTUDP{
SourcePortStart: matcher.Data.SourcePortStart,
SourcePortEnd: matcher.Data.SourcePortEnd,
DestinationPortStart: matcher.Data.DestinationPortStart,
DestinationPortEnd: matcher.Data.DestinationPortEnd,
InverseFlags: matcher.Data.InverseFlags,
}
linuxMatcher.Data = binary.Marshal(linuxMatcher.Data, usermem.ByteOrder, xtudp)
buf := make([]byte, 0, size)
buf = binary.Marshal(buf, usermem.ByteOrder, linuxMatcher)
buf = append(buf, make([]byte, size-len(buf))...)
nflog("convert to binary: marshalled UDP matcher into %v", buf)
return buf[:]
}
func marshalTarget(target iptables.Target) []byte {
switch target.(type) {
case iptables.UnconditionalAcceptTarget:
@ -218,6 +282,8 @@ func marshalTarget(target iptables.Target) []byte {
}
func marshalStandardTarget(verdict iptables.Verdict) []byte {
nflog("convert to binary: marshalling standard target with size %d", linux.SizeOfXTStandardTarget)
// The target's name will be the empty string.
target := linux.XTStandardTarget{
Target: linux.XTEntryTarget{
@ -285,8 +351,6 @@ func translateToStandardVerdict(val int32) (iptables.Verdict, *syserr.Error) {
// SetEntries sets iptables rules for a single table. See
// net/ipv4/netfilter/ip_tables.c:translate_table for reference.
func SetEntries(stack *stack.Stack, optVal []byte) *syserr.Error {
printReplace(optVal)
// Get the basic rules data (struct ipt_replace).
if len(optVal) < linux.SizeOfIPTReplace {
log.Warningf("netfilter.SetEntries: optVal has insufficient size for replace %d", len(optVal))
@ -307,10 +371,14 @@ func SetEntries(stack *stack.Stack, optVal []byte) *syserr.Error {
return syserr.ErrInvalidArgument
}
nflog("set entries: setting entries in table %q", replace.Name.String())
// Convert input into a list of rules and their offsets.
var offset uint32
var offsets []uint32
for entryIdx := uint32(0); entryIdx < replace.NumEntries; entryIdx++ {
nflog("set entries: processing entry at offset %d", offset)
// Get the struct ipt_entry.
if len(optVal) < linux.SizeOfIPTEntry {
log.Warningf("netfilter: optVal has insufficient size for entry %d", len(optVal))
@ -318,10 +386,12 @@ func SetEntries(stack *stack.Stack, optVal []byte) *syserr.Error {
}
var entry linux.IPTEntry
buf := optVal[:linux.SizeOfIPTEntry]
optVal = optVal[linux.SizeOfIPTEntry:]
binary.Unmarshal(buf, usermem.ByteOrder, &entry)
if entry.TargetOffset != linux.SizeOfIPTEntry {
// TODO(gvisor.dev/issue/170): Support matchers.
initialOptValLen := len(optVal)
optVal = optVal[linux.SizeOfIPTEntry:]
if entry.TargetOffset < linux.SizeOfIPTEntry {
log.Warningf("netfilter: entry has too-small target offset %d", entry.TargetOffset)
return syserr.ErrInvalidArgument
}
@ -332,19 +402,44 @@ func SetEntries(stack *stack.Stack, optVal []byte) *syserr.Error {
return err
}
// TODO(gvisor.dev/issue/170): Matchers and targets can specify
// that they only work for certain protocols, hooks, tables.
// Get matchers.
matchersSize := entry.TargetOffset - linux.SizeOfIPTEntry
if len(optVal) < int(matchersSize) {
log.Warningf("netfilter: entry doesn't have enough room for its matchers (only %d bytes remain)", len(optVal))
return syserr.ErrInvalidArgument
}
matchers, err := parseMatchers(filter, optVal[:matchersSize])
if err != nil {
log.Warningf("netfilter: failed to parse matchers: %v", err)
return err
}
optVal = optVal[matchersSize:]
// Get the target of the rule.
target, consumed, err := parseTarget(optVal)
targetSize := entry.NextOffset - entry.TargetOffset
if len(optVal) < int(targetSize) {
log.Warningf("netfilter: entry doesn't have enough room for its target (only %d bytes remain)", len(optVal))
return syserr.ErrInvalidArgument
}
target, err := parseTarget(optVal[:targetSize])
if err != nil {
return err
}
optVal = optVal[consumed:]
optVal = optVal[targetSize:]
table.Rules = append(table.Rules, iptables.Rule{
Filter: filter,
Target: target,
Filter: filter,
Target: target,
Matchers: matchers,
})
offsets = append(offsets, offset)
offset += linux.SizeOfIPTEntry + consumed
offset += uint32(entry.NextOffset)
if initialOptValLen-len(optVal) != int(entry.NextOffset) {
log.Warningf("netfilter: entry NextOffset is %d, but entry took up %d bytes", entry.NextOffset, initialOptValLen-len(optVal))
}
}
// Go through the list of supported hooks for this table and, for each
@ -401,12 +496,86 @@ func SetEntries(stack *stack.Stack, optVal []byte) *syserr.Error {
return nil
}
// parseTarget parses a target from the start of optVal and returns the target
// along with the number of bytes it occupies in optVal.
func parseTarget(optVal []byte) (iptables.Target, uint32, *syserr.Error) {
// parseMatchers parses 0 or more matchers from optVal. optVal should contain
// only the matchers.
func parseMatchers(filter iptables.IPHeaderFilter, optVal []byte) ([]iptables.Matcher, *syserr.Error) {
nflog("set entries: parsing matchers of size %d", len(optVal))
var matchers []iptables.Matcher
for len(optVal) > 0 {
nflog("set entries: optVal has len %d", len(optVal))
// Get the XTEntryMatch.
if len(optVal) < linux.SizeOfXTEntryMatch {
log.Warningf("netfilter: optVal has insufficient size for entry match: %d", len(optVal))
return nil, syserr.ErrInvalidArgument
}
var match linux.XTEntryMatch
buf := optVal[:linux.SizeOfXTEntryMatch]
binary.Unmarshal(buf, usermem.ByteOrder, &match)
nflog("set entries: parsed entry match %q: %+v", match.Name.String(), match)
// Check some invariants.
if match.MatchSize < linux.SizeOfXTEntryMatch {
log.Warningf("netfilter: match size is too small, must be at least %d", linux.SizeOfXTEntryMatch)
return nil, syserr.ErrInvalidArgument
}
if len(optVal) < int(match.MatchSize) {
log.Warningf("netfilter: optVal has insufficient size for match: %d", len(optVal))
return nil, syserr.ErrInvalidArgument
}
buf = optVal[linux.SizeOfXTEntryMatch:match.MatchSize]
var matcher iptables.Matcher
var err error
switch match.Name.String() {
case matcherNameUDP:
if len(buf) < linux.SizeOfXTUDP {
log.Warningf("netfilter: optVal has insufficient size for UDP match: %d", len(optVal))
return nil, syserr.ErrInvalidArgument
}
// For alignment reasons, the match's total size may
// exceed what's strictly necessary to hold matchData.
var matchData linux.XTUDP
binary.Unmarshal(buf[:linux.SizeOfXTUDP], usermem.ByteOrder, &matchData)
log.Infof("parseMatchers: parsed XTUDP: %+v", matchData)
matcher, err = iptables.NewUDPMatcher(filter, iptables.UDPMatcherParams{
SourcePortStart: matchData.SourcePortStart,
SourcePortEnd: matchData.SourcePortEnd,
DestinationPortStart: matchData.DestinationPortStart,
DestinationPortEnd: matchData.DestinationPortEnd,
InverseFlags: matchData.InverseFlags,
})
if err != nil {
log.Warningf("netfilter: failed to create UDP matcher: %v", err)
return nil, syserr.ErrInvalidArgument
}
default:
log.Warningf("netfilter: unsupported matcher with name %q", match.Name.String())
return nil, syserr.ErrInvalidArgument
}
matchers = append(matchers, matcher)
// TODO(gvisor.dev/issue/170): Check the revision field.
optVal = optVal[match.MatchSize:]
}
if len(optVal) != 0 {
log.Warningf("netfilter: optVal should be exhausted after parsing matchers")
return nil, syserr.ErrInvalidArgument
}
return matchers, nil
}
// parseTarget parses a target from optVal. optVal should contain only the
// target.
func parseTarget(optVal []byte) (iptables.Target, *syserr.Error) {
nflog("set entries: parsing target of size %d", len(optVal))
if len(optVal) < linux.SizeOfXTEntryTarget {
log.Warningf("netfilter: optVal has insufficient size for entry target %d", len(optVal))
return nil, 0, syserr.ErrInvalidArgument
return nil, syserr.ErrInvalidArgument
}
var target linux.XTEntryTarget
buf := optVal[:linux.SizeOfXTEntryTarget]
@ -414,9 +583,9 @@ func parseTarget(optVal []byte) (iptables.Target, uint32, *syserr.Error) {
switch target.Name.String() {
case "":
// Standard target.
if len(optVal) < linux.SizeOfXTStandardTarget {
log.Warningf("netfilter.SetEntries: optVal has insufficient size for standard target %d", len(optVal))
return nil, 0, syserr.ErrInvalidArgument
if len(optVal) != linux.SizeOfXTStandardTarget {
log.Warningf("netfilter.SetEntries: optVal has wrong size for standard target %d", len(optVal))
return nil, syserr.ErrInvalidArgument
}
var standardTarget linux.XTStandardTarget
buf = optVal[:linux.SizeOfXTStandardTarget]
@ -424,22 +593,23 @@ func parseTarget(optVal []byte) (iptables.Target, uint32, *syserr.Error) {
verdict, err := translateToStandardVerdict(standardTarget.Verdict)
if err != nil {
return nil, 0, err
return nil, err
}
switch verdict {
case iptables.Accept:
return iptables.UnconditionalAcceptTarget{}, linux.SizeOfXTStandardTarget, nil
return iptables.UnconditionalAcceptTarget{}, nil
case iptables.Drop:
return iptables.UnconditionalDropTarget{}, linux.SizeOfXTStandardTarget, nil
return iptables.UnconditionalDropTarget{}, nil
default:
panic(fmt.Sprintf("Unknown verdict: %v", verdict))
log.Warningf("Unknown verdict: %v", verdict)
return nil, syserr.ErrInvalidArgument
}
case errorTargetName:
// Error target.
if len(optVal) < linux.SizeOfXTErrorTarget {
if len(optVal) != linux.SizeOfXTErrorTarget {
log.Infof("netfilter.SetEntries: optVal has insufficient size for error target %d", len(optVal))
return nil, 0, syserr.ErrInvalidArgument
return nil, syserr.ErrInvalidArgument
}
var errorTarget linux.XTErrorTarget
buf = optVal[:linux.SizeOfXTErrorTarget]
@ -454,16 +624,16 @@ func parseTarget(optVal []byte) (iptables.Target, uint32, *syserr.Error) {
// rules have an error with the name of the chain.
switch errorTarget.Name.String() {
case errorTargetName:
return iptables.ErrorTarget{}, linux.SizeOfXTErrorTarget, nil
return iptables.ErrorTarget{}, nil
default:
log.Infof("Unknown error target %q doesn't exist or isn't supported yet.", errorTarget.Name.String())
return nil, 0, syserr.ErrInvalidArgument
return nil, syserr.ErrInvalidArgument
}
}
// Unknown target.
log.Infof("Unknown target %q doesn't exist or isn't supported yet.", target.Name.String())
return nil, 0, syserr.ErrInvalidArgument
return nil, syserr.ErrInvalidArgument
}
func filterFromIPTIP(iptip linux.IPTIP) (iptables.IPHeaderFilter, *syserr.Error) {
@ -508,51 +678,7 @@ func hookFromLinux(hook int) iptables.Hook {
panic(fmt.Sprintf("Unknown hook %d does not correspond to a builtin chain", hook))
}
// printReplace prints information about the struct ipt_replace in optVal. It
// is only for debugging.
func printReplace(optVal []byte) {
// Basic replace info.
var replace linux.IPTReplace
replaceBuf := optVal[:linux.SizeOfIPTReplace]
optVal = optVal[linux.SizeOfIPTReplace:]
binary.Unmarshal(replaceBuf, usermem.ByteOrder, &replace)
log.Infof("Replacing table %q: %+v", replace.Name.String(), replace)
// Read in the list of entries at the end of replace.
var totalOffset uint16
for entryIdx := uint32(0); entryIdx < replace.NumEntries; entryIdx++ {
var entry linux.IPTEntry
entryBuf := optVal[:linux.SizeOfIPTEntry]
binary.Unmarshal(entryBuf, usermem.ByteOrder, &entry)
log.Infof("Entry %d (total offset %d): %+v", entryIdx, totalOffset, entry)
totalOffset += entry.NextOffset
if entry.TargetOffset == linux.SizeOfIPTEntry {
log.Infof("Entry has no matches.")
} else {
log.Infof("Entry has matches.")
}
var target linux.XTEntryTarget
targetBuf := optVal[entry.TargetOffset : entry.TargetOffset+linux.SizeOfXTEntryTarget]
binary.Unmarshal(targetBuf, usermem.ByteOrder, &target)
log.Infof("Target named %q: %+v", target.Name.String(), target)
switch target.Name.String() {
case "":
var standardTarget linux.XTStandardTarget
stBuf := optVal[entry.TargetOffset : entry.TargetOffset+linux.SizeOfXTStandardTarget]
binary.Unmarshal(stBuf, usermem.ByteOrder, &standardTarget)
log.Infof("Standard target with verdict %q (%d).", linux.VerdictStrings[standardTarget.Verdict], standardTarget.Verdict)
case errorTargetName:
var errorTarget linux.XTErrorTarget
etBuf := optVal[entry.TargetOffset : entry.TargetOffset+linux.SizeOfXTErrorTarget]
binary.Unmarshal(etBuf, usermem.ByteOrder, &errorTarget)
log.Infof("Error target with name %q.", errorTarget.Name.String())
default:
log.Infof("Unknown target type.")
}
optVal = optVal[entry.NextOffset:]
}
// alignUp rounds a length up to an alignment. align must be a power of 2.
func alignUp(length int, align uint) int {
return (length + int(align) - 1) & ^(int(align) - 1)
}

View File

@ -8,6 +8,7 @@ go_library(
"iptables.go",
"targets.go",
"types.go",
"udp_matcher.go",
],
visibility = ["//visibility:public"],
deps = [

View File

@ -138,6 +138,8 @@ func EmptyFilterTable() Table {
// Check runs pkt through the rules for hook. It returns true when the packet
// should continue traversing the network stack and false when it should be
// dropped.
//
// Precondition: pkt.NetworkHeader is set.
func (it *IPTables) Check(hook Hook, pkt tcpip.PacketBuffer) bool {
// TODO(gvisor.dev/issue/170): A lot of this is uncomplicated because
// we're missing features. Jumps, the call stack, etc. aren't checked
@ -163,6 +165,7 @@ func (it *IPTables) Check(hook Hook, pkt tcpip.PacketBuffer) bool {
return true
}
// Precondition: pkt.NetworkHeader is set.
func (it *IPTables) checkTable(hook Hook, pkt tcpip.PacketBuffer, tablename string) Verdict {
// Start from ruleIdx and walk the list of rules until a rule gives us
// a verdict.

View File

@ -132,7 +132,7 @@ type Table struct {
// ValidHooks returns a bitmap of the builtin hooks for the given table.
func (table *Table) ValidHooks() uint32 {
hooks := uint32(0)
for hook, _ := range table.BuiltinChains {
for hook := range table.BuiltinChains {
hooks |= 1 << hook
}
return hooks
@ -174,6 +174,8 @@ type Matcher interface {
// Match returns whether the packet matches and whether the packet
// should be "hotdropped", i.e. dropped immediately. This is usually
// used for suspicious packets.
//
// Precondition: packet.NetworkHeader is set.
Match(hook Hook, packet tcpip.PacketBuffer, interfaceName string) (matches bool, hotdrop bool)
}

View File

@ -0,0 +1,109 @@
// 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 iptables
import (
"fmt"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/tcpip"
"gvisor.dev/gvisor/pkg/tcpip/header"
)
// TODO(gvisor.dev/issue/170): The following per-matcher params should be
// supported:
// - Table name
// - Match size
// - User size
// - Hooks
// - Proto
// - Family
// UDPMatcher matches UDP packets and their headers. It implements Matcher.
type UDPMatcher struct {
Data UDPMatcherParams
}
// UDPMatcherParams are the parameters used to create a UDPMatcher.
type UDPMatcherParams struct {
SourcePortStart uint16
SourcePortEnd uint16
DestinationPortStart uint16
DestinationPortEnd uint16
InverseFlags uint8
}
// NewUDPMatcher returns a new instance of UDPMatcher.
func NewUDPMatcher(filter IPHeaderFilter, data UDPMatcherParams) (Matcher, error) {
log.Infof("Adding rule with UDPMatcherParams: %+v", data)
if data.InverseFlags != 0 {
return nil, fmt.Errorf("unsupported UDP matcher inverse flags set")
}
if filter.Protocol != header.UDPProtocolNumber {
return nil, fmt.Errorf("UDP matching is only valid for protocol %d", header.UDPProtocolNumber)
}
return &UDPMatcher{Data: data}, nil
}
// Match implements Matcher.Match.
func (um *UDPMatcher) Match(hook Hook, pkt tcpip.PacketBuffer, interfaceName string) (bool, bool) {
netHeader := header.IPv4(pkt.NetworkHeader)
// TODO(gvisor.dev/issue/170): Proto checks should ultimately be moved
// into the iptables.Check codepath as matchers are added.
if netHeader.TransportProtocol() != header.UDPProtocolNumber {
return false, false
}
// We dont't match fragments.
if frag := netHeader.FragmentOffset(); frag != 0 {
if frag == 1 {
return false, true
}
return false, false
}
// Now we need the transport header. However, this may not have been set
// yet.
// TODO(gvisor.dev/issue/170): Parsing the transport header should
// ultimately be moved into the iptables.Check codepath as matchers are
// added.
var udpHeader header.UDP
if pkt.TransportHeader != nil {
udpHeader = header.UDP(pkt.TransportHeader)
} else {
// The UDP header hasn't been parsed yet. We have to do it here.
if len(pkt.Data.First()) < header.UDPMinimumSize {
// There's no valid UDP header here, so we hotdrop the
// packet.
return false, true
}
udpHeader = header.UDP(pkt.Data.First())
}
// Check whether the source and destination ports are within the
// matching range.
if sourcePort := udpHeader.SourcePort(); sourcePort < um.Data.SourcePortStart || um.Data.SourcePortEnd < sourcePort {
return false, false
}
if destinationPort := udpHeader.DestinationPort(); destinationPort < um.Data.DestinationPortStart || um.Data.DestinationPortEnd < destinationPort {
return false, false
}
return true, false
}

View File

@ -353,6 +353,11 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt tcpip.PacketBuffer) {
}
pkt.NetworkHeader = headerView[:h.HeaderLength()]
hlen := int(h.HeaderLength())
tlen := int(h.TotalLength())
pkt.Data.TrimFront(hlen)
pkt.Data.CapLength(tlen - hlen)
// iptables filtering. All packets that reach here are intended for
// this machine and will not be forwarded.
ipt := e.stack.IPTables()
@ -361,11 +366,6 @@ func (e *endpoint) HandlePacket(r *stack.Route, pkt tcpip.PacketBuffer) {
return
}
hlen := int(h.HeaderLength())
tlen := int(h.TotalLength())
pkt.Data.TrimFront(hlen)
pkt.Data.CapLength(tlen - hlen)
more := (h.Flags() & header.IPv4FlagMoreFragments) != 0
if more || h.FragmentOffset() != 0 {
if pkt.Data.Size() == 0 {

View File

@ -136,8 +136,13 @@ func FindFile(path string) (string, error) {
// TestConfig returns the default configuration to use in tests. Note that
// 'RootDir' must be set by caller if required.
func TestConfig() *boot.Config {
logDir := ""
if dir, ok := os.LookupEnv("TEST_UNDECLARED_OUTPUTS_DIR"); ok {
logDir = dir + "/"
}
return &boot.Config{
Debug: true,
DebugLog: logDir,
LogFormat: "text",
DebugLogFormat: "text",
AlsoLogToStderr: true,

View File

@ -15,6 +15,7 @@
package iptables
import (
"errors"
"fmt"
"net"
"time"
@ -248,3 +249,54 @@ func (FilterInputDropAll) ContainerAction(ip net.IP) error {
func (FilterInputDropAll) LocalAction(ip net.IP) error {
return sendUDPLoop(ip, dropPort, sendloopDuration)
}
// FilterInputMultiUDPRules verifies that multiple UDP rules are applied
// correctly. This has the added benefit of testing whether we're serializing
// rules correctly -- if we do it incorrectly, the iptables tool will
// misunderstand and save the wrong tables.
type FilterInputMultiUDPRules struct{}
// Name implements TestCase.Name.
func (FilterInputMultiUDPRules) Name() string {
return "FilterInputMultiUDPRules"
}
// ContainerAction implements TestCase.ContainerAction.
func (FilterInputMultiUDPRules) ContainerAction(ip net.IP) error {
if err := filterTable("-A", "INPUT", "-p", "udp", "-m", "udp", "--destination-port", fmt.Sprintf("%d", dropPort), "-j", "DROP"); err != nil {
return err
}
if err := filterTable("-A", "INPUT", "-p", "udp", "-m", "udp", "--destination-port", fmt.Sprintf("%d", acceptPort), "-j", "ACCEPT"); err != nil {
return err
}
return filterTable("-L")
}
// LocalAction implements TestCase.LocalAction.
func (FilterInputMultiUDPRules) LocalAction(ip net.IP) error {
// No-op.
return nil
}
// FilterInputRequireProtocolUDP checks that "-m udp" requires "-p udp" to be
// specified.
type FilterInputRequireProtocolUDP struct{}
// Name implements TestCase.Name.
func (FilterInputRequireProtocolUDP) Name() string {
return "FilterInputRequireProtocolUDP"
}
// ContainerAction implements TestCase.ContainerAction.
func (FilterInputRequireProtocolUDP) ContainerAction(ip net.IP) error {
if err := filterTable("-A", "INPUT", "-m", "udp", "--destination-port", fmt.Sprintf("%d", dropPort), "-j", "DROP"); err == nil {
return errors.New("expected iptables to fail with out \"-p udp\", but succeeded")
}
return nil
}
// LocalAction implements TestCase.LocalAction.
func (FilterInputRequireProtocolUDP) LocalAction(ip net.IP) error {
// No-op.
return nil
}