diff options
Diffstat (limited to 'pkg/sentry/socket/netlink')
-rw-r--r-- | pkg/sentry/socket/netlink/BUILD | 47 | ||||
-rw-r--r-- | pkg/sentry/socket/netlink/message.go | 159 | ||||
-rw-r--r-- | pkg/sentry/socket/netlink/port/BUILD | 28 | ||||
-rw-r--r-- | pkg/sentry/socket/netlink/port/port.go | 114 | ||||
-rw-r--r-- | pkg/sentry/socket/netlink/port/port_test.go | 82 | ||||
-rw-r--r-- | pkg/sentry/socket/netlink/provider.go | 104 | ||||
-rw-r--r-- | pkg/sentry/socket/netlink/route/BUILD | 33 | ||||
-rw-r--r-- | pkg/sentry/socket/netlink/route/protocol.go | 189 | ||||
-rw-r--r-- | pkg/sentry/socket/netlink/socket.go | 517 |
9 files changed, 1273 insertions, 0 deletions
diff --git a/pkg/sentry/socket/netlink/BUILD b/pkg/sentry/socket/netlink/BUILD new file mode 100644 index 000000000..9df3ab17c --- /dev/null +++ b/pkg/sentry/socket/netlink/BUILD @@ -0,0 +1,47 @@ +package(licenses = ["notice"]) # Apache 2.0 + +load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_stateify") + +go_stateify( + name = "netlink_state", + srcs = [ + "socket.go", + ], + out = "netlink_state.go", + package = "netlink", +) + +go_library( + name = "netlink", + srcs = [ + "message.go", + "netlink_state.go", + "provider.go", + "socket.go", + ], + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/socket/netlink", + visibility = ["//pkg/sentry:internal"], + deps = [ + "//pkg/abi/linux", + "//pkg/binary", + "//pkg/sentry/arch", + "//pkg/sentry/context", + "//pkg/sentry/device", + "//pkg/sentry/fs", + "//pkg/sentry/fs/fsutil", + "//pkg/sentry/kernel", + "//pkg/sentry/kernel/kdefs", + "//pkg/sentry/kernel/time", + "//pkg/sentry/socket", + "//pkg/sentry/socket/netlink/port", + "//pkg/sentry/socket/unix", + "//pkg/sentry/usermem", + "//pkg/state", + "//pkg/syserr", + "//pkg/syserror", + "//pkg/tcpip", + "//pkg/tcpip/transport/unix", + "//pkg/waiter", + ], +) diff --git a/pkg/sentry/socket/netlink/message.go b/pkg/sentry/socket/netlink/message.go new file mode 100644 index 000000000..b902d7ec9 --- /dev/null +++ b/pkg/sentry/socket/netlink/message.go @@ -0,0 +1,159 @@ +// Copyright 2018 Google Inc. +// +// 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 netlink + +import ( + "fmt" + "math" + + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/binary" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" +) + +// alignUp rounds a length up to an alignment. +// +// Preconditions: align is a power of two. +func alignUp(length int, align uint) int { + return (length + int(align) - 1) &^ (int(align) - 1) +} + +// Message contains a complete serialized netlink message. +type Message struct { + buf []byte +} + +// NewMessage creates a new Message containing the passed header. +// +// The header length will be updated by Finalize. +func NewMessage(hdr linux.NetlinkMessageHeader) *Message { + return &Message{ + buf: binary.Marshal(nil, usermem.ByteOrder, hdr), + } +} + +// Finalize returns the []byte containing the entire message, with the total +// length set in the message header. The Message must not be modified after +// calling Finalize. +func (m *Message) Finalize() []byte { + // Update length, which is the first 4 bytes of the header. + usermem.ByteOrder.PutUint32(m.buf, uint32(len(m.buf))) + + // Align the message. Note that the message length in the header (set + // above) is the useful length of the message, not the total aligned + // length. See net/netlink/af_netlink.c:__nlmsg_put. + aligned := alignUp(len(m.buf), linux.NLMSG_ALIGNTO) + m.putZeros(aligned - len(m.buf)) + return m.buf +} + +// putZeros adds n zeros to the message. +func (m *Message) putZeros(n int) { + for n > 0 { + m.buf = append(m.buf, 0) + n-- + } +} + +// Put serializes v into the message. +func (m *Message) Put(v interface{}) { + m.buf = binary.Marshal(m.buf, usermem.ByteOrder, v) +} + +// PutAttr adds v to the message as a netlink attribute. +// +// Preconditions: The serialized attribute (linux.NetlinkAttrHeaderSize + +// binary.Size(v) fits in math.MaxUint16 bytes. +func (m *Message) PutAttr(atype uint16, v interface{}) { + l := linux.NetlinkAttrHeaderSize + int(binary.Size(v)) + if l > math.MaxUint16 { + panic(fmt.Sprintf("attribute too large: %d", l)) + } + + m.Put(linux.NetlinkAttrHeader{ + Type: atype, + Length: uint16(l), + }) + m.Put(v) + + // Align the attribute. + aligned := alignUp(l, linux.NLA_ALIGNTO) + m.putZeros(aligned - l) +} + +// PutAttrString adds s to the message as a netlink attribute. +func (m *Message) PutAttrString(atype uint16, s string) { + l := linux.NetlinkAttrHeaderSize + len(s) + 1 + m.Put(linux.NetlinkAttrHeader{ + Type: atype, + Length: uint16(l), + }) + + // String + NUL-termination. + m.Put([]byte(s)) + m.putZeros(1) + + // Align the attribute. + aligned := alignUp(l, linux.NLA_ALIGNTO) + m.putZeros(aligned - l) +} + +// MessageSet contains a series of netlink messages. +type MessageSet struct { + // Multi indicates that this a multi-part message, to be terminated by + // NLMSG_DONE. NLMSG_DONE is sent even if the set contains only one + // Message. + // + // If Multi is set, all added messages will have NLM_F_MULTI set. + Multi bool + + // PortID is the destination port for all messages. + PortID int32 + + // Seq is the sequence counter for all messages in the set. + Seq uint32 + + // Messages contains the messages in the set. + Messages []*Message +} + +// NewMessageSet creates a new MessageSet. +// +// portID is the destination port to set as PortID in all messages. +// +// seq is the sequence counter to set as seq in all messages in the set. +func NewMessageSet(portID int32, seq uint32) *MessageSet { + return &MessageSet{ + PortID: portID, + Seq: seq, + } +} + +// AddMessage adds a new message to the set and returns it for further +// additions. +// +// The passed header will have Seq, PortID and the multi flag set +// automatically. +func (ms *MessageSet) AddMessage(hdr linux.NetlinkMessageHeader) *Message { + hdr.Seq = ms.Seq + hdr.PortID = uint32(ms.PortID) + if ms.Multi { + hdr.Flags |= linux.NLM_F_MULTI + } + + m := NewMessage(hdr) + ms.Messages = append(ms.Messages, m) + return m +} diff --git a/pkg/sentry/socket/netlink/port/BUILD b/pkg/sentry/socket/netlink/port/BUILD new file mode 100644 index 000000000..7340b95c9 --- /dev/null +++ b/pkg/sentry/socket/netlink/port/BUILD @@ -0,0 +1,28 @@ +package(licenses = ["notice"]) # Apache 2.0 + +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//tools/go_stateify:defs.bzl", "go_stateify") + +go_stateify( + name = "port_state", + srcs = ["port.go"], + out = "port_state.go", + package = "port", +) + +go_library( + name = "port", + srcs = [ + "port.go", + "port_state.go", + ], + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/socket/netlink/port", + visibility = ["//pkg/sentry:internal"], + deps = ["//pkg/state"], +) + +go_test( + name = "port_test", + srcs = ["port_test.go"], + embed = [":port"], +) diff --git a/pkg/sentry/socket/netlink/port/port.go b/pkg/sentry/socket/netlink/port/port.go new file mode 100644 index 000000000..4ccf0b84c --- /dev/null +++ b/pkg/sentry/socket/netlink/port/port.go @@ -0,0 +1,114 @@ +// Copyright 2018 Google Inc. +// +// 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 port provides port ID allocation for netlink sockets. +// +// A netlink port is any int32 value. Positive ports are typically equivalent +// to the PID of the binding process. If that port is unavailable, negative +// ports are searched to find a free port that will not conflict with other +// PIDS. +package port + +import ( + "fmt" + "math" + "math/rand" + "sync" +) + +// maxPorts is a sanity limit on the maximum number of ports to allocate per +// protocol. +const maxPorts = 10000 + +// Manager allocates netlink port IDs. +type Manager struct { + // mu protects the fields below. + mu sync.Mutex `state:"nosave"` + + // ports contains a map of allocated ports for each protocol. + ports map[int]map[int32]struct{} +} + +// New creates a new Manager. +func New() *Manager { + return &Manager{ + ports: make(map[int]map[int32]struct{}), + } +} + +// Allocate reserves a new port ID for protocol. hint will be taken if +// available. +func (m *Manager) Allocate(protocol int, hint int32) (int32, bool) { + m.mu.Lock() + defer m.mu.Unlock() + + proto, ok := m.ports[protocol] + if !ok { + proto = make(map[int32]struct{}) + // Port 0 is reserved for the kernel. + proto[0] = struct{}{} + m.ports[protocol] = proto + } + + if len(proto) >= maxPorts { + return 0, false + } + + if _, ok := proto[hint]; !ok { + // Hint is available, reserve it. + proto[hint] = struct{}{} + return hint, true + } + + // Search for any free port in [math.MinInt32, -4096). The positive + // port space is left open for pid-based allocations. This behavior is + // consistent with Linux. + start := int32(math.MinInt32 + rand.Int63n(math.MaxInt32-4096+1)) + curr := start + for { + if _, ok := proto[curr]; !ok { + proto[curr] = struct{}{} + return curr, true + } + + curr-- + if curr >= -4096 { + curr = -4097 + } + if curr == start { + // Nothing found. We should always find a free port + // because maxPorts < -4096 - MinInt32. + panic(fmt.Sprintf("No free port found in %+v", proto)) + } + } +} + +// Release frees the specified port for protocol. +// +// Preconditions: port is already allocated. +func (m *Manager) Release(protocol int, port int32) { + m.mu.Lock() + defer m.mu.Unlock() + + proto, ok := m.ports[protocol] + if !ok { + panic(fmt.Sprintf("Released port %d for protocol %d which has no allocations", port, protocol)) + } + + if _, ok := proto[port]; !ok { + panic(fmt.Sprintf("Released port %d for protocol %d is not allocated", port, protocol)) + } + + delete(proto, port) +} diff --git a/pkg/sentry/socket/netlink/port/port_test.go b/pkg/sentry/socket/netlink/port/port_test.go new file mode 100644 index 000000000..34565e2f9 --- /dev/null +++ b/pkg/sentry/socket/netlink/port/port_test.go @@ -0,0 +1,82 @@ +// Copyright 2018 Google Inc. +// +// 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 port + +import ( + "testing" +) + +func TestAllocateHint(t *testing.T) { + m := New() + + // We can get the hint port. + p, ok := m.Allocate(0, 1) + if !ok { + t.Errorf("m.Allocate got !ok want ok") + } + if p != 1 { + t.Errorf("m.Allocate(0, 1) got %d want 1", p) + } + + // Hint is taken. + p, ok = m.Allocate(0, 1) + if !ok { + t.Errorf("m.Allocate got !ok want ok") + } + if p == 1 { + t.Errorf("m.Allocate(0, 1) got 1 want anything else") + } + + // Hint is available for a different protocol. + p, ok = m.Allocate(1, 1) + if !ok { + t.Errorf("m.Allocate got !ok want ok") + } + if p != 1 { + t.Errorf("m.Allocate(1, 1) got %d want 1", p) + } + + m.Release(0, 1) + + // Hint is available again after release. + p, ok = m.Allocate(0, 1) + if !ok { + t.Errorf("m.Allocate got !ok want ok") + } + if p != 1 { + t.Errorf("m.Allocate(0, 1) got %d want 1", p) + } +} + +func TestAllocateExhausted(t *testing.T) { + m := New() + + // Fill all ports (0 is already reserved). + for i := int32(1); i < maxPorts; i++ { + p, ok := m.Allocate(0, i) + if !ok { + t.Fatalf("m.Allocate got !ok want ok") + } + if p != i { + t.Fatalf("m.Allocate(0, %d) got %d want %d", i, p, i) + } + } + + // Now no more can be allocated. + p, ok := m.Allocate(0, 1) + if ok { + t.Errorf("m.Allocate got %d, ok want !ok", p) + } +} diff --git a/pkg/sentry/socket/netlink/provider.go b/pkg/sentry/socket/netlink/provider.go new file mode 100644 index 000000000..36800da4d --- /dev/null +++ b/pkg/sentry/socket/netlink/provider.go @@ -0,0 +1,104 @@ +// Copyright 2018 Google Inc. +// +// 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 netlink + +import ( + "fmt" + + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket" + "gvisor.googlesource.com/gvisor/pkg/syserr" + "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" +) + +// Protocol is the implementation of a netlink socket protocol. +type Protocol interface { + // Protocol returns the Linux netlink protocol value. + Protocol() int + + // ProcessMessage processes a single message from userspace. + // + // If err == nil, any messages added to ms will be sent back to the + // other end of the socket. Setting ms.Multi will cause an NLMSG_DONE + // message to be sent even if ms contains no messages. + ProcessMessage(ctx context.Context, hdr linux.NetlinkMessageHeader, data []byte, ms *MessageSet) *syserr.Error +} + +// Provider is a function that creates a new Protocol for a specific netlink +// protocol. +// +// Note that this is distinct from socket.Provider, which is used for all +// socket families. +type Provider func(t *kernel.Task) (Protocol, *syserr.Error) + +// protocols holds a map of all known address protocols and their provider. +var protocols = make(map[int]Provider) + +// RegisterProvider registers the provider of a given address protocol so that +// netlink sockets of that type can be created via socket(2). +// +// Preconditions: May only be called before any netlink sockets are created. +func RegisterProvider(protocol int, provider Provider) { + if p, ok := protocols[protocol]; ok { + panic(fmt.Sprintf("Netlink protocol %d already provided by %+v", protocol, p)) + } + + protocols[protocol] = provider +} + +// socketProvider implements socket.Provider. +type socketProvider struct { +} + +// Socket implements socket.Provider.Socket. +func (*socketProvider) Socket(t *kernel.Task, stype unix.SockType, protocol int) (*fs.File, *syserr.Error) { + // Netlink sockets must be specified as datagram or raw, but they + // behave the same regardless of type. + if stype != unix.SockDgram && stype != unix.SockRaw { + return nil, syserr.ErrSocketNotSupported + } + + provider, ok := protocols[protocol] + if !ok { + return nil, syserr.ErrProtocolNotSupported + } + + p, err := provider(t) + if err != nil { + return nil, err + } + + s, err := NewSocket(t, p) + if err != nil { + return nil, err + } + + d := socket.NewDirent(t, netlinkSocketDevice) + return fs.NewFile(t, d, fs.FileFlags{Read: true, Write: true}, s), nil +} + +// Pair implements socket.Provider.Pair by returning an error. +func (*socketProvider) Pair(*kernel.Task, unix.SockType, int) (*fs.File, *fs.File, *syserr.Error) { + // Netlink sockets never supports creating socket pairs. + return nil, nil, syserr.ErrNotSupported +} + +// init registers the socket provider. +func init() { + socket.RegisterProvider(linux.AF_NETLINK, &socketProvider{}) +} diff --git a/pkg/sentry/socket/netlink/route/BUILD b/pkg/sentry/socket/netlink/route/BUILD new file mode 100644 index 000000000..ff3f7b7a4 --- /dev/null +++ b/pkg/sentry/socket/netlink/route/BUILD @@ -0,0 +1,33 @@ +package(licenses = ["notice"]) # Apache 2.0 + +load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//tools/go_stateify:defs.bzl", "go_stateify") + +go_stateify( + name = "route_state", + srcs = ["protocol.go"], + out = "route_state.go", + package = "route", +) + +go_library( + name = "route", + srcs = [ + "protocol.go", + "route_state.go", + ], + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/socket/netlink/route", + visibility = ["//pkg/sentry:internal"], + deps = [ + "//pkg/abi/linux", + "//pkg/sentry/context", + "//pkg/sentry/fs", + "//pkg/sentry/inet", + "//pkg/sentry/kernel", + "//pkg/sentry/kernel/auth", + "//pkg/sentry/socket/netlink", + "//pkg/sentry/usermem", + "//pkg/state", + "//pkg/syserr", + ], +) diff --git a/pkg/sentry/socket/netlink/route/protocol.go b/pkg/sentry/socket/netlink/route/protocol.go new file mode 100644 index 000000000..d611519d4 --- /dev/null +++ b/pkg/sentry/socket/netlink/route/protocol.go @@ -0,0 +1,189 @@ +// Copyright 2018 Google Inc. +// +// 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 route provides a NETLINK_ROUTE socket protocol. +package route + +import ( + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/inet" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/netlink" + "gvisor.googlesource.com/gvisor/pkg/syserr" +) + +// commandKind describes the operational class of a message type. +// +// The route message types use the lower 2 bits of the type to describe class +// of command. +type commandKind int + +const ( + kindNew commandKind = 0x0 + kindDel = 0x1 + kindGet = 0x2 + kindSet = 0x3 +) + +func typeKind(typ uint16) commandKind { + return commandKind(typ & 0x3) +} + +// Protocol implements netlink.Protocol. +type Protocol struct { + // stack is the network stack that this provider describes. + // + // May be nil. + stack inet.Stack +} + +var _ netlink.Protocol = (*Protocol)(nil) + +// NewProtocol creates a NETLINK_ROUTE netlink.Protocol. +func NewProtocol(t *kernel.Task) (netlink.Protocol, *syserr.Error) { + return &Protocol{ + stack: t.NetworkContext(), + }, nil +} + +// Protocol implements netlink.Protocol.Protocol. +func (p *Protocol) Protocol() int { + return linux.NETLINK_ROUTE +} + +// dumpLinks handles RTM_GETLINK + NLM_F_DUMP requests. +func (p *Protocol) dumpLinks(ctx context.Context, hdr linux.NetlinkMessageHeader, data []byte, ms *netlink.MessageSet) *syserr.Error { + // NLM_F_DUMP + RTM_GETLINK messages are supposed to include an + // ifinfomsg. However, Linux <3.9 only checked for rtgenmsg, and some + // userspace applications (including glibc) still include rtgenmsg. + // Linux has a workaround based on the total message length. + // + // We don't bother to check for either, since we don't support any + // extra attributes that may be included anyways. + // + // The message may also contain netlink attribute IFLA_EXT_MASK, which + // we don't support. + + // The RTM_GETLINK dump response is a set of messages each containing + // an InterfaceInfoMessage followed by a set of netlink attributes. + + // We always send back an NLMSG_DONE. + ms.Multi = true + + if p.stack == nil { + // No network devices. + return nil + } + + for id, i := range p.stack.Interfaces() { + m := ms.AddMessage(linux.NetlinkMessageHeader{ + Type: linux.RTM_NEWLINK, + }) + + m.Put(linux.InterfaceInfoMessage{ + Family: linux.AF_UNSPEC, + Type: i.DeviceType, + Index: id, + Flags: i.Flags, + }) + + m.PutAttrString(linux.IFLA_IFNAME, i.Name) + + // TODO: There are many more attributes, such as + // MAC address. + } + + return nil +} + +// dumpAddrs handles RTM_GETADDR + NLM_F_DUMP requests. +func (p *Protocol) dumpAddrs(ctx context.Context, hdr linux.NetlinkMessageHeader, data []byte, ms *netlink.MessageSet) *syserr.Error { + // RTM_GETADDR dump requests need not contain anything more than the + // netlink header and 1 byte protocol family common to all + // NETLINK_ROUTE requests. + // + // TODO: Filter output by passed protocol family. + + // The RTM_GETADDR dump response is a set of RTM_NEWADDR messages each + // containing an InterfaceAddrMessage followed by a set of netlink + // attributes. + + // We always send back an NLMSG_DONE. + ms.Multi = true + + if p.stack == nil { + // No network devices. + return nil + } + + for id, as := range p.stack.InterfaceAddrs() { + for _, a := range as { + m := ms.AddMessage(linux.NetlinkMessageHeader{ + Type: linux.RTM_NEWADDR, + }) + + m.Put(linux.InterfaceAddrMessage{ + Family: a.Family, + PrefixLen: a.PrefixLen, + Index: uint32(id), + }) + + m.PutAttr(linux.IFA_ADDRESS, []byte(a.Addr)) + + // TODO: There are many more attributes. + } + } + + return nil +} + +// ProcessMessage implements netlink.Protocol.ProcessMessage. +func (p *Protocol) ProcessMessage(ctx context.Context, hdr linux.NetlinkMessageHeader, data []byte, ms *netlink.MessageSet) *syserr.Error { + // All messages start with a 1 byte protocol family. + if len(data) < 1 { + // Linux ignores messages missing the protocol family. See + // net/core/rtnetlink.c:rtnetlink_rcv_msg. + return nil + } + + // Non-GET message types require CAP_NET_ADMIN. + if typeKind(hdr.Type) != kindGet { + creds := auth.CredentialsFromContext(ctx) + if !creds.HasCapability(linux.CAP_NET_ADMIN) { + return syserr.ErrPermissionDenied + } + } + + // TODO: Only the dump variant of the types below are + // supported. + if hdr.Flags&linux.NLM_F_DUMP != linux.NLM_F_DUMP { + return syserr.ErrNotSupported + } + + switch hdr.Type { + case linux.RTM_GETLINK: + return p.dumpLinks(ctx, hdr, data, ms) + case linux.RTM_GETADDR: + return p.dumpAddrs(ctx, hdr, data, ms) + default: + return syserr.ErrNotSupported + } +} + +// init registers the NETLINK_ROUTE provider. +func init() { + netlink.RegisterProvider(linux.NETLINK_ROUTE, NewProtocol) +} diff --git a/pkg/sentry/socket/netlink/socket.go b/pkg/sentry/socket/netlink/socket.go new file mode 100644 index 000000000..2d0e59ceb --- /dev/null +++ b/pkg/sentry/socket/netlink/socket.go @@ -0,0 +1,517 @@ +// Copyright 2018 Google Inc. +// +// 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 netlink provides core functionality for netlink sockets. +package netlink + +import ( + "sync" + + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/binary" + "gvisor.googlesource.com/gvisor/pkg/sentry/arch" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/device" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/kdefs" + ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket" + "gvisor.googlesource.com/gvisor/pkg/sentry/socket/netlink/port" + sunix "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/syserr" + "gvisor.googlesource.com/gvisor/pkg/syserror" + "gvisor.googlesource.com/gvisor/pkg/tcpip" + "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/unix" + "gvisor.googlesource.com/gvisor/pkg/waiter" +) + +// defaultSendBufferSize is the default size for the send buffer. +const defaultSendBufferSize = 16 * 1024 + +// netlinkSocketDevice is the netlink socket virtual device. +var netlinkSocketDevice = device.NewAnonDevice() + +// Socket is the base socket type for netlink sockets. +// +// This implementation only supports userspace sending and receiving messages +// to/from the kernel. +// +// Socket implements socket.Socket. +type Socket struct { + socket.ReceiveTimeout + fsutil.PipeSeek `state:"nosave"` + fsutil.NotDirReaddir `state:"nosave"` + fsutil.NoFsync `state:"nosave"` + fsutil.NoopFlush `state:"nosave"` + fsutil.NoMMap `state:"nosave"` + + // ports provides netlink port allocation. + ports *port.Manager + + // protocol is the netlink protocol implementation. + protocol Protocol + + // ep is a datagram unix endpoint used to buffer messages sent from the + // kernel to userspace. RecvMsg reads messages from this endpoint. + ep unix.Endpoint + + // connection is the kernel's connection to ep, used to write messages + // sent to userspace. + connection unix.ConnectedEndpoint + + // mu protects the fields below. + mu sync.Mutex `state:"nosave"` + + // bound indicates that portid is valid. + bound bool + + // portID is the port ID allocated for this socket. + portID int32 + + // sendBufferSize is the send buffer "size". We don't actually have a + // fixed buffer but only consume this many bytes. + sendBufferSize uint64 +} + +var _ socket.Socket = (*Socket)(nil) + +// NewSocket creates a new Socket. +func NewSocket(t *kernel.Task, protocol Protocol) (*Socket, *syserr.Error) { + // Datagram endpoint used to buffer kernel -> user messages. + ep := unix.NewConnectionless() + + // Bind the endpoint for good measure so we can connect to it. The + // bound address will never be exposed. + if terr := ep.Bind(tcpip.FullAddress{Addr: "dummy"}, nil); terr != nil { + ep.Close() + return nil, syserr.TranslateNetstackError(terr) + } + + // Create a connection from which the kernel can write messages. + connection, terr := ep.(unix.BoundEndpoint).UnidirectionalConnect() + if terr != nil { + ep.Close() + return nil, syserr.TranslateNetstackError(terr) + } + + return &Socket{ + ports: t.Kernel().NetlinkPorts(), + protocol: protocol, + ep: ep, + connection: connection, + sendBufferSize: defaultSendBufferSize, + }, nil +} + +// Release implements fs.FileOperations.Release. +func (s *Socket) Release() { + s.connection.Release() + s.ep.Close() + + if s.bound { + s.ports.Release(s.protocol.Protocol(), s.portID) + } +} + +// Readiness implements waiter.Waitable.Readiness. +func (s *Socket) Readiness(mask waiter.EventMask) waiter.EventMask { + // ep holds messages to be read and thus handles EventIn readiness. + ready := s.ep.Readiness(mask) + + if mask&waiter.EventOut == waiter.EventOut { + // sendMsg handles messages synchronously and is thus always + // ready for writing. + ready |= waiter.EventOut + } + + return ready +} + +// EventRegister implements waiter.Waitable.EventRegister. +func (s *Socket) EventRegister(e *waiter.Entry, mask waiter.EventMask) { + s.ep.EventRegister(e, mask) + // Writable readiness never changes, so no registration is needed. +} + +// EventUnregister implements waiter.Waitable.EventUnregister. +func (s *Socket) EventUnregister(e *waiter.Entry) { + s.ep.EventUnregister(e) +} + +// Ioctl implements fs.FileOperations.Ioctl. +func (s *Socket) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { + // TODO: no ioctls supported. + return 0, syserror.ENOTTY +} + +// ExtractSockAddr extracts the SockAddrNetlink from b. +func ExtractSockAddr(b []byte) (*linux.SockAddrNetlink, *syserr.Error) { + if len(b) < linux.SockAddrNetlinkSize { + return nil, syserr.ErrBadAddress + } + + var sa linux.SockAddrNetlink + binary.Unmarshal(b[:linux.SockAddrNetlinkSize], usermem.ByteOrder, &sa) + + if sa.Family != linux.AF_NETLINK { + return nil, syserr.ErrInvalidArgument + } + + return &sa, nil +} + +// bindPort binds this socket to a port, preferring 'port' if it is available. +// +// port of 0 defaults to the ThreadGroup ID. +// +// Preconditions: mu is held. +func (s *Socket) bindPort(t *kernel.Task, port int32) *syserr.Error { + if s.bound { + // Re-binding is only allowed if the port doesn't change. + if port != s.portID { + return syserr.ErrInvalidArgument + } + + return nil + } + + if port == 0 { + port = int32(t.ThreadGroup().ID()) + } + port, ok := s.ports.Allocate(s.protocol.Protocol(), port) + if !ok { + return syserr.ErrBusy + } + + s.portID = port + s.bound = true + return nil +} + +// Bind implements socket.Socket.Bind. +func (s *Socket) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error { + a, err := ExtractSockAddr(sockaddr) + if err != nil { + return err + } + + // No support for multicast groups yet. + if a.Groups != 0 { + return syserr.ErrPermissionDenied + } + + s.mu.Lock() + defer s.mu.Unlock() + + return s.bindPort(t, int32(a.PortID)) +} + +// Connect implements socket.Socket.Connect. +func (s *Socket) Connect(t *kernel.Task, sockaddr []byte, blocking bool) *syserr.Error { + a, err := ExtractSockAddr(sockaddr) + if err != nil { + return err + } + + // No support for multicast groups yet. + if a.Groups != 0 { + return syserr.ErrPermissionDenied + } + + s.mu.Lock() + defer s.mu.Unlock() + + if a.PortID == 0 { + // Netlink sockets default to connected to the kernel, but + // connecting anyways automatically binds if not already bound. + if !s.bound { + // Pass port 0 to get an auto-selected port ID. + return s.bindPort(t, 0) + } + return nil + } + + // We don't support non-kernel destination ports. Linux returns EPERM + // if applications attempt to do this without NL_CFG_F_NONROOT_SEND, so + // we emulate that. + return syserr.ErrPermissionDenied +} + +// Accept implements socket.Socket.Accept. +func (s *Socket) Accept(t *kernel.Task, peerRequested bool, flags int, blocking bool) (kdefs.FD, interface{}, uint32, *syserr.Error) { + // Netlink sockets never support accept. + return 0, nil, 0, syserr.ErrNotSupported +} + +// Listen implements socket.Socket.Listen. +func (s *Socket) Listen(t *kernel.Task, backlog int) *syserr.Error { + // Netlink sockets never support listen. + return syserr.ErrNotSupported +} + +// Shutdown implements socket.Socket.Shutdown. +func (s *Socket) Shutdown(t *kernel.Task, how int) *syserr.Error { + // Netlink sockets never support shutdown. + return syserr.ErrNotSupported +} + +// GetSockOpt implements socket.Socket.GetSockOpt. +func (s *Socket) GetSockOpt(t *kernel.Task, level int, name int, outLen int) (interface{}, *syserr.Error) { + // TODO: no sockopts supported. + return nil, syserr.ErrProtocolNotAvailable +} + +// SetSockOpt implements socket.Socket.SetSockOpt. +func (s *Socket) SetSockOpt(t *kernel.Task, level int, name int, opt []byte) *syserr.Error { + // TODO: no sockopts supported. + return syserr.ErrProtocolNotAvailable +} + +// GetSockName implements socket.Socket.GetSockName. +func (s *Socket) GetSockName(t *kernel.Task) (interface{}, uint32, *syserr.Error) { + s.mu.Lock() + defer s.mu.Unlock() + + sa := linux.SockAddrNetlink{ + Family: linux.AF_NETLINK, + PortID: uint32(s.portID), + } + return sa, uint32(binary.Size(sa)), nil +} + +// GetPeerName implements socket.Socket.GetPeerName. +func (s *Socket) GetPeerName(t *kernel.Task) (interface{}, uint32, *syserr.Error) { + sa := linux.SockAddrNetlink{ + Family: linux.AF_NETLINK, + // TODO: Support non-kernel peers. For now the peer + // must be the kernel. + PortID: 0, + } + return sa, uint32(binary.Size(sa)), nil +} + +// RecvMsg implements socket.Socket.RecvMsg. +func (s *Socket) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (int, interface{}, uint32, unix.ControlMessages, *syserr.Error) { + from := linux.SockAddrNetlink{ + Family: linux.AF_NETLINK, + PortID: 0, + } + fromLen := uint32(binary.Size(from)) + + trunc := flags&linux.MSG_TRUNC != 0 + + r := sunix.EndpointReader{ + Endpoint: s.ep, + Peek: flags&linux.MSG_PEEK != 0, + } + + if n, err := dst.CopyOutFrom(t, &r); err != syserror.ErrWouldBlock || flags&linux.MSG_DONTWAIT != 0 { + if trunc { + n = int64(r.MsgSize) + } + return int(n), from, fromLen, unix.ControlMessages{}, syserr.FromError(err) + } + + // We'll have to block. Register for notification and keep trying to + // receive all the data. + e, ch := waiter.NewChannelEntry(nil) + s.EventRegister(&e, waiter.EventIn) + defer s.EventUnregister(&e) + + for { + if n, err := dst.CopyOutFrom(t, &r); err != syserror.ErrWouldBlock { + if trunc { + n = int64(r.MsgSize) + } + return int(n), from, fromLen, unix.ControlMessages{}, syserr.FromError(err) + } + + if err := t.BlockWithDeadline(ch, haveDeadline, deadline); err != nil { + if err == syserror.ETIMEDOUT { + return 0, nil, 0, unix.ControlMessages{}, syserr.ErrTryAgain + } + return 0, nil, 0, unix.ControlMessages{}, syserr.FromError(err) + } + } +} + +// Read implements fs.FileOperations.Read. +func (s *Socket) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, _ int64) (int64, error) { + if dst.NumBytes() == 0 { + return 0, nil + } + return dst.CopyOutFrom(ctx, &sunix.EndpointReader{ + Endpoint: s.ep, + }) +} + +// sendResponse sends the response messages in ms back to userspace. +func (s *Socket) sendResponse(ctx context.Context, ms *MessageSet) *syserr.Error { + // Linux combines multiple netlink messages into a single datagram. + bufs := make([][]byte, 0, len(ms.Messages)) + for _, m := range ms.Messages { + bufs = append(bufs, m.Finalize()) + } + + if len(bufs) > 0 { + // RecvMsg never receives the address, so we don't need to send + // one. + _, notify, terr := s.connection.Send(bufs, unix.ControlMessages{}, tcpip.FullAddress{}) + // If the buffer is full, we simply drop messages, just like + // Linux. + if terr != nil && terr != tcpip.ErrWouldBlock { + return syserr.TranslateNetstackError(terr) + } + if notify { + s.connection.SendNotify() + } + } + + // N.B. multi-part messages should still send NLMSG_DONE even if + // MessageSet contains no messages. + // + // N.B. NLMSG_DONE is always sent in a different datagram. See + // net/netlink/af_netlink.c:netlink_dump. + if ms.Multi { + m := NewMessage(linux.NetlinkMessageHeader{ + Type: linux.NLMSG_DONE, + Flags: linux.NLM_F_MULTI, + Seq: ms.Seq, + PortID: uint32(ms.PortID), + }) + + _, notify, terr := s.connection.Send([][]byte{m.Finalize()}, unix.ControlMessages{}, tcpip.FullAddress{}) + if terr != nil && terr != tcpip.ErrWouldBlock { + return syserr.TranslateNetstackError(terr) + } + if notify { + s.connection.SendNotify() + } + } + + return nil +} + +// processMessages handles each message in buf, passing it to the protocol +// handler for final handling. +func (s *Socket) processMessages(ctx context.Context, buf []byte) *syserr.Error { + for len(buf) > 0 { + if len(buf) < linux.NetlinkMessageHeaderSize { + // Linux ignores messages that are too short. See + // net/netlink/af_netlink.c:netlink_rcv_skb. + break + } + + var hdr linux.NetlinkMessageHeader + binary.Unmarshal(buf[:linux.NetlinkMessageHeaderSize], usermem.ByteOrder, &hdr) + + if hdr.Length < linux.NetlinkMessageHeaderSize || uint64(hdr.Length) > uint64(len(buf)) { + // Linux ignores malformed messages. See + // net/netlink/af_netlink.c:netlink_rcv_skb. + break + } + + // Data from this message. + data := buf[linux.NetlinkMessageHeaderSize:hdr.Length] + + // Advance to the next message. + next := alignUp(int(hdr.Length), linux.NLMSG_ALIGNTO) + if next >= len(buf)-1 { + next = len(buf) - 1 + } + buf = buf[next:] + + // Ignore control messages. + if hdr.Type < linux.NLMSG_MIN_TYPE { + continue + } + + // TODO: ACKs not supported yet. + if hdr.Flags&linux.NLM_F_ACK == linux.NLM_F_ACK { + return syserr.ErrNotSupported + } + + ms := NewMessageSet(s.portID, hdr.Seq) + if err := s.protocol.ProcessMessage(ctx, hdr, data, ms); err != nil { + return err + } + + if err := s.sendResponse(ctx, ms); err != nil { + return err + } + } + + return nil +} + +// sendMsg is the core of message send, used for SendMsg and Write. +func (s *Socket) sendMsg(ctx context.Context, src usermem.IOSequence, to []byte, flags int, controlMessages unix.ControlMessages) (int, *syserr.Error) { + dstPort := int32(0) + + if len(to) != 0 { + a, err := ExtractSockAddr(to) + if err != nil { + return 0, err + } + + // No support for multicast groups yet. + if a.Groups != 0 { + return 0, syserr.ErrPermissionDenied + } + + dstPort = int32(a.PortID) + } + + if dstPort != 0 { + // Non-kernel destinations not supported yet. Treat as if + // NL_CFG_F_NONROOT_SEND is not set. + return 0, syserr.ErrPermissionDenied + } + + s.mu.Lock() + defer s.mu.Unlock() + + // For simplicity, and consistency with Linux, we copy in the entire + // message up front. + if uint64(src.NumBytes()) > s.sendBufferSize { + return 0, syserr.ErrMessageTooLong + } + + buf := make([]byte, src.NumBytes()) + n, err := src.CopyIn(ctx, buf) + if err != nil { + // Don't partially consume messages. + return 0, syserr.FromError(err) + } + + if err := s.processMessages(ctx, buf); err != nil { + return 0, err + } + + return n, nil +} + +// SendMsg implements socket.Socket.SendMsg. +func (s *Socket) SendMsg(t *kernel.Task, src usermem.IOSequence, to []byte, flags int, controlMessages unix.ControlMessages) (int, *syserr.Error) { + return s.sendMsg(t, src, to, flags, controlMessages) +} + +// Write implements fs.FileOperations.Write. +func (s *Socket) Write(ctx context.Context, _ *fs.File, src usermem.IOSequence, _ int64) (int64, error) { + n, err := s.sendMsg(ctx, src, nil, 0, unix.ControlMessages{}) + return int64(n), err.ToError() +} |