wrap and decouplin libseccomp

This commit is contained in:
criyle 2019-09-01 01:36:01 -07:00
parent 7b06ce65ac
commit 86c5e2f377
19 changed files with 357 additions and 325 deletions

View File

@ -12,6 +12,8 @@ import (
"github.com/criyle/go-sandbox/pkg/cgroup"
"github.com/criyle/go-sandbox/pkg/memfd"
"github.com/criyle/go-sandbox/pkg/rlimit"
"github.com/criyle/go-sandbox/pkg/seccomp"
"github.com/criyle/go-sandbox/pkg/seccomp/libseccomp"
"github.com/criyle/go-sandbox/runner"
"github.com/criyle/go-sandbox/runner/config"
"github.com/criyle/go-sandbox/runner/ptrace"
@ -206,6 +208,11 @@ func start() (*types.Result, error) {
Stack: stackLimit,
}
actionDefault := seccomp.ActionKill
if showDetails {
actionDefault = seccomp.ActionTrace.WithReturnCode(seccomp.MsgDisallow)
}
if useDeamon {
root, err := ioutil.TempDir("", "dm")
if err != nil {
@ -234,7 +241,14 @@ func start() (*types.Result, error) {
},
}
} else if namespace {
h.SyscallAllow = append(h.SyscallAllow, h.SyscallTrace...)
builder := libseccomp.Builder{
Allow: append(h.SyscallAllow, h.SyscallTrace...),
Default: actionDefault,
}
filter, err := builder.Build()
if err != nil {
return nil, fmt.Errorf("cannot build seccomp filter %v", err)
}
root, err := ioutil.TempDir("", "ns")
if err != nil {
return nil, fmt.Errorf("cannot make temp root for new namespace")
@ -248,11 +262,11 @@ func start() (*types.Result, error) {
WorkDir: "/w",
Files: fds,
RLimits: rlims,
ResLimits: types.Limit{
Limit: types.Limit{
TimeLimit: timeLimit * 1e3,
MemoryLimit: memoryLimit << 10,
},
SyscallAllowed: h.SyscallAllow,
Seccomp: filter,
Root: root,
Mounts: unshare.GetDefaultMounts(root, []unshare.AddBind{
{
@ -266,19 +280,27 @@ func start() (*types.Result, error) {
DomainName: "run_program",
}
} else {
builder := libseccomp.Builder{
Allow: h.SyscallAllow,
Trace: h.SyscallTrace,
Default: actionDefault,
}
filter, err := builder.Build()
if err != nil {
return nil, fmt.Errorf("failed to create seccomp filter %v", err)
}
runner = &ptrace.Runner{
Args: h.Args,
Env: []string{pathEnv},
ExecFile: execFile,
WorkDir: workPath,
RLimits: rlims,
TraceLimit: types.Limit{
Limit: types.Limit{
TimeLimit: timeLimit * 1e3,
MemoryLimit: memoryLimit << 10,
},
Files: fds,
SyscallAllowed: h.SyscallAllow,
SyscallTraced: h.SyscallTrace,
Seccomp: filter,
ShowDetails: showDetails,
Unsafe: unsafe,
Handler: h,

2
go.mod
View File

@ -4,5 +4,5 @@ go 1.12
require (
github.com/seccomp/libseccomp-golang v0.9.1
golang.org/x/sys v0.0.0-20190830023255-19e00faab6ad
golang.org/x/sys v0.0.0-20190830142957-1e83adbbebd0
)

4
go.sum
View File

@ -1,4 +1,4 @@
github.com/seccomp/libseccomp-golang v0.9.1 h1:NJjM5DNFOs0s3kYE1WUOr6G8V97sdt46rlXTMfXGWBo=
github.com/seccomp/libseccomp-golang v0.9.1/go.mod h1:GbW5+tmTXfcxTToHLXlScSlAvWlF4P2Ca7zGrPiEpWo=
golang.org/x/sys v0.0.0-20190830023255-19e00faab6ad h1:cCejgArrk10gX6kFqjWeLwXD7aVMqWoRpyUCaaJSggc=
golang.org/x/sys v0.0.0-20190830023255-19e00faab6ad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190830142957-1e83adbbebd0 h1:7z820YPX9pxWR59qM7BE5+fglp4D/mKqAwCvGt11b+8=
golang.org/x/sys v0.0.0-20190830142957-1e83adbbebd0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=

35
pkg/seccomp/action.go Normal file
View File

@ -0,0 +1,35 @@
package seccomp
// Action is seccomp trap action
type Action uint32
// Action defines seccomp action to the syscall
// default value 0 is invalid
const (
ActionAllow Action = iota + 1
ActionErrno
ActionTrace
ActionKill
)
// MsgDisallow, Msghandle defines the action needed when traped by
// seccomp filter
const (
MsgDisallow int16 = iota + 1
MsgHandle
)
// WithReturnCode set the return code when action is trace or ban
func (a Action) WithReturnCode(code int16) Action {
return a.Action() | Action(code)<<16
}
// ReturnCode get the return code
func (a Action) ReturnCode() int16 {
return int16(a >> 16)
}
// Action get the basic action
func (a Action) Action() Action {
return Action(a & 0xffff)
}

22
pkg/seccomp/filter.go Normal file
View File

@ -0,0 +1,22 @@
// Package seccomp provides a generated filter format for seccomp filter
// +build linux
package seccomp
import (
"syscall"
"unsafe"
)
// Filter is the BPF seccomp filter value
type Filter []byte
// SockFprog converts Filter to SockFprog for seccomp syscall
func (f Filter) SockFprog() *syscall.SockFprog {
b := []byte(f)
return &syscall.SockFprog{
Len: uint16(len(b) / 8),
Filter: (*syscall.SockFilter)(unsafe.Pointer(&b[0])),
}
}

View File

@ -0,0 +1,23 @@
package libseccomp
import (
"github.com/criyle/go-sandbox/pkg/seccomp"
libseccomp "github.com/seccomp/libseccomp-golang"
)
// ToSeccompAction convert action to libseccomp compatible action
func ToSeccompAction(a seccomp.Action) libseccomp.ScmpAction {
var action libseccomp.ScmpAction
switch a.Action() {
case seccomp.ActionAllow:
action = libseccomp.ActAllow
case seccomp.ActionErrno:
action = libseccomp.ActErrno
case seccomp.ActionTrace:
action = libseccomp.ActTrace
default:
action = libseccomp.ActKill
}
action = action.SetReturnCode(a.ReturnCode())
return action
}

View File

@ -0,0 +1,76 @@
// Package libseccomp provides wrapper to "github.com/seccomp/libseccomp-golang"
package libseccomp
import (
"io/ioutil"
"os"
"github.com/criyle/go-sandbox/pkg/seccomp"
libseccomp "github.com/seccomp/libseccomp-golang"
)
// Builder is used to build the filter
type Builder struct {
Allow, Trace []string
Default seccomp.Action
}
var actTrace = libseccomp.ActTrace.SetReturnCode(seccomp.MsgHandle)
// Build builds the filter
func (b *Builder) Build() (seccomp.Filter, error) {
filter, err := libseccomp.NewFilter(ToSeccompAction(b.Default))
if err != nil {
return nil, err
}
defer filter.Release()
if err = addFilterActions(filter, b.Allow, libseccomp.ActAllow); err != nil {
return nil, err
}
if err = addFilterActions(filter, b.Trace, actTrace); err != nil {
return nil, err
}
return ExportBPF(filter)
}
// ExportBPF convert libseccomp filter to kernel readable BPF content
func ExportBPF(filter *libseccomp.ScmpFilter) (seccomp.Filter, error) {
r, w, err := os.Pipe()
if err != nil {
return nil, err
}
defer r.Close()
// export BPF to pipe
go func() {
filter.ExportBPF(w)
w.Close()
}()
// get BPF binary
bin, err := ioutil.ReadAll(r)
if err != nil {
return nil, err
}
return seccomp.Filter(bin), nil
}
func addFilterActions(filter *libseccomp.ScmpFilter, names []string, action libseccomp.ScmpAction) error {
for _, s := range names {
if err := addFilterAction(filter, s, action); err != nil {
return err
}
}
return nil
}
func addFilterAction(filter *libseccomp.ScmpFilter, name string, action libseccomp.ScmpAction) error {
syscallID, err := libseccomp.GetSyscallFromName(name)
if err != nil {
return err
}
if err = filter.AddRule(syscallID, action); err != nil {
return err
}
return nil
}

View File

@ -0,0 +1,53 @@
// +build linux
package libseccomp
import (
"testing"
"github.com/criyle/go-sandbox/pkg/seccomp"
libseccomp "github.com/seccomp/libseccomp-golang"
)
var (
defaultSyscallAllows = []string{
"read", "write", "readv", "writev", "close", "fstat", "lseek", "dup", "dup2", "dup3", "ioctl", "fcntl", "fadvise64",
"mmap", "mprotect", "munmap", "brk", "mremap", "msync", "mincore", "madvise",
"rt_sigaction", "rt_sigprocmask", "rt_sigreturn", "rt_sigpending", "sigaltstack",
"getcwd", "exit", "exit_group", "arch_prctl",
"gettimeofday", "getrlimit", "getrusage", "times", "time", "clock_gettime", "restart_syscall",
}
defaultSyscallTraces = []string{
"execve", "open", "openat", "unlink", "unlinkat", "readlink", "readlinkat", "lstat", "stat", "access", "faccessat",
}
)
func TestBuildFilter(t *testing.T) {
defaultAction := libseccomp.ActKill
_, err := buildFilterMock(defaultAction)
if err != nil {
t.Error("BuildFilter failed")
}
}
// BenchmarkBuildDefaultFilter is about 0.2ms/op
func BenchmarkBuildDefaultFilter(b *testing.B) {
for i := 0; i < b.N; i++ {
builder := Builder{
Allow: defaultSyscallAllows,
Trace: defaultSyscallTraces,
Default: seccomp.ActionTrace,
}
builder.Build()
}
}
func buildFilterMock(d libseccomp.ScmpAction) (seccomp.Filter, error) {
b := Builder{
Allow: []string{"fork"},
Trace: []string{"execve"},
Default: seccomp.ActionTrace,
}
return b.Build()
}

View File

@ -0,0 +1,10 @@
package libseccomp
import (
libseccomp "github.com/seccomp/libseccomp-golang"
)
// ToSyscallName convert syscallno to syscall name
func ToSyscallName(sysno uint) (string, error) {
return libseccomp.ScmpSyscall(sysno).GetName()
}

View File

@ -1,73 +0,0 @@
// Package seccomp provides utility functions to manipulate seccomp filters
// provided by libseccomp
package seccomp
import (
"io/ioutil"
"os"
"syscall"
"unsafe"
libseccomp "github.com/seccomp/libseccomp-golang"
)
// FilterToBPF convert libseccomp filter to kernel readable BPF style
func FilterToBPF(filter *libseccomp.ScmpFilter) (*syscall.SockFprog, error) {
r, w, err := os.Pipe()
if err != nil {
return nil, err
}
// export to pipe
go func() {
filter.ExportBPF(w)
w.Close()
}()
// get BPF binary
bin, err := ioutil.ReadAll(r)
if err != nil {
return nil, err
}
// directly convert pointer
return &syscall.SockFprog{
Len: uint16(len(bin) / 8),
Filter: (*syscall.SockFilter)(unsafe.Pointer(&bin[0])),
}, nil
}
// BuildFilter builds libseccomp filter by defining the default action, trace action
// allow and trace syscall names
func BuildFilter(defaultAct, traceAct libseccomp.ScmpAction, allow, trace []string) (*libseccomp.ScmpFilter, error) {
filter, err := libseccomp.NewFilter(defaultAct)
if err != nil {
return nil, err
}
for _, s := range allow {
err := addFilterAction(filter, s, libseccomp.ActAllow)
if err != nil {
return nil, err
}
}
for _, s := range trace {
err := addFilterAction(filter, s, traceAct)
if err != nil {
return nil, err
}
}
return filter, nil
}
func addFilterAction(filter *libseccomp.ScmpFilter, name string, action libseccomp.ScmpAction) error {
syscallID, err := libseccomp.GetSyscallFromName(name)
if err != nil {
return err
}
err = filter.AddRule(syscallID, action)
if err != nil {
return err
}
return nil
}

View File

@ -1,95 +0,0 @@
package seccomp
import (
"testing"
libseccomp "github.com/pkg/seccomp/libseccomp-golang"
)
var (
defaultSyscallAllows = []string{
"read", "write", "readv", "writev", "close", "fstat", "lseek", "dup", "dup2", "dup3", "ioctl", "fcntl", "fadvise64",
"mmap", "mprotect", "munmap", "brk", "mremap", "msync", "mincore", "madvise",
"rt_sigaction", "rt_sigprocmask", "rt_sigreturn", "rt_sigpending", "sigaltstack",
"getcwd", "exit", "exit_group", "arch_prctl",
"gettimeofday", "getrlimit", "getrusage", "times", "time", "clock_gettime", "restart_syscall",
}
defaultSyscallTraces = []string{
"execve", "open", "openat", "unlink", "unlinkat", "readlink", "readlinkat", "lstat", "stat", "access", "faccessat",
}
)
func TestBuildFilter(t *testing.T) {
defaultAction := libseccomp.ActKill
filter, err := buildFilterMock(defaultAction)
if err != nil {
t.Error("BuildFilter failed")
}
if d, e := filter.GetDefaultAction(); e != nil || d != defaultAction {
t.Error("DefaultAction does not match by BuildFilter")
}
}
func TestFilterToBPF(t *testing.T) {
defaultAction := libseccomp.ActKill
filter, err := buildFilterMock(defaultAction)
if err != nil {
t.Error("BuildFilter failed")
}
prog, err := FilterToBPF(filter)
if err != nil || prog == nil || prog.Filter == nil {
t.Error("BuildFilter failed")
}
}
func TestBuildFilterFail(t *testing.T) {
defaultAction := libseccomp.ActKill
defaultTrace := libseccomp.ActTrace
allow := []string{"fork"}
trace := []string{"execve"}
allowf := append(allow, "fail")
filter, err := BuildFilter(defaultAction, defaultTrace, allowf, trace)
if err == nil || filter != nil {
t.Error("BuildFilter did not detect failure")
}
tracef := append(trace, "fail")
filter, err = BuildFilter(defaultAction, defaultTrace, allow, tracef)
if err == nil || filter != nil {
t.Error("BuildFilter did not detect failure")
}
filter, err = BuildFilter(libseccomp.ActInvalid, defaultTrace, allow, trace)
if err == nil || filter != nil {
t.Error("BuildFilter did not detect failure")
}
}
func TestAddActionFail(t *testing.T) {
defaultAction := libseccomp.ActKill
filter, _ := libseccomp.NewFilter(defaultAction)
arch, _ := libseccomp.GetNativeArch()
filter.RemoveArch(arch)
err := addFilterAction(filter, "fork", defaultAction)
if err == nil {
t.Error("addFilterAction did not detect failure")
}
}
// BenchmarkBuildDefaultFilter is about 0.2ms/op
func BenchmarkBuildDefaultFilter(b *testing.B) {
for i := 0; i < b.N; i++ {
filter, _ := BuildFilter(libseccomp.ActKill, libseccomp.ActTrace, defaultSyscallAllows, defaultSyscallTraces)
_, _ = FilterToBPF(filter)
filter.Release()
}
}
func buildFilterMock(d libseccomp.ScmpAction) (*libseccomp.ScmpFilter, error) {
defaultTrace := libseccomp.ActTrace
allow := []string{"fork"}
trace := []string{"execve"}
return BuildFilter(d, defaultTrace, allow, trace)
}

View File

@ -1,5 +1,7 @@
package ptracer
import "github.com/criyle/go-sandbox/types"
// TraceAction defines the action returned by TraceHandle
type TraceAction int
@ -12,6 +14,13 @@ const (
TraceKill
)
// Tracer defines a ptracer instance
type Tracer struct {
Handler
Runner
types.Limit
}
// Runner represents the process runner
type Runner interface {
// Starts starts the child process and return pid and error if failed

View File

@ -6,18 +6,12 @@ import (
unix "golang.org/x/sys/unix"
"github.com/criyle/go-sandbox/pkg/seccomp"
"github.com/criyle/go-sandbox/types"
)
// MsgDisallow, Msghandle defines the action needed when traped by
// seccomp filter
const (
MsgDisallow int16 = iota + 1
MsgHandle
)
// Trace starts new goroutine and trace runner with ptrace
func Trace(done <-chan struct{}, handler Handler, runner Runner, limits types.Limit) (<-chan types.Result, error) {
func (t *Tracer) Trace(done <-chan struct{}) (<-chan types.Result, error) {
var err error
result := make(chan types.Result, 1)
start := make(chan struct{})
@ -26,7 +20,7 @@ func Trace(done <-chan struct{}, handler Handler, runner Runner, limits types.Li
// run
go func() {
defer close(finish)
ret, err2 := TraceRun(done, start, handler, runner, limits)
ret, err2 := t.TraceRun(done, start)
err = err2
result <- ret
}()
@ -40,8 +34,7 @@ func Trace(done <-chan struct{}, handler Handler, runner Runner, limits types.Li
// TraceRun start and traces all child process by runner in the calling goroutine
// parameter done used to cancel work, start is used notify child starts
func TraceRun(done <-chan struct{}, start chan<- struct{},
handler Handler, runner Runner, limits types.Limit) (result types.Result, err error) {
func (t *Tracer) TraceRun(done <-chan struct{}, start chan<- struct{}) (result types.Result, err error) {
var (
wstatus unix.WaitStatus // wait4 wait status
rusage unix.Rusage // wait4 rusage
@ -58,10 +51,10 @@ func TraceRun(done <-chan struct{}, start chan<- struct{},
defer runtime.UnlockOSThread()
// Start the runner
pgid, err := runner.Start()
handler.Debug("tracer started: ", pgid, err)
pgid, err := t.Runner.Start()
t.Handler.Debug("tracer started: ", pgid, err)
if err != nil {
handler.Debug("start tracee failed: ", err)
t.Handler.Debug("start tracee failed: ", err)
result.Status = types.StatusRE
return result, err
}
@ -87,7 +80,7 @@ func TraceRun(done <-chan struct{}, start chan<- struct{},
err = types.StatusTLE
}
if err2 := recover(); err2 != nil {
handler.Debug(err2)
t.Handler.Debug(err2)
err = types.StatusFatal
}
// kill all tracee upon return
@ -107,10 +100,10 @@ func TraceRun(done <-chan struct{}, start chan<- struct{},
pid, err = unix.Wait4(pgid, &wstatus, unix.WALL, &rusage)
}
if err != nil {
handler.Debug("wait4 failed: ", err)
t.Handler.Debug("wait4 failed: ", err)
return result, types.StatusFatal
}
handler.Debug("------ ", pid, " ------")
t.Handler.Debug("------ ", pid, " ------")
status := types.StatusNormal
if pid == pgid {
@ -119,10 +112,10 @@ func TraceRun(done <-chan struct{}, start chan<- struct{},
userMem := uint64(rusage.Maxrss) // kb
// check tle / mle
if userTime > limits.TimeLimit {
if userTime > t.Limit.TimeLimit {
status = types.StatusTLE
}
if userMem > limits.MemoryLimit {
if userMem > t.Limit.MemoryLimit {
status = types.StatusMLE
}
result = types.Result{
@ -139,7 +132,7 @@ func TraceRun(done <-chan struct{}, start chan<- struct{},
switch {
case wstatus.Exited():
delete(traced, pid)
handler.Debug("process exited: ", pid, wstatus.ExitStatus())
t.Handler.Debug("process exited: ", pid, wstatus.ExitStatus())
if pid == pgid {
if execved {
result.ExitStatus = wstatus.ExitStatus()
@ -151,7 +144,7 @@ func TraceRun(done <-chan struct{}, start chan<- struct{},
case wstatus.Signaled():
sig := wstatus.Signal()
handler.Debug("ptrace signaled: ", sig)
t.Handler.Debug("ptrace signaled: ", sig)
if pid == pgid {
delete(traced, pid)
switch sig {
@ -172,7 +165,7 @@ func TraceRun(done <-chan struct{}, start chan<- struct{},
case wstatus.Stopped():
// Set option if the process is newly forked
if !traced[pid] {
handler.Debug("set ptrace option")
t.Handler.Debug("set ptrace option")
traced[pid] = true
// Ptrace set option valid if the tracee is stopped
err = setPtraceOption(pid)
@ -188,31 +181,31 @@ func TraceRun(done <-chan struct{}, start chan<- struct{},
case unix.PTRACE_EVENT_SECCOMP:
if execved {
// give the customized handle for syscall
err := handleTrap(handler, pid)
err := t.handleTrap(pid)
if err != nil {
result.Status = types.StatusBan
return result, err
}
} else {
handler.Debug("ptrace seccomp before execve (should be the execve syscall)")
t.Handler.Debug("ptrace seccomp before execve (should be the execve syscall)")
}
case unix.PTRACE_EVENT_CLONE:
handler.Debug("ptrace stop clone")
t.Handler.Debug("ptrace stop clone")
case unix.PTRACE_EVENT_VFORK:
handler.Debug("ptrace stop vfork")
t.Handler.Debug("ptrace stop vfork")
case unix.PTRACE_EVENT_FORK:
handler.Debug("ptrace stop fork")
t.Handler.Debug("ptrace stop fork")
case unix.PTRACE_EVENT_EXEC:
// forked tracee have successfully called execve
if !execved {
fTime = time.Now()
execved = true
}
handler.Debug("ptrace stop exec")
t.Handler.Debug("ptrace stop exec")
default:
handler.Debug("ptrace unexpected trap cause: ", trapCause)
t.Handler.Debug("ptrace unexpected trap cause: ", trapCause)
}
unix.PtraceCont(pid, 0)
} else {
@ -230,9 +223,9 @@ func TraceRun(done <-chan struct{}, start chan<- struct{},
// Likely encountered SIGSEGV (segment violation)
// Or compiler child exited
if stopSig != unix.SIGSTOP {
handler.Debug("ptrace unexpected stop signal: ", stopSig)
t.Handler.Debug("ptrace unexpected stop signal: ", stopSig)
}
handler.Debug("ptrace stopped")
t.Handler.Debug("ptrace stopped")
unix.PtraceCont(pid, int(stopSig))
}
}
@ -240,31 +233,31 @@ func TraceRun(done <-chan struct{}, start chan<- struct{},
}
// handleTrap handles the seccomp trap including the custom handle
func handleTrap(handler Handler, pid int) error {
handler.Debug("seccomp traced")
func (t *Tracer) handleTrap(pid int) error {
t.Handler.Debug("seccomp traced")
msg, err := unix.PtraceGetEventMsg(pid)
if err != nil {
handler.Debug("PtraceGetEventMsg failed:", err)
t.Handler.Debug("PtraceGetEventMsg failed:", err)
return err
}
switch int16(msg) {
case MsgDisallow:
case seccomp.MsgDisallow:
ctx, err := getTrapContext(pid)
if err != nil {
handler.Debug("getTrapContext failed:", err)
t.Handler.Debug("getTrapContext failed:", err)
return err
}
syscallName, err := handler.GetSyscallName(ctx)
handler.Debug("disallowed syscall: ", ctx.SyscallNo(), syscallName, err)
return handler.HandlerDisallow(syscallName)
syscallName, err := t.Handler.GetSyscallName(ctx)
t.Handler.Debug("disallowed syscall: ", ctx.SyscallNo(), syscallName, err)
return t.Handler.HandlerDisallow(syscallName)
case MsgHandle:
if handler != nil {
case seccomp.MsgHandle:
if t.Handler != nil {
ctx, err := getTrapContext(pid)
if err != nil {
return err
}
act := handler.Handle(ctx)
act := t.Handler.Handle(ctx)
switch act {
case TraceBan:
@ -279,7 +272,7 @@ func handleTrap(handler Handler, pid int) error {
default:
// undefined seccomp message, possible set up filter wrong
handler.Debug("unknown seccomp trap message: ", msg)
t.Handler.Debug("unknown seccomp trap message: ", msg)
}
return nil

View File

@ -4,7 +4,7 @@ import (
"fmt"
"os"
"github.com/criyle/go-sandbox/runner/ptrace"
"github.com/criyle/go-sandbox/ptracer"
)
// Handler defines file access restricted handler to call the ptrace
@ -17,51 +17,51 @@ type Handler struct {
}
// CheckRead checks whether the file have read permission
func (h *Handler) CheckRead(fn string) ptrace.TraceAction {
func (h *Handler) CheckRead(fn string) ptracer.TraceAction {
if !h.FileSet.IsReadableFile(fn) {
return h.onDgsFileDetect(fn)
}
return ptrace.TraceAllow
return ptracer.TraceAllow
}
// CheckWrite checks whether the file have write permission
func (h *Handler) CheckWrite(fn string) ptrace.TraceAction {
func (h *Handler) CheckWrite(fn string) ptracer.TraceAction {
if !h.FileSet.IsWritableFile(fn) {
return h.onDgsFileDetect(fn)
}
return ptrace.TraceAllow
return ptracer.TraceAllow
}
// CheckStat checks whether the file have stat permission
func (h *Handler) CheckStat(fn string) ptrace.TraceAction {
func (h *Handler) CheckStat(fn string) ptracer.TraceAction {
if !h.FileSet.IsStatableFile(fn) {
return h.onDgsFileDetect(fn)
}
return ptrace.TraceAllow
return ptracer.TraceAllow
}
// CheckSyscall checks syscalls other than allowed and traced agianst the
// SyscallCounter
func (h *Handler) CheckSyscall(syscallName string) ptrace.TraceAction {
func (h *Handler) CheckSyscall(syscallName string) ptracer.TraceAction {
// if it is traced, then try to count syscall
if inside, allow := h.SyscallCounter.Check(syscallName); inside {
if allow {
return ptrace.TraceAllow
return ptracer.TraceAllow
}
return ptrace.TraceKill
return ptracer.TraceKill
}
// if it is traced but not counted, it should be soft banned
return ptrace.TraceBan
return ptracer.TraceBan
}
// onDgsFileDetect soft ban file if in soft ban set
// otherwise stops the trace process
func (h *Handler) onDgsFileDetect(name string) ptrace.TraceAction {
func (h *Handler) onDgsFileDetect(name string) ptracer.TraceAction {
if h.FileSet.IsSoftBanFile(name) {
return ptrace.TraceBan
return ptracer.TraceBan
}
h.print("Dangerous fileopen: ", name)
return ptrace.TraceKill
return ptracer.TraceKill
}
// print is used to print debug information

View File

@ -6,8 +6,7 @@ import (
"path"
"syscall"
libseccomp "github.com/seccomp/libseccomp-golang"
"github.com/criyle/go-sandbox/pkg/seccomp/libseccomp"
"github.com/criyle/go-sandbox/ptracer"
"github.com/criyle/go-sandbox/types"
)
@ -27,7 +26,7 @@ func (h *tracerHandler) getString(ctx *ptracer.Context, addr uint) string {
return absPath(ctx.Pid, ctx.GetString(uintptr(addr)))
}
func (h *tracerHandler) checkOpen(ctx *ptracer.Context, addr uint, flags uint) TraceAction {
func (h *tracerHandler) checkOpen(ctx *ptracer.Context, addr uint, flags uint) ptracer.TraceAction {
fn := h.getString(ctx, addr)
isReadOnly := (flags&syscall.O_ACCMODE == syscall.O_RDONLY) &&
(flags&syscall.O_CREAT == 0) &&
@ -41,19 +40,19 @@ func (h *tracerHandler) checkOpen(ctx *ptracer.Context, addr uint, flags uint) T
return h.Handler.CheckWrite(fn)
}
func (h *tracerHandler) checkRead(ctx *ptracer.Context, addr uint) TraceAction {
func (h *tracerHandler) checkRead(ctx *ptracer.Context, addr uint) ptracer.TraceAction {
fn := h.getString(ctx, addr)
h.Debug("check read: ", fn)
return h.Handler.CheckRead(fn)
}
func (h *tracerHandler) checkWrite(ctx *ptracer.Context, addr uint) TraceAction {
func (h *tracerHandler) checkWrite(ctx *ptracer.Context, addr uint) ptracer.TraceAction {
fn := h.getString(ctx, addr)
h.Debug("check write: ", fn)
return h.Handler.CheckWrite(fn)
}
func (h *tracerHandler) checkStat(ctx *ptracer.Context, addr uint) TraceAction {
func (h *tracerHandler) checkStat(ctx *ptracer.Context, addr uint) ptracer.TraceAction {
fn := h.getString(ctx, addr)
h.Debug("check stat: ", fn)
return h.Handler.CheckStat(fn)
@ -61,11 +60,15 @@ func (h *tracerHandler) checkStat(ctx *ptracer.Context, addr uint) TraceAction {
func (h *tracerHandler) Handle(ctx *ptracer.Context) ptracer.TraceAction {
var (
action TraceAction
action ptracer.TraceAction
syscallNo = ctx.SyscallNo()
syscallName, err = libseccomp.ScmpSyscall(syscallNo).GetName()
syscallName, err = libseccomp.ToSyscallName(syscallNo)
)
h.Debug("syscall: ", syscallNo, syscallName, err)
if err != nil {
h.Debug("invalid syscall no")
return ptracer.TraceKill
}
switch syscallName {
case "open":
@ -107,9 +110,9 @@ func (h *tracerHandler) Handle(ctx *ptracer.Context) ptracer.TraceAction {
}
switch action {
case TraceAllow:
case ptracer.TraceAllow:
return ptracer.TraceAllow
case TraceBan:
case ptracer.TraceBan:
h.Debug("<soft ban syscall>")
return softBanSyscall(ctx)
default:
@ -119,7 +122,7 @@ func (h *tracerHandler) Handle(ctx *ptracer.Context) ptracer.TraceAction {
func (h *tracerHandler) GetSyscallName(ctx *ptracer.Context) (string, error) {
syscallNo := ctx.SyscallNo()
return libseccomp.ScmpSyscall(syscallNo).GetName()
return libseccomp.ToSyscallName(syscallNo)
}
func (h *tracerHandler) HandlerDisallow(name string) error {

View File

@ -1,30 +1,13 @@
package ptrace
import (
libseccomp "github.com/seccomp/libseccomp-golang"
"github.com/criyle/go-sandbox/pkg/forkexec"
"github.com/criyle/go-sandbox/pkg/seccomp"
"github.com/criyle/go-sandbox/ptracer"
"github.com/criyle/go-sandbox/types"
)
// Start starts the tracing process
func (r *Runner) Start(done <-chan struct{}) (<-chan types.Result, error) {
// build seccomp filter
filter, err := buildFilter(r.ShowDetails, r.SyscallAllowed, r.SyscallTraced)
if err != nil {
println(err)
return nil, err
}
defer filter.Release()
bpf, err := seccomp.FilterToBPF(filter)
if err != nil {
println(err)
return nil, err
}
ch := &forkexec.Runner{
Args: r.Args,
Env: r.Env,
@ -32,7 +15,7 @@ func (r *Runner) Start(done <-chan struct{}) (<-chan types.Result, error) {
RLimits: r.RLimits.PrepareRLimit(),
Files: r.Files,
WorkDir: r.WorkDir,
Seccomp: bpf,
Seccomp: r.Seccomp.SockFprog(),
Ptrace: true,
SyncFunc: r.SyncFunc,
}
@ -42,18 +25,11 @@ func (r *Runner) Start(done <-chan struct{}) (<-chan types.Result, error) {
Unsafe: r.Unsafe,
Handler: r.Handler,
}
return ptracer.Trace(done, th, ch, types.Limit(r.TraceLimit))
}
// build filter builds the libseccomp filter according to the allow, trace and show details
func buildFilter(showDetails bool, allow, trace []string) (*libseccomp.ScmpFilter, error) {
// make filter
var defaultAction libseccomp.ScmpAction
// if debug, allow all syscalls and output what was blocked
if showDetails {
defaultAction = libseccomp.ActTrace.SetReturnCode(ptracer.MsgDisallow)
} else {
defaultAction = libseccomp.ActKill
tracer := ptracer.Tracer{
Handler: th,
Runner: ch,
Limit: r.Limit,
}
return seccomp.BuildFilter(defaultAction, libseccomp.ActTrace.SetReturnCode(ptracer.MsgHandle), allow, trace)
return tracer.Trace(done)
}

View File

@ -4,6 +4,8 @@ import (
"syscall"
"github.com/criyle/go-sandbox/pkg/rlimit"
"github.com/criyle/go-sandbox/pkg/seccomp"
"github.com/criyle/go-sandbox/ptracer"
"github.com/criyle/go-sandbox/types"
)
@ -25,14 +27,13 @@ type Runner struct {
RLimits rlimit.RLimits
// Res limit enforced by tracer
TraceLimit types.Limit
Limit types.Limit
// Allowed / Traced syscall names
// Notice: file access syscalls should be traced
// If traced syscall is file access, it will checked by file access handler
// otherwise it will checked by syscall access handler
SyscallAllowed []string
SyscallTraced []string
// Defines seccomp filter for the ptrace runner
// file access syscalls need to set as ActionTrace
// allowed need to set as ActionAllow
// default action should be ActionTrace / ActionKill
Seccomp seccomp.Filter
// Traced syscall handler
Handler Handler
@ -44,24 +45,13 @@ type Runner struct {
SyncFunc func(pid int) error
}
// TraceAction defines action against a syscall check
type TraceAction int
// BanRet defines the return value for a syscall ban acction
var BanRet = syscall.EACCES
// TraceAllow allow the access, trace ban ignores the syscall and set the
// return value to BanRet, TraceKill stops the trace action
const (
TraceAllow = iota + 1
TraceBan
TraceKill
)
// Handler defines the action when a file access encountered
type Handler interface {
CheckRead(string) TraceAction
CheckWrite(string) TraceAction
CheckStat(string) TraceAction
CheckSyscall(string) TraceAction
CheckRead(string) ptracer.TraceAction
CheckWrite(string) ptracer.TraceAction
CheckStat(string) ptracer.TraceAction
CheckSyscall(string) ptracer.TraceAction
}

View File

@ -5,11 +5,9 @@ import (
"os"
"time"
libseccomp "github.com/seccomp/libseccomp-golang"
"golang.org/x/sys/unix"
"github.com/criyle/go-sandbox/pkg/forkexec"
"github.com/criyle/go-sandbox/pkg/seccomp"
"github.com/criyle/go-sandbox/types"
)
@ -20,19 +18,7 @@ const (
// Start starts the unshared process
func (r *Runner) Start(done <-chan struct{}) (<-chan types.Result, error) {
filter, err := seccomp.BuildFilter(libseccomp.ActKill, libseccomp.ActTrap, r.SyscallAllowed, []string{})
if err != nil {
println(err)
return nil, err
}
defer filter.Release()
bpf, err := seccomp.FilterToBPF(filter)
if err != nil {
println(err)
return nil, err
}
var err error
ch := &forkexec.Runner{
Args: r.Args,
Env: r.Env,
@ -40,7 +26,7 @@ func (r *Runner) Start(done <-chan struct{}) (<-chan types.Result, error) {
RLimits: r.RLimits.PrepareRLimit(),
Files: r.Files,
WorkDir: r.WorkDir,
Seccomp: bpf,
Seccomp: r.Seccomp.SockFprog(),
NoNewPrivs: true,
StopBeforeSeccomp: false,
UnshareFlags: UnshareFlags,
@ -117,6 +103,7 @@ func (r *Runner) Trace(done <-chan struct{}, start chan<- struct{},
}()
fTime = time.Now()
loop:
for {
_, err := unix.Wait4(pgid, &wstatus, 0, &rusage)
r.println("wait4: ", wstatus)
@ -129,10 +116,10 @@ func (r *Runner) Trace(done <-chan struct{}, start chan<- struct{},
userMem := uint64(rusage.Maxrss) // kb
// check tle / mle
if userTime > r.ResLimits.TimeLimit {
if userTime > r.Limit.TimeLimit {
status = types.StatusTLE
}
if userMem > r.ResLimits.MemoryLimit {
if userMem > r.Limit.MemoryLimit {
status = types.StatusMLE
}
result = types.Result{
@ -161,7 +148,7 @@ func (r *Runner) Trace(done <-chan struct{}, start chan<- struct{},
status = types.StatusRE
}
result.Status = status
return result, status
break loop
}
}
return result, status

View File

@ -3,6 +3,7 @@ package unshare
import (
"github.com/criyle/go-sandbox/pkg/mount"
"github.com/criyle/go-sandbox/pkg/rlimit"
"github.com/criyle/go-sandbox/pkg/seccomp"
"github.com/criyle/go-sandbox/types"
)
@ -25,10 +26,10 @@ type Runner struct {
RLimits rlimit.RLimits
// Resource limit enforced by tracer
ResLimits types.Limit
Limit types.Limit
// Allowed syscall names
SyscallAllowed []string
// Seccomp defines the seccomp filter attach to the process (should be whitelist only)
Seccomp seccomp.Filter
// New root
Root string