From 7a9bb17829ddd1eda0bc09b0725a0c19c428a1fc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gustavo=20I=C3=B1iguez=20Goia?= Date: Fri, 22 Sep 2023 00:36:26 +0200 Subject: [PATCH] allow to filter connections by process checksum Now you can create rules to filter processes by checksum. Only md5 is available at the moment. There's a global configuration option that you can use to enable or disable this feature, from the config file or from the Preferences dialog. As part of this feature there have been more changes: - New proc monitor method (PROCESS CONNECTOR) that listens for exec/exit events from the kernel. This feature depends on CONFIG_PROC_EVENTS kernel option. - Only one cache of active processes for ebpf and proc monitor methods. More info and details: #413. --- daemon/conman/connection.go | 45 +- daemon/default-config.json | 3 + daemon/netlink/procmon/procmon.go | 149 ++++++ daemon/procmon/activepids.go | 103 ++-- daemon/procmon/activepids_test.go | 104 ---- daemon/procmon/cache_events.go | 320 ++++++++++++ daemon/procmon/details.go | 299 +++++++++-- daemon/procmon/ebpf/cache.go | 102 +--- daemon/procmon/ebpf/ebpf.go | 37 +- daemon/procmon/ebpf/events.go | 59 ++- daemon/procmon/ebpf/find.go | 27 +- daemon/procmon/ebpf/monitor.go | 1 - daemon/procmon/monitor/init.go | 80 ++- daemon/procmon/parse.go | 6 +- daemon/procmon/process.go | 114 ++-- daemon/rule/loader.go | 28 +- daemon/rule/operator.go | 59 ++- daemon/rule/rule.go | 4 +- daemon/ui/client.go | 3 + daemon/ui/config/config.go | 15 +- daemon/ui/config_utils.go | 6 +- daemon/ui/notifications.go | 3 +- proto/ui.proto | 10 +- ui/opensnitch/dialogs/preferences.py | 19 + ui/opensnitch/dialogs/prompt.py | 19 +- ui/opensnitch/dialogs/ruleseditor.py | 31 ++ ui/opensnitch/res/preferences.ui | 750 +++++++++++++++------------ ui/opensnitch/res/prompt.ui | 283 +++++----- ui/opensnitch/res/ruleseditor.ui | 722 ++++++++++++++------------ 29 files changed, 2180 insertions(+), 1221 deletions(-) create mode 100644 daemon/netlink/procmon/procmon.go delete mode 100644 daemon/procmon/activepids_test.go create mode 100644 daemon/procmon/cache_events.go diff --git a/daemon/conman/connection.go b/daemon/conman/connection.go index 6448cf7393..b91a70160b 100644 --- a/daemon/conman/connection.go +++ b/daemon/conman/connection.go @@ -22,16 +22,17 @@ import ( // Connection represents an outgoing connection. type Connection struct { + Pkt *netfilter.Packet + Entry *netstat.Entry + Process *procmon.Process + Protocol string + DstHost string SrcIP net.IP - SrcPort uint DstIP net.IP - DstPort uint - DstHost string - Entry *netstat.Entry - Process *procmon.Process - Pkt *netfilter.Packet + SrcPort uint + DstPort uint } var showUnknownCons = false @@ -69,6 +70,7 @@ func Parse(nfp netfilter.Packet, interceptUnknown bool) *Connection { func newConnectionImpl(nfp *netfilter.Packet, c *Connection, protoType string) (cr *Connection, err error) { // no errors but not enough info neither if c.parseDirection(protoType) == false { + log.Debug("discarding conn: %+v", c) return nil, nil } log.Debug("new connection %s => %d:%v -> %v (%s):%d uid: %d, mark: %x", c.Protocol, c.SrcPort, c.SrcIP, c.DstIP, c.DstHost, c.DstPort, nfp.UID, nfp.Mark) @@ -98,8 +100,8 @@ func newConnectionImpl(nfp *netfilter.Packet, c *Connection, protoType string) ( } if err != nil { log.Debug("ebpf warning: %v", err) - return nil, nil } + log.Debug("[ebpf conn] PID not found via eBPF, falling back to proc") } else if procmon.MethodIsAudit() { if aevent := audit.GetEventByPid(pid); aevent != nil { audit.Lock.RLock() @@ -115,9 +117,10 @@ func newConnectionImpl(nfp *netfilter.Packet, c *Connection, protoType string) ( c.Process.ReadEnv() c.Process.CleanPath() - procmon.AddToActivePidsCache(uint64(pid), c.Process) + procmon.EventsCache.Add(*c.Process) return c, nil } + log.Debug("[auditd conn] PID not found via auditd, falling back to proc") } // Sometimes when using eBPF, the PID is not found by the connection's parameters, @@ -185,6 +188,7 @@ func NewConnection(nfp *netfilter.Packet) (c *Connection, err error) { DstHost: dns.HostOr(ip.DstIP, ""), Pkt: nfp, } + return newConnectionImpl(nfp, c, "") } @@ -317,17 +321,18 @@ func (c *Connection) String() string { // Serialize returns a connection serialized. func (c *Connection) Serialize() *protocol.Connection { return &protocol.Connection{ - Protocol: c.Protocol, - SrcIp: c.SrcIP.String(), - SrcPort: uint32(c.SrcPort), - DstIp: c.DstIP.String(), - DstHost: c.DstHost, - DstPort: uint32(c.DstPort), - UserId: uint32(c.Entry.UserId), - ProcessId: uint32(c.Process.ID), - ProcessPath: c.Process.Path, - ProcessArgs: c.Process.Args, - ProcessEnv: c.Process.Env, - ProcessCwd: c.Process.CWD, + Protocol: c.Protocol, + SrcIp: c.SrcIP.String(), + SrcPort: uint32(c.SrcPort), + DstIp: c.DstIP.String(), + DstHost: c.DstHost, + DstPort: uint32(c.DstPort), + UserId: uint32(c.Entry.UserId), + ProcessId: uint32(c.Process.ID), + ProcessPath: c.Process.Path, + ProcessArgs: c.Process.Args, + ProcessEnv: c.Process.Env, + ProcessCwd: c.Process.CWD, + ProcessChecksums: c.Process.Checksums, } } diff --git a/daemon/default-config.json b/daemon/default-config.json index 4c461a776d..bdde48ee4b 100644 --- a/daemon/default-config.json +++ b/daemon/default-config.json @@ -12,6 +12,9 @@ "LogUTC": true, "LogMicro": false, "Firewall": "nftables", + "Rules": { + "EnableChecksums": false + }, "Stats": { "MaxEvents": 150, "MaxStats": 25, diff --git a/daemon/netlink/procmon/procmon.go b/daemon/netlink/procmon/procmon.go new file mode 100644 index 0000000000..15a5a63f49 --- /dev/null +++ b/daemon/netlink/procmon/procmon.go @@ -0,0 +1,149 @@ +package procmon + +import ( + "runtime" + "time" + + "github.com/evilsocket/opensnitch/daemon/log" + "github.com/vishvananda/netlink" + "github.com/vishvananda/netns" +) + +var ( + // ProcEventsChannel channel of events to read + ProcEventsChannel = make(chan ProcEvent) +) + +// ProcEvent represents the struct returned from kernel +type ProcEvent struct { + ev netlink.ProcEvent + + TimeStamp uint64 + PID uint32 + PPID uint32 + TGID uint32 + PTGID uint32 +} + +// ProcEventsMonitor listens for process events from kernel. +// We listen for events via netlink, from the Process Events Conector: +// https://lwn.net/Articles/157150/ +// The kernel must have the options CONFIG_CONECTOR and CONFIG_PROC_EVENTS enabled. +func ProcEventsMonitor(done <-chan struct{}) { + log.Info("ProcEventMonitor started\n") + runtime.LockOSThread() + defer runtime.UnlockOSThread() + + pid1ns, err := netns.GetFromPid(1) + if err != nil { + log.Warning("unable to start netlink.ProcEventMonitor (0): %s", err) + return + } + + err = netns.Set(pid1ns) + if err != nil { + log.Warning("unable to start netlink.ProcEventMonitor (1): %s", err) + return + } + + ch := make(chan netlink.ProcEvent) + errChan := make(chan error) + if err := netlink.ProcEventMonitor(ch, done, errChan); err != nil { + log.Warning("unable to start netlink.ProcEventMonitor (2): %s", err) + return + } + + for { + select { + case <-done: + goto Exit + case errc := <-errChan: + // We may receive "no buffer space available" when: + // - the daemon is stopped (ptrace, signal, etc). + // - sometimes after coming back from suspend. + log.Error("ProcEventMonitor error: %s", errc) + goto Error + case e := <-ch: + p := NewProcEvent(e) + if !p.IsExec() && !p.IsExit() { + // Msg may be nil in case of error + if p.ev.Msg == nil { + log.Warning("ProcEventMonitor Msg == nil") + goto Error + } + continue + } + ProcEventsChannel <- p + } + } +Error: + log.Info("reinitiating ProcEventMonitor") + time.Sleep(time.Second) + ProcEventsMonitor(done) + return +Exit: + log.Debug("netlink.ProcEventsMonitor stopped") +} + +// NewProcEvent returns a new event received from kernel +func NewProcEvent(ev netlink.ProcEvent) ProcEvent { + pv := ProcEvent{ev: ev, TimeStamp: ev.Timestamp} + if pv.IsExec() { + if execEv, ok := pv.Msg().(*netlink.ExecProcEvent); ok { + pv.PID = execEv.ProcessPid + pv.TGID = execEv.ProcessTgid + } + } else if pv.IsExit() { + if exitEv, ok := pv.Msg().(*netlink.ExitProcEvent); ok { + pv.PID = exitEv.ProcessPid + pv.PPID = exitEv.ParentPid + pv.TGID = exitEv.ProcessTgid + pv.PTGID = exitEv.ParentTgid + } + } + /*else if pv.IsFork() { + if forkEv, ok := pv.Msg().(*netlink.ForkProcEvent); ok { + pv.PID = forkEv.ChildPid + pv.PPID = forkEv.ParentPid + pv.TGID = forkEv.ChildTgid + pv.PTGID = forkEv.ParentTgid + } + + } else if pv.IsComm() { + fmt.Printf("COMM: %d\n", ev.Msg.Pid()) + if commEv, ok := pv.Msg().(*netlink.CommProcEvent); ok { + fmt.Println("COMM EVENT ->", string(commEv.Comm[:])) + } + */ + return pv +} + +// Msg returns the message received from netlink +func (pe *ProcEvent) Msg() interface{} { + return pe.ev.Msg +} + +// Pid returns the pid of the event +func (pe *ProcEvent) Pid() uint32 { + return pe.ev.Msg.Pid() +} + +// IsFork returns if the event is fork +func (pe *ProcEvent) IsFork() bool { + return pe.ev.What == netlink.PROC_EVENT_FORK +} + +// IsExec returns if the event is exec +func (pe *ProcEvent) IsExec() bool { + return pe.ev.What == netlink.PROC_EVENT_EXEC +} + +// IsComm returns if the event is comm +func (pe *ProcEvent) IsComm() bool { + return pe.ev.What == netlink.PROC_EVENT_COMM +} + +// IsExit returns if the event is exit +func (pe *ProcEvent) IsExit() bool { + return pe.ev.What == netlink.PROC_EVENT_EXIT +} diff --git a/daemon/procmon/activepids.go b/daemon/procmon/activepids.go index 086642fa7a..8df86e9f7d 100644 --- a/daemon/procmon/activepids.go +++ b/daemon/procmon/activepids.go @@ -1,14 +1,10 @@ package procmon import ( - "fmt" - "io/ioutil" - "strconv" - "strings" "sync" - "time" "github.com/evilsocket/opensnitch/daemon/log" + "github.com/evilsocket/opensnitch/daemon/netlink/procmon" ) type value struct { @@ -23,68 +19,45 @@ var ( activePidsLock = sync.RWMutex{} ) -//MonitorActivePids checks that each process in activePids -//is still running and if not running (or another process with the same pid is running), -//removes the pid from activePids -func MonitorActivePids() { +// MonitorProcEvents listen for process events from kernel, via netlink. +func MonitorProcEvents(stop <-chan struct{}) { + log.Debug("MonitorProcEvents start") for { - time.Sleep(time.Second) - activePidsLock.Lock() - for k, v := range activePids { - data, err := ioutil.ReadFile(fmt.Sprintf("/proc/%d/stat", k)) - if err != nil { - //file does not exists, pid has quit - delete(activePids, k) - pidsCache.delete(int(k)) - continue - } - startTime, err := strconv.ParseInt(strings.Split(string(data), " ")[21], 10, 64) - if err != nil { - log.Error("Could not find or convert Starttime. This should never happen. Please report this incident to the Opensnitch developers: %v", err) - delete(activePids, k) - pidsCache.delete(int(k)) - continue - } - if uint64(startTime) != v.Starttime { - //extremely unlikely: the original process has quit and another process - //was started with the same PID - all this in less than 1 second - log.Error("Same PID but different Starttime. Please report this incident to the Opensnitch developers.") - delete(activePids, k) - pidsCache.delete(int(k)) - continue + select { + case <-stop: + goto Exit + case ev := <-procmon.ProcEventsChannel: + if ev.IsExec() { + // we don't receive the path of the process, therefore we need to discover it, + // to check if the PID has replaced the PPID. + proc := NewProcess(int(ev.PID), "") + proc.GetInfo() + proc.Parent = NewProcess(int(ev.TGID), "") + proc.Parent.GetInfo() + + log.Debug("[procmon exec event] %d, pid:%d tgid:%d %s, %s -> %s\n", ev.TimeStamp, ev.PID, ev.TGID, proc.Comm, proc.Path, proc.Parent.Path) + //log.Debug("[procmon exec event] %d, pid:%d tgid:%d\n", ev.TimeStamp, ev.PID, ev.TGID) + if _, needsHashUpdate, found := EventsCache.IsInStore(int(ev.PID), proc); found { + // check if this PID has replaced the PPID: + // systemd, pid:1234 -> curl, pid:1234 -> curl (i.e.: pid 1234) opens x.x.x.x:443 + // Without this, we would display that systemd is connecting to x.x.x.x:443 + // The previous pid+path will still exist as parent of the new child, in proc.Parent + if needsHashUpdate { + //log.Debug("[procmon inCache REPLACEMENT] rehashing, new: %d, %s -> inCache: %d -> %s", proc.ID, proc.Path, item.Proc.ID, item.Proc.Path) + EventsCache.ComputeChecksums(proc) + } + log.Debug("[procmon exec event inCache] %d, pid:%d tgid:%d\n", ev.TimeStamp, ev.PID, ev.TGID) + continue + } + EventsCache.Add(*proc) + } else if ev.IsExit() { + p, _, found := EventsCache.IsInStore(int(ev.PID), nil) + if found && p.Proc.IsAlive() == false { + EventsCache.Delete(p.Proc.ID) + } } } - activePidsLock.Unlock() - } -} - -func findProcessInActivePidsCache(pid uint64) *Process { - activePidsLock.Lock() - defer activePidsLock.Unlock() - if value, ok := activePids[pid]; ok { - return value.Process - } - return nil -} - -// AddToActivePidsCache adds the given pid to a list of known processes. -func AddToActivePidsCache(pid uint64, proc *Process) { - - data, err := ioutil.ReadFile(fmt.Sprintf("/proc/%d/stat", pid)) - if err != nil { - //most likely the process has quit by now - return - } - startTime, err2 := strconv.ParseInt(strings.Split(string(data), " ")[21], 10, 64) - if err2 != nil { - log.Error("Could not find or convert Starttime. This should never happen. Please report this incident to the Opensnitch developers: %v", err) - return - } - - activePidsLock.Lock() - activePids[pid] = value{ - Process: proc, - Starttime: uint64(startTime), } - activePidsLock.Unlock() +Exit: + log.Debug("MonitorProcEvents stopped") } diff --git a/daemon/procmon/activepids_test.go b/daemon/procmon/activepids_test.go deleted file mode 100644 index ccc2c44778..0000000000 --- a/daemon/procmon/activepids_test.go +++ /dev/null @@ -1,104 +0,0 @@ -package procmon - -import ( - "fmt" - "math/rand" - "os" - "os/exec" - "syscall" - "testing" - "time" -) - -//TestMonitorActivePids starts helper processes, adds them to activePids -//and then kills them and checks if monitorActivePids() removed the killed processes -//from activePids -func TestMonitorActivePids(t *testing.T) { - - if os.Getenv("helperBinaryMode") == "on" { - //we are in the "helper binary" mode, we were started with helperCmd.Start() (see below) - //do nothing, just wait to be killed - time.Sleep(time.Second * 10) - os.Exit(1) //will never get here; but keep it here just in case - } - - //we are in a normal "go test" mode - tmpDir := "/tmp/ostest_" + randString() - os.Mkdir(tmpDir, 0777) - fmt.Println("tmp dir", tmpDir) - defer os.RemoveAll(tmpDir) - - go MonitorActivePids() - - //build a "helper binary" with "go test -c -o /tmp/path" and put it into a tmp dir - helperBinaryPath := tmpDir + "/helper1" - goExecutable, _ := exec.LookPath("go") - cmd := exec.Command(goExecutable, "test", "-c", "-o", helperBinaryPath) - if err := cmd.Run(); err != nil { - t.Error("Error running go test -c", err) - } - - var numberOfHelpers = 5 - var helperProcs []*Process - //start helper binaries - for i := 0; i < numberOfHelpers; i++ { - var helperCmd *exec.Cmd - helperCmd = &exec.Cmd{ - Path: helperBinaryPath, - Args: []string{helperBinaryPath}, - Env: []string{"helperBinaryMode=on"}, - } - if err := helperCmd.Start(); err != nil { - t.Error("Error starting helper binary", err) - } - go func() { - helperCmd.Wait() //must Wait(), otherwise the helper process becomes a zombie when kill()ed - }() - - pid := helperCmd.Process.Pid - proc := NewProcess(pid, helperBinaryPath) - helperProcs = append(helperProcs, proc) - AddToActivePidsCache(uint64(pid), proc) - } - //sleep to make sure all processes started before we proceed - time.Sleep(time.Second * 1) - //make sure all PIDS are in the cache - for i := 0; i < numberOfHelpers; i++ { - proc := helperProcs[i] - pid := proc.ID - foundProc := findProcessInActivePidsCache(uint64(pid)) - if foundProc == nil { - t.Error("PID not found among active processes", pid) - } - if proc.Path != foundProc.Path || proc.ID != foundProc.ID { - t.Error("PID or path doesn't match with the found process") - } - } - //kill all helpers except for one - for i := 0; i < numberOfHelpers-1; i++ { - if err := syscall.Kill(helperProcs[i].ID, syscall.SIGTERM); err != nil { - t.Error("error in syscall.Kill", err) - } - } - //give the cache time to remove killed processes - time.Sleep(time.Second * 1) - - //make sure only the alive process is in the cache - foundProc := findProcessInActivePidsCache(uint64(helperProcs[numberOfHelpers-1].ID)) - if foundProc == nil { - t.Error("last alive PID is not found among active processes", foundProc) - } - if len(activePids) != 1 { - t.Error("more than 1 active PIDs left in cache") - } -} - -func randString() string { - rand.Seed(time.Now().UnixNano()) - var letterRunes = []rune("abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ") - b := make([]rune, 10) - for i := range b { - b[i] = letterRunes[rand.Intn(len(letterRunes))] - } - return string(b) -} diff --git a/daemon/procmon/cache_events.go b/daemon/procmon/cache_events.go new file mode 100644 index 0000000000..081da09acc --- /dev/null +++ b/daemon/procmon/cache_events.go @@ -0,0 +1,320 @@ +package procmon + +import ( + "sync" + "time" + + "github.com/evilsocket/opensnitch/daemon/log" +) + +var ( + // EventsCache is the cache of processes + EventsCache *EventsStore + eventsCacheTicker *time.Ticker + // When we receive an Exit event, we'll delete it from cache. + // This TTL defines how much time we retain a PID on cache, before we receive + // an Exit event. + pidTTL = 3600 // seconds + // the 2nd cache of items is by path. + // + pathTTL = 3600 * 24 // 1 day +) + +func init() { + EventsCache = NewEventsStore() + go monitorEventsCache() +} + +// ProcessEvent represents an process event +type ProcessEvent struct { + Filename string + Args string + Comm string + PID uint64 + PPID uint64 + UID uint64 +} + +// ExecEventItem represents an item of the cache +type ExecEventItem struct { + Proc Process + LastSeen int64 + TTL int32 +} + +func (e *ExecEventItem) isValid() bool { + lastSeen := time.Now().Sub( + time.Unix(0, e.LastSeen), + ) + return int(lastSeen.Seconds()) < pidTTL +} + +//EventsStore is the cache of exec events +type EventsStore struct { + eventByPID map[int]*ExecEventItem + // a path will have multiple pids, hashes will be computed only once by path + eventByPath map[string]*ExecEventItem + checksums map[string]uint + mu *sync.RWMutex + hashed uint64 + checksumsEnabled bool +} + +// NewEventsStore creates a new store of events. +func NewEventsStore() *EventsStore { + if eventsCacheTicker != nil { + eventsCacheTicker.Stop() + } + eventsCacheTicker = time.NewTicker(10 * time.Second) + + return &EventsStore{ + mu: &sync.RWMutex{}, + checksums: make(map[string]uint), + eventByPID: make(map[int]*ExecEventItem), + eventByPath: make(map[string]*ExecEventItem), + } +} + +// Add adds a new process to cache. +// If computing checksums is enabled, new checksums will be computed if needed, +// or reused existing ones otherwise. +func (e *EventsStore) Add(proc Process) { + log.Debug("[cache] EventsStore.Add() %d, %s", proc.ID, proc.Path) + if e.GetComputeChecksums() { + e.ComputeChecksums(&proc) + } + + e.updateItem(&proc) +} + +func (e *EventsStore) updateItem(proc *Process) { + log.Debug("[cache] updateItem() adding to events store (total: %d, hashed:%d), pid: %d, paths: %s", e.Len(), e.hashed, proc.ID, proc.Path) + if proc.Path == "" { + return + } + + e.mu.Lock() + defer e.mu.Unlock() + + ev := &ExecEventItem{ + Proc: *proc, + LastSeen: time.Now().UnixNano(), + } + e.eventByPID[proc.ID] = ev + e.eventByPath[proc.Path] = ev +} + +// IsInStore checks if a PID is in the store. +// If the PID is in cache, we may need to update it if the PID +// is reusing the PID of the parent. +func (e *EventsStore) IsInStore(key int, proc *Process) (item *ExecEventItem, needsHashUpdate bool, found bool) { + //fmt.Printf("IsInStore()\n") + item, found = e.IsInStoreByPID(key) + if !found { + return + } + /*if e.checksumsEnabled && len(item.Proc.Checksums) == 0 { + log.Info("RECALCULATING STORED item: %s", item.Proc.Path) + item.Proc.ComputeChecksums(e.checksums) + }*/ + log.Debug("[cache] Event found by PID: %d, %s", key, item.Proc.Path) + + // check if this PID has replaced the PPID: + // systemd, pid:1234 -> curl, pid:1234 -> curl (i.e.: pid 1234) opens x.x.x.x:443 + // Without this, we would display for example "systemd is connecting to x.x.x.x:443", + // instead of "curl is connecting to ..." + // The previous pid+path will still exist as parent of the new child, in proc.Parent + if proc != nil && proc.Path != "" && item.Proc.Path != proc.Path { + log.Debug("[event inCache, replacement] new: %d, %s -> inCache: %d -> %s", proc.ID, proc.Path, item.Proc.ID, item.Proc.Path) + //e.ComputeChecksums(proc) + e.updateItem(proc) + needsHashUpdate = true + } + + return +} + +// IsInStoreByPID checks if a pid exists in cache. +func (e *EventsStore) IsInStoreByPID(key int) (item *ExecEventItem, found bool) { + e.mu.RLock() + defer e.mu.RUnlock() + item, found = e.eventByPID[key] + return +} + +// IsInStoreByPath checks if a process exists in cache by path. +func (e *EventsStore) IsInStoreByPath(path string) (item *ExecEventItem, found bool) { + e.mu.RLock() + defer e.mu.RUnlock() + if path == "" || path == KernelConnection { + return + } + item, found = e.eventByPath[path] + if found { + log.Debug("[cache] event found by path: %s", path) + } + return +} + +// Delete an item from cache +func (e *EventsStore) Delete(key int) { + e.mu.Lock() + defer e.mu.Unlock() + delete(e.eventByPID, key) +} + +// Len returns the number of items in cache. +func (e *EventsStore) Len() int { + e.mu.RLock() + e.mu.RUnlock() + return len(e.eventByPID) +} + +// DeleteOldItems deletes items that have exceeded the TTL +func (e *EventsStore) DeleteOldItems() { + e.mu.Lock() + defer e.mu.Unlock() + + log.Debug("[cache] deleting old events, total byPID: %d, byPath: %d", len(e.eventByPID), len(e.eventByPath)) + for k, item := range e.eventByPID { + if item.Proc.IsAlive() == false { + log.Debug("[cache] deleting old PID: %d -> %s", k, item.Proc.Path) + delete(e.eventByPID, k) + } + } + for path, item := range e.eventByPath { + if item.Proc.IsAlive() == false { + log.Debug("[cache] deleting old path: %d -> %s", item.Proc.ID, item.Proc.Path) + delete(e.eventByPath, path) + } + } +} + +// ------------------------------------------------------------------------- +// TODO: Move to its own package. +// A hashing service than runs in background, and accepts paths to hash +// and returns the hashes for different algorithms (configurables) + +// ComputeChecksums decides if we need to compute the checksum of a process or not. +// We don't recalculate hashes during the life of the process. +func (e *EventsStore) ComputeChecksums(proc *Process) { + if !e.checksumsEnabled { + return + } + log.Debug("[cache] reuseChecksums %d, %s", proc.ID, proc.Path) + + // XXX: why double check if the PID is in cache? + // reuseChecksums is called from Add(), and before calling Add() we check if + // the PID is in cache. + // The problem is that we don't intercept some events (fork, clone*, dup*), + // and because of this sometimes we don't receive the event of the parent. + item, _, found := e.IsInStore(proc.ID, proc) + if !found { + //log.Debug("cache.reuseChecksums() %d not inCache, %s", proc.ID, proc.Path) + + // if parent path and current path are equal, and the parent is alive, see if we have the hash of the parent path + if !proc.IsChild() { + log.Debug("[cache] reuseChecksums() pid not in cache, not child of parent: %d, %s - %d", proc.ID, proc.Path, proc.Starttime) + proc.ComputeChecksums(e.checksums) + e.hashed++ + return + } + + // parent path is nil or paths differ or parent is not alive + // compute new checksums + log.Debug("[cache] reuseChecksums() proc is child, proc: %d, %d, %s parent: %d, %d, %s", proc.Starttime, proc.ID, proc.Path, proc.Parent.Starttime, proc.Parent.ID, proc.Parent.Path) + pit, found := e.IsInStoreByPath(proc.Parent.Path) + if !found { + //log.Info("cache.reuseChecksums() cache.add() pid not found byPath: %d, %s, parent: %d, %s", proc.ID, proc.Path, proc.Parent.ID, proc.Parent.Path) + proc.ComputeChecksums(e.checksums) + return + } + + // if the parent path is in cache reuse the checksums + log.Debug("[cache] reuseChecksums() inCache, found by parent path: %d:%s, parent alive: %v, %d:%s", pit.Proc.ID, pit.Proc.Path, proc.Parent.IsAlive(), proc.Parent.ID, proc.Parent.Path) + if len(pit.Proc.Checksums) == 0 { + proc.ComputeChecksums(e.checksums) + return + } + log.Debug("[cache] reuseCheckums() reusing checksums: %v", pit.Proc.Checksums) + proc.Checksums = pit.Proc.Checksums + return + } + + // pid found in cache + // we should check other parameters to see if the pid is really the same process + // proc//maps + if len(item.Proc.Checksums) > 0 && (item.Proc.IsAlive() && item.Proc.Path == proc.Path) { + log.Debug("[cache] reuseChecksums() cached PID alive, already hashed:%v, %s new: %s", item.Proc.Checksums, item.Proc.Path, proc.Path) + proc.Checksums = item.Proc.Checksums + return + } + log.Debug("[cache] reuseChecksums() PID found inCache, computing hashes: %s new: %s - hashes: |%v<>%v|", item.Proc.Path, proc.Path, item.Proc.Checksums, proc.Checksums) + proc.ComputeChecksums(e.checksums) + e.hashed++ +} + +// AddChecksumHash adds a new hash algorithm to compute checksums +func (e *EventsStore) AddChecksumHash(hash string) { + e.mu.Lock() + e.checksums[hash]++ + e.mu.Unlock() +} + +// DelChecksumHash deletes a hash algorithm from the list +func (e *EventsStore) DelChecksumHash(hash string) { + e.mu.Lock() + if _, found := e.checksums[hash]; found { + e.checksums[hash]-- + } + e.mu.Unlock() +} + +// SetComputeChecksums configures if we compute checksums of processes +// They can be disabled for example if there's no rule that requires checksums. +// When enabling this functionality, some already stored process may don't have +// the checksums computed, so when enabling compute them. +func (e *EventsStore) SetComputeChecksums(compute bool) { + e.mu.Lock() + defer e.mu.Unlock() + + e.checksumsEnabled = compute + if !compute { + for _, item := range e.eventByPID { + // XXX: reset saved checksums? or keep them in cache? + item.Proc.Checksums = make(map[string]string) + } + return + } + for _, item := range e.eventByPID { + if len(item.Proc.Checksums) == 0 { + item.Proc.ComputeChecksums(e.checksums) + } + } +} + +// DisableChecksums disables computing checksums functionality +func (e *EventsStore) DisableChecksums() { + e.mu.Lock() + defer e.mu.Unlock() + e.checksumsEnabled = false + e.checksums = make(map[string]uint) +} + +// GetComputeChecksums returns if computing checksums is enabled or not. +// Disabled -> if there're no rules with checksum field +// Disabled -> if events monitors are not available. +// TODO: Disabled -> if there were n rules with checksums, but the user delete them, or +// unchecked checksums. +func (e *EventsStore) GetComputeChecksums() bool { + e.mu.RLock() + defer e.mu.RUnlock() + return e.checksumsEnabled +} + +func monitorEventsCache() { + for { + <-eventsCacheTicker.C + EventsCache.DeleteOldItems() + } +} diff --git a/daemon/procmon/details.go b/daemon/procmon/details.go index d5fe5407d5..1fc25b47f1 100644 --- a/daemon/procmon/details.go +++ b/daemon/procmon/details.go @@ -2,12 +2,18 @@ package procmon import ( "bufio" + "crypto/md5" + "crypto/sha1" + "encoding/hex" "fmt" + "hash" + "io" "io/ioutil" "os" "regexp" "strconv" "strings" + "time" "github.com/evilsocket/opensnitch/daemon/core" "github.com/evilsocket/opensnitch/daemon/dns" @@ -16,6 +22,39 @@ import ( ) var socketsRegex, _ = regexp.Compile(`socket:\[([0-9]+)\]`) +var ppidPattern = regexp.MustCompile(`PPid:\s*(\d+)`) + +// GetParent obtains the information of this process' parent. +func (p *Process) GetParent() { + if p.Parent != nil { + log.Debug("%d already with parent: %v", p.ID, p.Parent) + return + } + + data, err := ioutil.ReadFile(p.pathStatus) + if err != nil { + return + } + + ppidMatches := ppidPattern.FindStringSubmatch(string(data)) + ppid := 0 + if len(ppidMatches) < 2 { + return + } + + ppid, err = strconv.Atoi(ppidMatches[1]) + if err != nil { + return + } + if ppid == 0 { + return + } + p.Parent = NewProcess(ppid, "") + p.Parent.GetInfo() + + // get process tree + //p.Parent.GetParent() +} // GetInfo collects information of a process. func (p *Process) GetInfo() error { @@ -39,9 +78,10 @@ func (p *Process) GetInfo() error { p.ReadCwd() if err := p.ReadPath(); err != nil { - log.Error("GetInfo() path can't be read") + log.Debug("GetInfo() path can't be read: %s", p.Path) return err } + // we need to load the env variables now, in order to be used with the rules. p.ReadEnv() return nil @@ -62,7 +102,7 @@ func (p *Process) ReadComm() error { if p.Comm != "" { return nil } - data, err := ioutil.ReadFile(fmt.Sprintf("/proc/%d/comm", p.ID)) + data, err := ioutil.ReadFile(p.pathComm) if err != nil { return err } @@ -75,7 +115,7 @@ func (p *Process) ReadCwd() error { if p.CWD != "" { return nil } - link, err := os.Readlink(fmt.Sprintf("/proc/%d/cwd", p.ID)) + link, err := os.Readlink(p.pathCwd) if err != nil { return err } @@ -85,7 +125,7 @@ func (p *Process) ReadCwd() error { // ReadEnv reads and parses the environment variables of a process. func (p *Process) ReadEnv() { - data, err := ioutil.ReadFile(fmt.Sprintf("/proc/%d/environ", p.ID)) + data, err := ioutil.ReadFile(p.pathEnviron) if err != nil { return } @@ -117,7 +157,7 @@ func (p *Process) ReadPath() error { defer func() { if p.Path == "" { // determine if this process might be of a kernel task. - if data, err := ioutil.ReadFile(fmt.Sprintf("/proc/%d/maps", p.ID)); err == nil && len(data) == 0 { + if data, err := ioutil.ReadFile(p.pathMaps); err == nil && len(data) == 0 { p.Path = "Kernel connection" p.Args = append(p.Args, p.Comm) return @@ -126,13 +166,12 @@ func (p *Process) ReadPath() error { } }() - linkName := fmt.Sprint("/proc/", p.ID, "/exe") - if _, err := os.Lstat(linkName); err != nil { + if _, err := os.Lstat(p.pathExe); err != nil { return err } // FIXME: this reading can give error: file name too long - link, err := os.Readlink(linkName) + link, err := os.Readlink(p.pathExe) if err != nil { return err } @@ -144,6 +183,11 @@ func (p *Process) ReadPath() error { func (p *Process) SetPath(path string) { p.Path = path p.CleanPath() + p.RealPath = fmt.Sprint(p.pathRoot, "/", p.Path) + if core.Exists(p.RealPath) == false { + p.RealPath = p.Path + // p.CleanPath() ? + } } // ReadCmdline reads the cmdline of the process from ProcFS /proc//cmdline @@ -153,7 +197,7 @@ func (p *Process) ReadCmdline() { if len(p.Args) > 0 { return } - if data, err := ioutil.ReadFile(fmt.Sprintf("/proc/%d/cmdline", p.ID)); err == nil { + if data, err := ioutil.ReadFile(p.pathCmdline); err == nil { if len(data) == 0 { return } @@ -184,7 +228,7 @@ func (p *Process) CleanArgs() { } func (p *Process) readDescriptors() { - f, err := os.Open(fmt.Sprint("/proc/", p.ID, "/fd/")) + f, err := os.Open(p.pathFd) if err != nil { return } @@ -196,32 +240,35 @@ func (p *Process) readDescriptors() { tempFd := &procDescriptors{ Name: fd.Name(), } - if link, err := os.Readlink(fmt.Sprint("/proc/", p.ID, "/fd/", fd.Name())); err == nil { - tempFd.SymLink = link - socket := socketsRegex.FindStringSubmatch(link) - if len(socket) > 0 { - socketInfo, err := netlink.GetSocketInfoByInode(socket[1]) - if err == nil { - tempFd.SymLink = fmt.Sprintf("socket:[%s] - %d:%s -> %s:%d, state: %s", fd.Name(), - socketInfo.ID.SourcePort, - socketInfo.ID.Source.String(), - dns.HostOr(socketInfo.ID.Destination, socketInfo.ID.Destination.String()), - socketInfo.ID.DestinationPort, - netlink.TCPStatesMap[socketInfo.State]) - } + link, err := os.Readlink(fmt.Sprint(p.pathFd, fd.Name())) + if err != nil { + continue + } + tempFd.SymLink = link + socket := socketsRegex.FindStringSubmatch(link) + if len(socket) > 0 { + socketInfo, err := netlink.GetSocketInfoByInode(socket[1]) + if err == nil { + tempFd.SymLink = fmt.Sprintf("socket:[%s] - %d:%s -> %s:%d, state: %s", fd.Name(), + socketInfo.ID.SourcePort, + socketInfo.ID.Source.String(), + dns.HostOr(socketInfo.ID.Destination, socketInfo.ID.Destination.String()), + socketInfo.ID.DestinationPort, + netlink.TCPStatesMap[socketInfo.State]) } + } - if linkInfo, err := os.Lstat(link); err == nil { - tempFd.Size = linkInfo.Size() - tempFd.ModTime = linkInfo.ModTime() - } + if linkInfo, err := os.Lstat(link); err == nil { + tempFd.Size = linkInfo.Size() + tempFd.ModTime = linkInfo.ModTime() } + p.Descriptors = append(p.Descriptors, tempFd) } } func (p *Process) readIOStats() { - f, err := os.Open(fmt.Sprint("/proc/", p.ID, "/io")) + f, err := os.Open(p.pathIO) if err != nil { return } @@ -250,19 +297,19 @@ func (p *Process) readIOStats() { } func (p *Process) readStatus() { - if data, err := ioutil.ReadFile(fmt.Sprint("/proc/", p.ID, "/status")); err == nil { + if data, err := ioutil.ReadFile(p.pathStatus); err == nil { p.Status = string(data) } - if data, err := ioutil.ReadFile(fmt.Sprint("/proc/", p.ID, "/stat")); err == nil { + if data, err := ioutil.ReadFile(p.pathStat); err == nil { p.Stat = string(data) } if data, err := ioutil.ReadFile(fmt.Sprint("/proc/", p.ID, "/stack")); err == nil { p.Stack = string(data) } - if data, err := ioutil.ReadFile(fmt.Sprint("/proc/", p.ID, "/maps")); err == nil { + if data, err := ioutil.ReadFile(p.pathMaps); err == nil { p.Maps = string(data) } - if data, err := ioutil.ReadFile(fmt.Sprint("/proc/", p.ID, "/statm")); err == nil { + if data, err := ioutil.ReadFile(p.pathStatm); err == nil { p.Statm = &procStatm{} fmt.Sscanf(string(data), "%d %d %d %d %d %d %d", &p.Statm.Size, &p.Statm.Resident, &p.Statm.Shared, &p.Statm.Text, &p.Statm.Lib, &p.Statm.Data, &p.Statm.Dt) } @@ -280,7 +327,7 @@ func (p *Process) CleanPath() { // to any process. // Therefore we cannot use /proc/self/exe directly, because it resolves to our own process. if p.Path == "/proc/self/exe" { - if link, err := os.Readlink(fmt.Sprint("/proc/", p.ID, "/exe")); err == nil { + if link, err := os.Readlink(p.pathExe); err == nil { p.Path = link return } @@ -309,5 +356,189 @@ func (p *Process) CleanPath() { // IsAlive checks if the process is still running func (p *Process) IsAlive() bool { - return core.Exists(fmt.Sprint("/proc/", p.ID)) + return core.Exists(p.pathProc) +} + +// IsChild determines if this process is child of its parent +func (p *Process) IsChild() bool { + return p.Parent != nil && p.Parent.Path == p.Path && p.Parent.IsAlive() //&& proc.Starttime != proc.Parent.Starttime +} + +// ComputeChecksums calculates the checksums of a the process path to the binary. +// Users may want to use different hashing alogrithms. +func (p *Process) ComputeChecksums(hashes map[string]uint) { + for hash := range hashes { + p.ComputeChecksum(hash) + } +} + +// ComputeChecksum calculates the checksum of a the process path to the binary +func (p *Process) ComputeChecksum(algo string) { + if p.Path == "" || p.Path == KernelConnection { + return + } + if p.Checksums[algo] != "" { + log.Debug("[hashing] %d already hasshed [%s]: %s\n", p.ID, algo, p.Checksums[algo]) + return + } + + // - hash first the exe link. That's the process that is currently running. + // If the binary has been updated while it's running, the checksum on disk + // will change and it won't match the one defined in the rules. + // However the exe link will match the one defined in the rules. + // So keep it valid until the user restarts the process. + // + // - If it can't be read, hash the RealPath, because containerized binaries' + // path usually won't exist on the host. + // Path cannot be trusted, because multiple processes with the same path + // can coexist in different namespaces. + // The real path is /proc//root/ + paths := []string{p.pathExe, p.RealPath, p.Path} + + var h hash.Hash + if algo == HashMD5 { + h = md5.New() + } else if algo == HashSHA1 { + h = sha1.New() + } else { + log.Debug("Unknown hashing algorithm: %s", algo) + return + } + + i := uint8(0) + for i = 0; i < 2; i++ { + log.Debug("[hashing %s], path %d: %s", algo, i, paths[i]) + + start := time.Now() + h.Reset() + // can this be instantiate outside of the loop? + f, err := os.Open(paths[i]) + if err != nil { + log.Debug("[hashing %s] Unable to open path: %s", algo, paths[i]) + + // one of the reasons to end here is when hashing AppImages + code, err := p.dumpImage() + if err != nil { + log.Debug("[hashing] Unable to dump process memory: %s", err) + continue + } + p.Checksums[algo] = hex.EncodeToString(h.Sum(code)) + log.Debug("[hashing] memory region hashed, elapsed: %v ,Hash: %s, %s\n", time.Since(start), p.Checksums[algo], paths[i]) + code = nil + break + } + defer f.Close() + + if _, err = io.Copy(h, f); err != nil { + log.Debug("[hashing %s] Error copying data: %s", algo, err) + continue + } + p.Checksums[algo] = hex.EncodeToString(h.Sum(nil)) + log.Debug("[hashing] elapsed: %v ,Hash: %s, %s\n", time.Since(start), p.Checksums[algo], paths[i]) + + break + } + + return +} + +// MemoryMapping represents a memory mapping region +type MemoryMapping struct { + StartAddr uint64 + EndAddr uint64 +} + +func (p *Process) dumpImage() ([]byte, error) { + return p.dumpFileImage(p.Path) +} + +// dumpFileImage will dump the memory region of a file mapped by this process. +// By default it'll dump the current image of this process. +func (p *Process) dumpFileImage(filePath string) ([]byte, error) { + var mappings []MemoryMapping + + // read memory mappings + mapsFile, err := os.Open(p.pathMaps) + if err != nil { + return nil, err + } + defer mapsFile.Close() + + if filePath == "" { + filePath = p.Path + } + + size := 0 + mapsScanner := bufio.NewScanner(mapsFile) + for mapsScanner.Scan() { + addrMap := mapsScanner.Text() + // filter by process path + // TODO: make it configurable + if !strings.Contains(addrMap, filePath) { + log.Debug("dumpFileImage() addr doesn't contain %s", filePath) + continue + } + fields := strings.Fields(addrMap) + if len(fields) < 6 { + log.Debug("dumpFileImage() line less than 6: %v", fields) + continue + } + + // TODO: make it configurable + /*permissions := fields[1] + if !strings.Contains(permissions, "r-xp") { + continue + } + */ + + addrRange := strings.Split(fields[0], "-") + addrStart, err := strconv.ParseUint(addrRange[0], 16, 64) + if err != nil { + //log.Debug("dumpFileImage() invalid addrStart: %v", addrRange) + continue + } + addrEnd, err := strconv.ParseUint(addrRange[1], 16, 64) + if err != nil { + log.Debug("dumpFileImage() invalid addrEnd: %v", addrRange) + continue + } + size += int(addrEnd - addrStart) + mappings = append(mappings, MemoryMapping{StartAddr: addrStart, EndAddr: addrEnd}) + } + + // read process memory + elfCode, err := p.readMem(mappings) + mappings = nil + //fmt.Printf(">>> READ MEM, regions size: %d, elfCode: %d\n", size, len(elfCode)) + + //if fInfo, err := os.Stat(filePath); err == nil { + // fmt.Printf("\t>>> on disk: %d\n", fInfo.Size()) + //} + + if err != nil { + return nil, err + } + + return elfCode, nil +} + +// given a range of addrs, read it from mem and return the content +func (p *Process) readMem(mappings []MemoryMapping) ([]byte, error) { + var elfCode []byte + memFile, err := os.Open(p.pathMem) + if err != nil { + return nil, err + } + defer memFile.Close() + + for _, mapping := range mappings { + memFile.Seek(int64(mapping.StartAddr), io.SeekStart) + code := make([]byte, mapping.EndAddr-mapping.StartAddr) + _, err = memFile.Read(code) + if err != nil { + return nil, err + } + } + + return elfCode, nil } diff --git a/daemon/procmon/ebpf/cache.go b/daemon/procmon/ebpf/cache.go index 6db489edbf..85b0531e32 100644 --- a/daemon/procmon/ebpf/cache.go +++ b/daemon/procmon/ebpf/cache.go @@ -7,90 +7,21 @@ import ( "github.com/evilsocket/opensnitch/daemon/procmon" ) -// NewExecEvent constructs a new execEvent from the arguments. -func NewExecEvent(pid, ppid, uid uint64, path string, comm [16]byte) *execEvent { - ev := &execEvent{ - Type: EV_TYPE_EXEC, - PID: pid, - PPID: ppid, - UID: uid, - Comm: comm, - } - length := MaxPathLen - if len(path) < MaxPathLen { - length = len(path) - } - copy(ev.Filename[:], path[:length]) - return ev -} - -type execEventItem struct { - Proc procmon.Process - Event execEvent - LastSeen int64 -} - -type eventsStore struct { - execEvents map[uint64]*execEventItem - sync.RWMutex -} - -// NewEventsStore creates a new store of events. -func NewEventsStore() *eventsStore { - return &eventsStore{ - execEvents: make(map[uint64]*execEventItem), - } -} - -func (e *eventsStore) add(key uint64, event execEvent, proc procmon.Process) { - e.Lock() - defer e.Unlock() - e.execEvents[key] = &execEventItem{ - Proc: proc, - Event: event, - } -} - -func (e *eventsStore) isInStore(key uint64) (item *execEventItem, found bool) { - e.RLock() - defer e.RUnlock() - item, found = e.execEvents[key] - return -} - -func (e *eventsStore) delete(key uint64) { - e.Lock() - defer e.Unlock() - delete(e.execEvents, key) -} - -func (e *eventsStore) DeleteOldItems() { - e.Lock() - defer e.Unlock() - - for k, item := range e.execEvents { - if item.Proc.IsAlive() == false { - delete(e.execEvents, k) - } - } -} - -//----------------------------------------------------------------------------- - type ebpfCacheItem struct { Proc procmon.Process - Key []byte LastSeen int64 + Key []byte } type ebpfCacheType struct { Items map[interface{}]*ebpfCacheItem - sync.RWMutex + mu *sync.RWMutex } var ( + // TODO: allow to configure these options maxTTL = 40 // Seconds - maxCacheItems = 5000 + maxCacheItems = 50000 ebpfCache *ebpfCacheType ebpfCacheTicker *time.Ticker ) @@ -116,19 +47,20 @@ func NewEbpfCache() *ebpfCacheType { ebpfCacheTicker = time.NewTicker(1 * time.Minute) return &ebpfCacheType{ Items: make(map[interface{}]*ebpfCacheItem, 0), + mu: &sync.RWMutex{}, } } func (e *ebpfCacheType) addNewItem(key interface{}, itemKey []byte, proc procmon.Process) { - e.Lock() + e.mu.Lock() e.Items[key] = NewEbpfCacheItem(itemKey, proc) - e.Unlock() + e.mu.Unlock() } func (e *ebpfCacheType) isInCache(key interface{}) (item *ebpfCacheItem, found bool) { leng := e.Len() - e.Lock() + e.mu.Lock() item, found = e.Items[key] if found { if item.isValid() { @@ -138,7 +70,7 @@ func (e *ebpfCacheType) isInCache(key interface{}) (item *ebpfCacheItem, found b delete(e.Items, key) } } - e.Unlock() + e.mu.Unlock() if leng > maxCacheItems { e.DeleteOldItems() @@ -152,16 +84,16 @@ func (e *ebpfCacheType) update(key interface{}, item *ebpfCacheItem) { } func (e *ebpfCacheType) Len() int { - e.RLock() - defer e.RUnlock() + e.mu.RLock() + defer e.mu.RUnlock() return len(e.Items) } func (e *ebpfCacheType) DeleteOldItems() { length := e.Len() - e.Lock() - defer e.Unlock() + e.mu.Lock() + defer e.mu.Unlock() for k, item := range e.Items { if length > maxCacheItems || (item != nil && !item.isValid()) { @@ -171,8 +103,8 @@ func (e *ebpfCacheType) DeleteOldItems() { } func (e *ebpfCacheType) delete(key interface{}) { - e.Lock() - defer e.Unlock() + e.mu.Lock() + defer e.mu.Unlock() if key, found := e.Items[key]; found { delete(e.Items, key) @@ -183,8 +115,8 @@ func (e *ebpfCacheType) clear() { if e == nil { return } - e.Lock() - defer e.Unlock() + e.mu.Lock() + defer e.mu.Unlock() for k := range e.Items { delete(e.Items, k) } diff --git a/daemon/procmon/ebpf/ebpf.go b/daemon/procmon/ebpf/ebpf.go index 74cde4ea86..a3e0c47a36 100644 --- a/daemon/procmon/ebpf/ebpf.go +++ b/daemon/procmon/ebpf/ebpf.go @@ -38,6 +38,19 @@ type alreadyEstablishedConns struct { sync.RWMutex } +// list of returned errors +const ( + NoError = iota + NotAvailable + EventsNotAvailable +) + +// Error returns the error type and a message with the explanation +type Error struct { + What int // 1 global error, 2 events error, 3 ... + Msg error +} + var ( m, perfMod *elf.Module lock = sync.RWMutex{} @@ -62,18 +75,19 @@ var ( ) //Start installs ebpf kprobes -func Start() error { +func Start() *Error { setRunning(false) if err := mountDebugFS(); err != nil { - log.Error("ebpf.Start -> mount debugfs error. Report on github please: %s", err) - return err + return &Error{ + NotAvailable, + fmt.Errorf("ebpf.Start: mount debugfs error. Report on github please: %s", err), + } } var err error m, err = core.LoadEbpfModule("opensnitch.o") if err != nil { - log.Error("%s", err) dispatchErrorEvent(fmt.Sprint("[eBPF]: ", err.Error())) - return err + return &Error{NotAvailable, fmt.Errorf("[eBPF] Error loading opensnitch.o: %s", err.Error())} } m.EnableOptionCompatProbe() @@ -83,12 +97,10 @@ func Start() error { if err := m.EnableKprobes(0); err != nil { m.Close() if err := m.Load(nil); err != nil { - log.Error("eBPF failed to load /etc/opensnitchd/opensnitch.o (2): %v", err) - return err + return &Error{NotAvailable, fmt.Errorf("eBPF failed to load /etc/opensnitchd/opensnitch.o (2): %v", err)} } if err := m.EnableKprobes(0); err != nil { - log.Error("eBPF error when enabling kprobes: %v", err) - return err + return &Error{NotAvailable, fmt.Errorf("eBPF error when enabling kprobes: %v", err)} } } determineHostByteOrder() @@ -105,13 +117,13 @@ func Start() error { } for prot, mfp := range ebpfMaps { if mfp.bpfmap == nil { - return fmt.Errorf("eBPF module opensnitch.o malformed, bpfmap[%s] nil", prot) + return &Error{NotAvailable, fmt.Errorf("eBPF module opensnitch.o malformed, bpfmap[%s] nil", prot)} } } ctxTasks, cancelTasks = context.WithCancel(context.Background()) ebpfCache = NewEbpfCache() - initEventsStreamer() + errf := initEventsStreamer() saveEstablishedConnections(uint8(syscall.AF_INET)) if core.IPv6Enabled { @@ -124,7 +136,7 @@ func Start() error { go monitorAlreadyEstablished() setRunning(true) - return nil + return errf } func saveEstablishedConnections(commDomain uint8) error { @@ -155,6 +167,7 @@ func setRunning(status bool) { // Stop stops monitoring connections using kprobes func Stop() { + log.Debug("ebpf.Stop()") lock.RLock() defer lock.RUnlock() if running == false { diff --git a/daemon/procmon/ebpf/events.go b/daemon/procmon/ebpf/events.go index e6f82f1cd6..fcc8e92736 100644 --- a/daemon/procmon/ebpf/events.go +++ b/daemon/procmon/ebpf/events.go @@ -47,6 +47,7 @@ type networkEventT struct { Pid uint64 UID uint64 Comm [TaskCommLen]byte + //Ns uint64 } // List of supported events @@ -59,7 +60,6 @@ const ( ) var ( - execEvents = NewEventsStore() perfMapList = make(map[*elf.PerfMap]*elf.Module) // total workers spawned by the different events PerfMaps eventWorkers = 0 @@ -71,20 +71,20 @@ var ( ringBuffSize = 64 // * PAGE_SIZE (4k usually) ) -func initEventsStreamer() { +func initEventsStreamer() *Error { elfOpts := make(map[string]elf.SectionParams) elfOpts["maps/"+perfMapName] = elf.SectionParams{PerfRingBufferPageCount: ringBuffSize} var err error perfMod, err = core.LoadEbpfModule("opensnitch-procs.o") if err != nil { dispatchErrorEvent(fmt.Sprint("[eBPF events]: ", err)) - return + return &Error{EventsNotAvailable, err} } perfMod.EnableOptionCompatProbe() if err = perfMod.Load(elfOpts); err != nil { dispatchErrorEvent(fmt.Sprint("[eBPF events]: ", err)) - return + return &Error{EventsNotAvailable, err} } tracepoints := []string{ @@ -99,7 +99,8 @@ func initEventsStreamer() { for _, tp := range tracepoints { err = perfMod.EnableTracepoint(tp) if err != nil { - dispatchErrorEvent(fmt.Sprintf("[eBPF events] error enabling tracepoint %s: %s", tp, err)) + dispatchErrorEvent(fmt.Sprintf(`[eBPF events] error enabling tracepoint %s: %s +Verify that your kernel has support for tracepoints (opensnitchd -check-requirements).`, tp, err)) } } @@ -109,7 +110,7 @@ func initEventsStreamer() { perfMod.Close() if err = perfMod.Load(elfOpts); err != nil { dispatchErrorEvent(fmt.Sprintf("[eBPF events] failed to load /etc/opensnitchd/opensnitch-procs.o (2): %v", err)) - return + return &Error{EventsNotAvailable, err} } if err = perfMod.EnableKprobes(0); err != nil { dispatchErrorEvent(fmt.Sprintf("[eBPF events] error enabling kprobes: %v", err)) @@ -123,28 +124,34 @@ func initEventsStreamer() { }(sig) eventWorkers = 0 - initPerfMap(perfMod) + if err := initPerfMap(perfMod); err != nil { + return &Error{EventsNotAvailable, err} + } + + return nil } -func initPerfMap(mod *elf.Module) { +func initPerfMap(mod *elf.Module) error { perfChan := make(chan []byte) lostEvents := make(chan uint64, 1) var err error perfMap, err := elf.InitPerfMap(mod, perfMapName, perfChan, lostEvents) if err != nil { dispatchErrorEvent(fmt.Sprintf("[eBPF events] Error initializing eBPF events perfMap: %s", err)) - return + return err } perfMapList[perfMap] = mod eventWorkers += 4 for i := 0; i < eventWorkers; i++ { - go streamEventsWorker(i, perfChan, lostEvents, kernelEvents, execEvents) + go streamEventsWorker(i, perfChan, lostEvents, kernelEvents) } perfMap.PollStart() + + return nil } -func streamEventsWorker(id int, chn chan []byte, lost chan uint64, kernelEvents chan interface{}, execEvents *eventsStore) { +func streamEventsWorker(id int, chn chan []byte, lost chan uint64, kernelEvents chan interface{}) { var event execEvent for { select { @@ -158,22 +165,32 @@ func streamEventsWorker(id int, chn chan []byte, lost chan uint64, kernelEvents } else { switch event.Type { case EV_TYPE_EXEC, EV_TYPE_EXECVEAT: - if _, found := execEvents.isInStore(event.PID); found { - log.Debug("[eBPF event inCache] -> %d", event.PID) - continue - } proc := event2process(&event) if proc == nil { continue } - execEvents.add(event.PID, event, *proc) + // TODO: store multiple executions with the same pid but different paths: forks, execves... + if p, needsHashUpdate, found := procmon.EventsCache.IsInStore(int(event.PID), proc); found { + if needsHashUpdate { + procmon.EventsCache.ComputeChecksums(proc) + } + log.Debug("[eBPF event inCache] -> %d, %v", event.PID, p.Proc.Checksums) + continue + } + procmon.EventsCache.Add(*proc) case EV_TYPE_SCHED_EXIT: - log.Debug("[eBPF exit event] -> %d", event.PID) - if _, found := execEvents.isInStore(event.PID); found { - log.Debug("[eBPF exit event inCache] -> %d", event.PID) - execEvents.delete(event.PID) + log.Debug("[eBPF exit event] total: %d, pid: %d, ppid: %d", 0 /*execEvents.Len()*/, event.PID, event.PPID) + ev, _, found := procmon.EventsCache.IsInStore(int(event.PID), nil) + if !found { + continue + } + log.Debug("[eBPF exit event inCache] pid: %d, tgid: %d", event.PID, event.PPID) + if ev.Proc.IsAlive() == false { + procmon.EventsCache.Delete(int(event.PID)) + log.Debug("[ebpf exit event] deleting DEAD pid: %d", event.PID) } + } } } @@ -184,7 +201,6 @@ Exit: } func event2process(event *execEvent) (proc *procmon.Process) { - proc = procmon.NewProcess(int(event.PID), byteArrayToString(event.Comm[:])) // trust process path received from kernel path := byteArrayToString(event.Filename[:]) @@ -207,6 +223,7 @@ func event2process(event *execEvent) (proc *procmon.Process) { } else { proc.ReadCmdline() } + proc.GetParent() log.Debug("[eBPF exec event] ppid: %d, pid: %d, %s -> %s", event.PPID, event.PID, proc.Path, proc.Args) return diff --git a/daemon/procmon/ebpf/find.go b/daemon/procmon/ebpf/find.go index 2d934bea3f..b0a1a54e97 100644 --- a/daemon/procmon/ebpf/find.go +++ b/daemon/procmon/ebpf/find.go @@ -164,31 +164,28 @@ func findConnProcess(value *networkEventT, connKey string) (proc *procmon.Proces // This is the UID that we've always used. proc.UID = int(value.UID) - err := proc.ReadPath() - if ev, found := execEvents.isInStore(value.Pid); found { + if ev, _, found := procmon.EventsCache.IsInStore(int(value.Pid), nil); found { // use socket's UID. See above why ^ ev.Proc.UID = proc.UID - ev.Proc.ReadCmdline() - // if proc's ReadPath() has been successfull, and the path received via the execve tracepoint differs, - // use proc's path. - // Sometimes we received from the tracepoint a wrong/non-existent path. - // Othertimes we receive a "helper" that executes the real binary which opens the connection. - // Downsides: for execveat() executions we won't display the original binary. - if err == nil && ev.Proc.Path != proc.Path { - proc.ReadCmdline() + if proc.Path == "" { + proc.Path = ev.Proc.Path + proc.Args = ev.Proc.Args + } + if /*err == nil &&*/ ev.Proc.Path != proc.Path { + //proc.ReadCmdline() ev.Proc.Path = proc.Path ev.Proc.Args = proc.Args } proc = &ev.Proc + // XXX: update cache? - log.Debug("[ebpf conn] not in cache, but in execEvents: %s, %d -> %s", connKey, proc.ID, proc.Path) + log.Debug("[ebpf conn] not in cache, but in execEvents: %s, %d -> %s -> %s", connKey, proc.ID, proc.Path, proc.Args) } else { - log.Debug("[ebpf conn] not in cache, NOR in execEvents: %s, %d -> %s", connKey, proc.ID, proc.Path) + log.Debug("[ebpf conn] not in cache, NOR in execEvents: %s, %d -> %s -> %s", connKey, proc.ID, proc.Path, proc.Args) // We'll end here if the events module has not been loaded, or if the process is not in cache. + proc.GetParent() proc.GetInfo() - execEvents.add(value.Pid, - *NewExecEvent(value.Pid, 0, value.UID, proc.Path, value.Comm), - *proc) + procmon.EventsCache.Add(*proc) } return diff --git a/daemon/procmon/ebpf/monitor.go b/daemon/procmon/ebpf/monitor.go index 958b5d1fab..c9e28613f8 100644 --- a/daemon/procmon/ebpf/monitor.go +++ b/daemon/procmon/ebpf/monitor.go @@ -39,7 +39,6 @@ func monitorCache() { goto Exit case <-ebpfCacheTicker.C: ebpfCache.DeleteOldItems() - execEvents.DeleteOldItems() } } Exit: diff --git a/daemon/procmon/monitor/init.go b/daemon/procmon/monitor/init.go index 4bad752d88..83df3b8574 100644 --- a/daemon/procmon/monitor/init.go +++ b/daemon/procmon/monitor/init.go @@ -1,21 +1,56 @@ package monitor import ( - "net" + "context" "github.com/evilsocket/opensnitch/daemon/log" + netlinkProcmon "github.com/evilsocket/opensnitch/daemon/netlink/procmon" "github.com/evilsocket/opensnitch/daemon/procmon" "github.com/evilsocket/opensnitch/daemon/procmon/audit" "github.com/evilsocket/opensnitch/daemon/procmon/ebpf" ) var ( - cacheMonitorsRunning = false + cacheMonitorsRunning = false + netlinkProcmonRunning = false + ctx, cancelTasks = context.WithCancel(context.Background()) ) -// ReconfigureMonitorMethod configures a new method for parsing connections. -func ReconfigureMonitorMethod(newMonitorMethod string) error { +// List of errors that this package may return. +const ( + NoError = iota + ProcFsErr + AuditdErr + EbpfErr + EbpfEventsErr +) + +// Error wraps the type of error with its message +type Error struct { + What int + Msg error +} + +func startProcMonitors() { + if netlinkProcmonRunning == false { + ctx, cancelTasks = context.WithCancel(context.Background()) + for i := 0; i < 4; i++ { + go procmon.MonitorProcEvents(ctx.Done()) + } + go netlinkProcmon.ProcEventsMonitor(ctx.Done()) + netlinkProcmonRunning = true + } +} + +func stopProcMonitors() { + if netlinkProcmonRunning { + cancelTasks() + netlinkProcmonRunning = false + } +} +// ReconfigureMonitorMethod configures a new method for parsing connections. +func ReconfigureMonitorMethod(newMonitorMethod string) *Error { if procmon.GetMonitorMethod() == newMonitorMethod { return nil } @@ -26,7 +61,9 @@ func ReconfigureMonitorMethod(newMonitorMethod string) error { // if the new monitor method fails to start, rollback the change and exit // without saving the configuration. Otherwise we can end up with the wrong // monitor method configured and saved to file. - if err := Init(); err != nil { + err := Init() + if err.What > NoError { + log.Error("Reconf() -> Init() error: %v", err) procmon.SetMonitorMethod(oldMethod) return err } @@ -36,6 +73,7 @@ func ReconfigureMonitorMethod(newMonitorMethod string) error { // End stops the way of parsing new connections. func End() { + stopProcMonitors() if procmon.MethodIsAudit() { audit.Stop() } else if procmon.MethodIsEbpf() { @@ -44,36 +82,52 @@ func End() { } // Init starts parsing connections using the method specified. -func Init() (err error) { +func Init() (errm *Error) { + errm = &Error{} + if cacheMonitorsRunning == false { - go procmon.MonitorActivePids() go procmon.CacheCleanerTask() cacheMonitorsRunning = true } if procmon.MethodIsEbpf() { - err = ebpf.Start() + err := ebpf.Start() if err == nil { log.Info("Process monitor method ebpf") - return nil + return errm + } + // ebpf main module loaded, we can use ebpf + + // XXX: this will have to be rewritten when we'll have more events (bind, listen, etc) + if err.What == ebpf.EventsNotAvailable { + log.Info("Process monitor method ebpf") + log.Warning("opensnitch-procs.o not available: %s", err.Msg) + + startProcMonitors() + return errm } // we need to stop this method even if it has failed to start, in order to clean up the kprobes // It helps with the error "cannot write...kprobe_events: file exists". ebpf.Stop() + errm.What = err.What + errm.Msg = err.Msg log.Warning("error starting ebpf monitor method: %v", err) + } else if procmon.MethodIsAudit() { - var auditConn net.Conn - auditConn, err = audit.Start() + auditConn, err := audit.Start() if err == nil { log.Info("Process monitor method audit") go audit.Reader(auditConn, (chan<- audit.Event)(audit.EventChan)) - return nil + return &Error{AuditdErr, err} } + errm.What = AuditdErr + errm.Msg = err log.Warning("error starting audit monitor method: %v", err) } + startProcMonitors() // if any of the above methods have failed, fallback to proc log.Info("Process monitor method /proc") procmon.SetMonitorMethod(procmon.MethodProc) - return err + return errm } diff --git a/daemon/procmon/parse.go b/daemon/procmon/parse.go index 224ff16db0..15b651dcda 100644 --- a/daemon/procmon/parse.go +++ b/daemon/procmon/parse.go @@ -97,8 +97,8 @@ func FindProcess(pid int, interceptUnknown bool) *Process { return NewProcess(0, "") } - if proc := findProcessInActivePidsCache(uint64(pid)); proc != nil { - return proc + if ev, _, found := EventsCache.IsInStore(pid, nil); found { + return &ev.Proc } proc := NewProcess(pid, "") @@ -107,6 +107,6 @@ func FindProcess(pid int, interceptUnknown bool) *Process { return nil } - AddToActivePidsCache(uint64(pid), proc) + EventsCache.Add(*proc) return proc } diff --git a/daemon/procmon/process.go b/daemon/procmon/process.go index ea89fa8732..f00eb9e49f 100644 --- a/daemon/procmon/process.go +++ b/daemon/procmon/process.go @@ -1,6 +1,8 @@ package procmon import ( + "context" + "fmt" "sync" "time" @@ -8,9 +10,9 @@ import ( ) var ( - cacheMonitorsRunning = false - lock = sync.RWMutex{} - monitorMethod = MethodProc + lock = sync.RWMutex{} + monitorMethod = MethodProc + Ctx, CancelTasks = context.WithCancel(context.Background()) ) // monitor method supported types @@ -18,6 +20,11 @@ const ( MethodProc = "proc" MethodAudit = "audit" MethodEbpf = "ebpf" + + KernelConnection = "Kernel connection" + + HashMD5 = "process.hash.md5" + HashSHA1 = "process.hash.sha1" ) // man 5 proc; man procfs @@ -36,10 +43,10 @@ type procNetStats struct { } type procDescriptors struct { + ModTime time.Time Name string SymLink string Size int64 - ModTime time.Time } type procStatm struct { @@ -54,12 +61,14 @@ type procStatm struct { // Process holds the details of a process. type Process struct { - ID int - PPID int - UID int - Comm string - // Path is the absolute path to the binary - Path string + Checksums map[string]string + Env map[string]string + Descriptors []*procDescriptors + Parent *Process + IOStats *procIOstats + NetStats *procNetStats + Statm *procStatm + // Args is the command that the user typed. It MAY contain the absolute path // of the binary: // $ curl https://... @@ -68,30 +77,75 @@ type Process struct { // $ /usr/bin/curl https://... // -> Path: /usr/bin/curl // -> Args: /usr/bin/curl https://.... - Args []string - Env map[string]string - CWD string - Descriptors []*procDescriptors - IOStats *procIOstats - NetStats *procNetStats - Status string - Stat string - Statm *procStatm - Stack string - Maps string + Args []string + Status string + Stat string + Stack string + Maps string + Comm string + + pathProc string + pathComm string + pathExe string + pathCmdline string + pathCwd string + pathEnviron string + pathRoot string + pathFd string + pathStatus string + pathStatm string + pathStat string + pathMaps string + pathMem string + pathIO string + + // Path is the absolute path to the binary + Path string + + // RealPath is the path to the binary taking into account its root fs. + // The simplest form of accessing the RealPath is by prepending /proc//root/ to the path: + // /usr/bin/curl -> /proc//root/usr/bin/curl + RealPath string + CWD string + Starttime int64 + ID int + PPID int + UID int } // NewProcess returns a new Process structure. func NewProcess(pid int, comm string) *Process { - return &Process{ - ID: pid, - Comm: comm, - Args: make([]string, 0), - Env: make(map[string]string), - IOStats: &procIOstats{}, - NetStats: &procNetStats{}, - Statm: &procStatm{}, + + p := &Process{ + Starttime: time.Now().UnixNano(), + ID: pid, + Comm: comm, + Args: make([]string, 0), + Env: make(map[string]string), + IOStats: &procIOstats{}, + NetStats: &procNetStats{}, + Statm: &procStatm{}, + Checksums: make(map[string]string), + } + if pid <= 0 { + return p } + p.pathProc = fmt.Sprint("/proc/", p.ID) + p.pathExe = fmt.Sprint(p.pathProc, "/exe") + p.pathCwd = fmt.Sprint(p.pathProc, "/cwd") + p.pathComm = fmt.Sprint(p.pathProc, "/comm") + p.pathCmdline = fmt.Sprint(p.pathProc, "/cmdline") + p.pathEnviron = fmt.Sprint(p.pathProc, "/environ") + p.pathStatus = fmt.Sprint(p.pathProc, "/status") + p.pathStatm = fmt.Sprint(p.pathProc, "/statm") + p.pathRoot = fmt.Sprint(p.pathProc, "/root") + p.pathMaps = fmt.Sprint(p.pathProc, "/maps") + p.pathStat = fmt.Sprint(p.pathProc, "/stat") + p.pathMem = fmt.Sprint(p.pathProc, "/mem") + p.pathFd = fmt.Sprint(p.pathProc, "/fd/") + p.pathIO = fmt.Sprint(p.pathProc, "/io") + + return p } //Serialize transforms a Process object to gRPC protocol object @@ -104,6 +158,7 @@ func (p *Process) Serialize() *protocol.Process { if netStats == nil { netStats = &procNetStats{} } + return &protocol.Process{ Pid: uint64(p.ID), Ppid: uint64(p.PPID), @@ -113,6 +168,7 @@ func (p *Process) Serialize() *protocol.Process { Args: p.Args, Env: p.Env, Cwd: p.CWD, + Checksums: p.Checksums, IoReads: uint64(ioStats.RChar), IoWrites: uint64(ioStats.WChar), NetReads: netStats.ReadBytes, diff --git a/daemon/rule/loader.go b/daemon/rule/loader.go index 5071a88dce..977164abcc 100644 --- a/daemon/rule/loader.go +++ b/daemon/rule/loader.go @@ -15,6 +15,7 @@ import ( "github.com/evilsocket/opensnitch/daemon/conman" "github.com/evilsocket/opensnitch/daemon/core" "github.com/evilsocket/opensnitch/daemon/log" + "github.com/evilsocket/opensnitch/daemon/procmon" "github.com/fsnotify/fsnotify" ) @@ -23,12 +24,13 @@ import ( // rules watcher. type Loader struct { sync.RWMutex - path string rules map[string]*Rule - rulesKeys []string + path string watcher *fsnotify.Watcher + rulesKeys []string liveReload bool liveReloadRunning bool + checkSums bool } // NewLoader loads rules from disk, and watches for changes made to the rules files @@ -61,6 +63,24 @@ func (l *Loader) GetAll() map[string]*Rule { return l.rules } +// EnableChecksums enables checksums field for rules globally. +func (l *Loader) EnableChecksums(enable bool) { + log.Debug("[rules loader] EnableChecksums:", enable) + l.checkSums = enable + procmon.EventsCache.SetComputeChecksums(enable) +} + +// HasChecksums checks if the rule will check for binary checksum matches +func (l *Loader) HasChecksums(op Operand) { + if op == OpProcessHashMD5 { + log.Debug("[rules loader] Adding MD5") + procmon.EventsCache.AddChecksumHash(string(OpProcessHashMD5)) + } else if op == OpProcessHashSHA1 { + log.Debug("[rules loader] Adding SHA1") + procmon.EventsCache.AddChecksumHash(string(OpProcessHashSHA1)) + } +} + // Load loads rules files from disk. func (l *Loader) Load(path string) error { if core.Exists(path) == false { @@ -119,11 +139,13 @@ func (l *Loader) loadRule(fileName string) error { if r.Enabled { if err := r.Operator.Compile(); err != nil { + l.HasChecksums(r.Operator.Operand) log.Warning("Operator.Compile() error: %s: %s", err, r.Operator.Data) return fmt.Errorf("(1) Error compiling rule: %s", err) } if r.Operator.Type == List { for i := 0; i < len(r.Operator.List); i++ { + l.HasChecksums(r.Operator.List[i].Operand) if err := r.Operator.List[i].Compile(); err != nil { log.Warning("Operator.Compile() error: %s: ", err) return fmt.Errorf("(1) Error compiling list rule: %s", err) @@ -407,7 +429,7 @@ func (l *Loader) FindFirstMatch(con *conman.Connection) (match *Rule) { if rule.Enabled == false { continue } - if rule.Match(con) { + if rule.Match(con, l.checkSums) { // We have a match. // Save the rule in order to don't ask the user to take action, // and keep iterating until a Deny or a Priority rule appears. diff --git a/daemon/rule/operator.go b/daemon/rule/operator.go index a4c1b45d1a..9425151784 100644 --- a/daemon/rule/operator.go +++ b/daemon/rule/operator.go @@ -41,6 +41,8 @@ const ( OpProcessCmd = Operand("process.command") OpProcessEnvPrefix = Operand("process.env.") OpProcessEnvPrefixLen = 12 + OpProcessHashMD5 = Operand("process.hash.md5") + OpProcessHashSHA1 = Operand("process.hash.sha1") OpUserID = Operand("user.id") OpSrcIP = Operand("source.ip") OpSrcPort = Operand("source.port") @@ -57,6 +59,8 @@ const ( OpDomainsRegexpLists = Operand("lists.domains_regexp") OpIPLists = Operand("lists.ips") OpNetLists = Operand("lists.nets") + // TODO + // OpHashMD5 = Operand("lists.hash.md5") ) type opCallback func(value interface{}) bool @@ -70,13 +74,13 @@ type Operator struct { List []Operator `json:"list"` sync.RWMutex - cb opCallback re *regexp.Regexp netMask *net.IPNet - isCompiled bool lists map[string]interface{} - listsMonitorRunning bool + cb opCallback exitMonitorChan chan (bool) + isCompiled bool + listsMonitorRunning bool } // NewOperator returns a new operator object @@ -108,7 +112,18 @@ func (o *Operator) Compile() error { return err } o.re = re - } else if o.Operand == OpDomainsLists { + } else if o.Type == List { + o.Operand = OpList + } else if o.Type == Network { + var err error + _, o.netMask, err = net.ParseCIDR(o.Data) + if err != nil { + return err + } + o.cb = o.cmpNetwork + } + + if o.Operand == OpDomainsLists { if o.Data == "" { return fmt.Errorf("Operand lists is empty, nothing to load: %s", o) } @@ -132,15 +147,8 @@ func (o *Operator) Compile() error { } o.loadLists() o.cb = o.ipNetCmp - } else if o.Type == List { - o.Operand = OpList - } else if o.Type == Network { - var err error - _, o.netMask, err = net.ParseCIDR(o.Data) - if err != nil { - return err - } - o.cb = o.cmpNetwork + } else if o.Operand == OpProcessHashMD5 || o.Operand == OpProcessHashSHA1 { + o.cb = o.hashCmp } log.Debug("Operator compiled: %s", o) o.isCompiled = true @@ -251,21 +259,29 @@ func (o *Operator) reListCmp(v interface{}) bool { return false } -func (o *Operator) listMatch(con interface{}) bool { +func (o *Operator) hashCmp(v interface{}) bool { + hash := v.(string) + if hash == "" { + return true // fake a match to avoid displaying a pop-up + } + return hash == o.Data +} + +func (o *Operator) listMatch(con interface{}, hasChecksums bool) bool { res := true for i := 0; i < len(o.List); i++ { - res = res && o.List[i].Match(con.(*conman.Connection)) + res = res && o.List[i].Match(con.(*conman.Connection), hasChecksums) } return res } // Match tries to match parts of a connection with the given operator. -func (o *Operator) Match(con *conman.Connection) bool { +func (o *Operator) Match(con *conman.Connection, hasChecksums bool) bool { if o.Operand == OpTrue { return true } else if o.Operand == OpList { - return o.listMatch(con) + return o.listMatch(con, hasChecksums) } else if o.Operand == OpProcessPath { return o.cb(con.Process.Path) } else if o.Operand == OpProcessCmd { @@ -298,6 +314,15 @@ func (o *Operator) Match(con *conman.Connection) bool { if ifname, err := net.InterfaceByIndex(con.Pkt.IfaceOutIdx); err == nil { return o.cb(ifname.Name) } + } else if o.Operand == OpProcessHashMD5 || o.Operand == OpProcessHashSHA1 { + ret := true + if !hasChecksums { + return ret + } + for algo := range con.Process.Checksums { + return o.cb(con.Process.Checksums[algo]) + } + return ret } else if o.Operand == OpProto { return o.cb(con.Protocol) } else if o.Operand == OpSrcIP { diff --git a/daemon/rule/rule.go b/daemon/rule/rule.go index 45e60b6af6..f9a15855f1 100644 --- a/daemon/rule/rule.go +++ b/daemon/rule/rule.go @@ -66,8 +66,8 @@ func (r *Rule) String() string { // Match performs on a connection the checks a Rule has, to determine if it // must be allowed or denied. -func (r *Rule) Match(con *conman.Connection) bool { - return r.Operator.Match(con) +func (r *Rule) Match(con *conman.Connection, hasChecksums bool) bool { + return r.Operator.Match(con, hasChecksums) } // Deserialize translates back the rule received to a Rule object diff --git a/daemon/ui/client.go b/daemon/ui/client.go index 541cc0d777..37e083a65a 100644 --- a/daemon/ui/client.go +++ b/daemon/ui/client.go @@ -10,6 +10,7 @@ import ( "github.com/evilsocket/opensnitch/daemon/firewall/iptables" "github.com/evilsocket/opensnitch/daemon/log" "github.com/evilsocket/opensnitch/daemon/log/loggers" + "github.com/evilsocket/opensnitch/daemon/procmon" "github.com/evilsocket/opensnitch/daemon/rule" "github.com/evilsocket/opensnitch/daemon/statistics" "github.com/evilsocket/opensnitch/daemon/ui/auth" @@ -80,6 +81,8 @@ func NewClient(socketPath string, stats *statistics.Statistics, rules *rule.Load if socketPath != "" { c.setSocketPath(c.getSocketPath(socketPath)) } + procmon.EventsCache.SetComputeChecksums(clientConfig.Rules.EnableChecksums) + rules.EnableChecksums(clientConfig.Rules.EnableChecksums) loggers.Load(clientConfig.Server.Loggers, clientConfig.Stats.Workers) stats.SetLimits(clientConfig.Stats) stats.SetLoggers(loggers) diff --git a/daemon/ui/config/config.go b/daemon/ui/config/config.go index 23d30fa323..ac337d81a3 100644 --- a/daemon/ui/config/config.go +++ b/daemon/ui/config/config.go @@ -44,19 +44,26 @@ type serverConfig struct { Loggers []loggers.LoggerConfig `json:"Loggers"` } +type rulesOptions struct { + // TODO: + //RulesPath string `json:"RulesPath"` + EnableChecksums bool `json:"EnableChecksums"` +} + // Config holds the values loaded from configFile type Config struct { sync.RWMutex Server serverConfig `json:"Server"` + Stats statistics.StatsConfig `json:"Stats"` + Rules rulesOptions `json:"Rules"` DefaultAction string `json:"DefaultAction"` DefaultDuration string `json:"DefaultDuration"` - InterceptUnknown bool `json:"InterceptUnknown"` ProcMonitorMethod string `json:"ProcMonitorMethod"` - LogLevel *uint32 `json:"LogLevel"` + Firewall string `json:"Firewall"` + LogLevel *int32 `json:"LogLevel"` + InterceptUnknown bool `json:"InterceptUnknown"` LogUTC bool `json:"LogUTC"` LogMicro bool `json:"LogMicro"` - Firewall string `json:"Firewall"` - Stats statistics.StatsConfig `json:"Stats"` } // Parse determines if the given configuration is ok. diff --git a/daemon/ui/config_utils.go b/daemon/ui/config_utils.go index 83012bcfde..c39505cf16 100644 --- a/daemon/ui/config_utils.go +++ b/daemon/ui/config_utils.go @@ -109,13 +109,15 @@ func (c *Client) loadConfiguration(rawConfig []byte) bool { clientErrorRule.Duration = rule.Duration(clientConfig.DefaultDuration) } if clientConfig.ProcMonitorMethod != "" { - if err := monitor.ReconfigureMonitorMethod(clientConfig.ProcMonitorMethod); err != nil { - msg := fmt.Sprintf("Unable to set new process monitor (%s) method from disk: %v", clientConfig.ProcMonitorMethod, err) + err := monitor.ReconfigureMonitorMethod(clientConfig.ProcMonitorMethod) + if err != nil { + msg := fmt.Sprintf("Unable to set new process monitor (%s) method from disk: %v", clientConfig.ProcMonitorMethod, err.Msg) log.Warning(msg) c.SendWarningAlert(msg) } } + c.rules.EnableChecksums(clientConfig.Rules.EnableChecksums) // TODO: //c.stats.SetLimits(clientConfig.Stats) //loggers.Load(clientConfig.Server.Loggers, clientConfig.Stats.Workers) diff --git a/daemon/ui/notifications.go b/daemon/ui/notifications.go index cf7d34b5cf..200d148340 100644 --- a/daemon/ui/notifications.go +++ b/daemon/ui/notifications.go @@ -61,6 +61,7 @@ func (c *Client) getClientConfig() *protocol.ClientConfig { func (c *Client) monitorProcessDetails(pid int, stream protocol.UI_NotificationsClient, notification *protocol.Notification) { p := procmon.NewProcess(pid, "") + p.GetParent() p.GetInfo() ticker := time.NewTicker(2 * time.Second) @@ -104,7 +105,7 @@ func (c *Client) handleActionChangeConfig(stream protocol.UI_NotificationsClient } if err := monitor.ReconfigureMonitorMethod(newConf.ProcMonitorMethod); err != nil { - c.sendNotificationReply(stream, notification.Id, "", err) + c.sendNotificationReply(stream, notification.Id, "", err.Msg) return } diff --git a/proto/ui.proto b/proto/ui.proto index 7826b1d736..cdada51816 100644 --- a/proto/ui.proto +++ b/proto/ui.proto @@ -114,10 +114,11 @@ message Process { repeated string args = 6; map env = 7; string cwd = 8; - uint64 io_reads = 9; - uint64 io_writes = 10; - uint64 net_reads = 11; - uint64 net_writes = 12; + map checksums = 9; + uint64 io_reads = 10; + uint64 io_writes = 11; + uint64 net_reads = 12; + uint64 net_writes = 13; } message Connection { @@ -133,6 +134,7 @@ message Connection { string process_cwd = 10; repeated string process_args = 11; map process_env = 12; + map process_checksums = 13; } message Operator { diff --git a/ui/opensnitch/dialogs/preferences.py b/ui/opensnitch/dialogs/preferences.py index b07c0d51de..6f4ecc12c9 100644 --- a/ui/opensnitch/dialogs/preferences.py +++ b/ui/opensnitch/dialogs/preferences.py @@ -190,6 +190,7 @@ def showEvent(self, event): self.comboNodeAction.currentIndexChanged.connect(self._cb_node_needs_update) self.checkNodeAuthSkipVerify.clicked.connect(self._cb_node_needs_update) self.comboNodeAuthVerifyType.currentIndexChanged.connect(self._cb_node_needs_update) + self.enableChecksums.clicked.connect(self._cb_node_needs_update) self.comboAuthType.currentIndexChanged.connect(self._cb_combo_auth_type_changed) self.comboNodeAuthType.currentIndexChanged.connect(self._cb_combo_node_auth_type_changed) @@ -371,6 +372,17 @@ def _load_node_settings(self): else: self.comboNodeAddress.setEnabled(False) self.comboNodeLogFile.setEnabled(False) + + + if node_config.get('Rules') != None: + self.enableChecksums.setChecked(node_config['Rules']['EnableChecksums']) + else: + node_config.update({"Rules":{"EnableChecksums":False}}) + self.enableChecksums.setChecked(False) + + + self._node_list[addr]['data'].config = json.dumps(node_config, indent=" ") + except Exception as e: print(self.LOG_TAG + "exception loading config: ", e) @@ -408,6 +420,13 @@ def _load_node_config(self, addr): node_config['Server'] = cfg else: print(addr, " doesn't have Server item") + + if node_config.get('Rules') != None: + node_config['Rules']['EnableChecksums'] = self.enableChecksums.isChecked() + else: + print(addr, "Doesn't have Rules config option") + node_config.update({"Rules":{"EnableChecksums":False}}) + return json.dumps(node_config, indent=" "), None except Exception as e: print(self.LOG_TAG + "exception loading node config on %s: " % addr, e) diff --git a/ui/opensnitch/dialogs/prompt.py b/ui/opensnitch/dialogs/prompt.py index 08c2102a41..9c9fe2ae3c 100644 --- a/ui/opensnitch/dialogs/prompt.py +++ b/ui/opensnitch/dialogs/prompt.py @@ -187,6 +187,9 @@ def _check_advanced_toggled(self, state): self.destIPLabel.setVisible(not state) self.checkDstPort.setVisible(state == True and (self._con != None and self._con.dst_port != 0)) self.checkUserID.setVisible(state) + self.checkSum.setVisible(self._con.process_checksums[Config.OPERAND_PROCESS_HASH_MD5] != "" and state) + self.checksumLabel_2.setVisible(self._con.process_checksums[Config.OPERAND_PROCESS_HASH_MD5] != "" and state) + self.checksumLabel.setVisible(self._con.process_checksums[Config.OPERAND_PROCESS_HASH_MD5] != "" and state) self._ischeckAdvanceded = state self.adjust_size() @@ -329,6 +332,9 @@ def _render_connection(self, con): self._set_app_path(app_name, app_args, con) self._set_app_args(app_name, app_args) + self.checksumLabel.setText(con.process_checksums[Config.OPERAND_PROCESS_HASH_MD5]) + self.checkSum.setChecked(False) + if app_name == "": self.appPathLabel.setVisible(False) self.argsLabel.setVisible(False) @@ -356,6 +362,9 @@ def _render_connection(self, con): else: self.destPortLabel.setText(str(con.dst_port)) self._hide_widget(self.destPortLabel, con.dst_port == 0) + self._hide_widget(self.checkSum, con.process_checksums[Config.OPERAND_PROCESS_HASH_MD5] == "" or not self._ischeckAdvanceded) + self._hide_widget(self.checksumLabel, con.process_checksums[Config.OPERAND_PROCESS_HASH_MD5] == "" or not self._ischeckAdvanceded) + self._hide_widget(self.checksumLabel_2, con.process_checksums[Config.OPERAND_PROCESS_HASH_MD5] == "" or not self._ischeckAdvanceded) self._hide_widget(self.destPortLabel_1, con.dst_port == 0) self._hide_widget(self.checkDstPort, con.dst_port == 0 or not self._ischeckAdvanceded) @@ -586,7 +595,10 @@ def _on_deny_btn_clicked(self, action): self._send_rule() def _is_list_rule(self): - return self.checkUserID.isChecked() or self.checkDstPort.isChecked() or self.checkDstIP.isChecked() + return self.checkUserID.isChecked() or \ + self.checkDstPort.isChecked() or \ + self.checkDstIP.isChecked() or \ + self.checkSum.isChecked() def _get_rule_name(self, rule): rule_temp_name = slugify("%s %s" % (rule.action, rule.duration)) @@ -637,6 +649,11 @@ def _send_rule(self): data.append({"type": Config.RULE_TYPE_SIMPLE, "operand": Config.OPERAND_USER_ID, "data": str(self._con.user_id)}) rule_temp_name = slugify("%s %s" % (rule_temp_name, str(self._con.user_id))) + if self.checkSum.isChecked() and self.checksumLabel.text() != "": + _type, _operand, _data = Config.RULE_TYPE_SIMPLE, Config.OPERAND_PROCESS_HASH_MD5, self.checksumLabel.text() + data.append({"type": _type, "operand": _operand, "data": _data}) + rule_temp_name = slugify("%s %s" % (rule_temp_name, _operand)) + is_list_rule = self._is_list_rule() # If the user has selected to filter by cmdline, but the launched diff --git a/ui/opensnitch/dialogs/ruleseditor.py b/ui/opensnitch/dialogs/ruleseditor.py index 98f631c005..27a828ff0d 100644 --- a/ui/opensnitch/dialogs/ruleseditor.py +++ b/ui/opensnitch/dialogs/ruleseditor.py @@ -255,6 +255,10 @@ def _cb_save_clicked(self): self._set_status_error(QC.translate("rules", "There's already a rule with this name.")) return + if self.md5Check.isChecked() and not self.procCheck.isChecked(): + self._set_status_error(QC.translate("rules", "Process path must be checked in order to verify checksums.")) + return + result, error = self._save_rule() if result == False: self._set_status_error(error) @@ -540,6 +544,13 @@ def _load_rule_operator(self, operator): self.dstListNetsLine.setText(operator.data) self.selectNetsListButton.setEnabled(True) + if operator.operand == Config.OPERAND_PROCESS_HASH_MD5: + self.md5Check.setChecked(True) + self.md5Line.setEnabled(True) + self.md5Line.setText(operator.data) + + + def _load_nodes(self, addr=None): try: self.nodesCombo.clear() @@ -976,6 +987,26 @@ def _save_rule(self): }) self.rule.operator.data = json.dumps(rule_data) + if self.md5Check.isChecked(): + if self.md5Line.text() == "": + return False, QC.translate("rules", "md5 line cannot be empty") + + self.rule.operator.operand = Config.OPERAND_PROCESS_HASH_MD5 + self.rule.operator.data = self.md5Line.text().lower() + rule_data.append( + { + 'type': Config.RULE_TYPE_SIMPLE, + 'operand': Config.OPERAND_PROCESS_HASH_MD5, + 'data': self.md5Line.text().lower(), + "sensitive": False + }) + if self._is_regex(self.md5Line.text()): + rule_data[len(rule_data)-1]['type'] = Config.RULE_TYPE_REGEXP + if self._is_valid_regex(self.pidLine.text()) == False: + return False, QC.translate("rules", "md5 field regexp error") + + + if len(rule_data) >= 2: self.rule.operator.type = Config.RULE_TYPE_LIST self.rule.operator.operand = Config.RULE_TYPE_LIST diff --git a/ui/opensnitch/res/preferences.ui b/ui/opensnitch/res/preferences.ui index a2577e8ce7..26c7b85700 100644 --- a/ui/opensnitch/res/preferences.ui +++ b/ui/opensnitch/res/preferences.ui @@ -36,7 +36,7 @@ - ../../../../../../../../../../.designer/backup../../../../../../../../../../.designer/backup + ../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../.designer/backup true @@ -63,7 +63,7 @@ - ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup + ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup @@ -74,7 +74,7 @@ - ../../../../../../../../../../.designer/backup../../../../../../../../../../.designer/backup + ../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../.designer/backup @@ -85,7 +85,7 @@ - ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup + ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup @@ -105,6 +105,9 @@ 0 + + Qt::ElideRight + @@ -116,23 +119,7 @@ Pop-ups - - - - - 0 - 0 - - - - <html><head/><body><p>This timeout is the countdown you see when a pop-up dialog is shown.</p><p>If the pop-up is not answered, the default options will be applied.</p></body></html> - - - Default timeout - - - - + 0 @@ -150,7 +137,7 @@ - ../../../../../../../../../../.designer/backup../../../../../../../../../../.designer/backup + ../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../.designer/backup true @@ -195,7 +182,7 @@ - ../../../../../../../../../../.designer/backup../../../../../../../../../../.designer/backup + ../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../.designer/backup true @@ -204,357 +191,388 @@ - - + + - + 0 0 + + <html><head/><body><p>This timeout is the countdown you see when a pop-up dialog is shown.</p><p>If the pop-up is not answered, the default options will be applied.</p></body></html> + - + Default timeout - - + + - + 0 0 - - Default options + + + + + + + + + Disable pop-ups, only display a notification + + + true + + + + + + + 0 - - - - - - - - 0 - 0 - + + + + 0 + 0 + 586 + 275 + + + + Default options + + + + + + <html><head/><body><p>Pop-up default action.</p><p>When a new outgoing connection is about to be established, this action will be selected by default, so if the timeout fires, this is the option that will be applied.</p><p>While a pop-up is asking the user to allow or deny a connection:</p><p>1. the daemon's default action will be applied (see Nodes tab).</p><p>2. known connections are allowed or denied based on the rules defined by the user.</p></body></html> + + + Action + + + + + + + + 0 + 0 + + + + + deny - - If checked, this field will be selected when a pop-up is displayed + + + ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup + + - User ID + allow + + + + ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup - - - - - - - 0 - 0 - + + + + reject - - If checked, this field will be selected when a pop-up is displayed + + + ../../../../../../.designer/backup../../../../../../.designer/backup + + + + + + + Pop-up default duration + + + Duration + + + + + + + + 0 + 0 + + + - Destination port + once - - - - - - - 0 - 0 - + + + + 30s - - If checked, this field will be selected when a pop-up is displayed + + + + 5m + + - Destination IP + 15m - - - - - - - - - 0 - 0 - - - - - deny - - - - ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup - - - + + + + 30m + + + + + 1h + + + + + until reboot + + + + + forever + + + + + + - allow - - - - ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup + Default target - - - - reject - - - - .. - - - - - - - - <html><head/><body><p>Pop-up default action.</p><p>When a new outgoing connection is about to be established, this action will be selected by default, so if the timeout fires, this is the option that will be applied.</p><p>While a pop-up is asking the user to allow or deny a connection:</p><p>1. the daemon's default action will be applied (see Nodes tab).</p><p>2. known connections are allowed or denied based on the rules defined by the user.</p></body></html> - - - Action - - - - - - - true - - - - 0 - 0 - - - - - center + + + + + + + 0 + 0 + - - + + + by executable + + + + + by command line + + + + + by destination port + + + + + by destination ip + + + + + by user id + + + + + by PID + + + + + + - top right + Default position on screen - - - - bottom right + + true - - - - top left + + + + + + true - - - - bottom left - - - - - - - - - 0 - 0 - - - - - once + + + 0 + 0 + - - - - 30s + + + center + + + + + top right + + + + + bottom right + + + + + top left + + + + + bottom left + + + + + + + + + + 0 + 0 + 586 + 275 + + + + More + + + + + + + 0 + 0 + - - - - 5m + + <html><head/><body><p>If checked, the pop-ups will be displayed with the advanced view active.</p></body></html> - - - 15m + - - - - 30m + + + + + + + 0 + 0 + - - - - 1h + + The advanced view allows you to easily select multiple fields to filter connections - - - until reboot + Show advanced view by default - - - - forever - - - - - - - - - 0 - 0 - - - - <html><head/><body><p>By default when a new pop-up appears, in its simplest form, you'll be able to filter connections or applications by one property of the connection (executable, port, IP, etc).</p><p>With these options, you can choose multiple fields to filter connections for.</p></body></html> - - - Filter connections also by: - - - true - - - - - - - - 0 - 0 - - - - - by executable + + true - - - - by command line + + + + + + + + + 0 + 0 + + + + If checked, this field will be selected when a pop-up is displayed + + + User ID + + + + + + + + 0 + 0 + + + + If checked, this field will be selected when a pop-up is displayed + + + Destination port + + + + + + + + 0 + 0 + + + + If checked, this field will be selected when a pop-up is displayed + + + Destination IP + + + + + + + + + + 0 + 0 + - - - - by destination port + + <html><head/><body><p>By default when a new pop-up appears, in its simplest form, you'll be able to filter connections or applications by one property of the connection (executable, port, IP, etc).</p><p>With these options, you can choose multiple fields to filter connections for.</p></body></html> - - - by destination ip + Filter connections also by: - - - - by user id + + true - - - - by PID - - - - - - - - Qt::Horizontal - - - - - - - Default target - - - - - - - Default position on screen - - - true - - - - - - - Pop-up default duration - - - Duration - - - - - - - - 0 - 0 - - - - The advanced view allows you to easily select multiple fields to filter connections - - - Show advanced view by default - - - true - - - - - - - - 0 - 0 - - - - <html><head/><body><p>If checked, the pop-ups will be displayed with the advanced view active.</p></body></html> - - - - - - - - - - - - - Disable pop-ups, only display a notification - - - true - + + + + @@ -653,8 +671,8 @@ 0 0 - 586 - 270 + 259 + 157 @@ -758,8 +776,8 @@ 0 0 - 586 - 209 + 321 + 112 @@ -854,8 +872,8 @@ 0 0 - 586 - 209 + 219 + 115 @@ -1099,7 +1117,7 @@ Temporary rules will still be valid, and you can use them when prompted to allow Nodes - + @@ -1190,7 +1208,7 @@ Temporary rules will still be valid, and you can use them when prompted to allow - 0 + 3 @@ -1198,7 +1216,7 @@ Temporary rules will still be valid, and you can use them when prompted to allow 0 0 586 - 229 + 200 @@ -1258,7 +1276,7 @@ Temporary rules will still be valid, and you can use them when prompted to allow - ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup + ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup @@ -1267,7 +1285,7 @@ Temporary rules will still be valid, and you can use them when prompted to allow - ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup + ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup @@ -1347,8 +1365,8 @@ Temporary rules will still be valid, and you can use them when prompted to allow 0 0 - 586 - 229 + 376 + 118 @@ -1479,6 +1497,14 @@ Temporary rules will still be valid, and you can use them when prompted to allow + + + 0 + 0 + 296 + 211 + + Authentication @@ -1602,6 +1628,44 @@ Temporary rules will still be valid, and you can use them when prompted to allow + + + + 0 + 0 + 586 + 200 + + + + Rules + + + + + + Compute and verify binaries checksums when they try to establish new connections + + + Enable checksums verification + + + + + + + Qt::Vertical + + + + 20 + 40 + + + + + + @@ -1689,7 +1753,7 @@ Temporary rules will still be valid, and you can use them when prompted to allow - ../../../../../../../../../../.designer/backup../../../../../../../../../../.designer/backup + ../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../.designer/backup @@ -1759,7 +1823,7 @@ Temporary rules will still be valid, and you can use them when prompted to allow - ../../../../../../../../../../.designer/backup../../../../../../../../../../.designer/backup + ../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../.designer/backup true @@ -1863,7 +1927,7 @@ Temporary rules will still be valid, and you can use them when prompted to allow - ../../../../../../../../../../.designer/backup../../../../../../../../../../.designer/backup + ../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../.designer/backup true @@ -1886,7 +1950,7 @@ Temporary rules will still be valid, and you can use them when prompted to allow - ../../../../../../../../../../.designer/backup../../../../../../../../../../.designer/backup + ../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../.designer/backup true @@ -1909,7 +1973,7 @@ Temporary rules will still be valid, and you can use them when prompted to allow - ../../../../../../../../../../.designer/backup../../../../../../../../../../.designer/backup + ../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../.designer/backup true diff --git a/ui/opensnitch/res/prompt.ui b/ui/opensnitch/res/prompt.ui index d090ae9ad5..87f6b5852f 100644 --- a/ui/opensnitch/res/prompt.ui +++ b/ui/opensnitch/res/prompt.ui @@ -10,7 +10,7 @@ 0 0 520 - 302 + 317 @@ -280,42 +280,8 @@ 3 - - - - - 0 - 0 - - - - - 10 - 75 - true - true - - - - User ID - - - - - - - Qt::Horizontal - - - - 20 - 20 - - - - - - + + 0 @@ -331,7 +297,7 @@ - <html><head/><body><p><span style=" font-weight:600;">Executed from</span></p></body></html> + Destination IP @@ -349,8 +315,8 @@ - - + + 0 @@ -370,7 +336,7 @@ - TextLabel + TextLabel Qt::PlainText @@ -379,12 +345,46 @@ Qt::AlignRight|Qt::AlignTrailing|Qt::AlignVCenter - Qt::TextSelectableByKeyboard|Qt::TextSelectableByMouse + Qt::LinksAccessibleByMouse|Qt::TextSelectableByKeyboard|Qt::TextSelectableByMouse - - + + + + + 0 + 0 + + + + + 10 + 75 + true + true + + + + Dst Port + + + + + + + Qt::Horizontal + + + + 20 + 20 + + + + + + 0 @@ -404,7 +404,7 @@ - TextLabel + TextLabel Qt::PlainText @@ -413,54 +413,80 @@ Qt::AlignRight|Qt::AlignTrailing|Qt::AlignVCenter - Qt::TextSelectableByKeyboard|Qt::TextSelectableByMouse + Qt::LinksAccessibleByMouse|Qt::TextSelectableByKeyboard|Qt::TextSelectableByMouse - - + + - + 0 0 + + + 90 + 0 + + 10 - 75 - true true - Source IP + + + + Qt::PlainText + + + Qt::AlignRight|Qt::AlignTrailing|Qt::AlignVCenter + + + Qt::LinksAccessibleByMouse|Qt::TextSelectableByKeyboard|Qt::TextSelectableByMouse - - + + - + 0 0 + + + 90 + 0 + + 10 - 75 - true true - Process ID + TextLabel + + + Qt::PlainText + + + Qt::AlignRight|Qt::AlignTrailing|Qt::AlignVCenter + + + Qt::LinksAccessibleByMouse|Qt::TextSelectableByKeyboard|Qt::TextSelectableByMouse - - + + 0 @@ -480,7 +506,7 @@ - TextLabel + TextLabel Qt::PlainText @@ -489,41 +515,70 @@ Qt::AlignRight|Qt::AlignTrailing|Qt::AlignVCenter - Qt::TextSelectableByKeyboard|Qt::TextSelectableByMouse + Qt::LinksAccessibleByMouse|Qt::TextSelectableByKeyboard|Qt::TextSelectableByMouse - - + + - + 0 0 - - - 90 - 0 - + + + 10 + 75 + true + true + + + + User ID + + + + + + + + 0 + 0 + 10 + 75 + true true - TextLabel + Process ID - - Qt::PlainText + + + + + + + 0 + 0 + - - Qt::AlignRight|Qt::AlignTrailing|Qt::AlignVCenter + + + 10 + 75 + true + true + - - Qt::TextSelectableByKeyboard|Qt::TextSelectableByMouse + + <html><head/><body><p><span style=" font-weight:600;">Executed from</span></p></body></html> @@ -540,8 +595,15 @@ - - + + + + + + + + + 0 @@ -561,7 +623,7 @@ - + TextLabel Qt::PlainText @@ -570,19 +632,12 @@ Qt::AlignRight|Qt::AlignTrailing|Qt::AlignVCenter - Qt::TextSelectableByKeyboard|Qt::TextSelectableByMouse - - - - - - - + Qt::LinksAccessibleByMouse|Qt::TextSelectableByKeyboard|Qt::TextSelectableByMouse - - + + 0 @@ -598,18 +653,12 @@ - Destination IP + Source IP - - - - - 0 - 0 - - + + 10 @@ -619,24 +668,12 @@ - Dst Port + Checksum - - - - 0 - 0 - - - - - 90 - 0 - - + 10 @@ -644,16 +681,20 @@ - TextLabel - - - Qt::PlainText + TextLabel Qt::AlignRight|Qt::AlignTrailing|Qt::AlignVCenter - Qt::TextSelectableByKeyboard|Qt::TextSelectableByMouse + Qt::LinksAccessibleByMouse|Qt::TextSelectableByKeyboard|Qt::TextSelectableByMouse + + + + + + + @@ -809,7 +850,8 @@ action - + + ../../../../../../.designer/backup../../../../../../.designer/backup QToolButton::MenuButtonPopup @@ -837,7 +879,8 @@ Allow - + + ../../../../../../.designer/backup../../../../../../.designer/backup diff --git a/ui/opensnitch/res/ruleseditor.ui b/ui/opensnitch/res/ruleseditor.ui index 310037674e..0dc4a9e42e 100644 --- a/ui/opensnitch/res/ruleseditor.ui +++ b/ui/opensnitch/res/ruleseditor.ui @@ -7,7 +7,7 @@ 0 0 552 - 559 + 580 @@ -20,248 +20,6 @@ Rule - - - - - 0 - 0 - - - - false - - - false - - - false - - - - QLayout::SetDefaultConstraint - - - 4 - - - 4 - - - 12 - - - - - Action - - - - - - - Qt::Horizontal - - - - 40 - 20 - - - - - - - - Qt::Horizontal - - - - 40 - 20 - - - - - - - - Duration - - - - - - - - once - - - - - 30s - - - - - 5m - - - - - 15m - - - - - 30m - - - - - 1h - - - - - until reboot - - - - - always - - - - - - - - - - - 0 - 0 - - - - Deny will just discard the connection - - - Deny - - - - ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup - - - true - - - - - - - Reject will drop the connection, and kill the socket that initiated it - - - Reject - - - - .. - - - - - - - - 0 - 0 - - - - Allow will allow the connection - - - Qt::LeftToRight - - - Allow - - - - ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup - - - - - - - - - - - - Enable - - - - - - - If checked, this rule will take precedence over the rest of the rules. No others rules will be checked after this one. - -You must name the rule in such manner that it'll be checked first, because they're checked in alphabetical order. For example: - -[x] Priority - 000-priority-rule -[ ] Priority - 001-less-priority-rule - - - Priority rule - - - - - - - true - - - - 0 - 0 - - - - The rules are checked in alphabetical order, so you can name them accordingly to prioritize them. - -000-allow-localhost -001-deny-broadcast -... - - - Name - - - true - - - - - - - true - - - - - - true - - - @@ -292,39 +50,31 @@ You must name the rule in such manner that it'll be checked first, because they' - - - - - - Node - - - - - - - - - - Qt::Horizontal - - - - 40 - 20 - - - - - - - - Apply rule to all nodes - - - - + + + + Qt::Horizontal + + + + + + + + 0 + 0 + + + + + 16777215 + 60 + + + + Description... + + @@ -340,7 +90,7 @@ You must name the rule in such manner that it'll be checked first, because they' - .. + ../../../../../../.designer/backup../../../../../../.designer/backup Applications @@ -440,6 +190,12 @@ You must name the rule in such manner that it'll be checked first, because they' QSizePolicy::MinimumExpanding + + + 0 + 0 + + @@ -479,7 +235,7 @@ You must name the rule in such manner that it'll be checked first, because they' - .. + ../../../../../../.designer/backup../../../../../../.designer/backup Network @@ -656,77 +412,77 @@ Note: Commas or spaces are not allowed to separate IPs or networks. - LAN + LAN - MULTICAST + MULTICAST - 127.0.0.0/8 + 127.0.0.0/8 - 192.168.0.0/24 + 192.168.0.0/24 - 192.168.1.0/24 + 192.168.1.0/24 - 192.168.2.0/24 + 192.168.2.0/24 - 192.168.0.0/16 + 192.168.0.0/16 - 169.254.0.0/16 + 169.254.0.0/16 - 172.16.0.0/12 + 172.16.0.0/12 - 10.0.0.0/8 + 10.0.0.0/8 - ::1/128 + ::1/128 - fc00::/7 + fc00::/7 - ff00::/8 + ff00::/8 - fe80::/10 + fe80::/10 - fd00::/8 + fd00::/8 @@ -920,7 +676,7 @@ Note: Commas or spaces are not allowed to separate IPs or networks. - .. + ../../../../../../.designer/backup../../../../../../.designer/backup List of domains/IPs @@ -952,7 +708,7 @@ Note: Commas or spaces are not allowed to separate IPs or networks. - .. + ../../../../../../.designer/backup../../../../../../.designer/backup @@ -987,7 +743,7 @@ Note: Commas or spaces are not allowed to separate IPs or networks. - .. + ../../../../../../.designer/backup../../../../../../.designer/backup @@ -1015,7 +771,7 @@ Note: Commas or spaces are not allowed to separate IPs or networks. - .. + ../../../../../../.designer/backup../../../../../../.designer/backup @@ -1057,7 +813,7 @@ Note: Commas or spaces are not allowed to separate IPs or networks. - .. + ../../../../../../.designer/backup../../../../../../.designer/backup @@ -1080,6 +836,31 @@ Note: Commas or spaces are not allowed to separate IPs or networks. More + + + + Qt::Horizontal + + + + + + + + + md5 + + + + + + + false + + + + + @@ -1090,17 +871,7 @@ Note: Commas or spaces are not allowed to separate IPs or networks. - - - - Don't log connections that match this rule - - - Don't log connections - - - - + Qt::Vertical @@ -1116,36 +887,343 @@ Note: Commas or spaces are not allowed to separate IPs or networks. + + + + Don't log connections that match this rule + + + Don't log connections + + + + + + + QLayout::SetMaximumSize + + + + + + 0 + 0 + + + + These options are experimental / in development, they may have bugs or not be completely finished. +Feedback is welcome + + + + + + + ../../../../../../.designer/backup../../../../../../.designer/backup + + + true + + + + + + + + 0 + 0 + + + + In development + + + Qt::AlignLeading|Qt::AlignLeft|Qt::AlignTop + + + + + - - - - Qt::Horizontal + + + + If checked, this rule will take precedence over the rest of the rules. No others rules will be checked after this one. + +You must name the rule in such manner that it'll be checked first, because they're checked in alphabetical order. For example: + +[x] Priority - 000-priority-rule +[ ] Priority - 001-less-priority-rule + + + Priority rule - - + + + + true + - + 0 0 - - - 16777215 - 60 - + + The rules are checked in alphabetical order, so you can name them accordingly to prioritize them. + +000-allow-localhost +001-deny-broadcast +... - Description... + Name + + + true + + + + true + + + + + + true + + + + + + + + 0 + 0 + + + + false + + + false + + + false + + + + QLayout::SetDefaultConstraint + + + 4 + + + 4 + + + 12 + + + + + Action + + + + + + + Qt::Horizontal + + + + 40 + 20 + + + + + + + + Qt::Horizontal + + + + 40 + 20 + + + + + + + + Duration + + + + + + + + once + + + + + 30s + + + + + 5m + + + + + 15m + + + + + 30m + + + + + 1h + + + + + until reboot + + + + + always + + + + + + + + + + + 0 + 0 + + + + Deny will just discard the connection + + + Deny + + + + ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup + + + true + + + + + + + Reject will drop the connection, and kill the socket that initiated it + + + Reject + + + + ../../../../../../.designer/backup../../../../../../.designer/backup + + + + + + + + 0 + 0 + + + + Allow will allow the connection + + + Qt::LeftToRight + + + Allow + + + + ../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../../.designer/backup + + + + + + + + + + + + Enable + + + + + + + + + Node + + + + + + + + + + Qt::Horizontal + + + + 40 + 20 + + + + + + + + Apply rule to all nodes + + + + +