summaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
authorgVisor bot <gvisor-bot@google.com>2020-02-20 23:23:04 +0000
committergVisor bot <gvisor-bot@google.com>2020-02-20 23:23:04 +0000
commitf7f6b5ae0fad1f5bfd050b3b290e63efc83d7a28 (patch)
treedc72bfda207b99be7ee6ab527728d67e524f1552
parentb1b3ef0213be56f5c3636b6399700ac313203f53 (diff)
parent4a73bae269ae9f52a962ae3b08a17ccaacf7ba80 (diff)
Merge release-20200211.0-57-g4a73bae (automated)
-rw-r--r--pkg/sentry/fs/proc/net.go5
-rw-r--r--pkg/sentry/fs/proc/sys_net.go4
-rwxr-xr-xpkg/sentry/inet/inet_state_autogen.go14
-rwxr-xr-xpkg/sentry/inet/namespace.go99
-rw-r--r--pkg/sentry/kernel/kernel.go26
-rwxr-xr-xpkg/sentry/kernel/kernel_state_autogen.go2
-rw-r--r--pkg/sentry/kernel/task.go9
-rw-r--r--pkg/sentry/kernel/task_clone.go16
-rw-r--r--pkg/sentry/kernel/task_net.go19
-rw-r--r--pkg/sentry/kernel/task_start.go8
-rwxr-xr-xpkg/tcpip/tcpip_state_autogen.go10
-rw-r--r--pkg/tcpip/time_unsafe.go2
-rwxr-xr-xrunsc/boot/boot_state_autogen.go21
-rw-r--r--runsc/boot/controller.go11
-rw-r--r--runsc/boot/loader.go121
-rw-r--r--runsc/boot/network.go27
-rwxr-xr-x[-rw-r--r--]runsc/boot/pprof/pprof.go (renamed from runsc/boot/pprof.go)6
-rwxr-xr-xrunsc/boot/pprof/pprof_state_autogen.go3
-rw-r--r--runsc/sandbox/network.go25
19 files changed, 324 insertions, 104 deletions
diff --git a/pkg/sentry/fs/proc/net.go b/pkg/sentry/fs/proc/net.go
index 6f2775344..95d5817ff 100644
--- a/pkg/sentry/fs/proc/net.go
+++ b/pkg/sentry/fs/proc/net.go
@@ -43,7 +43,10 @@ import (
// newNet creates a new proc net entry.
func (p *proc) newNetDir(ctx context.Context, k *kernel.Kernel, msrc *fs.MountSource) *fs.Inode {
var contents map[string]*fs.Inode
- if s := p.k.NetworkStack(); s != nil {
+ // TODO(gvisor.dev/issue/1833): Support for using the network stack in the
+ // network namespace of the calling process. We should make this per-process,
+ // a.k.a. /proc/PID/net, and make /proc/net a symlink to /proc/self/net.
+ if s := p.k.RootNetworkNamespace().Stack(); s != nil {
contents = map[string]*fs.Inode{
"dev": seqfile.NewSeqFileInode(ctx, &netDev{s: s}, msrc),
"snmp": seqfile.NewSeqFileInode(ctx, &netSnmp{s: s}, msrc),
diff --git a/pkg/sentry/fs/proc/sys_net.go b/pkg/sentry/fs/proc/sys_net.go
index 0772d4ae4..d4c4b533d 100644
--- a/pkg/sentry/fs/proc/sys_net.go
+++ b/pkg/sentry/fs/proc/sys_net.go
@@ -357,7 +357,9 @@ func (p *proc) newSysNetIPv4Dir(ctx context.Context, msrc *fs.MountSource, s ine
func (p *proc) newSysNetDir(ctx context.Context, msrc *fs.MountSource) *fs.Inode {
var contents map[string]*fs.Inode
- if s := p.k.NetworkStack(); s != nil {
+ // TODO(gvisor.dev/issue/1833): Support for using the network stack in the
+ // network namespace of the calling process.
+ if s := p.k.RootNetworkNamespace().Stack(); s != nil {
contents = map[string]*fs.Inode{
"ipv4": p.newSysNetIPv4Dir(ctx, msrc, s),
"core": p.newSysNetCore(ctx, msrc, s),
diff --git a/pkg/sentry/inet/inet_state_autogen.go b/pkg/sentry/inet/inet_state_autogen.go
index dc2f459e5..97047d993 100755
--- a/pkg/sentry/inet/inet_state_autogen.go
+++ b/pkg/sentry/inet/inet_state_autogen.go
@@ -21,6 +21,20 @@ func (x *TCPBufferSize) load(m state.Map) {
m.Load("Max", &x.Max)
}
+func (x *Namespace) beforeSave() {}
+func (x *Namespace) save(m state.Map) {
+ x.beforeSave()
+ m.Save("creator", &x.creator)
+ m.Save("isRoot", &x.isRoot)
+}
+
+func (x *Namespace) load(m state.Map) {
+ m.Load("creator", &x.creator)
+ m.Load("isRoot", &x.isRoot)
+ m.AfterLoad(x.afterLoad)
+}
+
func init() {
state.Register("pkg/sentry/inet.TCPBufferSize", (*TCPBufferSize)(nil), state.Fns{Save: (*TCPBufferSize).save, Load: (*TCPBufferSize).load})
+ state.Register("pkg/sentry/inet.Namespace", (*Namespace)(nil), state.Fns{Save: (*Namespace).save, Load: (*Namespace).load})
}
diff --git a/pkg/sentry/inet/namespace.go b/pkg/sentry/inet/namespace.go
new file mode 100755
index 000000000..c16667e7f
--- /dev/null
+++ b/pkg/sentry/inet/namespace.go
@@ -0,0 +1,99 @@
+// 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 inet
+
+// Namespace represents a network namespace. See network_namespaces(7).
+//
+// +stateify savable
+type Namespace struct {
+ // stack is the network stack implementation of this network namespace.
+ stack Stack `state:"nosave"`
+
+ // creator allows kernel to create new network stack for network namespaces.
+ // If nil, no networking will function if network is namespaced.
+ creator NetworkStackCreator
+
+ // isRoot indicates whether this is the root network namespace.
+ isRoot bool
+}
+
+// NewRootNamespace creates the root network namespace, with creator
+// allowing new network namespaces to be created. If creator is nil, no
+// networking will function if the network is namespaced.
+func NewRootNamespace(stack Stack, creator NetworkStackCreator) *Namespace {
+ return &Namespace{
+ stack: stack,
+ creator: creator,
+ isRoot: true,
+ }
+}
+
+// NewNamespace creates a new network namespace from the root.
+func NewNamespace(root *Namespace) *Namespace {
+ n := &Namespace{
+ creator: root.creator,
+ }
+ n.init()
+ return n
+}
+
+// Stack returns the network stack of n. Stack may return nil if no network
+// stack is configured.
+func (n *Namespace) Stack() Stack {
+ return n.stack
+}
+
+// IsRoot returns whether n is the root network namespace.
+func (n *Namespace) IsRoot() bool {
+ return n.isRoot
+}
+
+// RestoreRootStack restores the root network namespace with stack. This should
+// only be called when restoring kernel.
+func (n *Namespace) RestoreRootStack(stack Stack) {
+ if !n.isRoot {
+ panic("RestoreRootStack can only be called on root network namespace")
+ }
+ if n.stack != nil {
+ panic("RestoreRootStack called after a stack has already been set")
+ }
+ n.stack = stack
+}
+
+func (n *Namespace) init() {
+ // Root network namespace will have stack assigned later.
+ if n.isRoot {
+ return
+ }
+ if n.creator != nil {
+ var err error
+ n.stack, err = n.creator.CreateStack()
+ if err != nil {
+ panic(err)
+ }
+ }
+}
+
+// afterLoad is invoked by stateify.
+func (n *Namespace) afterLoad() {
+ n.init()
+}
+
+// NetworkStackCreator allows new instances of a network stack to be created. It
+// is used by the kernel to create new network namespaces when requested.
+type NetworkStackCreator interface {
+ // CreateStack creates a new network stack for a network namespace.
+ CreateStack() (Stack, error)
+}
diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go
index 7da0368f1..c62fd6eb1 100644
--- a/pkg/sentry/kernel/kernel.go
+++ b/pkg/sentry/kernel/kernel.go
@@ -111,7 +111,7 @@ type Kernel struct {
timekeeper *Timekeeper
tasks *TaskSet
rootUserNamespace *auth.UserNamespace
- networkStack inet.Stack `state:"nosave"`
+ rootNetworkNamespace *inet.Namespace
applicationCores uint
useHostCores bool
extraAuxv []arch.AuxEntry
@@ -260,8 +260,9 @@ type InitKernelArgs struct {
// RootUserNamespace is the root user namespace.
RootUserNamespace *auth.UserNamespace
- // NetworkStack is the TCP/IP network stack. NetworkStack may be nil.
- NetworkStack inet.Stack
+ // RootNetworkNamespace is the root network namespace. If nil, no networking
+ // will be available.
+ RootNetworkNamespace *inet.Namespace
// ApplicationCores is the number of logical CPUs visible to sandboxed
// applications. The set of logical CPU IDs is [0, ApplicationCores); thus
@@ -320,7 +321,10 @@ func (k *Kernel) Init(args InitKernelArgs) error {
k.rootUTSNamespace = args.RootUTSNamespace
k.rootIPCNamespace = args.RootIPCNamespace
k.rootAbstractSocketNamespace = args.RootAbstractSocketNamespace
- k.networkStack = args.NetworkStack
+ k.rootNetworkNamespace = args.RootNetworkNamespace
+ if k.rootNetworkNamespace == nil {
+ k.rootNetworkNamespace = inet.NewRootNamespace(nil, nil)
+ }
k.applicationCores = args.ApplicationCores
if args.UseHostCores {
k.useHostCores = true
@@ -543,8 +547,6 @@ func (ts *TaskSet) unregisterEpollWaiters() {
func (k *Kernel) LoadFrom(r io.Reader, net inet.Stack, clocks sentrytime.Clocks) error {
loadStart := time.Now()
- k.networkStack = net
-
initAppCores := k.applicationCores
// Load the pre-saved CPUID FeatureSet.
@@ -575,6 +577,10 @@ func (k *Kernel) LoadFrom(r io.Reader, net inet.Stack, clocks sentrytime.Clocks)
log.Infof("Kernel load stats: %s", &stats)
log.Infof("Kernel load took [%s].", time.Since(kernelStart))
+ // rootNetworkNamespace should be populated after loading the state file.
+ // Restore the root network stack.
+ k.rootNetworkNamespace.RestoreRootStack(net)
+
// Load the memory file's state.
memoryStart := time.Now()
if err := k.mf.LoadFrom(k.SupervisorContext(), r); err != nil {
@@ -905,6 +911,7 @@ func (k *Kernel) CreateProcess(args CreateProcessArgs) (*ThreadGroup, ThreadID,
FSContext: fsContext,
FDTable: args.FDTable,
Credentials: args.Credentials,
+ NetworkNamespace: k.RootNetworkNamespace(),
AllowedCPUMask: sched.NewFullCPUSet(k.applicationCores),
UTSNamespace: args.UTSNamespace,
IPCNamespace: args.IPCNamespace,
@@ -1255,10 +1262,9 @@ func (k *Kernel) RootAbstractSocketNamespace() *AbstractSocketNamespace {
return k.rootAbstractSocketNamespace
}
-// NetworkStack returns the network stack. NetworkStack may return nil if no
-// network stack is available.
-func (k *Kernel) NetworkStack() inet.Stack {
- return k.networkStack
+// RootNetworkNamespace returns the root network namespace, always non-nil.
+func (k *Kernel) RootNetworkNamespace() *inet.Namespace {
+ return k.rootNetworkNamespace
}
// GlobalInit returns the thread group with ID 1 in the root PID namespace, or
diff --git a/pkg/sentry/kernel/kernel_state_autogen.go b/pkg/sentry/kernel/kernel_state_autogen.go
index 8948ca886..deae6d118 100755
--- a/pkg/sentry/kernel/kernel_state_autogen.go
+++ b/pkg/sentry/kernel/kernel_state_autogen.go
@@ -132,6 +132,7 @@ func (x *Kernel) save(m state.Map) {
m.Save("timekeeper", &x.timekeeper)
m.Save("tasks", &x.tasks)
m.Save("rootUserNamespace", &x.rootUserNamespace)
+ m.Save("rootNetworkNamespace", &x.rootNetworkNamespace)
m.Save("applicationCores", &x.applicationCores)
m.Save("useHostCores", &x.useHostCores)
m.Save("extraAuxv", &x.extraAuxv)
@@ -165,6 +166,7 @@ func (x *Kernel) load(m state.Map) {
m.Load("timekeeper", &x.timekeeper)
m.Load("tasks", &x.tasks)
m.Load("rootUserNamespace", &x.rootUserNamespace)
+ m.Load("rootNetworkNamespace", &x.rootNetworkNamespace)
m.Load("applicationCores", &x.applicationCores)
m.Load("useHostCores", &x.useHostCores)
m.Load("extraAuxv", &x.extraAuxv)
diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go
index a3443ff21..e37e23231 100644
--- a/pkg/sentry/kernel/task.go
+++ b/pkg/sentry/kernel/task.go
@@ -486,13 +486,10 @@ type Task struct {
numaPolicy int32
numaNodeMask uint64
- // If netns is true, the task is in a non-root network namespace. Network
- // namespaces aren't currently implemented in full; being in a network
- // namespace simply prevents the task from observing any network devices
- // (including loopback) or using abstract socket addresses (see unix(7)).
+ // netns is the task's network namespace. netns is never nil.
//
- // netns is protected by mu. netns is owned by the task goroutine.
- netns bool
+ // netns is protected by mu.
+ netns *inet.Namespace
// If rseqPreempted is true, before the next call to p.Switch(),
// interrupt rseq critical regions as defined by rseqAddr and
diff --git a/pkg/sentry/kernel/task_clone.go b/pkg/sentry/kernel/task_clone.go
index ba74b4c1c..78866f280 100644
--- a/pkg/sentry/kernel/task_clone.go
+++ b/pkg/sentry/kernel/task_clone.go
@@ -17,6 +17,7 @@ package kernel
import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/bpf"
+ "gvisor.dev/gvisor/pkg/sentry/inet"
"gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/usermem"
)
@@ -54,8 +55,7 @@ type SharingOptions struct {
NewUserNamespace bool
// If NewNetworkNamespace is true, the task should have an independent
- // network namespace. (Note that network namespaces are not really
- // implemented; see comment on Task.netns for details.)
+ // network namespace.
NewNetworkNamespace bool
// If NewFiles is true, the task should use an independent file descriptor
@@ -199,6 +199,11 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
ipcns = NewIPCNamespace(userns)
}
+ netns := t.NetworkNamespace()
+ if opts.NewNetworkNamespace {
+ netns = inet.NewNamespace(netns)
+ }
+
// TODO(b/63601033): Implement CLONE_NEWNS.
mntnsVFS2 := t.mountNamespaceVFS2
if mntnsVFS2 != nil {
@@ -268,7 +273,7 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
FDTable: fdTable,
Credentials: creds,
Niceness: t.Niceness(),
- NetworkNamespaced: t.netns,
+ NetworkNamespace: netns,
AllowedCPUMask: t.CPUMask(),
UTSNamespace: utsns,
IPCNamespace: ipcns,
@@ -283,9 +288,6 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
} else {
cfg.InheritParent = t
}
- if opts.NewNetworkNamespace {
- cfg.NetworkNamespaced = true
- }
nt, err := t.tg.pidns.owner.NewTask(cfg)
if err != nil {
if opts.NewThreadGroup {
@@ -482,7 +484,7 @@ func (t *Task) Unshare(opts *SharingOptions) error {
t.mu.Unlock()
return syserror.EPERM
}
- t.netns = true
+ t.netns = inet.NewNamespace(t.netns)
}
if opts.NewUTSNamespace {
if !haveCapSysAdmin {
diff --git a/pkg/sentry/kernel/task_net.go b/pkg/sentry/kernel/task_net.go
index 172a31e1d..f7711232c 100644
--- a/pkg/sentry/kernel/task_net.go
+++ b/pkg/sentry/kernel/task_net.go
@@ -22,14 +22,23 @@ import (
func (t *Task) IsNetworkNamespaced() bool {
t.mu.Lock()
defer t.mu.Unlock()
- return t.netns
+ return !t.netns.IsRoot()
}
// NetworkContext returns the network stack used by the task. NetworkContext
// may return nil if no network stack is available.
+//
+// TODO(gvisor.dev/issue/1833): Migrate callers of this method to
+// NetworkNamespace().
func (t *Task) NetworkContext() inet.Stack {
- if t.IsNetworkNamespaced() {
- return nil
- }
- return t.k.networkStack
+ t.mu.Lock()
+ defer t.mu.Unlock()
+ return t.netns.Stack()
+}
+
+// NetworkNamespace returns the network namespace observed by the task.
+func (t *Task) NetworkNamespace() *inet.Namespace {
+ t.mu.Lock()
+ defer t.mu.Unlock()
+ return t.netns
}
diff --git a/pkg/sentry/kernel/task_start.go b/pkg/sentry/kernel/task_start.go
index f9236a842..a5035bb7f 100644
--- a/pkg/sentry/kernel/task_start.go
+++ b/pkg/sentry/kernel/task_start.go
@@ -17,6 +17,7 @@ package kernel
import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/sentry/arch"
+ "gvisor.dev/gvisor/pkg/sentry/inet"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/kernel/futex"
"gvisor.dev/gvisor/pkg/sentry/kernel/sched"
@@ -65,9 +66,8 @@ type TaskConfig struct {
// Niceness is the niceness of the new task.
Niceness int
- // If NetworkNamespaced is true, the new task should observe a non-root
- // network namespace.
- NetworkNamespaced bool
+ // NetworkNamespace is the network namespace to be used for the new task.
+ NetworkNamespace *inet.Namespace
// AllowedCPUMask contains the cpus that this task can run on.
AllowedCPUMask sched.CPUSet
@@ -133,7 +133,7 @@ func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) {
allowedCPUMask: cfg.AllowedCPUMask.Copy(),
ioUsage: &usage.IO{},
niceness: cfg.Niceness,
- netns: cfg.NetworkNamespaced,
+ netns: cfg.NetworkNamespace,
utsns: cfg.UTSNamespace,
ipcns: cfg.IPCNamespace,
abstractSockets: cfg.AbstractSocketNamespace,
diff --git a/pkg/tcpip/tcpip_state_autogen.go b/pkg/tcpip/tcpip_state_autogen.go
index 272b21d87..6753503f0 100755
--- a/pkg/tcpip/tcpip_state_autogen.go
+++ b/pkg/tcpip/tcpip_state_autogen.go
@@ -90,9 +90,19 @@ func (x *IPPacketInfo) load(m state.Map) {
m.Load("DestinationAddr", &x.DestinationAddr)
}
+func (x *StdClock) beforeSave() {}
+func (x *StdClock) save(m state.Map) {
+ x.beforeSave()
+}
+
+func (x *StdClock) afterLoad() {}
+func (x *StdClock) load(m state.Map) {
+}
+
func init() {
state.Register("pkg/tcpip.PacketBuffer", (*PacketBuffer)(nil), state.Fns{Save: (*PacketBuffer).save, Load: (*PacketBuffer).load})
state.Register("pkg/tcpip.FullAddress", (*FullAddress)(nil), state.Fns{Save: (*FullAddress).save, Load: (*FullAddress).load})
state.Register("pkg/tcpip.ControlMessages", (*ControlMessages)(nil), state.Fns{Save: (*ControlMessages).save, Load: (*ControlMessages).load})
state.Register("pkg/tcpip.IPPacketInfo", (*IPPacketInfo)(nil), state.Fns{Save: (*IPPacketInfo).save, Load: (*IPPacketInfo).load})
+ state.Register("pkg/tcpip.StdClock", (*StdClock)(nil), state.Fns{Save: (*StdClock).save, Load: (*StdClock).load})
}
diff --git a/pkg/tcpip/time_unsafe.go b/pkg/tcpip/time_unsafe.go
index 48764b978..2f98a996f 100644
--- a/pkg/tcpip/time_unsafe.go
+++ b/pkg/tcpip/time_unsafe.go
@@ -25,6 +25,8 @@ import (
)
// StdClock implements Clock with the time package.
+//
+// +stateify savable
type StdClock struct{}
var _ Clock = (*StdClock)(nil)
diff --git a/runsc/boot/boot_state_autogen.go b/runsc/boot/boot_state_autogen.go
index 23dd4b7b3..167d1cf02 100755
--- a/runsc/boot/boot_state_autogen.go
+++ b/runsc/boot/boot_state_autogen.go
@@ -1,3 +1,24 @@
// automatically generated by stateify.
package boot
+
+import (
+ "gvisor.dev/gvisor/pkg/state"
+)
+
+func (x *sandboxNetstackCreator) beforeSave() {}
+func (x *sandboxNetstackCreator) save(m state.Map) {
+ x.beforeSave()
+ m.Save("clock", &x.clock)
+ m.Save("uniqueID", &x.uniqueID)
+}
+
+func (x *sandboxNetstackCreator) afterLoad() {}
+func (x *sandboxNetstackCreator) load(m state.Map) {
+ m.Load("clock", &x.clock)
+ m.Load("uniqueID", &x.uniqueID)
+}
+
+func init() {
+ state.Register("runsc/boot.sandboxNetstackCreator", (*sandboxNetstackCreator)(nil), state.Fns{Save: (*sandboxNetstackCreator).save, Load: (*sandboxNetstackCreator).load})
+}
diff --git a/runsc/boot/controller.go b/runsc/boot/controller.go
index 9c9e94864..17e774e0c 100644
--- a/runsc/boot/controller.go
+++ b/runsc/boot/controller.go
@@ -32,6 +32,7 @@ import (
"gvisor.dev/gvisor/pkg/sentry/watchdog"
"gvisor.dev/gvisor/pkg/tcpip/stack"
"gvisor.dev/gvisor/pkg/urpc"
+ "gvisor.dev/gvisor/runsc/boot/pprof"
"gvisor.dev/gvisor/runsc/specutils"
)
@@ -142,7 +143,7 @@ func newController(fd int, l *Loader) (*controller, error) {
}
srv.Register(manager)
- if eps, ok := l.k.NetworkStack().(*netstack.Stack); ok {
+ if eps, ok := l.k.RootNetworkNamespace().Stack().(*netstack.Stack); ok {
net := &Network{
Stack: eps.Stack,
}
@@ -341,7 +342,7 @@ func (cm *containerManager) Restore(o *RestoreOpts, _ *struct{}) error {
return fmt.Errorf("creating memory file: %v", err)
}
k.SetMemoryFile(mf)
- networkStack := cm.l.k.NetworkStack()
+ networkStack := cm.l.k.RootNetworkNamespace().Stack()
cm.l.k = k
// Set up the restore environment.
@@ -365,9 +366,9 @@ func (cm *containerManager) Restore(o *RestoreOpts, _ *struct{}) error {
}
if cm.l.conf.ProfileEnable {
- // initializePProf opens /proc/self/maps, so has to be
- // called before installing seccomp filters.
- initializePProf()
+ // pprof.Initialize opens /proc/self/maps, so has to be called before
+ // installing seccomp filters.
+ pprof.Initialize()
}
// Seccomp filters have to be applied before parsing the state file.
diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go
index eef43b9df..e7ca98134 100644
--- a/runsc/boot/loader.go
+++ b/runsc/boot/loader.go
@@ -49,6 +49,7 @@ import (
"gvisor.dev/gvisor/pkg/sentry/watchdog"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/tcpip"
+ "gvisor.dev/gvisor/pkg/tcpip/link/loopback"
"gvisor.dev/gvisor/pkg/tcpip/link/sniffer"
"gvisor.dev/gvisor/pkg/tcpip/network/arp"
"gvisor.dev/gvisor/pkg/tcpip/network/ipv4"
@@ -60,6 +61,7 @@ import (
"gvisor.dev/gvisor/pkg/tcpip/transport/udp"
"gvisor.dev/gvisor/runsc/boot/filter"
_ "gvisor.dev/gvisor/runsc/boot/platforms" // register all platforms.
+ "gvisor.dev/gvisor/runsc/boot/pprof"
"gvisor.dev/gvisor/runsc/specutils"
// Include supported socket providers.
@@ -230,11 +232,8 @@ func New(args Args) (*Loader, error) {
return nil, fmt.Errorf("enabling strace: %v", err)
}
- // Create an empty network stack because the network namespace may be empty at
- // this point. Netns is configured before Run() is called. Netstack is
- // configured using a control uRPC message. Host network is configured inside
- // Run().
- networkStack, err := newEmptyNetworkStack(args.Conf, k, k)
+ // Create root network namespace/stack.
+ netns, err := newRootNetworkNamespace(args.Conf, k, k)
if err != nil {
return nil, fmt.Errorf("creating network: %v", err)
}
@@ -277,7 +276,7 @@ func New(args Args) (*Loader, error) {
FeatureSet: cpuid.HostFeatureSet(),
Timekeeper: tk,
RootUserNamespace: creds.UserNamespace,
- NetworkStack: networkStack,
+ RootNetworkNamespace: netns,
ApplicationCores: uint(args.NumCPU),
Vdso: vdso,
RootUTSNamespace: kernel.NewUTSNamespace(args.Spec.Hostname, args.Spec.Hostname, creds.UserNamespace),
@@ -466,7 +465,7 @@ func (l *Loader) run() error {
// Delay host network configuration to this point because network namespace
// is configured after the loader is created and before Run() is called.
log.Debugf("Configuring host network")
- stack := l.k.NetworkStack().(*hostinet.Stack)
+ stack := l.k.RootNetworkNamespace().Stack().(*hostinet.Stack)
if err := stack.Configure(); err != nil {
return err
}
@@ -485,7 +484,7 @@ func (l *Loader) run() error {
// l.restore is set by the container manager when a restore call is made.
if !l.restore {
if l.conf.ProfileEnable {
- initializePProf()
+ pprof.Initialize()
}
// Finally done with all configuration. Setup filters before user code
@@ -908,48 +907,92 @@ func (l *Loader) WaitExit() kernel.ExitStatus {
return l.k.GlobalInit().ExitStatus()
}
-func newEmptyNetworkStack(conf *Config, clock tcpip.Clock, uniqueID stack.UniqueID) (inet.Stack, error) {
+func newRootNetworkNamespace(conf *Config, clock tcpip.Clock, uniqueID stack.UniqueID) (*inet.Namespace, error) {
+ // Create an empty network stack because the network namespace may be empty at
+ // this point. Netns is configured before Run() is called. Netstack is
+ // configured using a control uRPC message. Host network is configured inside
+ // Run().
switch conf.Network {
case NetworkHost:
- return hostinet.NewStack(), nil
+ // No network namespacing support for hostinet yet, hence creator is nil.
+ return inet.NewRootNamespace(hostinet.NewStack(), nil), nil
case NetworkNone, NetworkSandbox:
- // NetworkNone sets up loopback using netstack.
- netProtos := []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol(), arp.NewProtocol()}
- transProtos := []stack.TransportProtocol{tcp.NewProtocol(), udp.NewProtocol(), icmp.NewProtocol4()}
- s := netstack.Stack{stack.New(stack.Options{
- NetworkProtocols: netProtos,
- TransportProtocols: transProtos,
- Clock: clock,
- Stats: netstack.Metrics,
- HandleLocal: true,
- // Enable raw sockets for users with sufficient
- // privileges.
- RawFactory: raw.EndpointFactory{},
- UniqueID: uniqueID,
- })}
-
- // Enable SACK Recovery.
- if err := s.Stack.SetTransportProtocolOption(tcp.ProtocolNumber, tcp.SACKEnabled(true)); err != nil {
- return nil, fmt.Errorf("failed to enable SACK: %v", err)
+ s, err := newEmptySandboxNetworkStack(clock, uniqueID)
+ if err != nil {
+ return nil, err
}
+ creator := &sandboxNetstackCreator{
+ clock: clock,
+ uniqueID: uniqueID,
+ }
+ return inet.NewRootNamespace(s, creator), nil
- // Set default TTLs as required by socket/netstack.
- s.Stack.SetNetworkProtocolOption(ipv4.ProtocolNumber, tcpip.DefaultTTLOption(netstack.DefaultTTL))
- s.Stack.SetNetworkProtocolOption(ipv6.ProtocolNumber, tcpip.DefaultTTLOption(netstack.DefaultTTL))
+ default:
+ panic(fmt.Sprintf("invalid network configuration: %v", conf.Network))
+ }
- // Enable Receive Buffer Auto-Tuning.
- if err := s.Stack.SetTransportProtocolOption(tcp.ProtocolNumber, tcpip.ModerateReceiveBufferOption(true)); err != nil {
- return nil, fmt.Errorf("SetTransportProtocolOption failed: %v", err)
- }
+}
- s.FillDefaultIPTables()
+func newEmptySandboxNetworkStack(clock tcpip.Clock, uniqueID stack.UniqueID) (inet.Stack, error) {
+ netProtos := []stack.NetworkProtocol{ipv4.NewProtocol(), ipv6.NewProtocol(), arp.NewProtocol()}
+ transProtos := []stack.TransportProtocol{tcp.NewProtocol(), udp.NewProtocol(), icmp.NewProtocol4()}
+ s := netstack.Stack{stack.New(stack.Options{
+ NetworkProtocols: netProtos,
+ TransportProtocols: transProtos,
+ Clock: clock,
+ Stats: netstack.Metrics,
+ HandleLocal: true,
+ // Enable raw sockets for users with sufficient
+ // privileges.
+ RawFactory: raw.EndpointFactory{},
+ UniqueID: uniqueID,
+ })}
- return &s, nil
+ // Enable SACK Recovery.
+ if err := s.Stack.SetTransportProtocolOption(tcp.ProtocolNumber, tcp.SACKEnabled(true)); err != nil {
+ return nil, fmt.Errorf("failed to enable SACK: %v", err)
+ }
- default:
- panic(fmt.Sprintf("invalid network configuration: %v", conf.Network))
+ // Set default TTLs as required by socket/netstack.
+ s.Stack.SetNetworkProtocolOption(ipv4.ProtocolNumber, tcpip.DefaultTTLOption(netstack.DefaultTTL))
+ s.Stack.SetNetworkProtocolOption(ipv6.ProtocolNumber, tcpip.DefaultTTLOption(netstack.DefaultTTL))
+
+ // Enable Receive Buffer Auto-Tuning.
+ if err := s.Stack.SetTransportProtocolOption(tcp.ProtocolNumber, tcpip.ModerateReceiveBufferOption(true)); err != nil {
+ return nil, fmt.Errorf("SetTransportProtocolOption failed: %v", err)
+ }
+
+ s.FillDefaultIPTables()
+
+ return &s, nil
+}
+
+// sandboxNetstackCreator implements kernel.NetworkStackCreator.
+//
+// +stateify savable
+type sandboxNetstackCreator struct {
+ clock tcpip.Clock
+ uniqueID stack.UniqueID
+}
+
+// CreateStack implements kernel.NetworkStackCreator.CreateStack.
+func (f *sandboxNetstackCreator) CreateStack() (inet.Stack, error) {
+ s, err := newEmptySandboxNetworkStack(f.clock, f.uniqueID)
+ if err != nil {
+ return nil, err
}
+
+ // Setup loopback.
+ n := &Network{Stack: s.(*netstack.Stack).Stack}
+ nicID := tcpip.NICID(f.uniqueID.UniqueID())
+ link := DefaultLoopbackLink
+ linkEP := loopback.New()
+ if err := n.createNICWithAddrs(nicID, link.Name, linkEP, link.Addresses); err != nil {
+ return nil, err
+ }
+
+ return s, nil
}
// signal sends a signal to one or more processes in a container. If PID is 0,
diff --git a/runsc/boot/network.go b/runsc/boot/network.go
index 6a8765ec8..bee6ee336 100644
--- a/runsc/boot/network.go
+++ b/runsc/boot/network.go
@@ -17,6 +17,7 @@ package boot
import (
"fmt"
"net"
+ "strings"
"syscall"
"gvisor.dev/gvisor/pkg/log"
@@ -31,6 +32,32 @@ import (
"gvisor.dev/gvisor/pkg/urpc"
)
+var (
+ // DefaultLoopbackLink contains IP addresses and routes of "127.0.0.1/8" and
+ // "::1/8" on "lo" interface.
+ DefaultLoopbackLink = LoopbackLink{
+ Name: "lo",
+ Addresses: []net.IP{
+ net.IP("\x7f\x00\x00\x01"),
+ net.IPv6loopback,
+ },
+ Routes: []Route{
+ {
+ Destination: net.IPNet{
+ IP: net.IPv4(0x7f, 0, 0, 0),
+ Mask: net.IPv4Mask(0xff, 0, 0, 0),
+ },
+ },
+ {
+ Destination: net.IPNet{
+ IP: net.IPv6loopback,
+ Mask: net.IPMask(strings.Repeat("\xff", net.IPv6len)),
+ },
+ },
+ },
+ }
+)
+
// Network exposes methods that can be used to configure a network stack.
type Network struct {
Stack *stack.Stack
diff --git a/runsc/boot/pprof.go b/runsc/boot/pprof/pprof.go
index 463362f02..1ded20dee 100644..100755
--- a/runsc/boot/pprof.go
+++ b/runsc/boot/pprof/pprof.go
@@ -12,7 +12,9 @@
// See the License for the specific language governing permissions and
// limitations under the License.
-package boot
+// Package pprof provides a stub to initialize custom profilers.
+package pprof
-func initializePProf() {
+// Initialize will be called at boot for initializing custom profilers.
+func Initialize() {
}
diff --git a/runsc/boot/pprof/pprof_state_autogen.go b/runsc/boot/pprof/pprof_state_autogen.go
new file mode 100755
index 000000000..cabd43173
--- /dev/null
+++ b/runsc/boot/pprof/pprof_state_autogen.go
@@ -0,0 +1,3 @@
+// automatically generated by stateify.
+
+package pprof
diff --git a/runsc/sandbox/network.go b/runsc/sandbox/network.go
index 99e143696..bc093fba5 100644
--- a/runsc/sandbox/network.go
+++ b/runsc/sandbox/network.go
@@ -21,7 +21,6 @@ import (
"path/filepath"
"runtime"
"strconv"
- "strings"
"syscall"
specs "github.com/opencontainers/runtime-spec/specs-go"
@@ -75,30 +74,8 @@ func setupNetwork(conn *urpc.Client, pid int, spec *specs.Spec, conf *boot.Confi
}
func createDefaultLoopbackInterface(conn *urpc.Client) error {
- link := boot.LoopbackLink{
- Name: "lo",
- Addresses: []net.IP{
- net.IP("\x7f\x00\x00\x01"),
- net.IPv6loopback,
- },
- Routes: []boot.Route{
- {
- Destination: net.IPNet{
-
- IP: net.IPv4(0x7f, 0, 0, 0),
- Mask: net.IPv4Mask(0xff, 0, 0, 0),
- },
- },
- {
- Destination: net.IPNet{
- IP: net.IPv6loopback,
- Mask: net.IPMask(strings.Repeat("\xff", net.IPv6len)),
- },
- },
- },
- }
if err := conn.Call(boot.NetworkCreateLinksAndRoutes, &boot.CreateLinksAndRoutesArgs{
- LoopbackLinks: []boot.LoopbackLink{link},
+ LoopbackLinks: []boot.LoopbackLink{boot.DefaultLoopbackLink},
}, nil); err != nil {
return fmt.Errorf("creating loopback link and routes: %v", err)
}