summaryrefslogtreecommitdiffhomepage
path: root/runsc/boot
diff options
context:
space:
mode:
Diffstat (limited to 'runsc/boot')
-rw-r--r--runsc/boot/BUILD88
-rw-r--r--runsc/boot/capability.go120
-rw-r--r--runsc/boot/config.go162
-rw-r--r--runsc/boot/controller.go128
-rw-r--r--runsc/boot/events.go81
-rw-r--r--runsc/boot/fds.go61
-rw-r--r--runsc/boot/filter/BUILD26
-rw-r--r--runsc/boot/filter/config.go175
-rw-r--r--runsc/boot/filter/extra_filters.go24
-rw-r--r--runsc/boot/filter/extra_filters_msan.go30
-rw-r--r--runsc/boot/filter/extra_filters_race.go33
-rw-r--r--runsc/boot/filter/filter.go67
-rw-r--r--runsc/boot/fs.go441
-rw-r--r--runsc/boot/limits.go60
-rw-r--r--runsc/boot/loader.go354
-rw-r--r--runsc/boot/loader_test.go238
-rw-r--r--runsc/boot/network.go213
-rw-r--r--runsc/boot/strace.go40
18 files changed, 2341 insertions, 0 deletions
diff --git a/runsc/boot/BUILD b/runsc/boot/BUILD
new file mode 100644
index 000000000..88736cfa4
--- /dev/null
+++ b/runsc/boot/BUILD
@@ -0,0 +1,88 @@
+package(licenses = ["notice"]) # Apache 2.0
+
+load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
+
+go_library(
+ name = "boot",
+ srcs = [
+ "capability.go",
+ "config.go",
+ "controller.go",
+ "events.go",
+ "fds.go",
+ "fs.go",
+ "limits.go",
+ "loader.go",
+ "network.go",
+ "strace.go",
+ ],
+ importpath = "gvisor.googlesource.com/gvisor/runsc/boot",
+ visibility = [
+ "//runsc:__subpackages__",
+ ],
+ deps = [
+ "//pkg/abi/linux",
+ "//pkg/control/server",
+ "//pkg/cpuid",
+ "//pkg/log",
+ "//pkg/sentry/context",
+ "//pkg/sentry/control",
+ "//pkg/sentry/fs",
+ "//pkg/sentry/fs/dev",
+ "//pkg/sentry/fs/gofer",
+ "//pkg/sentry/fs/host",
+ "//pkg/sentry/fs/proc",
+ "//pkg/sentry/fs/ramfs",
+ "//pkg/sentry/fs/sys",
+ "//pkg/sentry/fs/tmpfs",
+ "//pkg/sentry/inet",
+ "//pkg/sentry/kernel",
+ "//pkg/sentry/kernel/auth",
+ "//pkg/sentry/kernel/kdefs",
+ "//pkg/sentry/limits",
+ "//pkg/sentry/loader",
+ "//pkg/sentry/platform",
+ "//pkg/sentry/platform/kvm",
+ "//pkg/sentry/platform/ptrace",
+ "//pkg/sentry/sighandling",
+ "//pkg/sentry/socket/epsocket",
+ "//pkg/sentry/socket/hostinet",
+ "//pkg/sentry/socket/netlink",
+ "//pkg/sentry/socket/netlink/route",
+ "//pkg/sentry/socket/unix",
+ "//pkg/sentry/strace",
+ "//pkg/sentry/syscalls/linux",
+ "//pkg/sentry/time",
+ "//pkg/sentry/usage",
+ "//pkg/sentry/watchdog",
+ "//pkg/syserror",
+ "//pkg/tcpip",
+ "//pkg/tcpip/link/fdbased",
+ "//pkg/tcpip/link/loopback",
+ "//pkg/tcpip/link/sniffer",
+ "//pkg/tcpip/network/arp",
+ "//pkg/tcpip/network/ipv4",
+ "//pkg/tcpip/network/ipv6",
+ "//pkg/tcpip/stack",
+ "//pkg/tcpip/transport/tcp",
+ "//pkg/tcpip/transport/udp",
+ "//pkg/urpc",
+ "//runsc/boot/filter",
+ "//runsc/specutils",
+ "@com_github_opencontainers_runtime-spec//specs-go:go_default_library",
+ "@com_github_syndtr_gocapability//capability:go_default_library",
+ ],
+)
+
+go_test(
+ name = "boot_test",
+ size = "small",
+ srcs = ["loader_test.go"],
+ embed = [":boot"],
+ deps = [
+ "//pkg/control/server",
+ "//pkg/log",
+ "//pkg/sentry/context/contexttest",
+ "@com_github_opencontainers_runtime-spec//specs-go:go_default_library",
+ ],
+)
diff --git a/runsc/boot/capability.go b/runsc/boot/capability.go
new file mode 100644
index 000000000..4c6a59245
--- /dev/null
+++ b/runsc/boot/capability.go
@@ -0,0 +1,120 @@
+// 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 boot
+
+import (
+ "fmt"
+ "os"
+
+ specs "github.com/opencontainers/runtime-spec/specs-go"
+ "github.com/syndtr/gocapability/capability"
+)
+
+// ApplyCaps applies the capabilities in the spec to the current thread.
+//
+// Note that it must be called with current thread locked.
+func ApplyCaps(conf *Config, caps *specs.LinuxCapabilities) error {
+ setter, err := capability.NewPid2(os.Getpid())
+ if err != nil {
+ return err
+ }
+
+ bounding, err := capsFromNames(caps.Bounding)
+ if err != nil {
+ return err
+ }
+ effective, err := capsFromNames(caps.Effective)
+ if err != nil {
+ return err
+ }
+ permitted, err := capsFromNames(caps.Permitted)
+ if err != nil {
+ return err
+ }
+ inheritable, err := capsFromNames(caps.Inheritable)
+ if err != nil {
+ return err
+ }
+ ambient, err := capsFromNames(caps.Ambient)
+ if err != nil {
+ return err
+ }
+
+ // Ptrace platform requires extra capabilities.
+ if conf.Platform == PlatformPtrace {
+ bounding = append(bounding, capability.CAP_SYS_PTRACE)
+ effective = append(effective, capability.CAP_SYS_PTRACE)
+ permitted = append(permitted, capability.CAP_SYS_PTRACE)
+ }
+
+ setter.Set(capability.BOUNDS, bounding...)
+ setter.Set(capability.PERMITTED, permitted...)
+ setter.Set(capability.INHERITABLE, inheritable...)
+ setter.Set(capability.EFFECTIVE, effective...)
+ setter.Set(capability.AMBIENT, ambient...)
+ return setter.Apply(capability.CAPS | capability.BOUNDS | capability.AMBS)
+}
+
+func capsFromNames(names []string) ([]capability.Cap, error) {
+ var caps []capability.Cap
+ for _, name := range names {
+ cap, ok := capFromName[name]
+ if !ok {
+ return nil, fmt.Errorf("invalid capability %q", name)
+ }
+ caps = append(caps, cap)
+ }
+ return caps, nil
+}
+
+var capFromName = map[string]capability.Cap{
+ "CAP_CHOWN": capability.CAP_CHOWN,
+ "CAP_DAC_OVERRIDE": capability.CAP_DAC_OVERRIDE,
+ "CAP_DAC_READ_SEARCH": capability.CAP_DAC_READ_SEARCH,
+ "CAP_FOWNER": capability.CAP_FOWNER,
+ "CAP_FSETID": capability.CAP_FSETID,
+ "CAP_KILL": capability.CAP_KILL,
+ "CAP_SETGID": capability.CAP_SETGID,
+ "CAP_SETUID": capability.CAP_SETUID,
+ "CAP_SETPCAP": capability.CAP_SETPCAP,
+ "CAP_LINUX_IMMUTABLE": capability.CAP_LINUX_IMMUTABLE,
+ "CAP_NET_BIND_SERVICE": capability.CAP_NET_BIND_SERVICE,
+ "CAP_NET_BROAD_CAST": capability.CAP_NET_BROADCAST,
+ "CAP_NET_ADMIN": capability.CAP_NET_ADMIN,
+ "CAP_NET_RAW": capability.CAP_NET_RAW,
+ "CAP_IPC_LOCK": capability.CAP_IPC_LOCK,
+ "CAP_IPC_OWNER": capability.CAP_IPC_OWNER,
+ "CAP_SYS_MODULE": capability.CAP_SYS_MODULE,
+ "CAP_SYS_RAWIO": capability.CAP_SYS_RAWIO,
+ "CAP_SYS_CHROOT": capability.CAP_SYS_CHROOT,
+ "CAP_SYS_PTRACE": capability.CAP_SYS_PTRACE,
+ "CAP_SYS_PACCT": capability.CAP_SYS_PACCT,
+ "CAP_SYS_ADMIN": capability.CAP_SYS_ADMIN,
+ "CAP_SYS_BOOT": capability.CAP_SYS_BOOT,
+ "CAP_SYS_NICE": capability.CAP_SYS_NICE,
+ "CAP_SYS_RESOURCE": capability.CAP_SYS_RESOURCE,
+ "CAP_SYS_TIME": capability.CAP_SYS_TIME,
+ "CAP_SYS_TTY_CONFIG": capability.CAP_SYS_TTY_CONFIG,
+ "CAP_MKNOD": capability.CAP_MKNOD,
+ "CAP_LEASE": capability.CAP_LEASE,
+ "CAP_AUDIT_WRITE": capability.CAP_AUDIT_WRITE,
+ "CAP_AUDIT_CONTROL": capability.CAP_AUDIT_CONTROL,
+ "CAP_SETFCAP": capability.CAP_SETFCAP,
+ "CAP_MAC_OVERRIDE": capability.CAP_MAC_OVERRIDE,
+ "CAP_MAC_ADMIN": capability.CAP_MAC_ADMIN,
+ "CAP_SYSLOG": capability.CAP_SYSLOG,
+ "CAP_WAKE_ALARM": capability.CAP_WAKE_ALARM,
+ "CAP_BLOCK_SUSPEND": capability.CAP_BLOCK_SUSPEND,
+}
diff --git a/runsc/boot/config.go b/runsc/boot/config.go
new file mode 100644
index 000000000..f3e33e89a
--- /dev/null
+++ b/runsc/boot/config.go
@@ -0,0 +1,162 @@
+// 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 boot
+
+import "fmt"
+
+// PlatformType tells which platform to use.
+type PlatformType int
+
+const (
+ // Ptrace runs the sandbox with the ptrace platform.
+ PlatformPtrace PlatformType = iota
+
+ // KVM runs the sandbox with the KVM platform.
+ PlatformKVM
+)
+
+// MakePlatformType converts type from string.
+func MakePlatformType(s string) (PlatformType, error) {
+ switch s {
+ case "ptrace":
+ return PlatformPtrace, nil
+ case "kvm":
+ return PlatformKVM, nil
+ default:
+ return 0, fmt.Errorf("invalid platform type %q", s)
+ }
+}
+
+func (p PlatformType) String() string {
+ switch p {
+ case PlatformPtrace:
+ return "ptrace"
+ case PlatformKVM:
+ return "kvm"
+ default:
+ return fmt.Sprintf("unknown(%d)", p)
+ }
+}
+
+// FileAccessType tells how the filesystem is accessed.
+type FileAccessType int
+
+const (
+ // FileAccessProxy sends IO requests to a Gofer process that validates the
+ // requests and forwards them to the host.
+ FileAccessProxy FileAccessType = iota
+
+ // FileAccessDirect connects the sandbox directly to the host filesystem.
+ FileAccessDirect
+)
+
+// MakeFileAccessType converts type from string.
+func MakeFileAccessType(s string) (FileAccessType, error) {
+ switch s {
+ case "proxy":
+ return FileAccessProxy, nil
+ case "direct":
+ return FileAccessDirect, nil
+ default:
+ return 0, fmt.Errorf("invalid file access type %q", s)
+ }
+}
+
+func (f FileAccessType) String() string {
+ switch f {
+ case FileAccessProxy:
+ return "proxy"
+ case FileAccessDirect:
+ return "direct"
+ default:
+ return fmt.Sprintf("unknown(%d)", f)
+ }
+}
+
+// NetworkType tells which network stack to use.
+type NetworkType int
+
+const (
+ // NetworkSandbox uses internal network stack, isolated from the host.
+ NetworkSandbox NetworkType = iota
+
+ // NetworkHost redirects network related syscalls to the host network.
+ NetworkHost
+
+ // NetworkNone sets up just loopback using netstack.
+ NetworkNone
+)
+
+// MakeNetworkType converts type from string.
+func MakeNetworkType(s string) (NetworkType, error) {
+ switch s {
+ case "sandbox":
+ return NetworkSandbox, nil
+ case "host":
+ return NetworkHost, nil
+ case "none":
+ return NetworkNone, nil
+ default:
+ return 0, fmt.Errorf("invalid network type %q", s)
+ }
+}
+
+func (n NetworkType) String() string {
+ switch n {
+ case NetworkSandbox:
+ return "sandbox"
+ case NetworkHost:
+ return "host"
+ case NetworkNone:
+ return "none"
+ default:
+ return fmt.Sprintf("unknown(%d)", n)
+ }
+}
+
+// Config holds configuration that is not part of the runtime spec.
+type Config struct {
+ // RootDir is the runtime root directory.
+ RootDir string
+
+ // FileAccess indicates how the filesystem is accessed.
+ FileAccess FileAccessType
+
+ // Overlay is whether to wrap the root filesystem in an overlay.
+ Overlay bool
+
+ // Network indicates what type of network to use.
+ Network NetworkType
+
+ // LogPackets indicates that all network packets should be logged.
+ LogPackets bool
+
+ // Platform is the platform to run on.
+ Platform PlatformType
+
+ // Strace indicates that strace should be enabled.
+ Strace bool
+
+ // StraceSyscalls is the set of syscalls to trace. If StraceEnable is
+ // true and this list is empty, then all syscalls will be traced.
+ StraceSyscalls []string
+
+ // StraceLogSize is the max size of data blobs to display.
+ StraceLogSize uint
+
+ // DisableSeccomp indicates whether seccomp syscall filters should be
+ // disabled. Pardon the double negation, but default to enabled is important.
+ DisableSeccomp bool
+}
diff --git a/runsc/boot/controller.go b/runsc/boot/controller.go
new file mode 100644
index 000000000..4d4ef7256
--- /dev/null
+++ b/runsc/boot/controller.go
@@ -0,0 +1,128 @@
+// 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 boot
+
+import (
+ "fmt"
+
+ "gvisor.googlesource.com/gvisor/pkg/control/server"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/control"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/socket/epsocket"
+)
+
+const (
+ // ApplicationStart is the URPC endpoint for starting a sandboxed app.
+ ApplicationStart = "application.Start"
+
+ // ApplicationProcesses is the URPC endpoint for getting the list of
+ // processes running in a sandbox.
+ ApplicationProcesses = "application.Processes"
+
+ // ApplicationExecute is the URPC endpoint for executing a command in a
+ // sandbox.
+ ApplicationExecute = "application.Execute"
+
+ // ApplicationEvent is the URPC endpoint for getting stats about the
+ // container used by "runsc events".
+ ApplicationEvent = "application.Event"
+
+ // NetworkCreateLinksAndRoutes is the URPC endpoint for creating links
+ // and routes in a network stack.
+ NetworkCreateLinksAndRoutes = "Network.CreateLinksAndRoutes"
+)
+
+// ControlSocketAddr generates an abstract unix socket name for the given id.
+func ControlSocketAddr(id string) string {
+ return fmt.Sprintf("\x00runsc-sandbox.%s", id)
+}
+
+// controller holds the control server, and is used for communication into the
+// sandbox.
+type controller struct {
+ // srv is the contorl server.
+ srv *server.Server
+
+ // app holds the application methods.
+ app *application
+}
+
+// newController creates a new controller and starts it listening.
+func newController(fd int, k *kernel.Kernel) (*controller, error) {
+ srv, err := server.CreateFromFD(fd)
+ if err != nil {
+ return nil, err
+ }
+
+ app := &application{
+ startChan: make(chan struct{}),
+ startResultChan: make(chan error, 1),
+ k: k,
+ }
+ srv.Register(app)
+
+ if eps, ok := k.NetworkStack().(*epsocket.Stack); ok {
+ net := &Network{
+ Stack: eps.Stack,
+ }
+ srv.Register(net)
+ }
+
+ if err := srv.StartServing(); err != nil {
+ return nil, err
+ }
+
+ return &controller{
+ srv: srv,
+ app: app,
+ }, nil
+}
+
+// application contains methods that control the sandboxed application.
+type application struct {
+ // startChan is used to signal when the application process should be
+ // started.
+ startChan chan struct{}
+
+ // startResultChan is used to signal when the application has started. Any
+ // errors encountered during startup will be sent to the channel. A nil value
+ // indicates success.
+ startResultChan chan error
+
+ // k is the emulated linux kernel on which the sandboxed
+ // application runs.
+ k *kernel.Kernel
+}
+
+// Start will start the application process.
+func (a *application) Start(_, _ *struct{}) error {
+ // Tell the application to start and wait for the result.
+ a.startChan <- struct{}{}
+ return <-a.startResultChan
+}
+
+// Processes retrieves information about processes running in the sandbox.
+func (a *application) Processes(_, out *[]*control.Process) error {
+ return control.Processes(a.k, out)
+}
+
+// Execute runs a command on a created or running sandbox.
+func (a *application) Execute(e *control.ExecArgs, waitStatus *uint32) error {
+ proc := control.Proc{Kernel: a.k}
+ if err := proc.Exec(e, waitStatus); err != nil {
+ return fmt.Errorf("error executing: %+v: %v", e, err)
+ }
+ return nil
+}
diff --git a/runsc/boot/events.go b/runsc/boot/events.go
new file mode 100644
index 000000000..ef6459b01
--- /dev/null
+++ b/runsc/boot/events.go
@@ -0,0 +1,81 @@
+// 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 boot
+
+import (
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/usage"
+)
+
+// Event struct for encoding the event data to JSON. Corresponds to runc's
+// main.event struct.
+type Event struct {
+ Type string `json:"type"`
+ ID string `json:"id"`
+ Data interface{} `json:"data,omitempty"`
+}
+
+// Stats is the runc specific stats structure for stability when encoding and
+// decoding stats.
+// TODO: Many fields aren't obtainable due to a lack of cgroups.
+type Stats struct {
+ Memory Memory `json:"memory"`
+ Pids Pids `json:"pids"`
+}
+
+// Pids contains stats on processes.
+type Pids struct {
+ Current uint64 `json:"current,omitempty"`
+ Limit uint64 `json:"limit,omitempty"`
+}
+
+// MemoryEntry contains stats on a kind of memory.
+type MemoryEntry struct {
+ Limit uint64 `json:"limit"`
+ Usage uint64 `json:"usage,omitempty"`
+ Max uint64 `json:"max,omitempty"`
+ Failcnt uint64 `json:"failcnt"`
+}
+
+// Memory contains stats on memory.
+type Memory struct {
+ Cache uint64 `json:"cache,omitempty"`
+ Usage MemoryEntry `json:"usage,omitempty"`
+ Swap MemoryEntry `json:"swap,omitempty"`
+ Kernel MemoryEntry `json:"kernel,omitempty"`
+ KernelTCP MemoryEntry `json:"kernelTCP,omitempty"`
+ Raw map[string]uint64 `json:"raw,omitempty"`
+}
+
+func (a *application) Event(_ *struct{}, out *Event) error {
+ stats := &Stats{}
+ stats.populateMemory(a.k)
+ stats.populatePIDs(a.k)
+ *out = Event{Type: "stats", Data: stats}
+ return nil
+}
+
+func (s *Stats) populateMemory(k *kernel.Kernel) {
+ mem := k.Platform.Memory()
+ mem.UpdateUsage()
+ _, totalUsage := usage.MemoryAccounting.Copy()
+ s.Memory.Usage = MemoryEntry{
+ Usage: totalUsage,
+ }
+}
+
+func (s *Stats) populatePIDs(k *kernel.Kernel) {
+ s.Pids.Current = uint64(len(k.TaskSet().Root.ThreadGroups()))
+}
diff --git a/runsc/boot/fds.go b/runsc/boot/fds.go
new file mode 100644
index 000000000..0449e243d
--- /dev/null
+++ b/runsc/boot/fds.go
@@ -0,0 +1,61 @@
+// 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 boot
+
+import (
+ "fmt"
+ "syscall"
+
+ "gvisor.googlesource.com/gvisor/pkg/sentry/context"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/host"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/kdefs"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/limits"
+)
+
+// createFDMap creates an fd map that contains stdin, stdout, and stderr. If
+// console is true, then ioctl calls will be passed through to the host fd.
+//
+// TODO: We currently arn't passing any FDs in to the sandbox, so
+// there's not much else for this function to do. It will get more complicated
+// when gofers enter the picture. Also the LISTEN_FDS environment variable
+// allows passing arbitrary FDs to the sandbox, which we do not yet support.
+func createFDMap(ctx context.Context, k *kernel.Kernel, l *limits.LimitSet, console bool) (*kernel.FDMap, error) {
+ fdm := k.NewFDMap()
+ defer fdm.DecRef()
+
+ // Maps sandbox fd to host fd.
+ fdMap := map[int]int{
+ 0: syscall.Stdin,
+ 1: syscall.Stdout,
+ 2: syscall.Stderr,
+ }
+ mounter := fs.FileOwnerFromContext(ctx)
+
+ for sfd, hfd := range fdMap {
+ file, err := host.ImportFile(ctx, hfd, mounter, console /* allow ioctls */)
+ if err != nil {
+ return nil, fmt.Errorf("failed to import fd %d: %v", hfd, err)
+ }
+ defer file.DecRef()
+ if err := fdm.NewFDAt(kdefs.FD(sfd), file, kernel.FDFlags{}, l); err != nil {
+ return nil, fmt.Errorf("failed to add imported fd %d to FDMap: %v", hfd, err)
+ }
+ }
+
+ fdm.IncRef()
+ return fdm, nil
+}
diff --git a/runsc/boot/filter/BUILD b/runsc/boot/filter/BUILD
new file mode 100644
index 000000000..fd1b18717
--- /dev/null
+++ b/runsc/boot/filter/BUILD
@@ -0,0 +1,26 @@
+package(licenses = ["notice"]) # Apache 2.0
+
+load("@io_bazel_rules_go//go:def.bzl", "go_library")
+
+go_library(
+ name = "filter",
+ srcs = [
+ "config.go",
+ "extra_filters.go",
+ "extra_filters_msan.go",
+ "extra_filters_race.go",
+ "filter.go",
+ ],
+ importpath = "gvisor.googlesource.com/gvisor/runsc/boot/filter",
+ visibility = [
+ "//runsc/boot:__subpackages__",
+ ],
+ deps = [
+ "//pkg/log",
+ "//pkg/seccomp",
+ "//pkg/sentry/platform",
+ "//pkg/sentry/platform/kvm",
+ "//pkg/sentry/platform/ptrace",
+ "@org_golang_x_sys//unix:go_default_library",
+ ],
+)
diff --git a/runsc/boot/filter/config.go b/runsc/boot/filter/config.go
new file mode 100644
index 000000000..130e987df
--- /dev/null
+++ b/runsc/boot/filter/config.go
@@ -0,0 +1,175 @@
+// 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 filter
+
+import (
+ "syscall"
+
+ "golang.org/x/sys/unix"
+)
+
+// allowedSyscalls is the set of syscalls executed by the Sentry
+// to the host OS.
+var allowedSyscalls = []uintptr{
+ syscall.SYS_ACCEPT,
+ syscall.SYS_ARCH_PRCTL,
+ syscall.SYS_CLOCK_GETTIME,
+ syscall.SYS_CLONE,
+ syscall.SYS_CLOSE,
+ syscall.SYS_DUP,
+ syscall.SYS_DUP2,
+ syscall.SYS_EPOLL_CREATE1,
+ syscall.SYS_EPOLL_CTL,
+ syscall.SYS_EPOLL_PWAIT,
+ syscall.SYS_EPOLL_WAIT,
+ syscall.SYS_EVENTFD2,
+ syscall.SYS_EXIT,
+ syscall.SYS_EXIT_GROUP,
+ syscall.SYS_FALLOCATE,
+ syscall.SYS_FCHMOD,
+ syscall.SYS_FCNTL,
+ syscall.SYS_FSTAT,
+ syscall.SYS_FSYNC,
+ syscall.SYS_FTRUNCATE,
+ syscall.SYS_FUTEX,
+ syscall.SYS_GETDENTS64,
+ syscall.SYS_GETPID,
+ unix.SYS_GETRANDOM,
+ syscall.SYS_GETSOCKOPT,
+ syscall.SYS_GETTID,
+ syscall.SYS_GETTIMEOFDAY,
+ syscall.SYS_LISTEN,
+ syscall.SYS_LSEEK,
+ syscall.SYS_MADVISE,
+ syscall.SYS_MINCORE,
+ syscall.SYS_MMAP,
+ syscall.SYS_MPROTECT,
+ syscall.SYS_MUNMAP,
+ syscall.SYS_NEWFSTATAT,
+ syscall.SYS_POLL,
+ syscall.SYS_PREAD64,
+ syscall.SYS_PSELECT6,
+ syscall.SYS_PWRITE64,
+ syscall.SYS_READ,
+ syscall.SYS_READLINKAT,
+ syscall.SYS_READV,
+ syscall.SYS_RECVMSG,
+ syscall.SYS_RENAMEAT,
+ syscall.SYS_RESTART_SYSCALL,
+ syscall.SYS_RT_SIGACTION,
+ syscall.SYS_RT_SIGPROCMASK,
+ syscall.SYS_RT_SIGRETURN,
+ syscall.SYS_SCHED_YIELD,
+ syscall.SYS_SENDMSG,
+ syscall.SYS_SETITIMER,
+ syscall.SYS_SHUTDOWN,
+ syscall.SYS_SIGALTSTACK,
+ syscall.SYS_SYNC_FILE_RANGE,
+ syscall.SYS_TGKILL,
+ syscall.SYS_UTIMENSAT,
+ syscall.SYS_WRITE,
+ syscall.SYS_WRITEV,
+}
+
+// TODO: Ioctl is needed in order to support tty consoles.
+// Once filters support argument-checking, we should only allow ioctl
+// with tty-related arguments.
+func consoleFilters() []uintptr {
+ return []uintptr{
+ syscall.SYS_IOCTL,
+ }
+}
+
+// whitelistFSFilters returns syscalls made by whitelistFS. Using WhitelistFS
+// is less secure because it runs inside the Sentry and must be able to perform
+// file operations that would otherwise be disabled by seccomp when a Gofer is
+// used. When whitelistFS is not used, openning new FD in the Sentry is
+// disallowed.
+func whitelistFSFilters() []uintptr {
+ return []uintptr{
+ syscall.SYS_ACCESS,
+ syscall.SYS_FCHMOD,
+ syscall.SYS_FSTAT,
+ syscall.SYS_FSYNC,
+ syscall.SYS_FTRUNCATE,
+ syscall.SYS_GETCWD,
+ syscall.SYS_GETDENTS,
+ syscall.SYS_GETDENTS64,
+ syscall.SYS_LSEEK,
+ syscall.SYS_LSTAT,
+ syscall.SYS_MKDIR,
+ syscall.SYS_MKDIRAT,
+ syscall.SYS_NEWFSTATAT,
+ syscall.SYS_OPEN,
+ syscall.SYS_OPENAT,
+ syscall.SYS_PREAD64,
+ syscall.SYS_PWRITE64,
+ syscall.SYS_READ,
+ syscall.SYS_READLINK,
+ syscall.SYS_READLINKAT,
+ syscall.SYS_RENAMEAT,
+ syscall.SYS_STAT,
+ syscall.SYS_SYMLINK,
+ syscall.SYS_SYMLINKAT,
+ syscall.SYS_SYNC_FILE_RANGE,
+ syscall.SYS_UNLINK,
+ syscall.SYS_UNLINKAT,
+ syscall.SYS_UTIMENSAT,
+ syscall.SYS_WRITE,
+ }
+}
+
+// hostInetFilters contains syscalls that are needed by sentry/socket/hostinet.
+func hostInetFilters() []uintptr {
+ return []uintptr{
+ syscall.SYS_ACCEPT4,
+ syscall.SYS_BIND,
+ syscall.SYS_CONNECT,
+ syscall.SYS_GETPEERNAME,
+ syscall.SYS_GETSOCKNAME,
+ syscall.SYS_GETSOCKOPT,
+ syscall.SYS_IOCTL,
+ syscall.SYS_LISTEN,
+ syscall.SYS_READV,
+ syscall.SYS_RECVFROM,
+ syscall.SYS_RECVMSG,
+ syscall.SYS_SENDMSG,
+ syscall.SYS_SENDTO,
+ syscall.SYS_SETSOCKOPT,
+ syscall.SYS_SHUTDOWN,
+ syscall.SYS_SOCKET,
+ syscall.SYS_WRITEV,
+ }
+}
+
+// ptraceFilters returns syscalls made exclusively by the ptrace platform.
+func ptraceFilters() []uintptr {
+ return []uintptr{
+ syscall.SYS_PTRACE,
+ syscall.SYS_WAIT4,
+ unix.SYS_GETCPU,
+ unix.SYS_SCHED_SETAFFINITY,
+ }
+}
+
+// kvmFilters returns syscalls made exclusively by the KVM platform.
+func kvmFilters() []uintptr {
+ return []uintptr{
+ syscall.SYS_IOCTL,
+ syscall.SYS_RT_SIGSUSPEND,
+ syscall.SYS_RT_SIGTIMEDWAIT,
+ 0xffffffffffffffff, // KVM uses syscall -1 to transition to host.
+ }
+}
diff --git a/runsc/boot/filter/extra_filters.go b/runsc/boot/filter/extra_filters.go
new file mode 100644
index 000000000..e10d9bf4c
--- /dev/null
+++ b/runsc/boot/filter/extra_filters.go
@@ -0,0 +1,24 @@
+// 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.
+
+// +build !msan,!race
+
+package filter
+
+// instrumentationFilters returns additional filters for syscalls used by
+// Go intrumentation tools, e.g. -race, -msan.
+// Returns empty when disabled.
+func instrumentationFilters() []uintptr {
+ return nil
+}
diff --git a/runsc/boot/filter/extra_filters_msan.go b/runsc/boot/filter/extra_filters_msan.go
new file mode 100644
index 000000000..a862340f6
--- /dev/null
+++ b/runsc/boot/filter/extra_filters_msan.go
@@ -0,0 +1,30 @@
+// 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.
+
+// +build msan
+
+package filter
+
+import (
+ "syscall"
+)
+
+// instrumentationFilters returns additional filters for syscalls used by MSAN.
+func instrumentationFilters() []uintptr {
+ Report("MSAN is enabled: syscall filters less restrictive!")
+ return []uintptr{
+ syscall.SYS_SCHED_GETAFFINITY,
+ syscall.SYS_SET_ROBUST_LIST,
+ }
+}
diff --git a/runsc/boot/filter/extra_filters_race.go b/runsc/boot/filter/extra_filters_race.go
new file mode 100644
index 000000000..b0c74a58a
--- /dev/null
+++ b/runsc/boot/filter/extra_filters_race.go
@@ -0,0 +1,33 @@
+// 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.
+
+// +build race
+
+package filter
+
+import (
+ "syscall"
+)
+
+// instrumentationFilters returns additional filters for syscalls used by TSAN.
+func instrumentationFilters() []uintptr {
+ Report("TSAN is enabled: syscall filters less restrictive!")
+ return []uintptr{
+ syscall.SYS_BRK,
+ syscall.SYS_MUNLOCK,
+ syscall.SYS_NANOSLEEP,
+ syscall.SYS_OPEN,
+ syscall.SYS_SET_ROBUST_LIST,
+ }
+}
diff --git a/runsc/boot/filter/filter.go b/runsc/boot/filter/filter.go
new file mode 100644
index 000000000..3ba56a318
--- /dev/null
+++ b/runsc/boot/filter/filter.go
@@ -0,0 +1,67 @@
+// 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 filter defines all syscalls the sandbox is allowed to make
+// to the host, and installs seccomp filters to prevent prohibited
+// syscalls in case it's compromised.
+package filter
+
+import (
+ "fmt"
+
+ "gvisor.googlesource.com/gvisor/pkg/log"
+ "gvisor.googlesource.com/gvisor/pkg/seccomp"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/platform"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/platform/kvm"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/platform/ptrace"
+)
+
+// Install installs seccomp filters for based on the given platform.
+func Install(p platform.Platform, whitelistFS, console, hostNetwork bool) error {
+ s := allowedSyscalls
+
+ // Set of additional filters used by -race and -msan. Returns empty
+ // when not enabled.
+ s = append(s, instrumentationFilters()...)
+
+ if whitelistFS {
+ Report("direct file access allows unrestricted file access!")
+ s = append(s, whitelistFSFilters()...)
+ }
+ if console {
+ Report("console is enabled: syscall filters less restrictive!")
+ s = append(s, consoleFilters()...)
+ }
+ if hostNetwork {
+ Report("host networking enabled: syscall filters less restrictive!")
+ s = append(s, hostInetFilters()...)
+ }
+
+ switch p := p.(type) {
+ case *ptrace.PTrace:
+ s = append(s, ptraceFilters()...)
+ case *kvm.KVM:
+ s = append(s, kvmFilters()...)
+ default:
+ return fmt.Errorf("unknown platform type %T", p)
+ }
+
+ // TODO: Set kill=true when SECCOMP_RET_KILL_PROCESS is supported.
+ return seccomp.Install(s, false)
+}
+
+// Report writes a warning message to the log.
+func Report(msg string) {
+ log.Warningf("*** SECCOMP WARNING: %s", msg)
+}
diff --git a/runsc/boot/fs.go b/runsc/boot/fs.go
new file mode 100644
index 000000000..2073bd0b1
--- /dev/null
+++ b/runsc/boot/fs.go
@@ -0,0 +1,441 @@
+// 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 boot
+
+import (
+ "fmt"
+ "os"
+ "path/filepath"
+ "strings"
+
+ // Include filesystem types that OCI spec might mount.
+ _ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/dev"
+ _ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/gofer"
+ _ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/host"
+ _ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/proc"
+ _ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/sys"
+ _ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/tmpfs"
+
+ specs "github.com/opencontainers/runtime-spec/specs-go"
+ "gvisor.googlesource.com/gvisor/pkg/abi/linux"
+ "gvisor.googlesource.com/gvisor/pkg/log"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/context"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs/ramfs"
+ "gvisor.googlesource.com/gvisor/pkg/syserror"
+)
+
+type fdDispenser struct {
+ fds []int
+}
+
+func (f *fdDispenser) remove() int {
+ rv := f.fds[0]
+ f.fds = f.fds[1:]
+ return rv
+}
+
+func (f *fdDispenser) empty() bool {
+ return len(f.fds) == 0
+}
+
+// createMountNamespace creates a mount manager containing the root filesystem
+// and all mounts.
+func createMountNamespace(ctx context.Context, spec *specs.Spec, conf *Config, ioFDs []int) (*fs.MountNamespace, error) {
+ fds := &fdDispenser{fds: ioFDs}
+
+ // Create the MountNamespace from the root.
+ rootInode, err := createRootMount(ctx, spec, conf, fds)
+ if err != nil {
+ return nil, fmt.Errorf("failed to create root overlay: %v", err)
+ }
+ mns, err := fs.NewMountNamespace(ctx, rootInode)
+ if err != nil {
+ return nil, fmt.Errorf("failed to construct MountNamespace: %v", err)
+ }
+
+ // Keep track of whether proc, sys, and tmp were mounted.
+ var procMounted, sysMounted, tmpMounted bool
+
+ // Mount all submounts from the spec.
+ for _, m := range spec.Mounts {
+ // OCI spec uses many different mounts for the things inside of '/dev'. We
+ // have a single mount at '/dev' that is always mounted, regardless of
+ // whether it was asked for, as the spec says we SHOULD.
+ if strings.HasPrefix(m.Destination, "/dev") {
+ log.Warningf("ignoring dev mount at %q", m.Destination)
+ continue
+ }
+ switch m.Destination {
+ case "/proc":
+ procMounted = true
+ case "/sys":
+ sysMounted = true
+ case "/tmp":
+ tmpMounted = true
+ }
+
+ if err := mountSubmount(ctx, spec, conf, mns, fds, m); err != nil {
+ return nil, err
+ }
+ }
+
+ // Always mount /dev.
+ if err := mountSubmount(ctx, spec, conf, mns, nil, specs.Mount{
+ Type: "devtmpfs",
+ Destination: "/dev",
+ }); err != nil {
+ return nil, err
+ }
+
+ // Mount proc and sys even if the user did not ask for it, as the spec
+ // says we SHOULD.
+ if !procMounted {
+ if err := mountSubmount(ctx, spec, conf, mns, nil, specs.Mount{
+ Type: "proc",
+ Destination: "/proc",
+ }); err != nil {
+ return nil, err
+ }
+ }
+ if !sysMounted {
+ if err := mountSubmount(ctx, spec, conf, mns, nil, specs.Mount{
+ Type: "sysfs",
+ Destination: "/sys",
+ }); err != nil {
+ return nil, err
+ }
+ }
+
+ // Technically we don't have to mount tmpfs at /tmp, as we could just
+ // rely on the host /tmp, but this is a nice optimization, and fixes
+ // some apps that call mknod in /tmp.
+ if !tmpMounted {
+ if err := mountSubmount(ctx, spec, conf, mns, nil, specs.Mount{
+ Type: "tmpfs",
+ Destination: "/tmp",
+ }); err != nil {
+ return nil, err
+ }
+ }
+
+ if !fds.empty() {
+ return nil, fmt.Errorf("not all mount points were consumed, remaining: %v", fds)
+ }
+
+ return mns, nil
+}
+
+// createRootMount creates the root filesystem.
+func createRootMount(ctx context.Context, spec *specs.Spec, conf *Config, fds *fdDispenser) (*fs.Inode, error) {
+ // First construct the filesystem from the spec.Root.
+ mf := fs.MountSourceFlags{
+ ReadOnly: spec.Root.Readonly,
+ NoAtime: true,
+ }
+
+ var (
+ rootInode *fs.Inode
+ err error
+ )
+ switch conf.FileAccess {
+ case FileAccessProxy:
+ fd := fds.remove()
+ log.Infof("Mounting root over 9P, ioFD: %d", fd)
+ hostFS := mustFindFilesystem("9p")
+ rootInode, err = hostFS.Mount(ctx, "root", mf, fmt.Sprintf("trans=fd,rfdno=%d,wfdno=%d,privateunixsocket=true", fd, fd))
+ if err != nil {
+ return nil, fmt.Errorf("failed to generate root mount point: %v", err)
+ }
+
+ case FileAccessDirect:
+ hostFS := mustFindFilesystem("whitelistfs")
+ rootInode, err = hostFS.Mount(ctx, "root", mf, "root="+spec.Root.Path+",dont_translate_ownership=true")
+ if err != nil {
+ return nil, fmt.Errorf("failed to generate root mount point: %v", err)
+ }
+
+ default:
+ return nil, fmt.Errorf("invalid file access type: %v", conf.FileAccess)
+ }
+
+ // We need to overlay the root on top of a ramfs with stub directories
+ // for submount paths. "/dev" "/sys" "/proc" and "/tmp" are always
+ // mounted even if they are not in the spec.
+ submounts := append(subtargets("/", spec.Mounts), "/dev", "/sys", "/proc", "/tmp")
+ rootInode, err = addSubmountOverlay(ctx, rootInode, submounts)
+ if err != nil {
+ return nil, fmt.Errorf("error adding submount overlay: %v", err)
+ }
+
+ if conf.Overlay {
+ log.Debugf("Adding overlay on top of root mount")
+ // Overlay a tmpfs filesystem on top of the root.
+ rootInode, err = addOverlay(ctx, conf, rootInode, "root-overlay-upper", mf)
+ if err != nil {
+ return nil, err
+ }
+ }
+
+ log.Infof("Mounted %q to \"/\" type root", spec.Root.Path)
+ return rootInode, nil
+}
+
+func addOverlay(ctx context.Context, conf *Config, lower *fs.Inode, name string, lowerFlags fs.MountSourceFlags) (*fs.Inode, error) {
+ // Upper layer uses the same flags as lower, but it must be read-write.
+ lowerFlags.ReadOnly = false
+
+ tmpFS := mustFindFilesystem("tmpfs")
+ upper, err := tmpFS.Mount(ctx, name+"-upper", lowerFlags, "")
+ if err != nil {
+ return nil, fmt.Errorf("failed to create tmpfs overlay: %v", err)
+ }
+ return fs.NewOverlayRoot(ctx, upper, lower, lowerFlags)
+}
+
+func mountSubmount(ctx context.Context, spec *specs.Spec, conf *Config, mns *fs.MountNamespace, fds *fdDispenser, m specs.Mount) error {
+ // Map mount type to filesystem name, and parse out the options that we are
+ // capable of dealing with.
+ var data []string
+ var fsName string
+ var useOverlay bool
+ switch m.Type {
+ case "proc", "sysfs", "devtmpfs":
+ fsName = m.Type
+ case "none":
+ fsName = "sysfs"
+ case "tmpfs":
+ fsName = m.Type
+
+ // tmpfs has some extra supported options that we must pass through.
+ var err error
+ data, err = parseAndFilterOptions(m.Options, "mode", "uid", "gid")
+ if err != nil {
+ return err
+ }
+ case "bind":
+ switch conf.FileAccess {
+ case FileAccessProxy:
+ fd := fds.remove()
+ fsName = "9p"
+ data = []string{"trans=fd", fmt.Sprintf("rfdno=%d", fd), fmt.Sprintf("wfdno=%d", fd), "privateunixsocket=true"}
+ case FileAccessDirect:
+ fsName = "whitelistfs"
+ data = []string{"root=" + m.Source, "dont_translate_ownership=true"}
+ default:
+ return fmt.Errorf("invalid file access type: %v", conf.FileAccess)
+ }
+
+ fi, err := os.Stat(m.Source)
+ if err != nil {
+ return err
+ }
+ // Add overlay to all writable mounts, except when mapping an individual file.
+ useOverlay = conf.Overlay && !mountFlags(m.Options).ReadOnly && fi.Mode().IsDir()
+ default:
+ // TODO: Support all the mount types and make this a
+ // fatal error. Most applications will "just work" without
+ // them, so this is a warning for now.
+ // we do not support.
+ log.Warningf("ignoring unknown filesystem type %q", m.Type)
+ return nil
+ }
+
+ // All filesystem names should have been mapped to something we know.
+ filesystem := mustFindFilesystem(fsName)
+
+ mf := mountFlags(m.Options)
+ if useOverlay {
+ // All writes go to upper, be paranoid and make lower readonly.
+ mf.ReadOnly = true
+ }
+ mf.NoAtime = true
+
+ inode, err := filesystem.Mount(ctx, m.Type, mf, strings.Join(data, ","))
+ if err != nil {
+ return fmt.Errorf("failed to create mount with source %q: %v", m.Source, err)
+ }
+
+ // If there are submounts, we need to overlay the mount on top of a
+ // ramfs with stub directories for submount paths.
+ //
+ // We do not do this for /dev, since there will usually be submounts in
+ // the spec, but our devfs implementation contains all the necessary
+ // directories and files (well, most of them anyways).
+ if m.Destination != "/dev" {
+ submounts := subtargets(m.Destination, spec.Mounts)
+ if len(submounts) > 0 {
+ log.Infof("Adding submount overlay over %q", m.Destination)
+ inode, err = addSubmountOverlay(ctx, inode, submounts)
+ if err != nil {
+ return fmt.Errorf("error adding submount overlay: %v", err)
+ }
+ }
+ }
+
+ if useOverlay {
+ log.Debugf("Adding overlay on top of mount %q", m.Destination)
+ if inode, err = addOverlay(ctx, conf, inode, m.Type, mf); err != nil {
+ return err
+ }
+ }
+
+ root := mns.Root()
+ defer root.DecRef()
+ dirent, err := mns.FindInode(ctx, root, nil, m.Destination, linux.MaxSymlinkTraversals)
+ if err != nil {
+ return fmt.Errorf("failed to find mount destination %q: %v", m.Destination, err)
+ }
+ defer dirent.DecRef()
+ if err := mns.Mount(ctx, dirent, inode); err != nil {
+ return fmt.Errorf("failed to mount at destination %q: %v", m.Destination, err)
+ }
+
+ log.Infof("Mounted %q to %q type %s", m.Source, m.Destination, m.Type)
+ return nil
+}
+
+func mkdirAll(ctx context.Context, mns *fs.MountNamespace, path string) error {
+ root := mns.Root()
+ defer root.DecRef()
+
+ // Starting at the root, walk the path.
+ parent := root
+ ps := strings.Split(filepath.Clean(path), string(filepath.Separator))
+ for i := 0; i < len(ps); i++ {
+ if ps[i] == "" {
+ // This will be case for the first and last element, if the path
+ // begins or ends with '/'. Note that we always treat the path as
+ // absolute, regardless of what the first character contains.
+ continue
+ }
+ d, err := mns.FindInode(ctx, root, parent, ps[i], fs.DefaultTraversalLimit)
+ if err == syserror.ENOENT {
+ // If we encounter a path that does not exist, then
+ // create it.
+ if err := parent.CreateDirectory(ctx, root, ps[i], fs.FilePermsFromMode(0755)); err != nil {
+ return fmt.Errorf("failed to create directory %q: %v", ps[i], err)
+ }
+ if d, err = parent.Walk(ctx, root, ps[i]); err != nil {
+ return fmt.Errorf("walk to %q failed: %v", ps[i], err)
+ }
+ } else if err != nil {
+ return fmt.Errorf("failed to find inode %q: %v", ps[i], err)
+ }
+ parent = d
+ }
+ return nil
+}
+
+// parseAndFilterOptions parses a MountOptions slice and filters by the allowed
+// keys.
+func parseAndFilterOptions(opts []string, allowedKeys ...string) ([]string, error) {
+ var out []string
+ for _, o := range opts {
+ kv := strings.Split(o, "=")
+ switch len(kv) {
+ case 1:
+ if contains(allowedKeys, o) {
+ out = append(out, o)
+ continue
+ }
+ log.Warningf("ignoring unsupported key %q", kv)
+ case 2:
+ if contains(allowedKeys, kv[0]) {
+ out = append(out, o)
+ continue
+ }
+ log.Warningf("ignoring unsupported key %q", kv[0])
+ default:
+ return nil, fmt.Errorf("invalid option %q", o)
+ }
+ }
+ return out, nil
+}
+
+func destinations(mounts []specs.Mount, extra ...string) []string {
+ var ds []string
+ for _, m := range mounts {
+ ds = append(ds, m.Destination)
+ }
+ return append(ds, extra...)
+}
+
+func mountFlags(opts []string) fs.MountSourceFlags {
+ mf := fs.MountSourceFlags{}
+ for _, o := range opts {
+ switch o {
+ case "ro":
+ mf.ReadOnly = true
+ case "noatime":
+ mf.NoAtime = true
+ default:
+ log.Warningf("ignorning unknown mount option %q", o)
+ }
+ }
+ return mf
+}
+
+func contains(strs []string, str string) bool {
+ for _, s := range strs {
+ if s == str {
+ return true
+ }
+ }
+ return false
+}
+
+func mustFindFilesystem(name string) fs.Filesystem {
+ fs, ok := fs.FindFilesystem(name)
+ if !ok {
+ panic(fmt.Sprintf("could not find filesystem %q", name))
+ }
+ return fs
+}
+
+// addSubmountOverlay overlays the inode over a ramfs tree containing the given
+// paths.
+func addSubmountOverlay(ctx context.Context, inode *fs.Inode, submounts []string) (*fs.Inode, error) {
+ // There is no real filesystem backing this ramfs tree, so we pass in
+ // "nil" here.
+ mountTree, err := ramfs.MakeDirectoryTree(ctx, fs.NewNonCachingMountSource(nil, fs.MountSourceFlags{}), submounts)
+ if err != nil {
+ return nil, fmt.Errorf("error creating mount tree: %v", err)
+ }
+ overlayInode, err := fs.NewOverlayRoot(ctx, inode, mountTree, fs.MountSourceFlags{})
+ if err != nil {
+ return nil, fmt.Errorf("failed to make mount overlay: %v", err)
+ }
+ return overlayInode, err
+}
+
+// subtargets takes a set of Mounts and returns only the targets that are
+// children of the given root. The returned paths are relative to the root.
+func subtargets(root string, mnts []specs.Mount) []string {
+ r := filepath.Clean(root)
+ var targets []string
+ for _, mnt := range mnts {
+ t := filepath.Clean(mnt.Destination)
+ if strings.HasPrefix(t, r) {
+ // Make the mnt path relative to the root path. If the
+ // result is empty, then mnt IS the root mount, not a
+ // submount. We don't want to include those.
+ if t := strings.TrimPrefix(t, r); t != "" {
+ targets = append(targets, t)
+ }
+ }
+ }
+ return targets
+}
diff --git a/runsc/boot/limits.go b/runsc/boot/limits.go
new file mode 100644
index 000000000..ea72de8e9
--- /dev/null
+++ b/runsc/boot/limits.go
@@ -0,0 +1,60 @@
+// 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 boot
+
+import (
+ "fmt"
+
+ specs "github.com/opencontainers/runtime-spec/specs-go"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/limits"
+)
+
+// Mapping from linux resource names to limits.LimitType.
+var fromLinuxResource = map[string]limits.LimitType{
+ "RLIMIT_CPU": limits.CPU,
+ "RLIMIT_FSIZE": limits.FileSize,
+ "RLIMIT_DATA": limits.Data,
+ "RLIMIT_STACK": limits.Stack,
+ "RLIMIT_CORE": limits.Core,
+ "RLIMIT_RSS": limits.Rss,
+ "RLIMIT_NPROC": limits.ProcessCount,
+ "RLIMIT_NOFILE": limits.NumberOfFiles,
+ "RLIMIT_MEMLOCK": limits.MemoryPagesLocked,
+ "RLIMIT_AS": limits.AS,
+ "RLIMIT_LOCKS": limits.Locks,
+ "RLIMIT_SIGPENDING": limits.SignalsPending,
+ "RLIMIT_MSGQUEUE": limits.MessageQueueBytes,
+ "RLIMIT_NICE": limits.Nice,
+ "RLIMIT_RTPRIO": limits.RealTimePriority,
+ "RLIMIT_RTTIME": limits.Rttime,
+}
+
+func createLimitSet(spec *specs.Spec) (*limits.LimitSet, error) {
+ ls, err := limits.NewLinuxDistroLimitSet()
+ if err != nil {
+ return nil, err
+ }
+ for _, rl := range spec.Process.Rlimits {
+ lt, ok := fromLinuxResource[rl.Type]
+ if !ok {
+ return nil, fmt.Errorf("unknown resource %q", rl.Type)
+ }
+ ls.SetUnchecked(lt, limits.Limit{
+ Cur: rl.Soft,
+ Max: rl.Hard,
+ })
+ }
+ return ls, nil
+}
diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go
new file mode 100644
index 000000000..a470cb054
--- /dev/null
+++ b/runsc/boot/loader.go
@@ -0,0 +1,354 @@
+// 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 boot loads the kernel and runs the application.
+package boot
+
+import (
+ "fmt"
+ "math/rand"
+ "sync/atomic"
+ "syscall"
+ gtime "time"
+
+ specs "github.com/opencontainers/runtime-spec/specs-go"
+ "gvisor.googlesource.com/gvisor/pkg/abi/linux"
+ "gvisor.googlesource.com/gvisor/pkg/cpuid"
+ "gvisor.googlesource.com/gvisor/pkg/log"
+ "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/loader"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/platform"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/platform/kvm"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/platform/ptrace"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/sighandling"
+ slinux "gvisor.googlesource.com/gvisor/pkg/sentry/syscalls/linux"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/time"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/watchdog"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/link/sniffer"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/network/arp"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/network/ipv4"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/network/ipv6"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/stack"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/tcp"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/transport/udp"
+ "gvisor.googlesource.com/gvisor/runsc/boot/filter"
+ "gvisor.googlesource.com/gvisor/runsc/specutils"
+
+ // Include supported socket providers.
+ "gvisor.googlesource.com/gvisor/pkg/sentry/socket/epsocket"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/socket/hostinet"
+ _ "gvisor.googlesource.com/gvisor/pkg/sentry/socket/netlink"
+ _ "gvisor.googlesource.com/gvisor/pkg/sentry/socket/netlink/route"
+ _ "gvisor.googlesource.com/gvisor/pkg/sentry/socket/unix"
+)
+
+// Loader keeps state needed to start the kernel and run the application.
+type Loader struct {
+ // k is the kernel.
+ k *kernel.Kernel
+
+ // ctrl is the control server.
+ ctrl *controller
+
+ conf *Config
+
+ // console is set to true if terminal is enabled.
+ console bool
+
+ watchdog *watchdog.Watchdog
+
+ // stopSignalForwarding disables forwarding of signals to the sandboxed
+ // app. It should be called when a sandbox is destroyed.
+ stopSignalForwarding func()
+
+ // procArgs refers to the initial application task.
+ procArgs kernel.CreateProcessArgs
+}
+
+func init() {
+ // Initialize the random number generator.
+ rand.Seed(gtime.Now().UnixNano())
+
+ // Register the global syscall table.
+ kernel.RegisterSyscallTable(slinux.AMD64)
+}
+
+// New initializes a new kernel loader configured by spec.
+func New(spec *specs.Spec, conf *Config, controllerFD int, ioFDs []int, console bool) (*Loader, error) {
+ // Create kernel and platform.
+ p, err := createPlatform(conf)
+ if err != nil {
+ return nil, fmt.Errorf("error creating platform: %v", err)
+ }
+ k := &kernel.Kernel{
+ Platform: p,
+ }
+
+ // Create VDSO.
+ vdso, err := loader.PrepareVDSO(p)
+ if err != nil {
+ return nil, fmt.Errorf("error creating vdso: %v", err)
+ }
+
+ // Create timekeeper.
+ tk, err := kernel.NewTimekeeper(k, vdso.ParamPage.FileRange())
+ if err != nil {
+ return nil, fmt.Errorf("error creating timekeeper: %v", err)
+ }
+ tk.SetClocks(time.NewCalibratedClocks())
+
+ // Create initial limits.
+ ls, err := createLimitSet(spec)
+ if err != nil {
+ return nil, fmt.Errorf("error creating limits: %v", err)
+ }
+
+ // Create capabilities.
+ caps, err := specutils.Capabilities(spec.Process.Capabilities)
+ if err != nil {
+ return nil, fmt.Errorf("error creating capabilities: %v", err)
+ }
+
+ // Convert the spec's additional GIDs to KGIDs.
+ extraKGIDs := make([]auth.KGID, 0, len(spec.Process.User.AdditionalGids))
+ for _, GID := range spec.Process.User.AdditionalGids {
+ extraKGIDs = append(extraKGIDs, auth.KGID(GID))
+ }
+
+ // Create credentials.
+ creds := auth.NewUserCredentials(
+ auth.KUID(spec.Process.User.UID),
+ auth.KGID(spec.Process.User.GID),
+ extraKGIDs,
+ caps,
+ auth.NewRootUserNamespace())
+ if err != nil {
+ return nil, fmt.Errorf("error creating credentials: %v", err)
+ }
+
+ // Create user namespace.
+ // TODO: Not clear what domain name should be here. It is
+ // not configurable from runtime spec.
+ utsns := kernel.NewUTSNamespace(spec.Hostname, "", creds.UserNamespace)
+
+ ipcns := kernel.NewIPCNamespace()
+
+ if err := enableStrace(conf); err != nil {
+ return nil, fmt.Errorf("failed to enable strace: %v", err)
+ }
+
+ // Get the executable path, which is a bit tricky because we have to
+ // inspect the environment PATH which is relative to the root path.
+ exec, err := specutils.GetExecutablePath(spec.Process.Args[0], spec.Root.Path, spec.Process.Env)
+ if err != nil {
+ return nil, fmt.Errorf("error getting executable path: %v", err)
+ }
+
+ // Create the process arguments.
+ procArgs := kernel.CreateProcessArgs{
+ Filename: exec,
+ Argv: spec.Process.Args,
+ Envv: spec.Process.Env,
+ WorkingDirectory: spec.Process.Cwd,
+ Credentials: creds,
+ // Creating the FDMap requires that we have kernel.Kernel.fdMapUids, so
+ // it must wait until we have a Kernel.
+ Umask: uint(syscall.Umask(0)),
+ Limits: ls,
+ MaxSymlinkTraversals: linux.MaxSymlinkTraversals,
+ UTSNamespace: utsns,
+ IPCNamespace: ipcns,
+ }
+
+ // 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 := newEmptyNetworkStack(conf)
+
+ // Initiate the Kernel object, which is required by the Context passed
+ // to createVFS in order to mount (among other things) procfs.
+ if err = k.Init(kernel.InitKernelArgs{
+ FeatureSet: cpuid.HostFeatureSet(),
+ Timekeeper: tk,
+ RootUserNamespace: creds.UserNamespace,
+ NetworkStack: networkStack,
+ ApplicationCores: 8,
+ Vdso: vdso,
+ RootUTSNamespace: utsns,
+ RootIPCNamespace: ipcns,
+ }); err != nil {
+ return nil, fmt.Errorf("error initializing kernel: %v", err)
+ }
+
+ // Turn on packet logging if enabled.
+ if conf.LogPackets {
+ log.Infof("Packet logging enabled")
+ atomic.StoreUint32(&sniffer.LogPackets, 1)
+ } else {
+ log.Infof("Packet logging disabled")
+ atomic.StoreUint32(&sniffer.LogPackets, 0)
+ }
+
+ // Create the control server using the provided FD.
+ //
+ // This must be done *after* we have initialized the kernel since the
+ // controller is used to configure the kernel's network stack.
+ //
+ // This should also be *before* we create the process, since a
+ // misconfigured process will cause an error, and we want the control
+ // server up before that so that we don't time out trying to connect to
+ // it.
+ ctrl, err := newController(controllerFD, k)
+ if err != nil {
+ return nil, fmt.Errorf("error creating control server: %v", err)
+ }
+
+ ctx := procArgs.NewContext(k)
+
+ // Create the virtual filesystem.
+ mm, err := createMountNamespace(ctx, spec, conf, ioFDs)
+ if err != nil {
+ return nil, fmt.Errorf("error creating mounts: %v", err)
+ }
+ k.SetRootMountNamespace(mm)
+
+ // Create the FD map, which will set stdin, stdout, and stderr. If console
+ // is true, then ioctl calls will be passed through to the host fd.
+ fdm, err := createFDMap(ctx, k, ls, console)
+ if err != nil {
+ return nil, fmt.Errorf("error importing fds: %v", err)
+ }
+
+ // CreateProcess takes a reference on FDMap if successful. We
+ // won't need ours either way.
+ procArgs.FDMap = fdm
+
+ // We don't care about child signals; some platforms can generate a
+ // tremendous number of useless ones (I'm looking at you, ptrace).
+ if err := sighandling.IgnoreChildStop(); err != nil {
+ return nil, fmt.Errorf("failed to ignore child stop signals: %v", err)
+ }
+ // Ensure that most signals received in sentry context are forwarded to
+ // the emulated kernel.
+ stopSignalForwarding := sighandling.StartForwarding(k)
+
+ watchdog := watchdog.New(k, watchdog.DefaultTimeout, watchdog.LogWarning)
+ return &Loader{
+ k: k,
+ ctrl: ctrl,
+ conf: conf,
+ console: console,
+ watchdog: watchdog,
+ stopSignalForwarding: stopSignalForwarding,
+ procArgs: procArgs,
+ }, nil
+}
+
+// Destroy cleans up all resources used by the loader.
+func (l *Loader) Destroy() {
+ if l.ctrl != nil {
+ // Shut down control server.
+ l.ctrl.srv.Stop()
+ }
+ l.stopSignalForwarding()
+ l.watchdog.Stop()
+}
+
+func createPlatform(conf *Config) (platform.Platform, error) {
+ switch conf.Platform {
+ case PlatformPtrace:
+ log.Infof("Platform: ptrace")
+ return ptrace.New()
+ case PlatformKVM:
+ log.Infof("Platform: kvm")
+ return kvm.New()
+ default:
+ return nil, fmt.Errorf("invalid platform %v", conf.Platform)
+ }
+}
+
+// Run runs the application.
+func (l *Loader) Run() error {
+ err := l.run()
+ l.ctrl.app.startResultChan <- err
+ return err
+}
+
+func (l *Loader) run() error {
+ if l.conf.Network == NetworkHost {
+ // 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)
+ if err := stack.Configure(); err != nil {
+ return err
+ }
+ }
+
+ // Finally done with all configuration. Setup filters before user code
+ // is loaded.
+ if l.conf.DisableSeccomp {
+ filter.Report("syscall filter is DISABLED. Running in less secure mode.")
+ } else {
+ whitelistFS := l.conf.FileAccess == FileAccessDirect
+ hostNet := l.conf.Network == NetworkHost
+ if err := filter.Install(l.k.Platform, whitelistFS, l.console, hostNet); err != nil {
+ return fmt.Errorf("Failed to install seccomp filters: %v", err)
+ }
+ }
+
+ // Create the initial application task.
+ if _, err := l.k.CreateProcess(l.procArgs); err != nil {
+ return fmt.Errorf("failed to create init process: %v", err)
+ }
+
+ // CreateProcess takes a reference on FDMap if successful.
+ l.procArgs.FDMap.DecRef()
+
+ l.watchdog.Start()
+ return l.k.Start()
+}
+
+// WaitForStartSignal waits for a start signal from the control server.
+func (l *Loader) WaitForStartSignal() {
+ <-l.ctrl.app.startChan
+}
+
+// WaitExit waits for the application to exit, and returns the application's
+// exit status.
+func (l *Loader) WaitExit() kernel.ExitStatus {
+ // Wait for application.
+ l.k.WaitExited()
+
+ return l.k.GlobalInit().ExitStatus()
+}
+
+func newEmptyNetworkStack(conf *Config) inet.Stack {
+ switch conf.Network {
+ case NetworkHost:
+ return hostinet.NewStack()
+
+ case NetworkNone, NetworkSandbox:
+ // NetworkNone sets up loopback using netstack.
+ netProtos := []string{ipv4.ProtocolName, ipv6.ProtocolName, arp.ProtocolName}
+ protoNames := []string{tcp.ProtocolName, udp.ProtocolName}
+ return &epsocket.Stack{stack.New(netProtos, protoNames)}
+
+ default:
+ panic(fmt.Sprintf("invalid network configuration: %v", conf.Network))
+ }
+}
diff --git a/runsc/boot/loader_test.go b/runsc/boot/loader_test.go
new file mode 100644
index 000000000..2fc16b241
--- /dev/null
+++ b/runsc/boot/loader_test.go
@@ -0,0 +1,238 @@
+// 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 boot
+
+import (
+ "os"
+ "testing"
+ "time"
+
+ specs "github.com/opencontainers/runtime-spec/specs-go"
+ "gvisor.googlesource.com/gvisor/pkg/control/server"
+ "gvisor.googlesource.com/gvisor/pkg/log"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/context/contexttest"
+)
+
+func init() {
+ log.SetLevel(log.Debug)
+}
+
+// testSpec returns a simple spec that can be used in tests.
+func testSpec() *specs.Spec {
+ return &specs.Spec{
+ // The host filesystem root is the sandbox root.
+ Root: &specs.Root{
+ Path: "/",
+ Readonly: true,
+ },
+ Process: &specs.Process{
+ Args: []string{"/bin/true"},
+ },
+ }
+}
+
+func createLoader() (*Loader, error) {
+ fd, err := server.CreateSocket(ControlSocketAddr("123"))
+ if err != nil {
+ return nil, err
+ }
+ conf := &Config{
+ RootDir: "unused_root_dir",
+ Network: NetworkNone,
+ FileAccess: FileAccessDirect,
+ DisableSeccomp: true,
+ }
+ return New(testSpec(), conf, fd, nil, false)
+}
+
+// TestRun runs a simple application in a sandbox and checks that it succeeds.
+func TestRun(t *testing.T) {
+ s, err := createLoader()
+ if err != nil {
+ t.Fatalf("error creating loader: %v", err)
+ }
+ defer s.Destroy()
+
+ // Run the application.
+ if err := s.Run(); err != nil {
+ t.Errorf("error running application: %v", err)
+ }
+
+ // Wait for the application to exit. It should succeed.
+ if status := s.WaitExit(); status.Code != 0 || status.Signo != 0 {
+ t.Errorf("application exited with status %+v, want 0", status)
+ }
+}
+
+// TestStartSignal tests that the controller Start message will cause
+// WaitForStartSignal to return.
+func TestStartSignal(t *testing.T) {
+ s, err := createLoader()
+ if err != nil {
+ t.Fatalf("error creating loader: %v", err)
+ }
+ defer s.Destroy()
+
+ // We aren't going to wait on this application, so the control server
+ // needs to be shut down manually.
+ defer s.ctrl.srv.Stop()
+
+ // Start a goroutine that calls WaitForStartSignal and writes to a
+ // channel when it returns.
+ waitFinished := make(chan struct{})
+ go func() {
+ s.WaitForStartSignal()
+ // Pretent that Run() executed and returned no error.
+ s.ctrl.app.startResultChan <- nil
+ waitFinished <- struct{}{}
+ }()
+
+ // Nothing has been written to the channel, so waitFinished should not
+ // return. Give it a little bit of time to make sure the goroutine has
+ // started.
+ select {
+ case <-waitFinished:
+ t.Errorf("WaitForStartSignal completed but it should not have")
+ case <-time.After(50 * time.Millisecond):
+ // OK.
+ }
+
+ // Trigger the control server Start method.
+ if err := s.ctrl.app.Start(nil, nil); err != nil {
+ t.Errorf("error calling Start: %v", err)
+ }
+
+ // Now WaitForStartSignal should return (within a short amount of
+ // time).
+ select {
+ case <-waitFinished:
+ // OK.
+ case <-time.After(50 * time.Millisecond):
+ t.Errorf("WaitForStartSignal did not complete but it should have")
+ }
+
+}
+
+// Test that MountNamespace can be created with various specs.
+func TestCreateMountNamespace(t *testing.T) {
+ conf := &Config{
+ RootDir: "unused_root_dir",
+ FileAccess: FileAccessDirect,
+ DisableSeccomp: true,
+ }
+
+ testCases := []struct {
+ name string
+ // Spec that will be used to create the mount manager. Note
+ // that we can't mount procfs without a kernel, so each spec
+ // MUST contain something other than procfs mounted at /proc.
+ spec specs.Spec
+ // Paths that are expected to exist in the resulting fs.
+ expectedPaths []string
+ }{
+ {
+ // Only proc.
+ name: "only proc mount",
+ spec: specs.Spec{
+ Root: &specs.Root{
+ Path: os.TempDir(),
+ Readonly: true,
+ },
+ Mounts: []specs.Mount{
+ {
+ Destination: "/proc",
+ Type: "tmpfs",
+ },
+ },
+ },
+ // /proc, /dev, and /sys should always be mounted.
+ expectedPaths: []string{"/proc", "/dev", "/sys"},
+ },
+ {
+ // Mount at a deep path, with many components that do
+ // not exist in the root.
+ name: "deep mount path",
+ spec: specs.Spec{
+ Root: &specs.Root{
+ Path: os.TempDir(),
+ Readonly: true,
+ },
+ Mounts: []specs.Mount{
+ {
+ Destination: "/some/very/very/deep/path",
+ Type: "tmpfs",
+ },
+ {
+ Destination: "/proc",
+ Type: "tmpfs",
+ },
+ },
+ },
+ // /some/deep/path should be mounted, along with /proc,
+ // /dev, and /sys.
+ expectedPaths: []string{"/some/very/very/deep/path", "/proc", "/dev", "/sys"},
+ },
+ {
+ // Mounts are nested inside eachother.
+ name: "nested mounts",
+ spec: specs.Spec{
+ Root: &specs.Root{
+ Path: os.TempDir(),
+ Readonly: true,
+ },
+ Mounts: []specs.Mount{
+ {
+ Destination: "/proc",
+ Type: "tmpfs",
+ },
+ {
+ Destination: "/foo",
+ Type: "tmpfs",
+ },
+ {
+ Destination: "/foo/bar",
+ Type: "tmpfs",
+ },
+ {
+ Destination: "/foo/bar/baz",
+ Type: "tmpfs",
+ },
+ {
+ // A deep path that is in foo but not the other mounts.
+ Destination: "/foo/some/very/very/deep/path",
+ Type: "tmpfs",
+ },
+ },
+ },
+ expectedPaths: []string{"/foo", "/foo/bar", "/foo/bar/baz", "/foo/some/very/very/deep/path", "/proc", "/dev", "/sys"},
+ },
+ }
+
+ for _, tc := range testCases {
+ ctx := contexttest.Context(t)
+ mm, err := createMountNamespace(ctx, &tc.spec, conf, nil)
+ if err != nil {
+ t.Fatalf("createMountNamespace test case %q failed: %v", tc.name, err)
+ }
+ defer mm.DecRef()
+ root := mm.Root()
+ defer root.DecRef()
+ for _, p := range tc.expectedPaths {
+ if _, err := mm.FindInode(ctx, root, root, p, 0); err != nil {
+ t.Errorf("expected path %v to exist with spec %v, but got error %v", p, tc.spec, err)
+ }
+ }
+ }
+}
diff --git a/runsc/boot/network.go b/runsc/boot/network.go
new file mode 100644
index 000000000..d2b52c823
--- /dev/null
+++ b/runsc/boot/network.go
@@ -0,0 +1,213 @@
+// 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 boot
+
+import (
+ "fmt"
+ "math/rand"
+ "net"
+ "syscall"
+
+ "gvisor.googlesource.com/gvisor/pkg/log"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/link/fdbased"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/link/loopback"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/link/sniffer"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/network/arp"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/network/ipv4"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/network/ipv6"
+ "gvisor.googlesource.com/gvisor/pkg/tcpip/stack"
+ "gvisor.googlesource.com/gvisor/pkg/urpc"
+)
+
+// Network exposes methods that can be used to configure a network stack.
+type Network struct {
+ Stack *stack.Stack
+}
+
+// Route represents a route in the network stack.
+type Route struct {
+ Destination net.IP
+ Mask net.IPMask
+ Gateway net.IP
+}
+
+// DefaultRoute represents a catch all route to the default gateway.
+type DefaultRoute struct {
+ Route Route
+ Name string
+}
+
+// FDBasedLink configures an fd-based link.
+type FDBasedLink struct {
+ Name string
+ MTU int
+ Addresses []net.IP
+ Routes []Route
+}
+
+// LoopbackLink configures a loopback li nk.
+type LoopbackLink struct {
+ Name string
+ Addresses []net.IP
+ Routes []Route
+}
+
+// CreateLinksAndRoutesArgs are arguments to CreateLinkAndRoutes.
+type CreateLinksAndRoutesArgs struct {
+ // FilePayload contains the fds associated with the FDBasedLinks. The
+ // two slices must have the same length.
+ urpc.FilePayload
+
+ LoopbackLinks []LoopbackLink
+ FDBasedLinks []FDBasedLink
+
+ DefaultGateway DefaultRoute
+}
+
+// Empty returns true if route hasn't been set.
+func (r *Route) Empty() bool {
+ return r.Destination == nil && r.Mask == nil && r.Gateway == nil
+}
+
+func (r *Route) toTcpipRoute(id tcpip.NICID) tcpip.Route {
+ return tcpip.Route{
+ Destination: ipToAddress(r.Destination),
+ Gateway: ipToAddress(r.Gateway),
+ Mask: ipToAddress(net.IP(r.Mask)),
+ NIC: id,
+ }
+}
+
+// CreateLinksAndRoutes creates links and routes in a network stack. It should
+// only be called once.
+func (n *Network) CreateLinksAndRoutes(args *CreateLinksAndRoutesArgs, _ *struct{}) error {
+ if len(args.FilePayload.Files) != len(args.FDBasedLinks) {
+ return fmt.Errorf("FilePayload must be same length at FDBasedLinks")
+ }
+
+ var nicID tcpip.NICID
+ nicids := make(map[string]tcpip.NICID)
+
+ // Collect routes from all links.
+ var routes []tcpip.Route
+
+ // Loopback normally appear before other interfaces.
+ for _, link := range args.LoopbackLinks {
+ nicID++
+ nicids[link.Name] = nicID
+
+ linkEP := loopback.New()
+
+ log.Infof("Enabling loopback interface %q with id %d on addresses %+v", link.Name, nicID, link.Addresses)
+ if err := n.createNICWithAddrs(nicID, link.Name, linkEP, link.Addresses); err != nil {
+ return err
+ }
+
+ // Collect the routes from this link.
+ for _, r := range link.Routes {
+ routes = append(routes, r.toTcpipRoute(nicID))
+ }
+ }
+
+ for i, link := range args.FDBasedLinks {
+ nicID++
+ nicids[link.Name] = nicID
+
+ // Copy the underlying FD.
+ oldFD := args.FilePayload.Files[i].Fd()
+ newFD, err := syscall.Dup(int(oldFD))
+ if err != nil {
+ return fmt.Errorf("failed to dup FD %v: %v", oldFD, err)
+ }
+
+ linkEP := fdbased.New(&fdbased.Options{
+ FD: newFD,
+ MTU: uint32(link.MTU),
+ ChecksumOffload: false,
+ EthernetHeader: true,
+ Address: tcpip.LinkAddress(generateRndMac()),
+ })
+
+ log.Infof("Enabling interface %q with id %d on addresses %+v", link.Name, nicID, link.Addresses)
+ if err := n.createNICWithAddrs(nicID, link.Name, linkEP, link.Addresses); err != nil {
+ return err
+ }
+
+ // Collect the routes from this link.
+ for _, r := range link.Routes {
+ routes = append(routes, r.toTcpipRoute(nicID))
+ }
+ }
+
+ if !args.DefaultGateway.Route.Empty() {
+ nicID, ok := nicids[args.DefaultGateway.Name]
+ if !ok {
+ return fmt.Errorf("invalid interface name %q for default route", args.DefaultGateway.Name)
+ }
+ routes = append(routes, args.DefaultGateway.Route.toTcpipRoute(nicID))
+ }
+
+ log.Infof("Setting routes %+v", routes)
+ n.Stack.SetRouteTable(routes)
+ return nil
+}
+
+// createNICWithAddrs creates a NIC in the network stack and adds the given
+// addresses.
+func (n *Network) createNICWithAddrs(id tcpip.NICID, name string, linkEP tcpip.LinkEndpointID, addrs []net.IP) error {
+ if err := n.Stack.CreateNamedNIC(id, name, sniffer.New(linkEP)); err != nil {
+ return fmt.Errorf("CreateNamedNIC(%v, %v, %v) failed: %v", id, name, linkEP, err)
+ }
+
+ // Always start with an arp address for the NIC.
+ if err := n.Stack.AddAddress(id, arp.ProtocolNumber, arp.ProtocolAddress); err != nil {
+ return fmt.Errorf("AddAddress(%v, %v, %v) failed: %v", id, arp.ProtocolNumber, arp.ProtocolAddress, err)
+ }
+
+ for _, addr := range addrs {
+ proto, tcpipAddr := ipToAddressAndProto(addr)
+ if err := n.Stack.AddAddress(id, proto, tcpipAddr); err != nil {
+ return fmt.Errorf("AddAddress(%v, %v, %v) failed: %v", id, proto, tcpipAddr, err)
+ }
+ }
+ return nil
+}
+
+// ipToAddressAndProto converts IP to tcpip.Address and a protocol number.
+//
+// Note: don't use 'len(ip)' to determine IP version because length is always 16.
+func ipToAddressAndProto(ip net.IP) (tcpip.NetworkProtocolNumber, tcpip.Address) {
+ if i4 := ip.To4(); i4 != nil {
+ return ipv4.ProtocolNumber, tcpip.Address(i4)
+ }
+ return ipv6.ProtocolNumber, tcpip.Address(ip)
+}
+
+// ipToAddress converts IP to tcpip.Address, ignoring the protocol.
+func ipToAddress(ip net.IP) tcpip.Address {
+ _, addr := ipToAddressAndProto(ip)
+ return addr
+}
+
+// generateRndMac returns a random local MAC address.
+// Copied from eth_random_addr() (include/linux/etherdevice.h)
+func generateRndMac() net.HardwareAddr {
+ mac := make(net.HardwareAddr, 6)
+ rand.Read(mac)
+ mac[0] &^= 0x1 // clear multicast bit
+ mac[0] |= 0x2 // set local assignment bit (IEEE802)
+ return mac
+}
diff --git a/runsc/boot/strace.go b/runsc/boot/strace.go
new file mode 100644
index 000000000..1e898672b
--- /dev/null
+++ b/runsc/boot/strace.go
@@ -0,0 +1,40 @@
+// 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 boot
+
+import (
+ "gvisor.googlesource.com/gvisor/pkg/sentry/strace"
+)
+
+func enableStrace(conf *Config) error {
+ // We must initialize even if strace is not enabled.
+ strace.Initialize()
+
+ if !conf.Strace {
+ return nil
+ }
+
+ max := conf.StraceLogSize
+ if max == 0 {
+ max = 1024
+ }
+ strace.LogMaximumSize = max
+
+ if len(conf.StraceSyscalls) == 0 {
+ strace.EnableAll(strace.SinkTypeLog)
+ return nil
+ }
+ return strace.Enable(conf.StraceSyscalls, strace.SinkTypeLog)
+}