Skip to content

Commit

Permalink
allow to filter connections by process checksum
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
gustavo-iniguez-goya committed Sep 21, 2023
1 parent 79ac630 commit 7a9bb17
Show file tree
Hide file tree
Showing 29 changed files with 2,180 additions and 1,221 deletions.
45 changes: 25 additions & 20 deletions daemon/conman/connection.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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()
Expand All @@ -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,
Expand Down Expand Up @@ -185,6 +188,7 @@ func NewConnection(nfp *netfilter.Packet) (c *Connection, err error) {
DstHost: dns.HostOr(ip.DstIP, ""),
Pkt: nfp,
}

return newConnectionImpl(nfp, c, "")
}

Expand Down Expand Up @@ -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,
}
}
3 changes: 3 additions & 0 deletions daemon/default-config.json
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,9 @@
"LogUTC": true,
"LogMicro": false,
"Firewall": "nftables",
"Rules": {
"EnableChecksums": false
},
"Stats": {
"MaxEvents": 150,
"MaxStats": 25,
Expand Down
149 changes: 149 additions & 0 deletions daemon/netlink/procmon/procmon.go
Original file line number Diff line number Diff line change
@@ -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
}
103 changes: 38 additions & 65 deletions daemon/procmon/activepids.go
Original file line number Diff line number Diff line change
@@ -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 {
Expand All @@ -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")
}
Loading

0 comments on commit 7a9bb17

Please sign in to comment.