summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/socket/netlink
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry/socket/netlink')
-rw-r--r--pkg/sentry/socket/netlink/BUILD47
-rw-r--r--pkg/sentry/socket/netlink/message.go159
-rw-r--r--pkg/sentry/socket/netlink/port/BUILD28
-rw-r--r--pkg/sentry/socket/netlink/port/port.go114
-rw-r--r--pkg/sentry/socket/netlink/port/port_test.go82
-rw-r--r--pkg/sentry/socket/netlink/provider.go104
-rw-r--r--pkg/sentry/socket/netlink/route/BUILD33
-rw-r--r--pkg/sentry/socket/netlink/route/protocol.go189
-rw-r--r--pkg/sentry/socket/netlink/socket.go517
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()
+}