diff --git a/cmd/fibratus/app/rules/validate.go b/cmd/fibratus/app/rules/validate.go index 388b64297..9ab621d04 100644 --- a/cmd/fibratus/app/rules/validate.go +++ b/cmd/fibratus/app/rules/validate.go @@ -24,6 +24,7 @@ import ( "github.com/rabbitstack/fibratus/internal/bootstrap" "github.com/rabbitstack/fibratus/pkg/filter" "github.com/rabbitstack/fibratus/pkg/filter/fields" + "github.com/rabbitstack/fibratus/pkg/rules" "path/filepath" "strings" ) @@ -91,7 +92,7 @@ func validateRules() error { f := filter.New(rule.Condition, cfg) err := f.Compile() if err != nil { - return fmt.Errorf("%v %v", emoji.DisappointedFace, filter.ErrInvalidFilter(rule.Name, err)) + return fmt.Errorf("%v %v", emoji.DisappointedFace, rules.ErrInvalidFilter(rule.Name, err)) } w := warning{rule: rule.Name} diff --git a/configs/fibratus.yml b/configs/fibratus.yml index cb8223c9f..aca889811 100644 --- a/configs/fibratus.yml +++ b/configs/fibratus.yml @@ -131,6 +131,10 @@ filament: # For local file system rule paths, it is possible to use the glob expression to load the # rules from different directory locations. filters: + # Indicates if the rule engine match all strategy is enabled. When the match all strategy + # is enabled, a single event can trigger multiple rules. + match-all: true + rules: # Indicates if the rule engine is enabled and rules loaded enabled: true diff --git a/internal/bootstrap/bootstrap.go b/internal/bootstrap/bootstrap.go index d00bd5b64..6c9c67e5a 100644 --- a/internal/bootstrap/bootstrap.go +++ b/internal/bootstrap/bootstrap.go @@ -30,6 +30,7 @@ import ( "github.com/rabbitstack/fibratus/pkg/handle" "github.com/rabbitstack/fibratus/pkg/kcap" "github.com/rabbitstack/fibratus/pkg/ps" + "github.com/rabbitstack/fibratus/pkg/rules" "github.com/rabbitstack/fibratus/pkg/symbolize" "github.com/rabbitstack/fibratus/pkg/sys" "github.com/rabbitstack/fibratus/pkg/util/multierror" @@ -52,7 +53,7 @@ type App struct { config *config.Config evs *EventSourceControl symbolizer *symbolize.Symbolizer - rules *filter.Rules + engine *rules.Engine hsnap handle.Snapshotter psnap ps.Snapshotter filament filament.Filament @@ -134,34 +135,34 @@ func NewApp(cfg *config.Config, options ...Option) (*App, error) { hsnap := handle.NewSnapshotter(cfg, opts.handleSnapshotFn) psnap := ps.NewSnapshotter(hsnap, cfg) - var ( - rules *filter.Rules - res *config.RulesCompileResult - ) + var engine *rules.Engine + var rs *config.RulesCompileResult + if cfg.Filters.Rules.Enabled && !cfg.ForwardMode && !cfg.IsCaptureSet() { - rules = filter.NewRules(psnap, cfg) + engine = rules.NewEngine(psnap, cfg) var err error - res, err = rules.Compile() + rs, err = engine.Compile() if err != nil { return nil, err } - if res != nil { - log.Infof("rules compile summary: %s", res) + if rs != nil { + log.Infof("rules compile summary: %s", rs) } } else { log.Info("rule engine is disabled") } - evs := NewEventSourceControl(psnap, hsnap, cfg, res) + evs := NewEventSourceControl(psnap, hsnap, cfg, rs) app := &App{ config: cfg, evs: evs, - rules: rules, + engine: engine, hsnap: hsnap, psnap: psnap, signals: sigs, } + return app, nil } @@ -234,8 +235,8 @@ func (f *App) Run(args []string) error { f.evs.RegisterEventListener(f.symbolizer) } // register rule engine - if f.rules != nil { - f.evs.RegisterEventListener(f.rules) + if f.engine != nil { + f.evs.RegisterEventListener(f.engine) } // register YARA scanner if cfg.Yara.Enabled { diff --git a/pkg/config/config_windows.go b/pkg/config/config_windows.go index b960f9a76..2829bb2c8 100644 --- a/pkg/config/config_windows.go +++ b/pkg/config/config_windows.go @@ -380,6 +380,7 @@ func (c *Config) addFlags() { c.flags.StringSlice(rulesFromPaths, []string{filepath.Join(dir, "*")}, "Comma-separated list of rules files") c.flags.StringSlice(macrosFromPaths, []string{filepath.Join(dir, "Macros", "*")}, "Comma-separated list of macro files") c.flags.StringSlice(rulesFromURLs, []string{}, "Comma-separated list of rules URL resources") + c.flags.Bool(matchAll, true, "Indicates if the match all strategy is enabled for the rule engine. If the match all strategy is enabled, a single event can trigger multiple rules") } if c.opts.capture { c.flags.StringP(kcapFile, "o", "", "The path of the output kcap file") diff --git a/pkg/config/filters.go b/pkg/config/filters.go index 0a4434dcc..caabea5be 100644 --- a/pkg/config/filters.go +++ b/pkg/config/filters.go @@ -102,10 +102,13 @@ func (f FilterConfig) HasLabel(l string) bool { return f.Labels[l] != "" } // Filters contains references to rule and macro definitions. type Filters struct { - Rules Rules `json:"rules" yaml:"rules"` - Macros Macros `json:"macros" yaml:"macros"` - macros map[string]*Macro - filters []*FilterConfig + Rules Rules `json:"rules" yaml:"rules"` + Macros Macros `json:"macros" yaml:"macros"` + // MatchAll indicates if the match all strategy is enabled for the rule engine. + // If the match all strategy is enabled, a single event can trigger multiple rules. + MatchAll bool `json:"match-all" yaml:"match-all"` + macros map[string]*Macro + filters []*FilterConfig } // FiltersWithMacros builds the filter config with the map of @@ -241,6 +244,7 @@ const ( rulesFromPaths = "filters.rules.from-paths" rulesFromURLs = "filters.rules.from-urls" macrosFromPaths = "filters.macros.from-paths" + matchAll = "filters.match-all" ) func (f *Filters) initFromViper(v *viper.Viper) { @@ -248,6 +252,7 @@ func (f *Filters) initFromViper(v *viper.Viper) { f.Rules.FromPaths = v.GetStringSlice(rulesFromPaths) f.Rules.FromURLs = v.GetStringSlice(rulesFromURLs) f.Macros.FromPaths = v.GetStringSlice(macrosFromPaths) + f.MatchAll = v.GetBool(matchAll) } func (f Filters) HasMacros() bool { return len(f.macros) > 0 } diff --git a/pkg/config/filters_test.go b/pkg/config/filters_test.go index b26f0a9a1..16de977c4 100644 --- a/pkg/config/filters_test.go +++ b/pkg/config/filters_test.go @@ -37,6 +37,7 @@ func TestLoadRulesFromPaths(t *testing.T) { }, }, Macros{FromPaths: nil}, + false, map[string]*Macro{}, []*FilterConfig{}, } @@ -78,6 +79,7 @@ func TestLoadRulesFromPathsWithTemplate(t *testing.T) { }, }, Macros{FromPaths: nil}, + false, map[string]*Macro{}, []*FilterConfig{}, } @@ -116,6 +118,7 @@ func TestLoadGroupsFromURLs(t *testing.T) { }, }, Macros{FromPaths: nil}, + false, map[string]*Macro{}, []*FilterConfig{}, } diff --git a/pkg/config/schema_windows.go b/pkg/config/schema_windows.go index 347e393d2..d91d45d0c 100644 --- a/pkg/config/schema_windows.go +++ b/pkg/config/schema_windows.go @@ -146,6 +146,7 @@ var schema = ` "filters": { "type": "object", "properties": { + "match-all": {"type": "boolean"}, "rules": { "type": "object", "properties": { @@ -510,7 +511,7 @@ var rulesSchema = ` "id": {"type": "string", "minLength": 36, "pattern": "^[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}$"}, "version": {"type": "string", "minLength": 5, "pattern": "^([0-9]+.)([0-9]+.)([0-9]+)$"}, "name": {"type": "string", "minLength": 3}, - "description": {"type": "string"}, + "description": {"type": "string"}, "output": {"type": "string", "minLength": 5}, "notes": {"type": "string"}, "severity": {"type": "string", "enum": ["low", "medium", "high", "critical"]}, diff --git a/pkg/filter/_fixtures/sequence_gc.yml b/pkg/filter/_fixtures/sequence_gc.yml deleted file mode 100644 index 552b2c25e..000000000 --- a/pkg/filter/_fixtures/sequence_gc.yml +++ /dev/null @@ -1,16 +0,0 @@ -name: Out of order sequence -id: 272902be-76e9-4ee7-a48a-6275fa571cf4 -version: 1.0.0 -condition: > - sequence - by ps.uuid - |kevt.name = 'OpenProcess'| - |kevt.name = 'CreateFile' and file.operation = 'CREATE' - and - file.extension = '.dmp' - | -output: > - Detected an attempt by `%1.ps.name` process to access and read - the memory of the **Local Security And Authority Subsystem Service** - and subsequently write the `%2.file.name` dump file to the disk device -min-engine-version: 2.0.0 diff --git a/pkg/filter/_fixtures/sequence_rule_bound_fields.yml b/pkg/filter/_fixtures/sequence_rule_bound_fields.yml deleted file mode 100644 index 84dd55132..000000000 --- a/pkg/filter/_fixtures/sequence_rule_bound_fields.yml +++ /dev/null @@ -1,15 +0,0 @@ -name: Command shell created a temp file with network outbound -id: 372902be-76e9-4ee7-a48a-6275fa571cf4 -version: 1.0.0 -condition: > - sequence - maxspan 200ms - |kevt.name = 'CreateProcess' and ps.name = 'cmd.exe'| as e1 - |kevt.name = 'CreateFile' - and - file.path icontains 'temp' - and - $e1.ps.sid = ps.sid - | as e2 - |kevt.name = 'Connect' and ps.sid != $e2.ps.sid and ps.sid = $e1.ps.sid| -min-engine-version: 2.0.0 diff --git a/pkg/filter/_fixtures/sequence_rule_bound_fields_with_functions.yml b/pkg/filter/_fixtures/sequence_rule_bound_fields_with_functions.yml deleted file mode 100644 index bc9bac25f..000000000 --- a/pkg/filter/_fixtures/sequence_rule_bound_fields_with_functions.yml +++ /dev/null @@ -1,26 +0,0 @@ -name: Suspicious password filter DLL registered -id: 472902be-76e9-4ee7-a48a-6275fa571cf4 -version: 1.0.0 -description: | - Detects attempts to register malicious password filters to harvest credentials - from local computers and/or entire domains. To perform proper validation, - filters must receive plain-text credentials from the LSA. A malicious password - filter would receive these plain-text credentials every time a password request is made. -condition: > - sequence - maxspan 5m - |kevt.name = 'CreateFile' - and - file.path imatches '?:\\Windows\\System32\\*.dll' - | as e1 - |kevt.name = 'RegSetValue' - and - registry.path ~= 'HKEY_CURRENT_USER\\Volatile Environment\\Notification Packages' - and - get_reg_value(registry.path) iin (base($e1.file.path, false)) - | -output: > - %1.ps.exe process dropped potentially malicious - %1.file.path password filter and %2.ps.name - registered the password filter DLL under %2.registry.path registry key -min-engine-version: 2.0.0 diff --git a/pkg/filter/_fixtures/sequence_rule_expire.yml b/pkg/filter/_fixtures/sequence_rule_expire.yml deleted file mode 100644 index 4fc3476f3..000000000 --- a/pkg/filter/_fixtures/sequence_rule_expire.yml +++ /dev/null @@ -1,12 +0,0 @@ -name: Command shell created a temp file -id: 672902be-76e9-4ee7-a48a-6275fa571cf4 -version: 1.0.0 -condition: > - sequence - maxspan 1m - |kevt.name = 'OpenProcess' and ps.name = 'cmd.exe'| by ps.exe - |kevt.name = 'CreateFile' - and - file.path icontains 'temp' - | by file.path -min-engine-version: 2.0.0 diff --git a/pkg/filter/_fixtures/sequence_rule_out_of_order.yml b/pkg/filter/_fixtures/sequence_rule_out_of_order.yml deleted file mode 100644 index 8d9f588b4..000000000 --- a/pkg/filter/_fixtures/sequence_rule_out_of_order.yml +++ /dev/null @@ -1,17 +0,0 @@ -name: Out of order sequence -id: 772902be-76e9-4ee7-a48a-6275fa571cf4 -version: 1.0.0 -condition: > - sequence - maxspan 2m - by ps.uuid - |kevt.name = 'OpenProcess'| - |kevt.name = 'CreateFile' and file.operation = 'CREATE' - and - file.extension = '.dmp' - | -output: > - Detected an attempt by `%1.ps.name` process to access and read - the memory of the **Local Security And Authority Subsystem Service** - and subsequently write the `%2.file.path` dump file to the disk device -min-engine-version: 2.0.0 diff --git a/pkg/filter/_fixtures/sequence_rule_simple_max_span.yml b/pkg/filter/_fixtures/sequence_rule_simple_max_span.yml deleted file mode 100644 index 701c94c34..000000000 --- a/pkg/filter/_fixtures/sequence_rule_simple_max_span.yml +++ /dev/null @@ -1,13 +0,0 @@ -name: Command shell created a temp file -id: 40ffc2a8-0bde-45c4-9e20-46158250fa91 -version: 1.0.0 -condition: > - sequence - maxspan 200ms - by ps.pid - |kevt.name = 'CreateProcess' and ps.name = 'cmd.exe'| - |kevt.name = 'CreateFile' - and - file.path icontains 'temp' - | -min-engine-version: 2.0.0 diff --git a/pkg/filter/filter.go b/pkg/filter/filter.go index bfe15e639..05e301d0f 100644 --- a/pkg/filter/filter.go +++ b/pkg/filter/filter.go @@ -53,7 +53,7 @@ type Filter interface { // RunSequence runs a filter with sequence expressions. Sequence rules depend // on the state machine transitions and partial matches to decide whether the // rule is fired. - RunSequence(evt *kevent.Kevent, seqID uint16, partials map[uint16][]*kevent.Kevent, rawMatch bool) bool + RunSequence(evt *kevent.Kevent, seqID int, partials map[int][]*kevent.Kevent, rawMatch bool) bool // GetStringFields returns field names mapped to their string values. GetStringFields() map[fields.Field][]string // GetFields returns all fields used in the filter expression. @@ -87,7 +87,7 @@ type filter struct { segments []fields.Segment boundFields []*ql.BoundFieldLiteral // seqBoundFields contains per-sequence bound fields resolved from bound field literals - seqBoundFields map[uint16][]BoundField + seqBoundFields map[int][]BoundField // stringFields contains filter field names mapped to their string values stringFields map[fields.Field][]string hasFunctions bool @@ -182,22 +182,22 @@ func (f *filter) Compile() error { return f.checkBoundRefs() } -func (f *filter) Run(kevt *kevent.Kevent) bool { +func (f *filter) Run(e *kevent.Kevent) bool { if f.expr == nil { return false } - return ql.Eval(f.expr, f.mapValuer(kevt), f.hasFunctions) + return ql.Eval(f.expr, f.mapValuer(e), f.hasFunctions) } -func (f *filter) RunSequence(kevt *kevent.Kevent, seqID uint16, partials map[uint16][]*kevent.Kevent, rawMatch bool) bool { +func (f *filter) RunSequence(e *kevent.Kevent, seqID int, partials map[int][]*kevent.Kevent, rawMatch bool) bool { if f.seq == nil { return false } - nseqs := uint16(len(f.seq.Expressions)) + nseqs := len(f.seq.Expressions) if seqID > nseqs-1 { return false } - valuer := f.mapValuer(kevt) + valuer := f.mapValuer(e) expr := f.seq.Expressions[seqID] if rawMatch { @@ -213,12 +213,12 @@ func (f *filter) RunSequence(kevt *kevent.Kevent, seqID uint16, partials map[uin // aliases p := make(map[string][]*kevent.Kevent) nslots := len(partials[seqID]) - for i := uint16(0); i < seqID; i++ { + for i := 0; i < seqID; i++ { alias := f.seq.Expressions[i].Alias if alias == "" { continue } - p[alias] = partials[i+1] + p[alias] = partials[i] if len(p[alias]) > nslots { nslots = len(p[alias]) } @@ -292,8 +292,8 @@ func (f *filter) RunSequence(kevt *kevent.Kevent, seqID uint16, partials map[uin match = ql.Eval(expr.Expr, valuer, f.hasFunctions) if match { // compute sequence key hash to tie the events - evt.AddMeta(kevent.RuleSequenceByKey, hashers.FnvUint64(hash)) - kevt.AddMeta(kevent.RuleSequenceByKey, hashers.FnvUint64(hash)) + evt.AddMeta(kevent.RuleSequenceLink, hashers.FnvUint64(hash)) + e.AddMeta(kevent.RuleSequenceLink, hashers.FnvUint64(hash)) break } } @@ -308,9 +308,9 @@ func (f *filter) RunSequence(kevt *kevent.Kevent, seqID uint16, partials map[uin joins := make([]bool, seqID) joinID := valuer[by.Value] outer: - for i := uint16(0); i < seqID; i++ { - for _, p := range partials[i+1] { - if compareSeqJoin(joinID, p.SequenceBy()) { + for i := 0; i < seqID; i++ { + for _, p := range partials[i] { + if CompareSeqLink(joinID, p.SequenceLink()) { joins[i] = true continue outer } @@ -323,7 +323,7 @@ func (f *filter) RunSequence(kevt *kevent.Kevent, seqID uint16, partials map[uin if match && by != nil { if v := valuer[by.Value]; v != nil { - kevt.AddMeta(kevent.RuleSequenceByKey, v) + e.AddMeta(kevent.RuleSequenceLink, v) } } } @@ -456,7 +456,7 @@ func (f *filter) addSegment(segment *ql.BoundSegmentLiteral) { // addSeqBoundFields receives the sequence id and the list of bound field literals // and populates the list of bound fields containing the field structure convenient // for accessors. -func (f *filter) addSeqBoundFields(seqID uint16, fields []*ql.BoundFieldLiteral) []BoundField { +func (f *filter) addSeqBoundFields(seqID int, fields []*ql.BoundFieldLiteral) []BoundField { flds := make([]BoundField, 0, len(fields)) for _, field := range fields { flds = append(flds, @@ -497,9 +497,9 @@ func (f *filter) checkBoundRefs() error { return nil } -// compareSeqJoin returns true if both values +// CompareSeqLink returns true if both values // representing the sequence joins are equal. -func compareSeqJoin(s1, s2 any) bool { +func CompareSeqLink(s1, s2 any) bool { if s1 == nil || s2 == nil { return false } diff --git a/pkg/filter/filter_windows.go b/pkg/filter/filter_windows.go index 239e1d395..dae664161 100644 --- a/pkg/filter/filter_windows.go +++ b/pkg/filter/filter_windows.go @@ -99,7 +99,7 @@ func New(expr string, config *config.Config, options ...Option) Filter { segments: make([]fields.Segment, 0), stringFields: make(map[fields.Field][]string), boundFields: make([]*ql.BoundFieldLiteral, 0), - seqBoundFields: make(map[uint16][]BoundField), + seqBoundFields: make(map[int][]BoundField), } } @@ -129,7 +129,7 @@ func NewFromCLIWithAllAccessors(args []string) (Filter, error) { segments: make([]fields.Segment, 0), stringFields: make(map[fields.Field][]string), boundFields: make([]*ql.BoundFieldLiteral, 0), - seqBoundFields: make(map[uint16][]BoundField), + seqBoundFields: make(map[int][]BoundField), } if err := filter.Compile(); err != nil { return nil, fmt.Errorf("bad filter:\n %v", err) diff --git a/pkg/filter/rules.go b/pkg/filter/rules.go deleted file mode 100644 index e15a572e4..000000000 --- a/pkg/filter/rules.go +++ /dev/null @@ -1,922 +0,0 @@ -/* - * Copyright 2020-2021 by Nedim Sabic Sabic - * https://www.fibratus.io - * All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package filter - -import ( - "context" - "expvar" - "fmt" - fsm "github.com/qmuntal/stateless" - "github.com/rabbitstack/fibratus/pkg/filter/action" - "github.com/rabbitstack/fibratus/pkg/filter/fields" - "github.com/rabbitstack/fibratus/pkg/kevent/kparams" - "github.com/rabbitstack/fibratus/pkg/kevent/ktypes" - "github.com/rabbitstack/fibratus/pkg/ps" - "github.com/rabbitstack/fibratus/pkg/util/atomic" - "github.com/rabbitstack/fibratus/pkg/util/hashers" - "github.com/rabbitstack/fibratus/pkg/util/version" - "sort" - "sync" - "time" - - semver "github.com/hashicorp/go-version" - "github.com/rabbitstack/fibratus/pkg/config" - "github.com/rabbitstack/fibratus/pkg/kevent" - log "github.com/sirupsen/logrus" -) - -const ( - // maxOutstandingPartials determines the maximum number of partials per sequence index - maxOutstandingPartials = 1000 -) - -var ( - filterMatches = expvar.NewMap("filter.matches") - filtersCount = expvar.NewInt("filter.filters.count") - - matchTransitionErrors = expvar.NewInt("sequence.match.transition.errors") - partialsPerSequence = expvar.NewMap("sequence.partials.count") - partialExpirations = expvar.NewMap("sequence.partial.expirations") - partialBreaches = expvar.NewMap("sequence.partial.breaches") - - ErrInvalidFilter = func(rule string, err error) error { - return fmt.Errorf("syntax error in rule %q: \n%v", rule, err) - } - ErrRuleAction = func(rule string, err error) error { - return fmt.Errorf("fail to execute action for %q rule: %v", rule, err) - } - ErrIncompatibleFilter = func(rule, v string) error { - return fmt.Errorf("rule %q needs engine version [%s] but current version is [%s]", rule, v, version.Get()) - } - ErrMalformedMinEngineVer = func(rule, v string, err error) error { - return fmt.Errorf("rule %q has a malformed minimum engine version: %s: %v", rule, v, err) - } - - // sequenceGcInterval determines how often sequence GC kicks in - sequenceGcInterval = time.Minute - // maxSequencePartialLifetime indicates the maximum time for the - // partial to exist in the sequence state. If the partial has been - // placed in the sequence state more than allowed, it is removed - maxSequencePartialLifetime = time.Hour * 4 -) - -var ( - // sequenceTerminalState represents the final state in the FSM. - // This state is transitioned when the last rule in the sequence - // produces a match - sequenceTerminalState = fsm.State("terminal") - // sequenceDeadlineState represents the state to which other - // states transition if the rule's max span is reached - sequenceDeadlineState = fsm.State("deadline") - // sequenceExpiredState designates the state to which other - // states transition when the sequence is expired - sequenceExpiredState = fsm.State("expired") - - // transitions for match, cancel, reset, and expire triggers - matchTransition = fsm.Trigger("match") - cancelTransition = fsm.Trigger("cancel") - resetTransition = fsm.Trigger("reset") - expireTransition = fsm.Trigger("expire") -) - -// Rules stores the compiled filter exprs -// and for each incoming event, it applies -// the corresponding filter to the event. -// If the filter rule has an action, the -// former is executed when the rule fires. -type Rules struct { - filters map[uint32][]*compiledFilter - config *config.Config - psnap ps.Snapshotter - - matches []*ruleMatch - sequences []*sequenceState - - scavenger *time.Ticker -} - -type ruleMatch struct { - ctx *config.ActionContext -} - -type compiledFilter struct { - filter Filter - ss *sequenceState - config *config.FilterConfig -} - -// sequenceState represents the state of the -// ordered sequence of multiple events that -// may have time-frame constraints. A deterministic -// finite state machine tracks the matching status of -// each rule (state) in the machine. -type sequenceState struct { - name string - maxSpan time.Duration - - // partials keeps the state of all matched events per expression - partials map[uint16][]*kevent.Kevent - // mu guards the partials map - mu sync.RWMutex - - // matches stores only the event that matched - // the upstream partials. These events will be propagated - // in the rule action context - matches map[uint16]*kevent.Kevent - // mmu guards the matches map - mmu sync.RWMutex - - fsm *fsm.StateMachine - - // rule to rule index mapping. Indices start at 1 - idxs map[fsm.State]uint16 - spanDeadlines map[fsm.State]*time.Timer - inDeadline atomic.Bool - inExpired atomic.Bool - initialState fsm.State - - // matchedRules keeps the mapping between rule indexes and - // their matches. Indices start at 1 - matchedRules map[uint16]bool - // mrm guards the matchedRules map - mrm sync.RWMutex -} - -func newSequenceState(name, initialState string, maxSpan time.Duration) *sequenceState { - ss := &sequenceState{ - name: name, - maxSpan: maxSpan, - partials: make(map[uint16][]*kevent.Kevent), - matchedRules: make(map[uint16]bool), - matches: make(map[uint16]*kevent.Kevent), - idxs: make(map[fsm.State]uint16), - spanDeadlines: make(map[fsm.State]*time.Timer), - initialState: fsm.State(initialState), - inDeadline: atomic.MakeBool(false), - } - - ss.initFSM(initialState) - - return ss -} - -func (s *sequenceState) events() []*kevent.Kevent { - s.mmu.RLock() - defer s.mmu.RUnlock() - events := make([]*kevent.Kevent, 0, len(s.matches)) - for _, e := range s.matches { - events = append(events, e) - } - sort.Slice(events, func(i, j int) bool { return events[i].Timestamp.Before(events[j].Timestamp) }) - return events -} - -func (s *sequenceState) isStateSchedulable(state fsm.State) bool { - return state != s.initialState && state != sequenceTerminalState && state != sequenceExpiredState && state != sequenceDeadlineState -} - -func (s *sequenceState) initFSM(initialState string) { - s.fsm = fsm.NewStateMachine(initialState) - s.fsm.OnTransitioned(func(ctx context.Context, transition fsm.Transition) { - // schedule span deadline for the current state unless initial/meta states - if s.maxSpan != 0 && s.isStateSchedulable(s.currentState()) { - log.Debugf("scheduling max span deadline of %v for rule %s", s.maxSpan, s.currentState()) - s.scheduleMaxSpanDeadline(s.currentState(), s.maxSpan) - } - // if the sequence was deadlined/expired, we can disable the deadline - // status when the first rule in the sequence is reevaluated - if transition.Source == s.initialState && s.inDeadline.Load() { - s.inDeadline.Store(false) - } - if transition.Source == s.initialState && s.inExpired.Load() { - s.inExpired.Store(false) - } - // clear state in case of expire/deadline transitions - if transition.Trigger == expireTransition || transition.Trigger == cancelTransition { - s.clear() - } - if transition.Trigger == matchTransition { - log.Debugf("state trigger from rule [%s]", transition.Source) - // a match occurred from current to next state. - // Stop deadline execution for the old current state - if span, ok := s.spanDeadlines[transition.Source]; ok { - log.Debugf("stopped max span deadline for rule %s", transition.Source) - span.Stop() - delete(s.spanDeadlines, transition.Source) - } - // save rule matches - s.matchedRules[s.idxs[transition.Source]] = true - } - }) -} - -func (s *sequenceState) matchTransition(rule string, kevt *kevent.Kevent) error { - s.mrm.Lock() - defer s.mrm.Unlock() - shouldFire := !s.matchedRules[s.idxs[rule]] - if shouldFire { - return s.fsm.Fire(matchTransition, kevt) - } - return nil -} - -func (s *sequenceState) cancelTransition(rule fsm.State) error { - return s.fsm.Fire(cancelTransition, rule) -} - -func (s *sequenceState) expireTransition() error { - return s.fsm.Fire(expireTransition) -} - -func (s *sequenceState) isTerminalState() bool { - isFinal := s.currentState() == sequenceTerminalState - if isFinal { - err := s.fsm.Fire(resetTransition) - if err != nil { - log.Warnf("unable to transition to initial state: %v", err) - } - } - return isFinal -} - -func (s *sequenceState) isInitialState() bool { - return s.currentState() == s.initialState -} - -func (s *sequenceState) currentState() fsm.State { - return s.fsm.MustState() -} - -func (s *sequenceState) addPartial(rule string, kevt *kevent.Kevent, outOfOrder bool) { - i := s.idxs[rule] - s.mu.Lock() - defer s.mu.Unlock() - if len(s.partials[i]) > maxOutstandingPartials { - partialBreaches.Add(s.name, 1) - log.Warnf("max partials encountered in sequence %s slot [%d]. "+ - "Dropping incoming partial", s.name, s.idxs[rule]) - return - } - key := kevt.PartialKey() - if key != 0 { - for _, p := range s.partials[i] { - if key == p.PartialKey() { - log.Debugf("%s event tuple already in sequence state", kevt.Name) - return - } - } - } - if outOfOrder { - kevt.AddMeta(kevent.RuleExpressionKey, rule) - kevt.AddMeta(kevent.RuleSequenceOutOfOrderKey, true) - } - log.Debugf("adding partial to slot [%d] for rule %q: %s", i, rule, kevt) - partialsPerSequence.Add(s.name, 1) - s.partials[i] = append(s.partials[i], kevt) - sort.Slice(s.partials[i], func(n, m int) bool { return s.partials[i][n].Timestamp.Before(s.partials[i][m].Timestamp) }) -} - -// gc prunes the sequence partial if it remained -// more time than specified by max span or if max -// span is omitted, the partial is allowed to remain -// in sequence state for four hours. -func (s *sequenceState) gc() { - s.mu.Lock() - defer s.mu.Unlock() - dur := s.maxSpan - if dur == 0 { - dur = maxSequencePartialLifetime - } - for _, idx := range s.idxs { - for i := len(s.partials[idx]) - 1; i >= 0; i-- { - if len(s.partials[idx]) > 0 && time.Since(s.partials[idx][i].Timestamp) > dur { - log.Debugf("garbage collecting partial: [%s]", s.partials[idx][i]) - // remove partial event from the corresponding slot - s.partials[idx] = append( - s.partials[idx][:i], - s.partials[idx][i+1:]...) - partialsPerSequence.Add(s.name, -1) - } - } - } -} - -func (s *sequenceState) clear() { - s.partials = make(map[uint16][]*kevent.Kevent) - s.matches = make(map[uint16]*kevent.Kevent) - s.matchedRules = make(map[uint16]bool) - s.spanDeadlines = make(map[fsm.State]*time.Timer) - partialsPerSequence.Delete(s.name) -} - -func (s *sequenceState) clearLocked() { - s.mu.Lock() - defer s.mu.Unlock() - s.mrm.Lock() - defer s.mrm.Unlock() - s.mmu.Lock() - defer s.mmu.Unlock() - s.clear() -} - -// next determines whether the next rule in the -// sequence should be evaluated. The rule is evaluated -// if all its upstream sequence rules produced a match and -// the sequence is not stuck in deadline or expired state. -func (s *sequenceState) next(i int) bool { - // always evaluate the first rule in the sequence - if i == 0 { - return true - } - var next bool - s.mrm.RLock() - defer s.mrm.RUnlock() - for n := 0; n < i; n++ { - next = s.matchedRules[uint16(n+1)] - if !next { - break - } - } - return next && !s.inDeadline.Load() && !s.inExpired.Load() -} - -func (s *sequenceState) scheduleMaxSpanDeadline(rule fsm.State, maxSpan time.Duration) { - t := time.AfterFunc(maxSpan, func() { - inState, _ := s.fsm.IsInState(rule) - if inState { - log.Infof("max span of %v exceded for rule %s", maxSpan, rule) - s.inDeadline.Store(true) - s.mu.Lock() - defer s.mu.Unlock() - s.mrm.Lock() - defer s.mrm.Unlock() - // transitions to deadline state - err := s.cancelTransition(rule) - if err != nil { - s.inDeadline.Store(false) - log.Warnf("deadline transition failed: %v", err) - } - // transitions from deadline state to initial state - err = s.fsm.Fire(resetTransition) - if err != nil { - log.Warnf("unable to transition to initial state: %v", err) - } - } - }) - s.spanDeadlines[rule] = t -} - -func (s *sequenceState) expire(e *kevent.Kevent) bool { - if !e.IsTerminateProcess() { - return false - } - canExpire := func(lhs, rhs *kevent.Kevent, isFinalSlot bool) bool { - // if the TerminateProcess event arrives for the - // process spawned by CreateProcess, and it pertains - // to the final sequence slot, it is safe to expire - // the whole sequence - if lhs.Type == ktypes.CreateProcess && isFinalSlot { - p1, _ := lhs.Kparams.GetPid() - p2, _ := rhs.Kparams.GetPid() - return p1 == p2 - } - pid, _ := rhs.Kparams.GetPid() - return lhs.PID == pid - } - s.mu.Lock() - defer s.mu.Unlock() - s.mrm.RLock() - defer s.mrm.RUnlock() - for _, idx := range s.idxs { - for i := len(s.partials[idx]) - 1; i >= 0; i-- { - if len(s.partials[idx]) > 0 && !canExpire(s.partials[idx][i], e, idx == uint16(len(s.idxs))) { - continue - } - - log.Debugf("removing event originated from %s (%d) "+ - "in partials pertaining to sequence [%s] and slot [%d]", - e.Kparams.MustGetString(kparams.ProcessName), - e.Kparams.MustGetPid(), - s.name, - idx) - // remove partial event from the corresponding slot - s.partials[idx] = append( - s.partials[idx][:i], - s.partials[idx][i+1:]...) - partialsPerSequence.Add(s.name, -1) - - if len(s.partials[idx]) == 0 { - partialExpirations.Add(s.name, 1) - log.Debugf("%q sequence expired. All partials retracted", s.name) - s.inExpired.Store(true) - err := s.expireTransition() - if err != nil { - s.inExpired.Store(false) - log.Warnf("expire transition failed: %v", err) - } - // transitions from expired state to initial state - err = s.fsm.Fire(resetTransition) - if err != nil { - log.Warnf("unable to transition to initial state: %v", err) - } - return true - } - } - } - return false -} - -func newCompiledFilter(f Filter, filterConfig *config.FilterConfig, ss *sequenceState) *compiledFilter { - return &compiledFilter{config: filterConfig, filter: f, ss: ss} -} - -// isScoped determines if this filter is scoped, i.e. it has the event name or category -// conditions. -func (f compiledFilter) isScoped() bool { - for name := range f.filter.GetStringFields() { - if name == fields.KevtName || name == fields.KevtCategory { - return true - } - } - return false -} - -// run execute the filter with either simple or sequence expressions. -func (f compiledFilter) run(kevt *kevent.Kevent, i int, rawMatch, lock bool) bool { - if f.ss != nil { - if lock { - f.ss.mu.RLock() - defer f.ss.mu.RUnlock() - } - return f.filter.RunSequence(kevt, uint16(i), f.ss.partials, rawMatch) - } - return f.filter.Run(kevt) -} - -// NewRules produces a fresh rules engine instance. -func NewRules(psnap ps.Snapshotter, config *config.Config) *Rules { - rules := &Rules{ - filters: make(map[uint32][]*compiledFilter), - matches: make([]*ruleMatch, 0), - sequences: make([]*sequenceState, 0), - psnap: psnap, - config: config, - scavenger: time.NewTicker(sequenceGcInterval), - } - - go rules.gcSequences() - - return rules -} - -// Compile loads macros and rules from all -// indicated resources and compiles the filters. -// It also sets up the state machine transitions -// for sequence rules. -func (r *Rules) Compile() (*config.RulesCompileResult, error) { - if err := r.config.Filters.LoadMacros(); err != nil { - return nil, err - } - if err := r.config.Filters.LoadFilters(); err != nil { - return nil, err - } - - for _, f := range r.config.GetFilters() { - if f.IsDisabled() { - log.Warnf("[%s] rule is disabled", f.Name) - continue - } - - filtersCount.Add(1) - - // compile filter and for sequence rules - // configure the FSM states and transitions - fltr := New(f.Condition, r.config, WithPSnapshotter(r.psnap)) - err := fltr.Compile() - if err != nil { - return nil, ErrInvalidFilter(f.Name, err) - } - // check version requirements - if !version.IsDev() { - minEngineVer, err := semver.NewSemver(f.MinEngineVersion) - if err != nil { - return nil, ErrMalformedMinEngineVer(f.Name, f.MinEngineVersion, err) - } - if minEngineVer.GreaterThan(version.Sem()) { - return nil, ErrIncompatibleFilter(f.Name, f.MinEngineVersion) - } - } - for _, field := range fltr.GetFields() { - deprecated, d := fields.IsDeprecated(field.Name) - if deprecated { - log.Warnf("%s rule uses the [%s] field which "+ - "was deprecated starting from version %s. "+ - "Please consider migrating to %s field(s) "+ - "because [%s] will be removed in future versions.", - f.Name, field, d.Since, d.Fields, field) - } - } - cf := newCompiledFilter(fltr, f, configureFSM(f, fltr)) - if fltr.IsSequence() && cf.ss != nil { - // store the sequences in rules - // for more convenient tracking - r.sequences = append(r.sequences, cf.ss) - } - - // traverse all event name or category fields and determine - // the event type from the filter field name expression. - // We end up with a map of rules indexed by event name - // or event category hash - if !cf.isScoped() { - log.Warnf("%q rule doesn't have "+ - "event type or event category condition! "+ - "This rule is being discarded by "+ - "the engine. Please consider narrowing the "+ - "scope of this rule by including the `kevt.name` "+ - "or `kevt.category` condition", - f.Name) - continue - } - for name, values := range fltr.GetStringFields() { - for _, v := range values { - if name == fields.KevtName || name == fields.KevtCategory { - hash := hashers.FnvUint32([]byte(v)) - r.filters[hash] = append(r.filters[hash], cf) - } - } - } - } - - if len(r.filters) == 0 { - return nil, nil - } - - return r.buildCompileResult(), nil -} - -func configureFSM(filter *config.FilterConfig, f Filter) *sequenceState { - if !f.IsSequence() { - return nil - } - seq := f.GetSequence() - expressions := seq.Expressions - if len(expressions) == 0 { - return nil - } - initialState := expressions[0].Expr.String() - seqState := newSequenceState(filter.Name, initialState, seq.MaxSpan) - // setup finite state machine states. The last rule - // in the sequence transitions to the terminal state - // if all rules match - for i, expr := range expressions { - n := expr.Expr.String() - seqState.idxs[n] = uint16(i + 1) - if i >= len(expressions)-1 { - seqState.fsm. - Configure(n). - Permit(matchTransition, sequenceTerminalState). - Permit(cancelTransition, sequenceDeadlineState). - Permit(expireTransition, sequenceExpiredState) - } else { - seqState.fsm. - Configure(n). - Permit(matchTransition, expressions[i+1].Expr.String()). - Permit(cancelTransition, sequenceDeadlineState). - Permit(expireTransition, sequenceExpiredState) - } - } - // configure reset transitions that are triggered - // when the final state is reached of when a deadline - // or sequence expiration happens - seqState.fsm. - Configure(sequenceTerminalState). - Permit(resetTransition, initialState) - seqState.fsm. - Configure(sequenceDeadlineState). - Permit(resetTransition, initialState) - seqState.fsm. - Configure(sequenceExpiredState). - Permit(resetTransition, initialState) - return seqState -} - -func (r *Rules) buildCompileResult() *config.RulesCompileResult { - rs := &config.RulesCompileResult{} - - m := make(map[ktypes.Ktype]bool) - events := make([]ktypes.Ktype, 0) - - for _, fltrs := range r.filters { - for _, cf := range fltrs { - rs.NumberRules++ - for name, values := range cf.filter.GetStringFields() { - for _, v := range values { - if name == fields.KevtName || name == fields.KevtCategory { - types := ktypes.KeventNameToKtypes(v) - for _, typ := range types { - switch typ.Category() { - case ktypes.Process: - rs.HasProcEvents = true - case ktypes.Thread: - rs.HasThreadEvents = true - case ktypes.Image: - rs.HasImageEvents = true - case ktypes.File: - rs.HasFileEvents = true - case ktypes.Net: - rs.HasNetworkEvents = true - case ktypes.Registry: - rs.HasRegistryEvents = true - case ktypes.Mem: - rs.HasMemEvents = true - case ktypes.Handle: - rs.HasHandleEvents = true - case ktypes.Threadpool: - rs.HasThreadpoolEvents = true - } - if typ == ktypes.MapViewFile || typ == ktypes.UnmapViewFile { - rs.HasVAMapEvents = true - } - if typ == ktypes.OpenProcess || typ == ktypes.OpenThread || typ == ktypes.SetThreadContext || - typ == ktypes.CreateSymbolicLinkObject { - rs.HasAuditAPIEvents = true - } - if typ.Subcategory() == ktypes.DNS { - rs.HasDNSEvents = true - } - if m[typ] { - continue - } - events = append(events, typ) - m[typ] = true - } - } - } - } - } - } - - rs.UsedEvents = events - return rs -} - -// hasRules checks if rules were loaded into -// the engine. If there are no rules the event is -// forwarded to the aggregator. -func (r *Rules) hasRules() bool { return len(r.filters) > 0 } - -// findFilters collects all compiled filters for a -// particular event type or category. If no filters -// are found the event is rejected from the aggregator -// batch. -func (r *Rules) findFilters(evt *kevent.Kevent) []*compiledFilter { - filters1 := r.filters[evt.Type.Hash()] - filters2 := r.filters[evt.Category.Hash()] - if filters1 == nil && filters2 == nil { - return nil - } - return append(filters1, filters2...) -} - -func (*Rules) CanEnqueue() bool { return true } - -func (r *Rules) ProcessEvent(evt *kevent.Kevent) (bool, error) { - if !r.hasRules() { - return true, nil - } - if evt.IsTerminateProcess() { - // expire all sequences if the - // process referenced in any - // partials has terminated - for _, seq := range r.sequences { - seq.expire(evt) - } - } - return r.runRules(r.findFilters(evt), evt), nil -} - -func (r *Rules) gcSequences() { - for { - <-r.scavenger.C - for _, seq := range r.sequences { - seq.gc() - } - } -} - -func (r *Rules) runSequence(kevt *kevent.Kevent, f *compiledFilter) bool { - seq := f.filter.GetSequence() - if seq == nil { - return false - } - for i, expr := range seq.Expressions { - // only try to evaluate the expression - // if upstream expressions have matched - if !f.ss.next(i) { - if !seq.IsUnordered { - continue - } - // it could be the event arrived out - // of order because certain provider - // flushed its buffers first. When this - // happens the event timestamp serves as - // a temporal reference. - // If this sequence expression can evaluate - // against the current event, mark it as - // out-of-order and store in partials list - if expr.IsEvaluable(kevt) && f.run(kevt, i, true, true) { - f.ss.addPartial(expr.Expr.String(), kevt, true) - } - continue - } - // prevent running the filter if the expression - // can't be matched against the current event - if !expr.IsEvaluable(kevt) { - continue - } - rule := expr.Expr.String() - matches := f.run(kevt, i, false, true) - // append the partial and transition state machine - if matches { - f.ss.addPartial(rule, kevt, false) - err := f.ss.matchTransition(rule, kevt) - if err != nil { - matchTransitionErrors.Add(1) - log.Warnf("match transition failure: %v", err) - } - // now try to match all pending out-of-order - // events from downstream sequence slots if - // the previous match hasn't reached terminal - // state - if seq.IsUnordered && f.ss.currentState() != sequenceTerminalState { - r.matchUnorderedPartials(f) - } - } - } - // if both the terminal state is reached and the partials - // in the sequence state could be joined by the specified - // field(s), the rule has matched successfully, and we can - // collect all events involved in the rule match - isTerminal := f.ss.isTerminalState() - if isTerminal { - f.ss.mu.RLock() - defer f.ss.mu.RUnlock() - nseqs := uint16(len(f.ss.partials)) - - setMatch := func(idx uint16, e *kevent.Kevent) { - f.ss.mmu.Lock() - defer f.ss.mmu.Unlock() - if f.ss.matches[idx] == nil { - f.ss.matches[idx] = e - } - } - - for i := uint16(1); i < nseqs+1; i++ { - for _, outer := range f.ss.partials[i] { - for _, inner := range f.ss.partials[i+1] { - if compareSeqJoin(outer.SequenceBy(), inner.SequenceBy()) { - setMatch(i, outer) - setMatch(i+1, inner) - } - } - } - } - } - return isTerminal -} - -func (r *Rules) matchUnorderedPartials(f *compiledFilter) { - f.ss.mu.Lock() - defer f.ss.mu.Unlock() - for n, partials := range f.ss.partials { - for _, partial := range partials { - if !partial.ContainsMeta(kevent.RuleSequenceOutOfOrderKey) { - continue - } - matches := f.run(partial, int(n)-1, false, false) - rule := partial.GetMetaAsString(kevent.RuleExpressionKey) - // transition the state machine - if matches { - err := f.ss.matchTransition(rule, partial) - if err != nil { - matchTransitionErrors.Add(1) - log.Warnf("out of order match transition failure: %v", err) - } - partial.RemoveMeta(kevent.RuleSequenceOutOfOrderKey) - } - } - } -} - -func (r *Rules) triggerSequencesInFilters(e *kevent.Kevent, filters []*compiledFilter) { - for _, f := range filters { - if !f.filter.IsSequence() || f.ss == nil { - continue - } - if r.runSequence(e, f) { - r.appendMatch(f.config, f.ss.events()...) - f.ss.clearLocked() - } - } -} - -func (r *Rules) runRules(filters []*compiledFilter, kevt *kevent.Kevent) bool { - for i, f := range filters { - var match bool - if f.ss != nil { - match = r.runSequence(kevt, f) - } else { - match = f.run(kevt, i, false, false) - if match { - // transition sequence states since a match - // in a simple rule could trigger multiple - // matches in sequence rules - r.triggerSequencesInFilters(kevt, filters) - } - } - if match { - if f.ss != nil { - r.appendMatch(f.config, f.ss.events()...) - f.ss.clearLocked() - } else { - r.appendMatch(f.config, kevt) - } - err := r.processActions() - if err != nil { - log.Errorf("unable to execute rule action: %v", err) - } - return true - } - } - return false -} - -// processActions executes rule actions -// on behalf of rule matches. Actions are -// categorized into implicit and explicit -// actions. -// Sending an alert is an implicit action -// carried out each time there is a rule -// match. Other actions are executed if -// defined in the rule definition. -func (r *Rules) processActions() error { - defer r.clearMatches() - for _, m := range r.matches { - f, evts := m.ctx.Filter, m.ctx.Events - filterMatches.Add(f.Name, 1) - log.Debugf("[%s] rule matched", f.Name) - err := action.Emit(m.ctx, f.Name, InterpolateFields(f.Output, evts), f.Severity, f.Tags) - if err != nil { - return ErrRuleAction(f.Name, err) - } - - actions, err := f.DecodeActions() - if err != nil { - return err - } - for _, act := range actions { - switch act.(type) { - case config.KillAction: - log.Infof("executing kill action: pids=%v rule=%s", m.ctx.UniquePids(), f.Name) - if err := action.Kill(m.ctx.UniquePids()); err != nil { - return ErrRuleAction(f.Name, err) - } - } - } - } - return nil -} - -func (r *Rules) appendMatch(f *config.FilterConfig, evts ...*kevent.Kevent) { - for _, evt := range evts { - evt.AddMeta(kevent.RuleNameKey, f.Name) - for k, v := range f.Labels { - evt.AddMeta(kevent.MetadataKey(k), v) - } - } - ctx := &config.ActionContext{ - Events: evts, - Filter: f, - } - r.matches = append(r.matches, &ruleMatch{ctx: ctx}) -} - -func (r *Rules) clearMatches() { - r.matches = make([]*ruleMatch, 0) -} diff --git a/pkg/filter/rules_test.go b/pkg/filter/rules_test.go deleted file mode 100644 index c698f38f8..000000000 --- a/pkg/filter/rules_test.go +++ /dev/null @@ -1,1243 +0,0 @@ -/* - * Copyright 2020-2021 by Nedim Sabic Sabic - * https://www.fibratus.io - * All Rights Reserved. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package filter - -import ( - "github.com/rabbitstack/fibratus/pkg/alertsender" - "github.com/rabbitstack/fibratus/pkg/config" - "github.com/rabbitstack/fibratus/pkg/fs" - "github.com/rabbitstack/fibratus/pkg/kevent" - "github.com/rabbitstack/fibratus/pkg/kevent/kparams" - "github.com/rabbitstack/fibratus/pkg/kevent/ktypes" - "github.com/rabbitstack/fibratus/pkg/ps" - "github.com/rabbitstack/fibratus/pkg/ps/types" - "github.com/rabbitstack/fibratus/pkg/sys" - "github.com/rabbitstack/fibratus/pkg/util/version" - log "github.com/sirupsen/logrus" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "golang.org/x/sys/windows" - "golang.org/x/sys/windows/registry" - "net" - "os" - "testing" - "time" -) - -type mockNoopSender struct{} -type mockNoneSender struct{} - -var emitAlert *alertsender.Alert -var seqAlert *alertsender.Alert - -func (s *mockNoopSender) Send(a alertsender.Alert) error { - emitAlert = &a - return nil -} - -func (s *mockNoopSender) Type() alertsender.Type { - return alertsender.Noop -} - -func (s *mockNoopSender) Shutdown() error { return nil } -func (s *mockNoopSender) SupportsMarkdown() bool { return true } - -func makeNoopSender(config alertsender.Config) (alertsender.Sender, error) { - return &mockNoopSender{}, nil -} - -func (s *mockNoneSender) Send(a alertsender.Alert) error { - seqAlert = &a - return nil -} - -func (s *mockNoneSender) Type() alertsender.Type { - return alertsender.None -} - -func (s *mockNoneSender) Shutdown() error { return nil } -func (s *mockNoneSender) SupportsMarkdown() bool { return true } - -func makeNoneSender(config alertsender.Config) (alertsender.Sender, error) { - return &mockNoneSender{}, nil -} - -func init() { - alertsender.Register(alertsender.Noop, makeNoopSender) - alertsender.Register(alertsender.None, makeNoneSender) -} - -func wrapProcessEvent(e *kevent.Kevent, fn func(*kevent.Kevent) (bool, error)) bool { - match, err := fn(e) - if err != nil { - panic(err) - } - return match -} - -func compileRules(t *testing.T, rules *Rules) { - stats, err := rules.Compile() - require.NoError(t, err) - require.NotNil(t, stats) -} - -func fireRules(t *testing.T, c *config.Config) bool { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, c) - - kevt := &kevent.Kevent{ - Type: ktypes.RecvTCPv4, - Name: "Recv", - Tid: 2484, - PID: 859, - Category: ktypes.Net, - Kparams: kevent.Kparams{ - kparams.NetDport: {Name: kparams.NetDport, Type: kparams.Uint16, Value: uint16(443)}, - kparams.NetSport: {Name: kparams.NetSport, Type: kparams.Uint16, Value: uint16(43123)}, - kparams.NetSIP: {Name: kparams.NetSIP, Type: kparams.IPv4, Value: net.ParseIP("127.0.0.1")}, - kparams.NetDIP: {Name: kparams.NetDIP, Type: kparams.IPv4, Value: net.ParseIP("216.58.201.174")}, - }, - Metadata: make(map[kevent.MetadataKey]any), - } - compileRules(t, rules) - return wrapProcessEvent(kevt, rules.ProcessEvent) -} - -func newConfig(fromFiles ...string) *config.Config { - var kstreamConfig = config.KstreamConfig{ - EnableHandleKevents: true, - EnableNetKevents: true, - EnableRegistryKevents: true, - EnableFileIOKevents: true, - EnableImageKevents: true, - EnableThreadKevents: true, - } - c := &config.Config{ - Kstream: kstreamConfig, - Filters: &config.Filters{ - Rules: config.Rules{ - FromPaths: fromFiles, - }, - }, - } - return c -} - -func TestCompileMergeFilters(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/merged_filters/filter*.yml")) - compileRules(t, rules) - - assert.Len(t, rules.filters, 2) - assert.Len(t, rules.filters[ktypes.RecvTCPv4.Hash()], 3) - assert.Len(t, rules.filters[ktypes.Net.Hash()], 1) - - assert.Len(t, rules.findFilters(&kevent.Kevent{Type: ktypes.RecvUDPv6}), 3) - assert.Len(t, rules.findFilters(&kevent.Kevent{Type: ktypes.RecvTCPv4}), 3) - assert.Len(t, rules.findFilters(&kevent.Kevent{Type: ktypes.RecvTCPv4, Category: ktypes.Net}), 4) -} - -func TestProcessRules(t *testing.T) { - var tests = []struct { - config *config.Config - matches bool - }{ - {newConfig("_fixtures/simple_matches.yml"), true}, - {newConfig("_fixtures/simple_matches/filter*.yml"), true}, - } - - for i, tt := range tests { - matches := fireRules(t, tt.config) - if matches != tt.matches { - t.Errorf("%d. %v process rules mismatch: exp=%t got=%t", i, tt.config.Filters, tt.matches, matches) - } - } -} - -func TestSequenceState(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_rule_simple.yml")) - compileRules(t, rules) - log.SetLevel(log.DebugLevel) - - e1 := &kevent.Kevent{ - Type: ktypes.CreateProcess, - Name: "CreateProcess", - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(4143)}, - kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.AnsiString, Value: "powershell.exe"}, - }, - } - e2 := &kevent.Kevent{ - Type: ktypes.CreateFile, - Name: "CreateFile", - Tid: 2484, - PID: 4143, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper"}, - }, - } - - assert.Len(t, rules.filters, 2) - - ss := rules.filters[ktypes.CreateProcess.Hash()][0].ss - ss1 := rules.filters[ktypes.CreateFile.Hash()][0].ss - - // should reference the same sequence state - assert.Equal(t, ss, ss1) - require.NotNil(t, ss) - - assert.Equal(t, "kevt.name = CreateProcess AND ps.name = cmd.exe", ss.currentState()) - assert.True(t, ss.isInitialState()) - assert.Equal(t, "kevt.name = CreateProcess AND ps.name = cmd.exe", ss.initialState) - - ss.addPartial("kevt.name = CreateProcess AND ps.name = cmd.exe", e1, false) - require.NoError(t, ss.matchTransition("kevt.name = CreateProcess AND ps.name = cmd.exe", e1)) - assert.False(t, ss.isInitialState()) - assert.Equal(t, "kevt.name = CreateFile AND file.path ICONTAINS temp", ss.currentState()) - - ss.addPartial("kevt.name = CreateFile AND file.path ICONTAINS temp", e2, false) - require.NoError(t, ss.matchTransition("kevt.name = CreateFile AND file.path ICONTAINS temp", e2)) - - assert.Len(t, ss.partials[1], 1) - assert.Len(t, ss.partials[2], 1) - - assert.Equal(t, sequenceTerminalState, ss.currentState()) - assert.True(t, ss.isTerminalState()) - - ss.clear() - - // reset transition leads back to initial state - assert.Equal(t, "kevt.name = CreateProcess AND ps.name = cmd.exe", ss.currentState()) - // deadline exceeded - require.NoError(t, ss.matchTransition("kevt.name = CreateProcess AND ps.name = cmd.exe", e1)) - assert.Equal(t, "kevt.name = CreateFile AND file.path ICONTAINS temp", ss.currentState()) - time.Sleep(time.Millisecond * 120) - assert.True(t, ss.isInitialState()) - - require.True(t, ss.inDeadline.Load()) - require.False(t, ss.next(1)) - if ss.next(1) { - // this shouldn't happen - require.NoError(t, ss.matchTransition("kevt.name = CreateFile AND file.path ICONTAINS temp", e2)) - } - - ss.clear() - assert.True(t, ss.isInitialState()) - require.NoError(t, ss.matchTransition("kevt.name = CreateProcess AND ps.name = cmd.exe", e1)) - ss.addPartial("kevt.name = CreateProcess AND ps.name = cmd.exe", e2, false) - ss.addPartial("kevt.name = CreateFile AND file.path ICONTAINS temp", e2, false) - require.False(t, ss.inDeadline.Load()) - - // test expiration - terminateProcess := &kevent.Kevent{ - Type: ktypes.TerminateProcess, - Name: "TerminateProcess", - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(4143)}, - kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.AnsiString, Value: "powershell.exe"}, - }, - } - - require.True(t, ss.expire(terminateProcess)) - require.True(t, ss.inExpired.Load()) - - require.NoError(t, ss.matchTransition("kevt.name = CreateProcess AND ps.name = cmd.exe", e1)) - require.False(t, ss.inExpired.Load()) - - assert.Equal(t, "kevt.name = CreateFile AND file.path ICONTAINS temp", ss.currentState()) -} - -func TestSequenceStateNext(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_rule_simple.yml")) - compileRules(t, rules) - log.SetLevel(log.DebugLevel) - - assert.Len(t, rules.filters, 2) - - ss := rules.filters[ktypes.CreateProcess.Hash()][0].ss - - assert.True(t, ss.next(0)) - assert.False(t, ss.next(1)) - - // first rule matched, should be able to proceed - // to the next rule but can't still reach the third rule - ss.matchedRules[1] = true - assert.True(t, ss.next(1)) - assert.False(t, ss.next(2)) - - // should be able to reach the third rule - ss.matchedRules[2] = true - assert.True(t, ss.next(2)) -} - -func TestExpireSequences(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_rule_expire.yml")) - compileRules(t, rules) - log.SetLevel(log.DebugLevel) - - e1 := &kevent.Kevent{ - Type: ktypes.OpenProcess, - Name: "OpenProcess", - Tid: 2484, - PID: 4143, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(4143)}, - kparams.DesiredAccess: {Name: kparams.DesiredAccess, Type: kparams.Uint32, Value: uint32(5)}, - kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.AnsiString, Value: "powershell.exe"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - e2 := &kevent.Kevent{ - Type: ktypes.TerminateProcess, - Name: "TerminateProcess", - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(4143)}, - kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.AnsiString, Value: "powershell.exe"}, - }, - } - - ss := rules.filters[ktypes.OpenProcess.Hash()][0].ss - - require.False(t, wrapProcessEvent(e1, rules.ProcessEvent)) - require.False(t, wrapProcessEvent(e2, rules.ProcessEvent)) - require.True(t, ss.inExpired.Load()) -} - -func TestMinEngineVersion(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/min_engine_version/fail/*.yml")) - version.Set("2.0.0") - _, err := rules.Compile() - require.Error(t, err) - rules = NewRules(psnap, newConfig("_fixtures/min_engine_version/ok/*.yml")) - compileRules(t, rules) -} - -func TestRuleCompileStats(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/default/*.yml")) - stats, err := rules.Compile() - require.NoError(t, err) - require.NotNil(t, stats) - - assert.True(t, stats.HasImageEvents) - assert.True(t, stats.HasProcEvents) - assert.False(t, stats.HasMemEvents) - assert.False(t, stats.HasAuditAPIEvents) - assert.True(t, stats.HasDNSEvents) - assert.Contains(t, stats.UsedEvents, ktypes.CreateProcess) - assert.Contains(t, stats.UsedEvents, ktypes.LoadImage) - assert.Contains(t, stats.UsedEvents, ktypes.QueryDNS) - assert.Contains(t, stats.UsedEvents, ktypes.ConnectTCPv4) - assert.Contains(t, stats.UsedEvents, ktypes.ConnectTCPv6) -} - -func TestSimpleSequenceRule(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_rule_simple.yml")) - compileRules(t, rules) - - e1 := &kevent.Kevent{ - Type: ktypes.CreateProcess, - Timestamp: time.Now(), - Name: "CreateProcess", - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost-temp.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - e2 := &kevent.Kevent{ - Type: ktypes.CreateFile, - Timestamp: time.Now(), - Name: "CreateFile", - Tid: 2484, - PID: 859, - Category: ktypes.File, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Windows\\system32\\svchost-temp.exe"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - require.False(t, wrapProcessEvent(e1, rules.ProcessEvent)) - require.True(t, wrapProcessEvent(e2, rules.ProcessEvent)) - - // if we alter the process executable in the first event, it shouldn't match - e1.PS.Exe = "C:\\System32\\cmd.exe" - - require.False(t, wrapProcessEvent(e1, rules.ProcessEvent)) - require.False(t, wrapProcessEvent(e2, rules.ProcessEvent)) -} - -func TestSimpleSequenceRuleMultiplePartials(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_rule_simple_max_span.yml")) - compileRules(t, rules) - - // create random matches which don't satisfy the BY statement - for i, pid := range []uint32{2343, 1024, 11122, 3450, 12319} { - e := &kevent.Kevent{ - Type: ktypes.CreateProcess, - Timestamp: time.Now().Add(time.Duration(i) * time.Millisecond), - Name: "CreateProcess", - Tid: 2484, - PID: pid, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\cmd.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: pid % 2}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - e1 := &kevent.Kevent{ - Type: ktypes.CreateFile, - Timestamp: time.Now().Add(time.Duration(i) * time.Millisecond * 2), - Name: "CreateFile", - Tid: 2484, - PID: pid * 2, - Category: ktypes.File, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\cmd.exe", - }, - Kparams: kevent.Kparams{ - kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Windows\\system32\\svchost-temp.exe"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - require.False(t, wrapProcessEvent(e, rules.ProcessEvent)) - require.False(t, wrapProcessEvent(e1, rules.ProcessEvent)) - } - - ss := rules.filters[ktypes.CreateProcess.Hash()][0].ss - assert.Len(t, ss.partials[1], 5) - assert.Len(t, ss.partials[2], 0) - - e1 := &kevent.Kevent{ - Seq: 20, - Type: ktypes.CreateProcess, - Timestamp: time.Now().Add(time.Second), - Name: "CreateProcess", - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost-temp.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - e2 := &kevent.Kevent{ - Type: ktypes.CreateFile, - Seq: 22, - Timestamp: time.Now().Add(time.Second * time.Duration(2)), - Name: "CreateFile", - Tid: 2484, - PID: 859, - Category: ktypes.File, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Windows\\system32\\svchost-temp.exe"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - require.False(t, wrapProcessEvent(e1, rules.ProcessEvent)) - assert.Len(t, ss.partials[1], 6) - assert.Len(t, ss.partials[2], 0) - require.True(t, wrapProcessEvent(e2, rules.ProcessEvent)) -} - -func TestSimpleSequenceRuleWithMaxSpanReached(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_rule_simple_max_span.yml")) - compileRules(t, rules) - - e1 := &kevent.Kevent{ - Type: ktypes.CreateProcess, - Timestamp: time.Now(), - Name: "CreateProcess", - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - e2 := &kevent.Kevent{ - Type: ktypes.CreateFile, - Timestamp: time.Now(), - Name: "CreateFile", - Tid: 2484, - PID: 859, - Category: ktypes.File, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - require.False(t, wrapProcessEvent(e1, rules.ProcessEvent)) - time.Sleep(time.Millisecond * 110) - require.True(t, wrapProcessEvent(e2, rules.ProcessEvent)) - - // now the state machine has transitioned - // to the initial state, which means we should - // be able to match the sequence if we reinsert - // the events - require.False(t, wrapProcessEvent(e1, rules.ProcessEvent)) - require.True(t, wrapProcessEvent(e2, rules.ProcessEvent)) -} - -func TestSimpleSequencePolicyWithMaxSpanNotReached(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_rule_simple_max_span.yml")) - compileRules(t, rules) - - e1 := &kevent.Kevent{ - Type: ktypes.CreateProcess, - Timestamp: time.Now(), - Name: "CreateProcess", - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - e2 := &kevent.Kevent{ - Type: ktypes.CreateFile, - Timestamp: time.Now(), - Name: "CreateFile", - Tid: 2484, - PID: 859, - Category: ktypes.File, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - require.False(t, wrapProcessEvent(e1, rules.ProcessEvent)) - time.Sleep(time.Millisecond * 110) - require.True(t, wrapProcessEvent(e2, rules.ProcessEvent)) -} - -func TestComplexSequenceRule(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_rule_complex.yml")) - compileRules(t, rules) - log.SetLevel(log.DebugLevel) - - e1 := &kevent.Kevent{ - Seq: 1, - Type: ktypes.CreateProcess, - Timestamp: time.Now(), - Category: ktypes.Process, - Name: "CreateProcess", - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "explorer.exe", - Exe: "C:\\Windows\\system32\\explorer.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(2243)}, - kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "firefox.exe"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - e2 := &kevent.Kevent{ - Seq: 2, - Type: ktypes.CreateFile, - Timestamp: time.Now().Add(time.Millisecond * 250), - Name: "CreateFile", - Tid: 2484, - PID: 2243, - Category: ktypes.File, - PS: &types.PS{ - Name: "firefox.exe", - Exe: "C:\\Program Files\\Mozilla Firefox\\firefox.exe", - Cmdline: "C:\\Program Files\\Mozilla Firefox\\firefox.exe\" -contentproc --channel=\"10464.7.539748228\\1366525930\" -childID 6 -isF", - }, - Kparams: kevent.Kparams{ - kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper.exe"}, - kparams.FileOperation: {Name: kparams.FileOperation, Type: kparams.Enum, Value: uint32(2), Enum: fs.FileCreateDispositions}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - e3 := &kevent.Kevent{ - Seq: 4, - Type: ktypes.ConnectTCPv4, - Timestamp: time.Now().Add(time.Second), - Category: ktypes.Net, - Name: "Connect", - Tid: 244, - PID: 2243, - PS: &types.PS{ - Name: "firefox.exe", - Exe: "C:\\Program Files\\Mozilla Firefox\\firefox.exe", - Cmdline: "C:\\Program Files\\Mozilla Firefox\\firefox.exe\" -contentproc --channel=\"10464.7.539748228\\1366525930\" -childID 6 -isF", - }, - Kparams: kevent.Kparams{ - kparams.NetDIP: {Name: kparams.NetDIP, Type: kparams.IPv4, Value: net.ParseIP("10.0.2.3")}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - // register alert sender - require.NoError(t, alertsender.LoadAll([]alertsender.Config{{Type: alertsender.None}})) - - require.False(t, wrapProcessEvent(e1, rules.ProcessEvent)) - require.False(t, wrapProcessEvent(e2, rules.ProcessEvent)) - - ss := rules.filters[ktypes.CreateProcess.Hash()][0].ss - assert.Len(t, ss.partials[1], 1) - assert.Len(t, ss.partials[2], 1) - - time.Sleep(time.Millisecond * 30) - require.True(t, wrapProcessEvent(e3, rules.ProcessEvent)) - - time.Sleep(time.Millisecond * 50) - - // check the format of the generated alert - require.NotNil(t, seqAlert) - assert.Equal(t, "572902be-76e9-4ee7-a48a-6275fa571cf4", seqAlert.ID) - assert.Len(t, seqAlert.Events, 3) - assert.Equal(t, "Phishing dropper outbound communication", seqAlert.Title) - assert.Equal(t, "firefox.exe process initiated outbound communication to 10.0.2.3", seqAlert.Text) - seqAlert = nil - - // FSM should transition from terminal to initial state - require.False(t, wrapProcessEvent(e1, rules.ProcessEvent)) - require.False(t, wrapProcessEvent(e2, rules.ProcessEvent)) - time.Sleep(time.Millisecond * 15) - require.True(t, wrapProcessEvent(e3, rules.ProcessEvent)) -} - -func TestSequencePsUUID(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_rule_ps_uuid.yml")) - compileRules(t, rules) - log.SetLevel(log.DebugLevel) - - kevt1 := &kevent.Kevent{ - Seq: 1, - Type: ktypes.CreateProcess, - Timestamp: time.Now(), - Category: ktypes.Process, - Name: "CreateProcess", - Tid: 2484, - PID: uint32(os.Getpid()), - PS: &types.PS{ - PID: uint32(os.Getpid()), - Name: "explorer.exe", - Exe: "C:\\Windows\\system32\\explorer.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(2243)}, - kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "firefox.exe"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - kevt2 := &kevent.Kevent{ - Seq: 2, - Type: ktypes.CreateFile, - Timestamp: time.Now(), - Name: "CreateFile", - Tid: 2484, - PID: uint32(os.Getpid()), - Category: ktypes.File, - PS: &types.PS{ - PID: uint32(os.Getpid()), - Name: "firefox.exe", - Exe: "C:\\Program Files\\Mozilla Firefox\\firefox.exe", - Cmdline: "C:\\Program Files\\Mozilla Firefox\\firefox.exe\" -contentproc --channel=\"10464.7.539748228\\1366525930\" -childID 6 -isF", - }, - Kparams: kevent.Kparams{ - kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper.exe"}, - kparams.FileOperation: {Name: kparams.FileOperation, Type: kparams.Enum, Value: uint32(2), Enum: fs.FileCreateDispositions}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - require.False(t, wrapProcessEvent(kevt1, rules.ProcessEvent)) - require.True(t, wrapProcessEvent(kevt2, rules.ProcessEvent)) -} - -func TestSequenceOutOfOrder(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_rule_out_of_order.yml")) - compileRules(t, rules) - - now := time.Now() - e1 := &kevent.Kevent{ - Type: ktypes.OpenProcess, - Timestamp: now, - Name: "OpenProcess", - Tid: 2484, - PID: 859, - PS: &types.PS{ - PID: 859, - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost-temp.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, - kparams.DesiredAccess: {Name: kparams.DesiredAccess, Type: kparams.Uint32, Value: uint32(5)}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - e2 := &kevent.Kevent{ - Type: ktypes.CreateFile, - Timestamp: now.Add(time.Millisecond * 200), - Name: "CreateFile", - Tid: 2484, - PID: 859, - Category: ktypes.File, - PS: &types.PS{ - PID: 859, - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Windows\\system32\\lsass.dmp"}, - kparams.FileOperation: {Name: kparams.FileOperation, Type: kparams.Enum, Value: uint32(2), Enum: fs.FileCreateDispositions}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - ss := rules.filters[ktypes.CreateFile.Hash()][0].ss - - require.False(t, wrapProcessEvent(e2, rules.ProcessEvent)) - assert.Len(t, ss.partials[2], 1) - assert.True(t, ss.partials[2][0].ContainsMeta(kevent.RuleSequenceOutOfOrderKey)) - - require.True(t, wrapProcessEvent(e1, rules.ProcessEvent)) -} - -func TestGCSequence(t *testing.T) { - sequenceGcInterval = time.Millisecond * 300 - maxSequencePartialLifetime = time.Millisecond * 500 - - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_gc.yml")) - compileRules(t, rules) - log.SetLevel(log.DebugLevel) - - now := time.Now() - kevt1 := &kevent.Kevent{ - Type: ktypes.OpenProcess, - Timestamp: now, - Name: "OpenProcess", - Tid: 2484, - PID: 859, - PS: &types.PS{ - PID: 859, - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost-temp.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, - kparams.DesiredAccess: {Name: kparams.DesiredAccess, Type: kparams.Uint32, Value: uint32(5)}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - ss := rules.filters[ktypes.OpenProcess.Hash()][0].ss - - require.False(t, wrapProcessEvent(kevt1, rules.ProcessEvent)) - - assert.Len(t, ss.partials[1], 1) - - time.Sleep(time.Second) - - assert.Len(t, ss.partials[1], 0) -} - -func TestSequenceAndSimpleRuleMix(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/simple_and_sequence_rules/*.yml")) - compileRules(t, rules) - log.SetLevel(log.DebugLevel) - - kevt1 := &kevent.Kevent{ - Seq: 1, - Type: ktypes.CreateProcess, - Timestamp: time.Now(), - Category: ktypes.Process, - Name: "CreateProcess", - Tid: 2484, - PID: 2243, - PS: &types.PS{ - Name: "powershell.exe", - Exe: "C:\\Windows\\system32\\powershell.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(2243)}, - kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "firefox.exe"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - kevt2 := &kevent.Kevent{ - Seq: 2, - Type: ktypes.CreateFile, - Timestamp: time.Now().Add(time.Millisecond * 544), - Name: "CreateFile", - Tid: 2484, - PID: 2243, - Category: ktypes.File, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\cmd.exe", - }, - Kparams: kevent.Kparams{ - kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper.exe"}, - kparams.FileOperation: {Name: kparams.FileOperation, Type: kparams.Enum, Value: uint32(2)}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - require.True(t, wrapProcessEvent(kevt1, rules.ProcessEvent)) - require.True(t, wrapProcessEvent(kevt2, rules.ProcessEvent)) - - kevt3 := &kevent.Kevent{ - Seq: 10, - Type: ktypes.CreateProcess, - Timestamp: time.Now().Add(time.Second * 2), - Category: ktypes.Process, - Name: "CreateProcess", - Tid: 2484, - PID: 2243, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\cmd.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(2243)}, - kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "chrome.exe"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - require.True(t, wrapProcessEvent(kevt3, rules.ProcessEvent)) -} - -func TestSequenceRuleBoundsFields(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_rule_bound_fields.yml")) - compileRules(t, rules) - - kevt := &kevent.Kevent{ - Type: ktypes.CreateProcess, - Timestamp: time.Now(), - Name: "CreateProcess", - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost-temp.exe", - SID: "zinet", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - kevt1 := &kevent.Kevent{ - Type: ktypes.CreateProcess, - Timestamp: time.Now().Add(time.Millisecond * 20), - Name: "CreateProcess", - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost-temp.exe", - SID: "nusret", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - kevt2 := &kevent.Kevent{ - Type: ktypes.CreateFile, - Timestamp: time.Now().Add(time.Second), - Name: "CreateFile", - Tid: 2484, - PID: 859, - Category: ktypes.File, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - SID: "nusret", - }, - Kparams: kevent.Kparams{ - kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Windows\\system32\\svchost-temp.exe"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - kevt3 := &kevent.Kevent{ - Type: ktypes.ConnectTCPv4, - Timestamp: time.Now().Add(time.Second * 3), - Name: "Connect", - Tid: 2484, - PID: 859, - Category: ktypes.File, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - SID: "zinet", - }, - Kparams: kevent.Kparams{ - kparams.NetDport: {Name: kparams.NetDport, Type: kparams.Uint16, Value: uint16(80)}, - kparams.NetDIP: {Name: kparams.NetDIP, Type: kparams.IPv4, Value: net.ParseIP("172.1.2.3")}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - require.False(t, wrapProcessEvent(kevt, rules.ProcessEvent)) - require.False(t, wrapProcessEvent(kevt1, rules.ProcessEvent)) - require.False(t, wrapProcessEvent(kevt2, rules.ProcessEvent)) - require.True(t, wrapProcessEvent(kevt3, rules.ProcessEvent)) -} - -func TestFilterActionEmitAlert(t *testing.T) { - psnap := new(ps.SnapshotterMock) - require.NoError(t, alertsender.LoadAll([]alertsender.Config{{Type: alertsender.Noop}})) - rules := NewRules(psnap, newConfig("_fixtures/simple_emit_alert.yml")) - compileRules(t, rules) - - kevt := &kevent.Kevent{ - Type: ktypes.RecvTCPv4, - Name: "Recv", - Tid: 2484, - PID: 859, - Category: ktypes.Net, - PS: &types.PS{ - Name: "cmd.exe", - }, - Kparams: kevent.Kparams{ - kparams.NetDport: {Name: kparams.NetDport, Type: kparams.Uint16, Value: uint16(443)}, - kparams.NetSport: {Name: kparams.NetSport, Type: kparams.Uint16, Value: uint16(43123)}, - kparams.NetSIP: {Name: kparams.NetSIP, Type: kparams.IPv4, Value: net.ParseIP("127.0.0.1")}, - kparams.NetDIP: {Name: kparams.NetDIP, Type: kparams.IPv4, Value: net.ParseIP("216.58.201.174")}, - }, - Metadata: make(map[kevent.MetadataKey]any), - } - - require.True(t, wrapProcessEvent(kevt, rules.ProcessEvent)) - time.Sleep(time.Millisecond * 25) - require.NotNil(t, emitAlert) - assert.Equal(t, "match https connections", emitAlert.Title) - assert.Equal(t, "cmd.exe process received data on port 443", emitAlert.Text) - assert.Equal(t, alertsender.Critical, emitAlert.Severity) - assert.Equal(t, []string{"tag1", "tag2"}, emitAlert.Tags) - emitAlert = nil -} - -func TestIsExpressionEvaluable(t *testing.T) { - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_rule_simple.yml")) - compileRules(t, rules) - log.SetLevel(log.DebugLevel) - - e1 := &kevent.Kevent{ - Type: ktypes.CreateProcess, - Name: "CreateProcess", - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - e2 := &kevent.Kevent{ - Type: ktypes.RenameFile, - Name: "RenameFile", - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost.exe", - }, - Kparams: kevent.Kparams{ - kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - for _, f := range rules.filters { - for _, cf := range f { - e := cf.filter.GetSequence().Expressions[0] - assert.False(t, e.IsEvaluable(e2)) - assert.True(t, e.IsEvaluable(e1)) - } - } -} - -func TestBoundFieldsWithFunctions(t *testing.T) { - log.SetLevel(log.DebugLevel) - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/sequence_rule_bound_fields_with_functions.yml")) - compileRules(t, rules) - - kevt1 := &kevent.Kevent{ - Type: ktypes.CreateFile, - Name: "CreateFile", - Category: ktypes.File, - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\cmd.exe", - }, - Kparams: kevent.Kparams{ - kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Windows\\System32\\passwdflt.dll"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - kevt2 := &kevent.Kevent{ - Type: ktypes.RegSetValue, - Name: "RegSetValue", - Category: ktypes.Registry, - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\cmd.exe", - }, - Kparams: kevent.Kparams{ - kparams.RegPath: {Name: kparams.RegPath, Type: kparams.UnicodeString, Value: "HKEY_CURRENT_USER\\Volatile Environment\\Notification Packages"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - key, err := registry.OpenKey(registry.CURRENT_USER, "Volatile Environment", registry.SET_VALUE) - require.NoError(t, err) - defer key.Close() - - defer func() { - _ = key.DeleteValue("Notification Packages") - }() - - require.NoError(t, key.SetStringsValue("Notification Packages", []string{"secli", "passwdflt"})) - - require.False(t, wrapProcessEvent(kevt1, rules.ProcessEvent)) - require.True(t, wrapProcessEvent(kevt2, rules.ProcessEvent)) -} - -func TestKillAction(t *testing.T) { - log.SetLevel(log.DebugLevel) - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/kill_action.yml")) - compileRules(t, rules) - - // register alert sender - require.NoError(t, alertsender.LoadAll([]alertsender.Config{{Type: alertsender.None}})) - - var si windows.StartupInfo - var pi windows.ProcessInformation - argv, err := windows.UTF16PtrFromString("calc.exe") - require.NoError(t, err) - err = windows.CreateProcess( - nil, - argv, - nil, - nil, - true, - 0, - nil, - nil, - &si, - &pi) - require.NoError(t, err) - - i := 0 - for !sys.IsProcessRunning(pi.Process) && i < 10 { - i++ - time.Sleep(time.Millisecond * 100 * time.Duration(i)) - } - - e := &kevent.Kevent{ - Type: ktypes.CreateProcess, - Timestamp: time.Now(), - Name: "CreateProcess", - Tid: 2484, - PID: 859, - Category: ktypes.Process, - PS: &types.PS{ - Name: "cmd.exe", - Exe: "C:\\Windows\\system32\\svchost-temp.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: pi.ProcessId}, - kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "calc.exe"}, - }, - Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, - } - - require.True(t, sys.IsProcessRunning(pi.Process)) - require.True(t, wrapProcessEvent(e, rules.ProcessEvent)) - require.False(t, sys.IsProcessRunning(pi.Process)) -} - -func BenchmarkRunRules(b *testing.B) { - b.ReportAllocs() - psnap := new(ps.SnapshotterMock) - rules := NewRules(psnap, newConfig("_fixtures/default/*.yml")) - stats, err := rules.Compile() - require.NoError(b, err) - require.NotNil(b, stats) - - b.ResetTimer() - kevts := []*kevent.Kevent{ - { - Type: ktypes.ConnectTCPv4, - Name: "Recv", - Tid: 2484, - PID: 859, - Category: ktypes.Net, - PS: &types.PS{ - Name: "cmd.exe", - }, - Kparams: kevent.Kparams{ - kparams.NetDport: {Name: kparams.NetDport, Type: kparams.Uint16, Value: uint16(443)}, - kparams.NetSport: {Name: kparams.NetSport, Type: kparams.Uint16, Value: uint16(43123)}, - kparams.NetSIP: {Name: kparams.NetSIP, Type: kparams.IPv4, Value: net.ParseIP("127.0.0.1")}, - kparams.NetDIP: {Name: kparams.NetDIP, Type: kparams.IPv4, Value: net.ParseIP("216.58.201.174")}, - }, - Metadata: make(map[kevent.MetadataKey]any), - }, - { - Type: ktypes.CreateProcess, - Name: "CreateProcess", - Category: ktypes.Process, - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "powershell.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: 2323}, - kparams.ProcessParentID: {Name: kparams.ProcessParentID, Type: kparams.PID, Value: uint32(8390)}, - kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "spotify.exe"}, - kparams.Cmdline: {Name: kparams.Cmdline, Type: kparams.UnicodeString, Value: `C:\Users\admin\AppData\Roaming\Spotify\Spotify.exe --type=crashpad-handler /prefetch:7 --max-uploads=5 --max-db-size=20 --max-db-age=5 --monitor-self-annotation=ptype=crashpad-handler "--metrics-dir=C:\Users\admin\AppData\Local\Spotify\User Data" --url=https://crashdump.spotify.com:443/ --annotation=platform=win32 --annotation=product=spotify --annotation=version=1.1.4.197 --initial-client-data=0x5a4,0x5a0,0x5a8,0x59c,0x5ac,0x6edcbf60,0x6edcbf70,0x6edcbf7c`}, - kparams.Exe: {Name: kparams.Exe, Type: kparams.UnicodeString, Value: `C:\Users\admin\AppData\Roaming\Spotify\Spotify.exe`}, - kparams.UserSID: {Name: kparams.UserSID, Type: kparams.UnicodeString, Value: `admin\SYSTEM`}, - }, - Metadata: make(map[kevent.MetadataKey]any), - }, - { - Type: ktypes.CreateHandle, - Name: "CreateHandle", - Category: ktypes.Handle, - Tid: 2484, - PID: 859, - PS: &types.PS{ - Name: "powershell.exe", - }, - Kparams: kevent.Kparams{ - kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: 2323}, - }, - Metadata: make(map[kevent.MetadataKey]any), - }, - } - - for i := 0; i < b.N; i++ { - for _, kevt := range kevts { - _, _ = rules.ProcessEvent(kevt) - } - } -} diff --git a/pkg/kevent/kevent.go b/pkg/kevent/kevent.go index 7aaa7d9db..6b358788b 100644 --- a/pkg/kevent/kevent.go +++ b/pkg/kevent/kevent.go @@ -44,15 +44,11 @@ const ( YaraMatchesKey MetadataKey = "yara.matches" // RuleNameKey identifies the rule that was triggered by the event RuleNameKey MetadataKey = "rule.name" - // RuleGroupKey identifies the group to which the triggered rule pertains - RuleGroupKey MetadataKey = "rule.group" - // RuleSequenceByKey represents the join field value in sequence rules - RuleSequenceByKey MetadataKey = "rule.seq.by" - // RuleExpressionKey represents the rule filter expression - RuleExpressionKey MetadataKey = "rule.expr" - // RuleSequenceOutOfOrderKey the presence of this metadata key indicates the + // RuleSequenceLink represents the join link value in sequence rules + RuleSequenceLink MetadataKey = "rule.seq.link" + // RuleSequenceOOOKey the presence of this metadata key indicates the // event in the partials list arrived out of order and requires reevaluation - RuleSequenceOutOfOrderKey MetadataKey = "rule.seq.outoforder" + RuleSequenceOOOKey MetadataKey = "rule.seq.ooo" ) func (key MetadataKey) String() string { return string(key) } @@ -308,9 +304,9 @@ func (e *Kevent) GetFlagsAsSlice(name string) []string { return strings.Split(e.GetParamAsString(name), "|") } -// SequenceBy returns the BY statement join field from event metadata. -func (e *Kevent) SequenceBy() any { +// SequenceLink returns the sequence link value from event metadata. +func (e *Kevent) SequenceLink() any { e.mmux.RLock() defer e.mmux.RUnlock() - return e.Metadata[RuleSequenceByKey] + return e.Metadata[RuleSequenceLink] } diff --git a/pkg/filter/_fixtures/default/microsoft_edge.yml b/pkg/rules/_fixtures/default/microsoft_edge.yml similarity index 100% rename from pkg/filter/_fixtures/default/microsoft_edge.yml rename to pkg/rules/_fixtures/default/microsoft_edge.yml diff --git a/pkg/filter/_fixtures/sequence_rule_simple.yml b/pkg/rules/_fixtures/default/sequence_rule_simple.yml similarity index 100% rename from pkg/filter/_fixtures/sequence_rule_simple.yml rename to pkg/rules/_fixtures/default/sequence_rule_simple.yml diff --git a/pkg/filter/_fixtures/default/suspicious_domains.yml b/pkg/rules/_fixtures/default/suspicious_domains.yml similarity index 100% rename from pkg/filter/_fixtures/default/suspicious_domains.yml rename to pkg/rules/_fixtures/default/suspicious_domains.yml diff --git a/pkg/filter/_fixtures/default/suspicious_module_loaded.yml b/pkg/rules/_fixtures/default/suspicious_module_loaded.yml similarity index 100% rename from pkg/filter/_fixtures/default/suspicious_module_loaded.yml rename to pkg/rules/_fixtures/default/suspicious_module_loaded.yml diff --git a/pkg/filter/_fixtures/default/suspicious_network_connecting_binaries.yml b/pkg/rules/_fixtures/default/suspicious_network_connecting_binaries.yml similarity index 100% rename from pkg/filter/_fixtures/default/suspicious_network_connecting_binaries.yml rename to pkg/rules/_fixtures/default/suspicious_network_connecting_binaries.yml diff --git a/pkg/filter/_fixtures/default/windows_error_reporting_and_wmi_provider_host.yml b/pkg/rules/_fixtures/default/windows_error_reporting_and_wmi_provider_host.yml similarity index 100% rename from pkg/filter/_fixtures/default/windows_error_reporting_and_wmi_provider_host.yml rename to pkg/rules/_fixtures/default/windows_error_reporting_and_wmi_provider_host.yml diff --git a/pkg/filter/_fixtures/kill_action.yml b/pkg/rules/_fixtures/kill_action.yml similarity index 100% rename from pkg/filter/_fixtures/kill_action.yml rename to pkg/rules/_fixtures/kill_action.yml diff --git a/pkg/filter/_fixtures/merged_filters/filter1.yml b/pkg/rules/_fixtures/merged_filters/filter1.yml similarity index 100% rename from pkg/filter/_fixtures/merged_filters/filter1.yml rename to pkg/rules/_fixtures/merged_filters/filter1.yml diff --git a/pkg/filter/_fixtures/merged_filters/filter2.yml b/pkg/rules/_fixtures/merged_filters/filter2.yml similarity index 100% rename from pkg/filter/_fixtures/merged_filters/filter2.yml rename to pkg/rules/_fixtures/merged_filters/filter2.yml diff --git a/pkg/filter/_fixtures/merged_filters/filter3.yml b/pkg/rules/_fixtures/merged_filters/filter3.yml similarity index 100% rename from pkg/filter/_fixtures/merged_filters/filter3.yml rename to pkg/rules/_fixtures/merged_filters/filter3.yml diff --git a/pkg/filter/_fixtures/merged_filters/filter4.yml b/pkg/rules/_fixtures/merged_filters/filter4.yml similarity index 100% rename from pkg/filter/_fixtures/merged_filters/filter4.yml rename to pkg/rules/_fixtures/merged_filters/filter4.yml diff --git a/pkg/filter/_fixtures/min_engine_version/fail/filter1.yml b/pkg/rules/_fixtures/min_engine_version/fail/filter1.yml similarity index 100% rename from pkg/filter/_fixtures/min_engine_version/fail/filter1.yml rename to pkg/rules/_fixtures/min_engine_version/fail/filter1.yml diff --git a/pkg/filter/_fixtures/min_engine_version/fail/filter2.yml b/pkg/rules/_fixtures/min_engine_version/fail/filter2.yml similarity index 100% rename from pkg/filter/_fixtures/min_engine_version/fail/filter2.yml rename to pkg/rules/_fixtures/min_engine_version/fail/filter2.yml diff --git a/pkg/filter/_fixtures/min_engine_version/fail/filter3.yml b/pkg/rules/_fixtures/min_engine_version/fail/filter3.yml similarity index 100% rename from pkg/filter/_fixtures/min_engine_version/fail/filter3.yml rename to pkg/rules/_fixtures/min_engine_version/fail/filter3.yml diff --git a/pkg/filter/_fixtures/min_engine_version/ok/filter1.yml b/pkg/rules/_fixtures/min_engine_version/ok/filter1.yml similarity index 100% rename from pkg/filter/_fixtures/min_engine_version/ok/filter1.yml rename to pkg/rules/_fixtures/min_engine_version/ok/filter1.yml diff --git a/pkg/filter/_fixtures/min_engine_version/ok/filter2.yml b/pkg/rules/_fixtures/min_engine_version/ok/filter2.yml similarity index 100% rename from pkg/filter/_fixtures/min_engine_version/ok/filter2.yml rename to pkg/rules/_fixtures/min_engine_version/ok/filter2.yml diff --git a/pkg/filter/_fixtures/min_engine_version/ok/filter3.yml b/pkg/rules/_fixtures/min_engine_version/ok/filter3.yml similarity index 100% rename from pkg/filter/_fixtures/min_engine_version/ok/filter3.yml rename to pkg/rules/_fixtures/min_engine_version/ok/filter3.yml diff --git a/pkg/filter/_fixtures/sequence_rule_complex.yml b/pkg/rules/_fixtures/sequence_rule_complex.yml similarity index 100% rename from pkg/filter/_fixtures/sequence_rule_complex.yml rename to pkg/rules/_fixtures/sequence_rule_complex.yml diff --git a/pkg/filter/_fixtures/sequence_rule_ps_uuid.yml b/pkg/rules/_fixtures/sequence_rule_ps_uuid.yml similarity index 100% rename from pkg/filter/_fixtures/sequence_rule_ps_uuid.yml rename to pkg/rules/_fixtures/sequence_rule_ps_uuid.yml diff --git a/pkg/filter/_fixtures/simple_and_sequence_rules/command_shell_spawned_chrome_browser.yml b/pkg/rules/_fixtures/simple_and_sequence_rules/command_shell_spawned_chrome_browser.yml similarity index 100% rename from pkg/filter/_fixtures/simple_and_sequence_rules/command_shell_spawned_chrome_browser.yml rename to pkg/rules/_fixtures/simple_and_sequence_rules/command_shell_spawned_chrome_browser.yml diff --git a/pkg/filter/_fixtures/simple_and_sequence_rules/powershell_created_temp_file.yml b/pkg/rules/_fixtures/simple_and_sequence_rules/powershell_created_temp_file.yml similarity index 100% rename from pkg/filter/_fixtures/simple_and_sequence_rules/powershell_created_temp_file.yml rename to pkg/rules/_fixtures/simple_and_sequence_rules/powershell_created_temp_file.yml diff --git a/pkg/filter/_fixtures/simple_and_sequence_rules/process_spawned_by_powershell.yml b/pkg/rules/_fixtures/simple_and_sequence_rules/process_spawned_by_powershell.yml similarity index 100% rename from pkg/filter/_fixtures/simple_and_sequence_rules/process_spawned_by_powershell.yml rename to pkg/rules/_fixtures/simple_and_sequence_rules/process_spawned_by_powershell.yml diff --git a/pkg/filter/_fixtures/simple_and_sequence_rules/spawn_chrome_browser.yml b/pkg/rules/_fixtures/simple_and_sequence_rules/spawn_chrome_browser.yml similarity index 100% rename from pkg/filter/_fixtures/simple_and_sequence_rules/spawn_chrome_browser.yml rename to pkg/rules/_fixtures/simple_and_sequence_rules/spawn_chrome_browser.yml diff --git a/pkg/filter/_fixtures/simple_emit_alert.yml b/pkg/rules/_fixtures/simple_emit_alert.yml similarity index 100% rename from pkg/filter/_fixtures/simple_emit_alert.yml rename to pkg/rules/_fixtures/simple_emit_alert.yml diff --git a/pkg/filter/_fixtures/simple_matches.yml b/pkg/rules/_fixtures/simple_matches.yml similarity index 100% rename from pkg/filter/_fixtures/simple_matches.yml rename to pkg/rules/_fixtures/simple_matches.yml diff --git a/pkg/filter/_fixtures/simple_matches/filter1.yml b/pkg/rules/_fixtures/simple_matches/filter1.yml similarity index 100% rename from pkg/filter/_fixtures/simple_matches/filter1.yml rename to pkg/rules/_fixtures/simple_matches/filter1.yml diff --git a/pkg/filter/_fixtures/simple_matches/filter2.yml b/pkg/rules/_fixtures/simple_matches/filter2.yml similarity index 100% rename from pkg/filter/_fixtures/simple_matches/filter2.yml rename to pkg/rules/_fixtures/simple_matches/filter2.yml diff --git a/pkg/filter/_fixtures/simple_matches/filter3.yml b/pkg/rules/_fixtures/simple_matches/filter3.yml similarity index 100% rename from pkg/filter/_fixtures/simple_matches/filter3.yml rename to pkg/rules/_fixtures/simple_matches/filter3.yml diff --git a/pkg/filter/_fixtures/simple_matches/filter4.yml b/pkg/rules/_fixtures/simple_matches/filter4.yml similarity index 100% rename from pkg/filter/_fixtures/simple_matches/filter4.yml rename to pkg/rules/_fixtures/simple_matches/filter4.yml diff --git a/pkg/filter/action/emit.go b/pkg/rules/action/alert.go similarity index 91% rename from pkg/filter/action/emit.go rename to pkg/rules/action/alert.go index 144898148..e6edfe218 100644 --- a/pkg/filter/action/emit.go +++ b/pkg/rules/action/alert.go @@ -27,8 +27,8 @@ import ( "strings" ) -// Emit sends the rule alert via all configured alert senders. -func Emit(ctx *config.ActionContext, title string, text string, severity string, tags []string) error { +// Alert sends the rule alert via all configured alert senders. +func Alert(ctx *config.ActionContext, title string, text string, severity string, tags []string) error { var b strings.Builder for _, evt := range ctx.Events { b.WriteString(evt.String()) diff --git a/pkg/filter/action/kill_windows.go b/pkg/rules/action/kill_windows.go similarity index 100% rename from pkg/filter/action/kill_windows.go rename to pkg/rules/action/kill_windows.go diff --git a/pkg/rules/compiler.go b/pkg/rules/compiler.go new file mode 100644 index 000000000..35b8919f6 --- /dev/null +++ b/pkg/rules/compiler.go @@ -0,0 +1,173 @@ +/* + * Copyright 2021-present by Nedim Sabic Sabic + * https://www.fibratus.io + * All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rules + +import ( + "expvar" + "fmt" + semver "github.com/hashicorp/go-version" + "github.com/rabbitstack/fibratus/pkg/config" + "github.com/rabbitstack/fibratus/pkg/filter" + "github.com/rabbitstack/fibratus/pkg/filter/fields" + "github.com/rabbitstack/fibratus/pkg/kevent/ktypes" + "github.com/rabbitstack/fibratus/pkg/ps" + "github.com/rabbitstack/fibratus/pkg/util/version" + log "github.com/sirupsen/logrus" +) + +var ( + // filtersCount computes the total number of filters in the ruleset + filtersCount = expvar.NewInt("filter.filters.count") + + ErrInvalidFilter = func(rule string, err error) error { + return fmt.Errorf("syntax error in rule %q: \n%v", rule, err) + } + ErrIncompatibleFilter = func(rule, v string) error { + return fmt.Errorf("rule %q needs engine version [%s] but current version is [%s]", rule, v, version.Get()) + } + ErrMalformedMinEngineVer = func(rule, v string, err error) error { + return fmt.Errorf("rule %q has a malformed minimum engine version: %s: %v", rule, v, err) + } +) + +type compiler struct { + psnap ps.Snapshotter + config *config.Config +} + +func newCompiler(psnap ps.Snapshotter, config *config.Config) *compiler { + return &compiler{psnap: psnap, config: config} +} + +func (c *compiler) compile() (map[*config.FilterConfig]filter.Filter, *config.RulesCompileResult, error) { + if err := c.config.Filters.LoadMacros(); err != nil { + return nil, nil, err + } + if err := c.config.Filters.LoadFilters(); err != nil { + return nil, nil, err + } + + filters := make(map[*config.FilterConfig]filter.Filter) + + for _, f := range c.config.GetFilters() { + if f.IsDisabled() { + log.Warnf("[%s] rule is disabled", f.Name) + continue + } + + filtersCount.Add(1) + + // compile the filter + fltr := filter.New(f.Condition, c.config, filter.WithPSnapshotter(c.psnap)) + err := fltr.Compile() + if err != nil { + return nil, nil, ErrInvalidFilter(f.Name, err) + } + // check version requirements + if !version.IsDev() { + minEngineVer, err := semver.NewSemver(f.MinEngineVersion) + if err != nil { + return nil, nil, ErrMalformedMinEngineVer(f.Name, f.MinEngineVersion, err) + } + if minEngineVer.GreaterThan(version.Sem()) { + return nil, nil, ErrIncompatibleFilter(f.Name, f.MinEngineVersion) + } + } + // output warning for deprecated fields + for _, field := range fltr.GetFields() { + deprecated, d := fields.IsDeprecated(field.Name) + if deprecated { + log.Warnf("%s rule uses the [%s] field which "+ + "was deprecated starting from version %s. "+ + "Please consider migrating to %s field(s) "+ + "because [%s] will be removed in future versions.", + f.Name, field, d.Since, d.Fields, field) + } + } + + filters[f] = fltr + } + + if len(filters) == 0 { + return filters, nil, nil + } + + return filters, c.buildCompileResult(filters), nil +} + +func (c *compiler) buildCompileResult(filters map[*config.FilterConfig]filter.Filter) *config.RulesCompileResult { + rs := &config.RulesCompileResult{} + + m := make(map[ktypes.Ktype]bool) + events := make([]ktypes.Ktype, 0) + + for _, f := range filters { + rs.NumberRules++ + for name, values := range f.GetStringFields() { + for _, v := range values { + if name == fields.KevtName || name == fields.KevtCategory { + types := ktypes.KeventNameToKtypes(v) + for _, typ := range types { + switch typ.Category() { + case ktypes.Process: + rs.HasProcEvents = true + case ktypes.Thread: + rs.HasThreadEvents = true + case ktypes.Image: + rs.HasImageEvents = true + case ktypes.File: + rs.HasFileEvents = true + case ktypes.Net: + rs.HasNetworkEvents = true + case ktypes.Registry: + rs.HasRegistryEvents = true + case ktypes.Mem: + rs.HasMemEvents = true + case ktypes.Handle: + rs.HasHandleEvents = true + case ktypes.Threadpool: + rs.HasThreadpoolEvents = true + } + if typ.Subcategory() == ktypes.DNS { + rs.HasDNSEvents = true + } + if typ == ktypes.MapViewFile || typ == ktypes.UnmapViewFile { + rs.HasVAMapEvents = true + } + if typ == ktypes.OpenProcess || typ == ktypes.OpenThread || typ == ktypes.SetThreadContext || + typ == ktypes.CreateSymbolicLinkObject { + rs.HasAuditAPIEvents = true + } + + if m[typ] { + continue + } + + events = append(events, typ) + m[typ] = true + } + } + } + } + } + + rs.UsedEvents = events + + return rs +} diff --git a/pkg/rules/compiler_test.go b/pkg/rules/compiler_test.go new file mode 100644 index 000000000..f093713fd --- /dev/null +++ b/pkg/rules/compiler_test.go @@ -0,0 +1,72 @@ +/* + * Copyright 2021-present by Nedim Sabic Sabic + * https://www.fibratus.io + * All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rules + +import ( + "github.com/rabbitstack/fibratus/pkg/kevent/ktypes" + "github.com/rabbitstack/fibratus/pkg/ps" + "github.com/rabbitstack/fibratus/pkg/util/version" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "testing" +) + +func TestCompile(t *testing.T) { + c := newCompiler(new(ps.SnapshotterMock), newConfig("_fixtures/default/*.yml")) + filters, rs, err := c.compile() + require.NoError(t, err) + require.NotNil(t, rs) + require.Len(t, filters, 6) + + assert.True(t, rs.HasImageEvents) + assert.True(t, rs.HasProcEvents) + assert.False(t, rs.HasMemEvents) + assert.False(t, rs.HasAuditAPIEvents) + assert.True(t, rs.HasDNSEvents) + assert.Contains(t, rs.UsedEvents, ktypes.CreateProcess) + assert.Contains(t, rs.UsedEvents, ktypes.LoadImage) + assert.Contains(t, rs.UsedEvents, ktypes.QueryDNS) + assert.Contains(t, rs.UsedEvents, ktypes.ConnectTCPv4) + assert.Contains(t, rs.UsedEvents, ktypes.ConnectTCPv6) +} + +func TestCompileMinEngineVersion(t *testing.T) { + var tests = []struct { + rules string + ver string + e string + }{ + {"_fixtures/min_engine_version/fail/*.yml", "2.0.0", `rule "accept events where source port = 44123" needs engine version [2.2.0] but current version is [2.0.0]`}, + {"_fixtures/min_engine_version/ok/*.yml", "2.0.0", ""}, + } + + for _, tt := range tests { + t.Run(tt.rules, func(t *testing.T) { + c := newCompiler(new(ps.SnapshotterMock), newConfig(tt.rules)) + version.Set(tt.ver) + _, _, err := c.compile() + if err != nil && tt.e == "" { + require.Error(t, err) + } + if err != nil { + require.EqualError(t, err, tt.e) + } + }) + } +} diff --git a/pkg/rules/doc.go b/pkg/rules/doc.go new file mode 100644 index 000000000..ad5f7b5dc --- /dev/null +++ b/pkg/rules/doc.go @@ -0,0 +1,21 @@ +/* + * Copyright 2021-present by Nedim Sabic Sabic + * https://www.fibratus.io + * All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// Package rules provides the implementation of the rule engine for both +// simple filter expressions and sequences. +package rules diff --git a/pkg/rules/engine.go b/pkg/rules/engine.go new file mode 100644 index 000000000..3a2a812d2 --- /dev/null +++ b/pkg/rules/engine.go @@ -0,0 +1,355 @@ +/* + * Copyright 2021-present by Nedim Sabic Sabic + * https://www.fibratus.io + * All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rules + +import ( + "expvar" + "fmt" + "github.com/rabbitstack/fibratus/pkg/config" + "github.com/rabbitstack/fibratus/pkg/filter" + "github.com/rabbitstack/fibratus/pkg/filter/fields" + "github.com/rabbitstack/fibratus/pkg/kevent" + "github.com/rabbitstack/fibratus/pkg/kevent/ktypes" + "github.com/rabbitstack/fibratus/pkg/ps" + "github.com/rabbitstack/fibratus/pkg/rules/action" + "github.com/rabbitstack/fibratus/pkg/util/hashers" + log "github.com/sirupsen/logrus" + "sync" + "time" +) + +// RuleMatchFunc is rule match function definition. It accepts +// the filter (rule) config and the group of events that fired +// the rule +type RuleMatchFunc func(f *config.FilterConfig, evts ...*kevent.Kevent) + +var ( + // sequenceGcInterval determines how often sequence GC kicks in + sequenceGcInterval = time.Minute + + filterMatches = expvar.NewMap("filter.matches") + + ErrRuleAction = func(rule string, err error) error { + return fmt.Errorf("fail to execute action for %q rule: %v", rule, err) + } +) + +// Engine asserts the full-fledged system event against +// the collection of compiled filters that are derived +// from the loaded ruleset. +type Engine struct { + filters compiledFilters + config *config.Config + psnap ps.Snapshotter + + matches []*ruleMatch + sequences []*sequenceState + + scavenger *time.Ticker + + compiler *compiler + + hashCache *hashCache + + matchFunc RuleMatchFunc +} + +type ruleMatch struct { + ctx *config.ActionContext +} + +// hashCache caches the event type/category FNV hashes +type hashCache struct { + mu sync.RWMutex + types map[ktypes.Ktype]uint32 + cats map[ktypes.Category]uint32 + lookupCategory bool +} + +func newHashCache() *hashCache { + return &hashCache{types: make(map[ktypes.Ktype]uint32), cats: make(map[ktypes.Category]uint32)} +} + +func (c *hashCache) typeHash(e *kevent.Kevent) uint32 { + c.mu.RLock() + defer c.mu.RUnlock() + return c.types[e.Type] +} + +func (c *hashCache) categoryHash(e *kevent.Kevent) uint32 { + c.mu.RLock() + defer c.mu.RUnlock() + return c.cats[e.Category] +} + +func (c *hashCache) addTypeHash(e *kevent.Kevent) uint32 { + c.mu.Lock() + defer c.mu.Unlock() + h := e.Type.Hash() + c.types[e.Type] = h + return h +} + +func (c *hashCache) addCategoryHash(e *kevent.Kevent) uint32 { + c.mu.Lock() + defer c.mu.Unlock() + h := e.Category.Hash() + c.cats[e.Category] = h + return h +} + +type compiledFilter struct { + filter filter.Filter + config *config.FilterConfig + ss *sequenceState +} + +type compiledFilters map[uint32][]*compiledFilter + +// collect collects all compiled filters for a +// particular event type or category. If no filters +// are found, the event is not asserted against the +// ruleset. +func (filters compiledFilters) collect(hashCache *hashCache, e *kevent.Kevent) []*compiledFilter { + h := hashCache.typeHash(e) + if h == 0 { + h = hashCache.addTypeHash(e) + } + + if !hashCache.lookupCategory { + return filters[h] + } + + c := hashCache.categoryHash(e) + if c == 0 { + c = hashCache.addCategoryHash(e) + } + return append(filters[h], filters[c]...) +} + +func newCompiledFilter(f filter.Filter, c *config.FilterConfig, ss *sequenceState) *compiledFilter { + return &compiledFilter{filter: f, config: c, ss: ss} +} + +// isScoped determines if this filter is scoped, i.e. it has the event name or category +// conditions. +func (f *compiledFilter) isScoped() bool { + for name := range f.filter.GetStringFields() { + if name == fields.KevtName || name == fields.KevtCategory { + return true + } + } + return false +} + +func (f *compiledFilter) isSequence() bool { + return f.ss != nil +} + +func (f *compiledFilter) run(e *kevent.Kevent) bool { + if f.ss != nil { + return f.ss.runSequence(e) + } + return f.filter.Run(e) +} + +// NewEngine builds a fresh rules engine instance. +func NewEngine(psnap ps.Snapshotter, config *config.Config) *Engine { + e := &Engine{ + filters: make(map[uint32][]*compiledFilter), + matches: make([]*ruleMatch, 0), + sequences: make([]*sequenceState, 0), + psnap: psnap, + config: config, + scavenger: time.NewTicker(sequenceGcInterval), + compiler: newCompiler(psnap, config), + hashCache: newHashCache(), + } + + go e.gcSequences() + + return e +} + +func (e *Engine) gcSequences() { + for { + <-e.scavenger.C + for _, seq := range e.sequences { + seq.gc() + } + } +} + +// Compile loads macros/rules and builds an indexable filter set. +// For every rule in the ruleset the condition is compiled and +// converted into a filter. The filter is indexed by either the +// event name or event category. +func (e *Engine) Compile() (*config.RulesCompileResult, error) { + filters, rs, err := e.compiler.compile() + if err != nil { + return nil, err + } + + for c, f := range filters { + var ss *sequenceState + if f.IsSequence() { + ss = newSequenceState(f, c) + } + fltr := newCompiledFilter(f, c, ss) + if ss != nil { + // store the sequences in engine + // for more convenient tracking + e.sequences = append(e.sequences, ss) + } + if !fltr.isScoped() { + log.Warnf("%q rule doesn't have "+ + "event type or event category condition! "+ + "This rule is being discarded by "+ + "the engine. Please consider narrowing the "+ + "scope of the rule by including the `kevt.name` "+ + "or `kevt.category` condition", + c.Name) + continue + } + // traverse all event name or category fields and determine + // the event type from the filter field name expression. + // We end up with a map of rules indexed by event name + // or event category hash + for name, values := range f.GetStringFields() { + for _, v := range values { + if name == fields.KevtName || name == fields.KevtCategory { + if name == fields.KevtCategory { + e.hashCache.lookupCategory = true + } + hash := hashers.FnvUint32([]byte(v)) + e.filters[hash] = append(e.filters[hash], fltr) + } + } + } + } + + return rs, nil +} + +func (e *Engine) RegisterMatchFunc(fn RuleMatchFunc) { + e.matchFunc = fn +} + +func (*Engine) CanEnqueue() bool { return true } + +// ProcessEvent processes the system event against compiled filters. +// Filter is the internal lingo that designates a rule condition. +// Filters can be simple direct-event matchers or sequence states that +// track an ordered series of events over a short period of time. +func (e *Engine) ProcessEvent(evt *kevent.Kevent) (bool, error) { + if len(e.filters) == 0 { + return true, nil + } + var matches bool + if evt.IsTerminateProcess() { + // expire all sequences if the + // process referenced in any + // partials has terminated + for _, seq := range e.sequences { + seq.expire(evt) + } + } + filters := e.filters.collect(e.hashCache, evt) + for _, f := range filters { + match := f.run(evt) + if !match { + continue + } + if f.isSequence() { + e.appendMatch(f.config, f.ss.events()...) + f.ss.clearLocked() + } else { + e.appendMatch(f.config, evt) + } + err := e.processActions() + if err != nil { + log.Errorf("unable to execute rule action: %v", err) + } + switch { + case e.config.Filters.MatchAll: + matches = true + default: + return true, nil + } + } + return matches, nil +} + +// processActions executes rule actions +// on behalf of rule matches. Actions are +// categorized into implicit and explicit +// actions. +// Sending an alert is an implicit action +// carried out each time there is a rule +// match. Other actions are executed if +// defined in the rule definition. +func (e *Engine) processActions() error { + defer e.clearMatches() + for _, m := range e.matches { + f, evts := m.ctx.Filter, m.ctx.Events + filterMatches.Add(f.Name, 1) + log.Debugf("[%s] rule matched", f.Name) + err := action.Alert(m.ctx, f.Name, filter.InterpolateFields(f.Output, evts), f.Severity, f.Tags) + if err != nil { + return ErrRuleAction(f.Name, err) + } + + actions, err := f.DecodeActions() + if err != nil { + return err + } + + for _, act := range actions { + switch act.(type) { + case config.KillAction: + log.Infof("executing kill action: pids=%v rule=%s", m.ctx.UniquePids(), f.Name) + if err := action.Kill(m.ctx.UniquePids()); err != nil { + return ErrRuleAction(f.Name, err) + } + } + } + } + return nil +} + +func (e *Engine) appendMatch(f *config.FilterConfig, evts ...*kevent.Kevent) { + for _, evt := range evts { + evt.AddMeta(kevent.RuleNameKey, f.Name) + for k, v := range f.Labels { + evt.AddMeta(kevent.MetadataKey(k), v) + } + } + ctx := &config.ActionContext{ + Events: evts, + Filter: f, + } + e.matches = append(e.matches, &ruleMatch{ctx: ctx}) + if e.matchFunc != nil { + e.matchFunc(f, evts...) + } +} + +func (e *Engine) clearMatches() { + e.matches = make([]*ruleMatch, 0) +} diff --git a/pkg/rules/engine_test.go b/pkg/rules/engine_test.go new file mode 100644 index 000000000..37782053a --- /dev/null +++ b/pkg/rules/engine_test.go @@ -0,0 +1,584 @@ +/* + * Copyright 2021-present by Nedim Sabic Sabic + * https://www.fibratus.io + * All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rules + +import ( + "github.com/rabbitstack/fibratus/pkg/alertsender" + "github.com/rabbitstack/fibratus/pkg/config" + "github.com/rabbitstack/fibratus/pkg/fs" + "github.com/rabbitstack/fibratus/pkg/kevent" + "github.com/rabbitstack/fibratus/pkg/kevent/kparams" + "github.com/rabbitstack/fibratus/pkg/kevent/ktypes" + "github.com/rabbitstack/fibratus/pkg/ps" + "github.com/rabbitstack/fibratus/pkg/ps/types" + "github.com/rabbitstack/fibratus/pkg/sys" + log "github.com/sirupsen/logrus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "golang.org/x/sys/windows" + "net" + "os" + "testing" + "time" +) + +type mockNoopSender struct{} +type mockNoneSender struct{} + +var emitAlert *alertsender.Alert +var seqAlert *alertsender.Alert + +func (s *mockNoopSender) Send(a alertsender.Alert) error { + emitAlert = &a + return nil +} + +func (s *mockNoopSender) Type() alertsender.Type { + return alertsender.Noop +} + +func (s *mockNoopSender) Shutdown() error { return nil } +func (s *mockNoopSender) SupportsMarkdown() bool { return true } + +func makeNoopSender(config alertsender.Config) (alertsender.Sender, error) { + return &mockNoopSender{}, nil +} + +func (s *mockNoneSender) Send(a alertsender.Alert) error { + seqAlert = &a + return nil +} + +func (s *mockNoneSender) Type() alertsender.Type { + return alertsender.None +} + +func (s *mockNoneSender) Shutdown() error { return nil } +func (s *mockNoneSender) SupportsMarkdown() bool { return true } + +func makeNoneSender(config alertsender.Config) (alertsender.Sender, error) { + return &mockNoneSender{}, nil +} + +func init() { + alertsender.Register(alertsender.Noop, makeNoopSender) + alertsender.Register(alertsender.None, makeNoneSender) +} + +func newConfig(fromFiles ...string) *config.Config { + var kstreamConfig = config.KstreamConfig{ + EnableHandleKevents: true, + EnableNetKevents: true, + EnableRegistryKevents: true, + EnableFileIOKevents: true, + EnableImageKevents: true, + EnableThreadKevents: true, + } + c := &config.Config{ + Kstream: kstreamConfig, + Filters: &config.Filters{ + Rules: config.Rules{ + FromPaths: fromFiles, + }, + }, + } + return c +} + +func compileRules(t *testing.T, e *Engine) { + rs, err := e.Compile() + require.NoError(t, err) + require.NotNil(t, rs) +} + +func wrapProcessEvent(e *kevent.Kevent, fn func(*kevent.Kevent) (bool, error)) bool { + match, err := fn(e) + if err != nil { + panic(err) + } + return match +} + +func fireRules(t *testing.T, c *config.Config) bool { + e := NewEngine(new(ps.SnapshotterMock), c) + evt := &kevent.Kevent{ + Type: ktypes.RecvTCPv4, + Name: "Recv", + Tid: 2484, + PID: 859, + Category: ktypes.Net, + Kparams: kevent.Kparams{ + kparams.NetDport: {Name: kparams.NetDport, Type: kparams.Uint16, Value: uint16(443)}, + kparams.NetSport: {Name: kparams.NetSport, Type: kparams.Uint16, Value: uint16(43123)}, + kparams.NetSIP: {Name: kparams.NetSIP, Type: kparams.IPv4, Value: net.ParseIP("127.0.0.1")}, + kparams.NetDIP: {Name: kparams.NetDIP, Type: kparams.IPv4, Value: net.ParseIP("216.58.201.174")}, + }, + Metadata: make(map[kevent.MetadataKey]any), + } + compileRules(t, e) + return wrapProcessEvent(evt, e.ProcessEvent) +} + +func TestCompileIndexableFilters(t *testing.T) { + e := NewEngine(new(ps.SnapshotterMock), newConfig( + "_fixtures/merged_filters/filter*.yml", + "_fixtures/default/microsoft_edge.yml", + "_fixtures/default/windows_error_*.yml")) + + compileRules(t, e) + + assert.Len(t, e.filters, 3) + + var tests = []struct { + evt *kevent.Kevent + wants int + }{ + {&kevent.Kevent{Type: ktypes.CreateProcess}, 2}, + {&kevent.Kevent{Type: ktypes.RecvUDPv6}, 3}, + {&kevent.Kevent{Type: ktypes.RecvTCPv4}, 3}, + {&kevent.Kevent{Type: ktypes.RecvTCPv4, Category: ktypes.Net}, 4}, + {&kevent.Kevent{Category: ktypes.Net}, 1}, + } + + for _, tt := range tests { + t.Run(tt.evt.Type.String(), func(t *testing.T) { + assert.Len(t, e.filters.collect(e.hashCache, tt.evt), tt.wants) + }) + } + + assert.Len(t, e.hashCache.types, 4) + + evt := &kevent.Kevent{Type: ktypes.RecvTCPv4} + + h1, h2 := e.hashCache.typeHash(evt), e.hashCache.categoryHash(evt) + assert.Equal(t, uint32(0xfa4dab59), h1) + assert.Equal(t, uint32(0x811c9dc5), h2) +} + +func TestRunSimpleRules(t *testing.T) { + var tests = []struct { + config *config.Config + matches bool + }{ + {newConfig("_fixtures/simple_matches.yml"), true}, + {newConfig("_fixtures/simple_matches/filter*.yml"), true}, + } + + for i, tt := range tests { + matches := fireRules(t, tt.config) + if matches != tt.matches { + t.Errorf("%d. %v process rules mismatch: exp=%t got=%t", i, tt.config.Filters, tt.matches, matches) + } + } +} + +func TestRunSequenceRule(t *testing.T) { + log.SetLevel(log.DebugLevel) + + e := NewEngine(new(ps.SnapshotterMock), newConfig("_fixtures/sequence_rule_complex.yml")) + compileRules(t, e) + + e1 := &kevent.Kevent{ + Seq: 1, + Type: ktypes.CreateProcess, + Timestamp: time.Now(), + Category: ktypes.Process, + Name: "CreateProcess", + Tid: 2484, + PID: 859, + PS: &types.PS{ + Name: "explorer.exe", + Exe: "C:\\Windows\\system32\\explorer.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(2243)}, + kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "firefox.exe"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + e2 := &kevent.Kevent{ + Seq: 2, + Type: ktypes.CreateFile, + Timestamp: time.Now().Add(time.Millisecond * 250), + Name: "CreateFile", + Tid: 2484, + PID: 2243, + Category: ktypes.File, + PS: &types.PS{ + Name: "firefox.exe", + Exe: "C:\\Program Files\\Mozilla Firefox\\firefox.exe", + Cmdline: "C:\\Program Files\\Mozilla Firefox\\firefox.exe\" -contentproc --channel=\"10464.7.539748228\\1366525930\" -childID 6 -isF", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper.exe"}, + kparams.FileOperation: {Name: kparams.FileOperation, Type: kparams.Enum, Value: uint32(2), Enum: fs.FileCreateDispositions}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + e3 := &kevent.Kevent{ + Seq: 4, + Type: ktypes.ConnectTCPv4, + Timestamp: time.Now().Add(time.Second), + Category: ktypes.Net, + Name: "Connect", + Tid: 244, + PID: 2243, + PS: &types.PS{ + Name: "firefox.exe", + Exe: "C:\\Program Files\\Mozilla Firefox\\firefox.exe", + Cmdline: "C:\\Program Files\\Mozilla Firefox\\firefox.exe\" -contentproc --channel=\"10464.7.539748228\\1366525930\" -childID 6 -isF", + }, + Kparams: kevent.Kparams{ + kparams.NetDIP: {Name: kparams.NetDIP, Type: kparams.IPv4, Value: net.ParseIP("10.0.2.3")}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + // register alert sender + require.NoError(t, alertsender.LoadAll([]alertsender.Config{{Type: alertsender.None}})) + + require.False(t, wrapProcessEvent(e1, e.ProcessEvent)) + require.False(t, wrapProcessEvent(e2, e.ProcessEvent)) + + time.Sleep(time.Millisecond * 30) + require.True(t, wrapProcessEvent(e3, e.ProcessEvent)) + + time.Sleep(time.Millisecond * 50) + + // check the format of the generated alert + require.NotNil(t, seqAlert) + assert.Equal(t, "572902be-76e9-4ee7-a48a-6275fa571cf4", seqAlert.ID) + assert.Len(t, seqAlert.Events, 3) + assert.Equal(t, "Phishing dropper outbound communication", seqAlert.Title) + assert.Equal(t, "firefox.exe process initiated outbound communication to 10.0.2.3", seqAlert.Text) + seqAlert = nil + + // FSM should transition from terminal to initial state + require.False(t, wrapProcessEvent(e1, e.ProcessEvent)) + require.False(t, wrapProcessEvent(e2, e.ProcessEvent)) + time.Sleep(time.Millisecond * 15) + require.True(t, wrapProcessEvent(e3, e.ProcessEvent)) +} + +func TestRunSequenceRuleWithPsUUIDLink(t *testing.T) { + log.SetLevel(log.DebugLevel) + + e := NewEngine(new(ps.SnapshotterMock), newConfig("_fixtures/sequence_rule_ps_uuid.yml")) + compileRules(t, e) + + e1 := &kevent.Kevent{ + Seq: 1, + Type: ktypes.CreateProcess, + Timestamp: time.Now(), + Category: ktypes.Process, + Name: "CreateProcess", + Tid: 2484, + PID: uint32(os.Getpid()), + PS: &types.PS{ + PID: uint32(os.Getpid()), + Name: "explorer.exe", + Exe: "C:\\Windows\\system32\\explorer.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(2243)}, + kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "firefox.exe"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + e2 := &kevent.Kevent{ + Seq: 2, + Type: ktypes.CreateFile, + Timestamp: time.Now(), + Name: "CreateFile", + Tid: 2484, + PID: uint32(os.Getpid()), + Category: ktypes.File, + PS: &types.PS{ + PID: uint32(os.Getpid()), + Name: "firefox.exe", + Exe: "C:\\Program Files\\Mozilla Firefox\\firefox.exe", + Cmdline: "C:\\Program Files\\Mozilla Firefox\\firefox.exe\" -contentproc --channel=\"10464.7.539748228\\1366525930\" -childID 6 -isF", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper.exe"}, + kparams.FileOperation: {Name: kparams.FileOperation, Type: kparams.Enum, Value: uint32(2), Enum: fs.FileCreateDispositions}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + require.False(t, wrapProcessEvent(e1, e.ProcessEvent)) + require.True(t, wrapProcessEvent(e2, e.ProcessEvent)) +} + +func TestRunSimpleAndSequenceRules(t *testing.T) { + log.SetLevel(log.DebugLevel) + + expectedMatches := make(map[string][]uint64) + c := newConfig("_fixtures/simple_and_sequence_rules/*.yml") + c.Filters.MatchAll = true + e := NewEngine(new(ps.SnapshotterMock), c) + e.RegisterMatchFunc(func(f *config.FilterConfig, evts ...*kevent.Kevent) { + ids := make([]uint64, 0) + for _, evt := range evts { + ids = append(ids, evt.Seq) + } + expectedMatches[f.Name] = ids + }) + + compileRules(t, e) + + evts := []*kevent.Kevent{ + { + Seq: 1, + Type: ktypes.CreateProcess, + Timestamp: time.Now(), + Category: ktypes.Process, + Name: "CreateProcess", + Tid: 2484, + PID: 2243, + PS: &types.PS{ + Name: "powershell.exe", + Exe: "C:\\Windows\\system32\\powershell.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(2243)}, + kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "firefox.exe"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + }, + { + Seq: 2, + Type: ktypes.CreateFile, + Timestamp: time.Now().Add(time.Millisecond * 544), + Name: "CreateFile", + Tid: 2484, + PID: 2243, + Category: ktypes.File, + PS: &types.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\cmd.exe", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper.exe"}, + kparams.FileOperation: {Name: kparams.FileOperation, Type: kparams.Enum, Value: uint32(2)}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + }, + { + Seq: 10, + Type: ktypes.CreateProcess, + Timestamp: time.Now().Add(time.Second * 2), + Category: ktypes.Process, + Name: "CreateProcess", + Tid: 2484, + PID: 2243, + PS: &types.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\cmd.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(2243)}, + kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "chrome.exe"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + }, + } + + for _, evt := range evts { + require.True(t, wrapProcessEvent(evt, e.ProcessEvent)) + } + + assert.Len(t, expectedMatches, 4) + + var tests = []struct { + rule string + eventIDs []uint64 + }{ + {"Process spawned by powershell", []uint64{1}}, + {"Powershell created a temp file", []uint64{1, 2}}, + {"Spawn Chrome browser", []uint64{10}}, + {"Command shell spawned Chrome browser", []uint64{1, 10}}, + } + + for _, tt := range tests { + t.Run(tt.rule, func(t *testing.T) { + assert.Equal(t, expectedMatches[tt.rule], tt.eventIDs) + }) + } +} + +func TestAlertAction(t *testing.T) { + require.NoError(t, alertsender.LoadAll([]alertsender.Config{{Type: alertsender.Noop}})) + e := NewEngine(new(ps.SnapshotterMock), newConfig("_fixtures/simple_emit_alert.yml")) + compileRules(t, e) + + evt := &kevent.Kevent{ + Type: ktypes.RecvTCPv4, + Name: "Recv", + Tid: 2484, + PID: 859, + Category: ktypes.Net, + PS: &types.PS{ + Name: "cmd.exe", + }, + Kparams: kevent.Kparams{ + kparams.NetDport: {Name: kparams.NetDport, Type: kparams.Uint16, Value: uint16(443)}, + kparams.NetSport: {Name: kparams.NetSport, Type: kparams.Uint16, Value: uint16(43123)}, + kparams.NetSIP: {Name: kparams.NetSIP, Type: kparams.IPv4, Value: net.ParseIP("127.0.0.1")}, + kparams.NetDIP: {Name: kparams.NetDIP, Type: kparams.IPv4, Value: net.ParseIP("216.58.201.174")}, + }, + Metadata: make(map[kevent.MetadataKey]any), + } + + require.True(t, wrapProcessEvent(evt, e.ProcessEvent)) + time.Sleep(time.Millisecond * 25) + require.NotNil(t, emitAlert) + assert.Equal(t, "match https connections", emitAlert.Title) + assert.Equal(t, "cmd.exe process received data on port 443", emitAlert.Text) + assert.Equal(t, alertsender.Critical, emitAlert.Severity) + assert.Equal(t, []string{"tag1", "tag2"}, emitAlert.Tags) + emitAlert = nil +} + +func TestKillAction(t *testing.T) { + log.SetLevel(log.DebugLevel) + + e := NewEngine(new(ps.SnapshotterMock), newConfig("_fixtures/kill_action.yml")) + compileRules(t, e) + + // register alert sender + require.NoError(t, alertsender.LoadAll([]alertsender.Config{{Type: alertsender.None}})) + + var si windows.StartupInfo + var pi windows.ProcessInformation + argv, err := windows.UTF16PtrFromString("calc.exe") + require.NoError(t, err) + err = windows.CreateProcess( + nil, + argv, + nil, + nil, + true, + 0, + nil, + nil, + &si, + &pi) + require.NoError(t, err) + + i := 0 + for !sys.IsProcessRunning(pi.Process) && i < 10 { + i++ + time.Sleep(time.Millisecond * 100 * time.Duration(i)) + } + + evt := &kevent.Kevent{ + Type: ktypes.CreateProcess, + Timestamp: time.Now(), + Name: "CreateProcess", + Tid: 2484, + PID: 859, + Category: ktypes.Process, + PS: &types.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost-temp.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: pi.ProcessId}, + kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "calc.exe"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + require.True(t, sys.IsProcessRunning(pi.Process)) + require.True(t, wrapProcessEvent(evt, e.ProcessEvent)) + require.False(t, sys.IsProcessRunning(pi.Process)) +} + +func BenchmarkRunRules(b *testing.B) { + b.ReportAllocs() + e := NewEngine(new(ps.SnapshotterMock), newConfig("_fixtures/default/*.yml")) + rs, err := e.Compile() + require.NoError(b, err) + require.NotNil(b, rs) + + b.ResetTimer() + + evts := []*kevent.Kevent{ + { + Type: ktypes.ConnectTCPv4, + Name: "Recv", + Tid: 2484, + PID: 859, + Category: ktypes.Net, + PS: &types.PS{ + Name: "cmd.exe", + }, + Kparams: kevent.Kparams{ + kparams.NetDport: {Name: kparams.NetDport, Type: kparams.Uint16, Value: uint16(443)}, + kparams.NetSport: {Name: kparams.NetSport, Type: kparams.Uint16, Value: uint16(43123)}, + kparams.NetSIP: {Name: kparams.NetSIP, Type: kparams.IPv4, Value: net.ParseIP("127.0.0.1")}, + kparams.NetDIP: {Name: kparams.NetDIP, Type: kparams.IPv4, Value: net.ParseIP("216.58.201.174")}, + }, + Metadata: make(map[kevent.MetadataKey]any), + }, + { + Type: ktypes.CreateProcess, + Name: "CreateProcess", + Category: ktypes.Process, + Tid: 2484, + PID: 859, + PS: &types.PS{ + Name: "powershell.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: 2323}, + kparams.ProcessParentID: {Name: kparams.ProcessParentID, Type: kparams.PID, Value: uint32(8390)}, + kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "spotify.exe"}, + kparams.Cmdline: {Name: kparams.Cmdline, Type: kparams.UnicodeString, Value: `C:\Users\admin\AppData\Roaming\Spotify\Spotify.exe --type=crashpad-handler /prefetch:7 --max-uploads=5 --max-db-size=20 --max-db-age=5 --monitor-self-annotation=ptype=crashpad-handler "--metrics-dir=C:\Users\admin\AppData\Local\Spotify\User Data" --url=https://crashdump.spotify.com:443/ --annotation=platform=win32 --annotation=product=spotify --annotation=version=1.1.4.197 --initial-client-data=0x5a4,0x5a0,0x5a8,0x59c,0x5ac,0x6edcbf60,0x6edcbf70,0x6edcbf7c`}, + kparams.Exe: {Name: kparams.Exe, Type: kparams.UnicodeString, Value: `C:\Users\admin\AppData\Roaming\Spotify\Spotify.exe`}, + kparams.UserSID: {Name: kparams.UserSID, Type: kparams.UnicodeString, Value: `admin\SYSTEM`}, + }, + Metadata: make(map[kevent.MetadataKey]any), + }, + { + Type: ktypes.CreateHandle, + Name: "CreateHandle", + Category: ktypes.Handle, + Tid: 2484, + PID: 859, + PS: &types.PS{ + Name: "powershell.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: 2323}, + }, + Metadata: make(map[kevent.MetadataKey]any), + }, + } + + for i := 0; i < b.N; i++ { + for _, evt := range evts { + _, _ = e.ProcessEvent(evt) + } + } +} diff --git a/pkg/rules/sequence.go b/pkg/rules/sequence.go new file mode 100644 index 000000000..28f181882 --- /dev/null +++ b/pkg/rules/sequence.go @@ -0,0 +1,591 @@ +/* + * Copyright 2021-present by Nedim Sabic Sabic + * https://www.fibratus.io + * All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rules + +import ( + "context" + "expvar" + fsm "github.com/qmuntal/stateless" + "github.com/rabbitstack/fibratus/pkg/config" + "github.com/rabbitstack/fibratus/pkg/filter" + "github.com/rabbitstack/fibratus/pkg/filter/ql" + "github.com/rabbitstack/fibratus/pkg/kevent" + "github.com/rabbitstack/fibratus/pkg/kevent/kparams" + "github.com/rabbitstack/fibratus/pkg/kevent/ktypes" + "github.com/rabbitstack/fibratus/pkg/util/atomic" + log "github.com/sirupsen/logrus" + "sort" + "sync" + "time" +) + +const ( + // maxOutstandingPartials determines the maximum number of partials per sequence index + maxOutstandingPartials = 1000 +) + +var ( + partialsPerSequence = expvar.NewMap("sequence.partials.count") + partialExpirations = expvar.NewMap("sequence.partial.expirations") + partialBreaches = expvar.NewMap("sequence.partial.breaches") + matchTransitionErrors = expvar.NewInt("sequence.match.transition.errors") + + // maxSequencePartialLifetime indicates the maximum time for the + // partial to exist in the sequence state. If the partial has been + // placed in the sequence state more than allowed, it is removed + maxSequencePartialLifetime = time.Hour * 4 +) + +var ( + // sequenceTerminalState represents the final state in the FSM. + // This state is transitioned when the last rule in the sequence + // produces a match + sequenceTerminalState = fsm.State("terminal") + // sequenceDeadlineState represents the state to which other + // states transition if the rule's max span is reached + sequenceDeadlineState = fsm.State("deadline") + // sequenceExpiredState designates the state to which other + // states transition when the sequence is expired + sequenceExpiredState = fsm.State("expired") + // sequenceInitialState represents the initial sequence state + sequenceInitialState = fsm.State(0) + + // transitions for match, cancel, reset, and expire triggers + matchTransition = fsm.Trigger("match") + cancelTransition = fsm.Trigger("cancel") + resetTransition = fsm.Trigger("reset") + expireTransition = fsm.Trigger("expire") +) + +// sequenceState represents the state of the +// ordered sequence of multiple events that +// may have time-frame constraints. A deterministic +// finite state machine tracks the matching status of +// each expression (state) in the machine. +type sequenceState struct { + filter filter.Filter + seq *ql.Sequence + name string + maxSpan time.Duration + + // partials keeps the state of all matched events per expression + partials map[int][]*kevent.Kevent + // mu guards the partials map + mu sync.RWMutex + + // matches stores only the event that matched + // the upstream partials. These events will + // be propagated in the rule action context + matches map[int]*kevent.Kevent + // mmu guards the matches map + mmu sync.RWMutex + + fsm *fsm.StateMachine + + // exprs stores the expression index to + // its respective string representation + exprs map[int]string + spanDeadlines map[fsm.State]*time.Timer + inDeadline atomic.Bool + inExpired atomic.Bool + initialState fsm.State + isPartialsBreached atomic.Bool + + // states keeps the mapping between expression + // index and its matching state. Whenever the expression + // evaluates to true the state is updated for the index + // pertaining to the expression sequence slot + states map[fsm.State]bool + // smu guards the states map + smu sync.RWMutex +} + +func newSequenceState(f filter.Filter, c *config.FilterConfig) *sequenceState { + ss := &sequenceState{ + filter: f, + seq: f.GetSequence(), + name: c.Name, + maxSpan: f.GetSequence().MaxSpan, + partials: make(map[int][]*kevent.Kevent), + states: make(map[fsm.State]bool), + matches: make(map[int]*kevent.Kevent), + exprs: make(map[int]string), + spanDeadlines: make(map[fsm.State]*time.Timer), + initialState: sequenceInitialState, + inDeadline: atomic.MakeBool(false), + } + + ss.initFSM() + + ss.configureFSM() + + return ss +} + +func (s *sequenceState) events() []*kevent.Kevent { + s.mmu.RLock() + defer s.mmu.RUnlock() + events := make([]*kevent.Kevent, 0, len(s.matches)) + for _, e := range s.matches { + events = append(events, e) + } + sort.Slice(events, func(i, j int) bool { return events[i].Timestamp.Before(events[j].Timestamp) }) + return events +} + +func (s *sequenceState) isStateSchedulable(state fsm.State) bool { + return state != s.initialState && state != sequenceTerminalState && state != sequenceExpiredState && state != sequenceDeadlineState +} + +// initFSM initializes the state machine and installs transition callbacks +// that are triggered when the expression in the sequence matches, it expires +// or the deadline occurs. +func (s *sequenceState) initFSM() { + s.fsm = fsm.NewStateMachine(s.initialState) + s.fsm.OnTransitioned(func(ctx context.Context, transition fsm.Transition) { + // schedule span deadline for the current state unless initial/meta states + if s.maxSpan != 0 && s.isStateSchedulable(s.currentState()) { + log.Debugf("scheduling max span deadline of %v for expression [%s] of sequence [%s]", s.maxSpan, s.expr(s.currentState()), s.name) + s.scheduleMaxSpanDeadline(s.currentState(), s.maxSpan) + } + // if the sequence was deadlined/expired, we can disable the deadline + // status when the first expression in the sequence is reevaluated + if transition.Source == s.initialState && s.inDeadline.Load() { + s.inDeadline.Store(false) + } + if transition.Source == s.initialState && s.inExpired.Load() { + s.inExpired.Store(false) + } + // clear state in case of expire/deadline transitions + if transition.Trigger == expireTransition || transition.Trigger == cancelTransition { + s.clear() + } + if transition.Trigger == matchTransition { + log.Debugf("state trigger from expression [%s] of sequence [%s]", s.expr(transition.Source), s.name) + // a match occurred from current to next state. + // Stop deadline execution for the old current state + if span, ok := s.spanDeadlines[transition.Source]; ok { + log.Debugf("stopped max span deadline for expression [%s] of sequence [%s]", s.expr(transition.Source), s.name) + span.Stop() + delete(s.spanDeadlines, transition.Source) + } + // save expression match + s.states[transition.Source] = true + } + }) +} + +// configureFSM sets up the states and transitions of the state automata. +// A simplified representation of the constructed automata is better +// visualized in the diagram above +// +// +-----+ +-----+ +-----+ +// ----> | 0 | -----> | 1 | -----> | 2 | -----> terminal +// + +-----+ +-----+ +-----+ +// + | | | +// + +----+----+ +----+----+ +----+----+ +// + | | | | | | +// + v v v v v v +// +---------+ +---------+ +---------+ +---------+ +---------+ +---------+ +// +// | deadline| | expired | | deadline| | expired | | deadline| | expired | +// +---------+ +---------+ +---------+ +---------+ +---------+ +---------+ +// +// The diagram is based on the assumption that there are three expressions +// inside the sequence. In the course of normal circumstances, the initial +// state transitions to the state 1 and the state 1 to the state 2 whenever +// the expression associated to the state is evaluated to true. +// Once the final state is reached, it transitions to the terminal state and +// the sequence is considered to yield a match. +// +// However, it can happen that the maximum time span defined in the sequence +// elapses. In this situation, the sequence is promoted to the deadline state +// and the state machine is reset to the initial state. The similar behaviour +// occurs when the process attributed to any of the pending partials in the +// sequence terminates. In this case, the state machine transitions to the +// expired state. +func (s *sequenceState) configureFSM() { + for seqID, expr := range s.seq.Expressions { + // sequence expression index is the state name + s.exprs[seqID] = expr.Expr.String() + // is this the last state? + if seqID >= len(s.seq.Expressions)-1 { + s.fsm. + Configure(seqID). + Permit(matchTransition, sequenceTerminalState). + Permit(cancelTransition, sequenceDeadlineState). + Permit(expireTransition, sequenceExpiredState) + } else { + // the previous state can transition to the next one + // via the match transition, or can either go to the + // deadline or expired states via cancel and expire + // transitions respectively + s.fsm. + Configure(seqID). + Permit(matchTransition, seqID+1). + Permit(cancelTransition, sequenceDeadlineState). + Permit(expireTransition, sequenceExpiredState) + } + } + // configure reset transitions that are triggered + // when the final state is reached of when a deadline + // or sequence expiration happens + s.fsm. + Configure(sequenceTerminalState). + Permit(resetTransition, sequenceInitialState) + s.fsm. + Configure(sequenceDeadlineState). + Permit(resetTransition, sequenceInitialState) + s.fsm. + Configure(sequenceExpiredState). + Permit(resetTransition, sequenceInitialState) +} + +func (s *sequenceState) matchTransition(seqID int, e *kevent.Kevent) error { + s.smu.Lock() + defer s.smu.Unlock() + shouldFire := !s.states[seqID] + if shouldFire { + return s.fsm.Fire(matchTransition, e) + } + return nil +} + +func (s *sequenceState) cancelTransition(seqID fsm.State) error { + return s.fsm.Fire(cancelTransition, seqID) +} + +func (s *sequenceState) expireTransition() error { + return s.fsm.Fire(expireTransition) +} + +func (s *sequenceState) isTerminalState() bool { + isFinal := s.currentState() == sequenceTerminalState + if isFinal { + err := s.fsm.Fire(resetTransition) + if err != nil { + log.Warnf("unable to transition to initial state: %v", err) + } + } + return isFinal +} + +func (s *sequenceState) isInitialState() bool { + return s.currentState() == s.initialState +} + +func (s *sequenceState) currentState() fsm.State { + return s.fsm.MustState() +} + +func (s *sequenceState) expr(state fsm.State) string { + seqID, ok := state.(int) + if !ok { + return "" + } + return s.exprs[seqID] +} + +// addPartial appends the event that matched the expression at the +// sequence index. If the event arrived out of order, then the isOOO +// parameter is equal to false. +func (s *sequenceState) addPartial(seqID int, e *kevent.Kevent, isOOO bool) { + s.mu.Lock() + defer s.mu.Unlock() + if len(s.partials[seqID]) > maxOutstandingPartials { + partialBreaches.Add(s.name, 1) + if !s.isPartialsBreached.Load() { + log.Warnf("max partials encountered in sequence %s slot [%d]. "+ + "Dropping incoming partial: %s", s.name, seqID, e) + } + s.isPartialsBreached.Store(true) + return + } + key := e.PartialKey() + if key != 0 { + for _, p := range s.partials[seqID] { + if key == p.PartialKey() { + log.Debugf("event %s for tuple %d already in sequence state", e, key) + return + } + } + } + if isOOO { + e.AddMeta(kevent.RuleSequenceOOOKey, true) + } + log.Debugf("adding partial to sequence [%s] slot [%d] for expression %q, ooo: %t: %s", s.name, seqID, s.expr(seqID), isOOO, e) + partialsPerSequence.Add(s.name, 1) + s.partials[seqID] = append(s.partials[seqID], e) + sort.Slice(s.partials[seqID], func(n, m int) bool { return s.partials[seqID][n].Timestamp.Before(s.partials[seqID][m].Timestamp) }) +} + +// gc prunes the sequence partial if it remained +// more time than specified by max span or if max +// span is omitted, the partial is allowed to remain +// in sequence state for four hours. +func (s *sequenceState) gc() { + s.mu.Lock() + defer s.mu.Unlock() + dur := s.maxSpan + if dur == 0 { + dur = maxSequencePartialLifetime + } + for idx := range s.exprs { + for i := len(s.partials[idx]) - 1; i >= 0; i-- { + if len(s.partials[idx]) > 0 && time.Since(s.partials[idx][i].Timestamp) > dur { + log.Debugf("garbage collecting partial: [%s] of sequence [%s]", s.partials[idx][i], s.name) + // remove partial event from the corresponding slot + s.partials[idx] = append( + s.partials[idx][:i], + s.partials[idx][i+1:]...) + partialsPerSequence.Add(s.name, -1) + } + } + } +} + +func (s *sequenceState) clear() { + s.partials = make(map[int][]*kevent.Kevent) + s.matches = make(map[int]*kevent.Kevent) + s.states = make(map[fsm.State]bool) + s.spanDeadlines = make(map[fsm.State]*time.Timer) + s.isPartialsBreached.Store(false) + partialsPerSequence.Delete(s.name) +} + +func (s *sequenceState) clearLocked() { + s.mu.Lock() + defer s.mu.Unlock() + s.smu.Lock() + defer s.smu.Unlock() + s.mmu.Lock() + defer s.mmu.Unlock() + s.clear() +} + +// next determines whether the next expression in the +// sequence should be evaluated. The expression is evaluated +// if all its upstream sequence expression produced a match and +// the sequence is not stuck in deadline or expired state. +func (s *sequenceState) next(seqID int) bool { + // always evaluate the first expression in the sequence + if seqID == 0 { + return true + } + var next bool + s.smu.RLock() + defer s.smu.RUnlock() + for n := 0; n < seqID; n++ { + next = s.states[n] + if !next { + break + } + } + return next && !s.inDeadline.Load() && !s.inExpired.Load() +} + +func (s *sequenceState) scheduleMaxSpanDeadline(seqID fsm.State, maxSpan time.Duration) { + t := time.AfterFunc(maxSpan, func() { + inState, _ := s.fsm.IsInState(seqID) + if inState { + log.Debugf("max span of %v exceded for expression [%s] of sequence [%s]", maxSpan, s.expr(seqID), s.name) + s.inDeadline.Store(true) + s.mu.Lock() + defer s.mu.Unlock() + s.smu.Lock() + defer s.smu.Unlock() + // transitions to deadline state + err := s.cancelTransition(seqID) + if err != nil { + s.inDeadline.Store(false) + log.Warnf("deadline transition failed: %v", err) + } + // transitions from deadline state to initial state + err = s.fsm.Fire(resetTransition) + if err != nil { + log.Warnf("unable to transition to initial state: %v", err) + } + } + }) + s.spanDeadlines[seqID] = t +} + +func (s *sequenceState) runSequence(e *kevent.Kevent) bool { + for i, expr := range s.seq.Expressions { + // only try to evaluate the expression + // if upstream expressions have matched + if !s.next(i) { + if !s.seq.IsUnordered { + continue + } + // it could be the event arrived out + // of order because certain provider + // flushed its buffers first. When this + // happens the event timestamp serves as + // a temporal reference. + // If this sequence expression can evaluate + // against the current event, mark it as + // out-of-order and store in partials list + s.mu.RLock() + ok := expr.IsEvaluable(e) && s.filter.RunSequence(e, i, s.partials, true) + s.mu.RUnlock() + if ok { + s.addPartial(i, e, true) + } + continue + } + + // prevent running the filter if the expression + // can't be matched against the current event + if !expr.IsEvaluable(e) { + continue + } + + s.mu.RLock() + matches := s.filter.RunSequence(e, i, s.partials, false) + s.mu.RUnlock() + + // append the partial and transition state machine + if matches { + s.addPartial(i, e, false) + err := s.matchTransition(i, e) + if err != nil { + matchTransitionErrors.Add(1) + log.Warnf("match transition failure: %v", err) + } + // now try to match all pending out-of-order + // events from downstream sequence slots if + // the previous match hasn't reached terminal + // state + if s.seq.IsUnordered && s.currentState() != sequenceTerminalState { + s.mu.RLock() + for seqID := range s.partials { + for _, evt := range s.partials[seqID] { + if !evt.ContainsMeta(kevent.RuleSequenceOOOKey) { + continue + } + matches = s.filter.RunSequence(evt, seqID, s.partials, false) + // transition the state machine + if matches { + err := s.matchTransition(seqID, evt) + if err != nil { + matchTransitionErrors.Add(1) + log.Warnf("out of order match transition failure: %v", err) + } + evt.RemoveMeta(kevent.RuleSequenceOOOKey) + } + } + } + s.mu.RUnlock() + } + } + + // if both the terminal state is reached and the partials + // in the sequence state could be joined by the specified + // field(s), the rule has matched successfully, and we can + // collect all events involved in the rule match + isTerminal := s.isTerminalState() + if isTerminal { + setMatch := func(seqID int, e *kevent.Kevent) { + s.mmu.Lock() + defer s.mmu.Unlock() + if s.matches[seqID] == nil { + s.matches[seqID] = e + } + } + + s.mu.RLock() + for seqID := 0; seqID < len(s.partials); seqID++ { + for _, outer := range s.partials[seqID] { + for _, inner := range s.partials[seqID+1] { + if filter.CompareSeqLink(outer.SequenceLink(), inner.SequenceLink()) { + setMatch(seqID, outer) + setMatch(seqID+1, inner) + } + } + } + } + s.mu.RUnlock() + + return true + } + } + return false +} + +func (s *sequenceState) expire(e *kevent.Kevent) bool { + if !e.IsTerminateProcess() { + return false + } + canExpire := func(lhs, rhs *kevent.Kevent, isFinalSlot bool) bool { + // if the TerminateProcess event arrives for the + // process spawned by CreateProcess, and it pertains + // to the final sequence slot, it is safe to expire + // the whole sequence + if lhs.Type == ktypes.CreateProcess && isFinalSlot { + p1, _ := lhs.Kparams.GetPid() + p2, _ := rhs.Kparams.GetPid() + return p1 == p2 + } + pid, _ := rhs.Kparams.GetPid() + return lhs.PID == pid + } + s.mu.Lock() + defer s.mu.Unlock() + s.smu.RLock() + defer s.smu.RUnlock() + + for idx := range s.exprs { + for i := len(s.partials[idx]) - 1; i >= 0; i-- { + if len(s.partials[idx]) > 0 && !canExpire(s.partials[idx][i], e, idx == len(s.exprs)-1) { + continue + } + log.Debugf("removing event originated from %s (%d) "+ + "in partials pertaining to sequence [%s] and slot [%d]", + e.Kparams.MustGetString(kparams.ProcessName), + e.Kparams.MustGetPid(), + s.name, + idx) + // remove partial event from the corresponding slot + s.partials[idx] = append( + s.partials[idx][:i], + s.partials[idx][i+1:]...) + partialsPerSequence.Add(s.name, -1) + + if len(s.partials[idx]) == 0 { + partialExpirations.Add(s.name, 1) + log.Debugf("%q sequence expired. All partials retracted", s.name) + s.inExpired.Store(true) + err := s.expireTransition() + if err != nil { + s.inExpired.Store(false) + log.Warnf("expire transition failed: %v", err) + } + // transitions from expired state to initial state + err = s.fsm.Fire(resetTransition) + if err != nil { + log.Warnf("unable to transition to initial state: %v", err) + } + return true + } + } + } + + return false +} diff --git a/pkg/rules/sequence_test.go b/pkg/rules/sequence_test.go new file mode 100644 index 000000000..998a613bb --- /dev/null +++ b/pkg/rules/sequence_test.go @@ -0,0 +1,979 @@ +/* + * Copyright 2021-present by Nedim Sabic Sabic + * https://www.fibratus.io + * All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rules + +import ( + "github.com/rabbitstack/fibratus/pkg/config" + "github.com/rabbitstack/fibratus/pkg/filter" + "github.com/rabbitstack/fibratus/pkg/fs" + "github.com/rabbitstack/fibratus/pkg/kevent" + "github.com/rabbitstack/fibratus/pkg/kevent/kparams" + "github.com/rabbitstack/fibratus/pkg/kevent/ktypes" + pstypes "github.com/rabbitstack/fibratus/pkg/ps/types" + log "github.com/sirupsen/logrus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "golang.org/x/sys/windows/registry" + "net" + "strconv" + "testing" + "time" +) + +func TestSequenceState(t *testing.T) { + log.SetLevel(log.DebugLevel) + + c := &config.FilterConfig{Name: "Command shell created and executed by file"} + f := filter.New(` + sequence + maxspan 100ms + |kevt.name = 'CreateProcess' and ps.name = 'cmd.exe'| by ps.exe + |kevt.name = 'CreateFile' and file.path icontains 'temp'| by file.path + |kevt.name = 'CreateProcess'| by ps.child.exe`, + &config.Config{Kstream: config.KstreamConfig{}, Filters: &config.Filters{}}) + + require.NoError(t, f.Compile()) + + ss := newSequenceState(f, c) + + assert.Equal(t, 0, ss.currentState()) + assert.True(t, ss.isInitialState()) + assert.Equal(t, "kevt.name = CreateProcess AND ps.name = cmd.exe", ss.expr(ss.initialState)) + + e1 := &kevent.Kevent{ + Type: ktypes.CreateProcess, + Name: "CreateProcess", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(4143)}, + kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.AnsiString, Value: "powershell.exe"}, + }, + } + require.True(t, ss.next(0)) + require.False(t, ss.next(1)) + require.NoError(t, ss.matchTransition(0, e1)) + ss.addPartial(0, e1, false) + require.True(t, ss.next(1)) + assert.True(t, ss.states[0]) + require.False(t, ss.next(2)) + + assert.False(t, ss.isInitialState()) + assert.Equal(t, "kevt.name = CreateFile AND file.path ICONTAINS temp", ss.expr(ss.currentState())) + + e2 := &kevent.Kevent{ + Type: ktypes.CreateFile, + Name: "CreateFile", + Tid: 2484, + PID: 4143, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost.exe", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper"}, + }, + } + // can't go to the next transitions as the expr hasn't matched + require.False(t, ss.next(2)) + require.NoError(t, ss.matchTransition(1, e2)) + ss.addPartial(1, e2, false) + require.True(t, ss.states[1]) + require.True(t, ss.next(2)) + + assert.Len(t, ss.partials[0], 1) + assert.Len(t, ss.partials[1], 1) + + assert.Equal(t, 2, ss.currentState()) + assert.Equal(t, "kevt.name = CreateProcess", ss.expr(ss.currentState())) + + e3 := &kevent.Kevent{ + Type: ktypes.CreateProcess, + Name: "CreateProcess", + Tid: 2484, + PID: 4143, + Kparams: kevent.Kparams{ + kparams.Exe: {Name: kparams.Exe, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper.exe"}, + }, + } + require.NoError(t, ss.matchTransition(2, e3)) + ss.addPartial(2, e3, false) + + assert.Len(t, ss.partials[2], 1) + + assert.Equal(t, sequenceTerminalState, ss.currentState()) + assert.True(t, ss.isTerminalState()) + + // reset sequence state + ss.clear() + + // reset transition leads back to initial state + assert.Equal(t, 0, ss.currentState()) + assert.Equal(t, "kevt.name = CreateProcess AND ps.name = cmd.exe", ss.expr(ss.currentState())) + // deadline exceeded + require.NoError(t, ss.matchTransition(0, e1)) + assert.Equal(t, "kevt.name = CreateFile AND file.path ICONTAINS temp", ss.expr(ss.currentState())) + time.Sleep(time.Millisecond * 120) + // transition to initial state + assert.True(t, ss.isInitialState()) + + // sequence in deadline state + require.True(t, ss.inDeadline.Load()) + require.True(t, ss.next(0)) + require.False(t, ss.next(1)) + if ss.next(1) { + // this shouldn't happen + require.NoError(t, ss.matchTransition(1, e2)) + } + + ss.clear() + + assert.True(t, ss.isInitialState()) + require.NoError(t, ss.matchTransition(0, e1)) + ss.addPartial(0, e1, false) + ss.addPartial(1, e2, false) + require.False(t, ss.inDeadline.Load()) + + // expire entire sequence + e4 := &kevent.Kevent{ + Type: ktypes.TerminateProcess, + Name: "TerminateProcess", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(4143)}, + kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.AnsiString, Value: "powershell.exe"}, + }, + } + require.True(t, ss.expire(e4)) + require.True(t, ss.inExpired.Load()) + + require.NoError(t, ss.matchTransition(0, e1)) + require.False(t, ss.inExpired.Load()) + + assert.Equal(t, "kevt.name = CreateFile AND file.path ICONTAINS temp", ss.expr(ss.currentState())) +} + +func TestSimpleSequence(t *testing.T) { + log.SetLevel(log.DebugLevel) + + c := &config.FilterConfig{Name: "Command shell created a temp file"} + f := filter.New(` + sequence + maxspan 100ms + |kevt.name = 'CreateProcess' and ps.name = 'cmd.exe'| by ps.exe + |kevt.name = 'CreateFile' and file.path icontains 'temp'| by file.path + `, &config.Config{Kstream: config.KstreamConfig{EnableFileIOKevents: true}, Filters: &config.Filters{}}) + require.NoError(t, f.Compile()) + + ss := newSequenceState(f, c) + + var tests = []struct { + evts []*kevent.Kevent + matches []bool + }{ + {[]*kevent.Kevent{{ + Type: ktypes.CreateProcess, + Name: "CreateProcess", + Timestamp: time.Now(), + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost-temp.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + }, { + Type: ktypes.CreateFile, + Name: "CreateFile", + Timestamp: time.Now(), + Tid: 2484, + PID: 859, + Category: ktypes.File, + PS: &pstypes.PS{ + Name: "cmd.exe", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Windows\\system32\\svchost-temp.exe"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}}}, []bool{false, true}}, + {[]*kevent.Kevent{{ + Type: ktypes.CreateProcess, + Name: "CreateProcess", + Timestamp: time.Now(), + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\cmd.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + }, { + Type: ktypes.CreateFile, + Name: "CreateFile", + Timestamp: time.Now(), + Tid: 2484, + PID: 859, + Category: ktypes.File, + PS: &pstypes.PS{ + Name: "cmd.exe", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Windows\\system32\\svchost-temp.exe"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}}}, []bool{false, false}}, + } + + for i, tt := range tests { + t.Run(strconv.Itoa(i), func(t *testing.T) { + for idx, e := range tt.evts { + assert.Equal(t, tt.matches[idx], ss.runSequence(e)) + } + }) + } +} + +func TestSimpleSequenceMultiplePartials(t *testing.T) { + log.SetLevel(log.DebugLevel) + + c := &config.FilterConfig{Name: "Command shell created a temp file"} + f := filter.New(` + sequence + maxspan 200ms + by ps.pid + |kevt.name = 'CreateProcess' and ps.name = 'cmd.exe'| + |kevt.name = 'CreateFile' and file.path icontains 'temp'| + `, &config.Config{Kstream: config.KstreamConfig{EnableFileIOKevents: true}, Filters: &config.Filters{}}) + require.NoError(t, f.Compile()) + + ss := newSequenceState(f, c) + + // create random matches which don't satisfy the sequence link + for i, pid := range []uint32{2343, 1024, 11122, 3450, 12319} { + e1 := &kevent.Kevent{ + Type: ktypes.CreateProcess, + Timestamp: time.Now().Add(time.Duration(i) * time.Millisecond), + Name: "CreateProcess", + Tid: 2484, + PID: pid, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\cmd.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: pid % 2}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + e2 := &kevent.Kevent{ + Type: ktypes.CreateFile, + Timestamp: time.Now().Add(time.Duration(i) * time.Millisecond * 2), + Name: "CreateFile", + Tid: 2484, + PID: pid * 2, + Category: ktypes.File, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\cmd.exe", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Windows\\system32\\svchost-temp.exe"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + require.False(t, ss.runSequence(e1)) + require.False(t, ss.runSequence(e2)) + } + + // expression matched multiple partials + assert.Len(t, ss.partials[0], 5) + assert.Len(t, ss.partials[1], 0) + + e1 := &kevent.Kevent{ + Seq: 20, + Type: ktypes.CreateProcess, + Timestamp: time.Now().Add(time.Second), + Name: "CreateProcess", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\System32\\cmd.exe", + Parent: &pstypes.PS{ + Name: "WmiPrvSE.exe", + }, + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + e2 := &kevent.Kevent{ + Type: ktypes.CreateFile, + Seq: 22, + Timestamp: time.Now().Add(time.Second * time.Duration(2)), + Name: "CreateFile", + Tid: 2484, + PID: 859, + Category: ktypes.File, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost.exe", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\file.tmp"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + require.False(t, ss.runSequence(e1)) + // expression matched the partial that satisfy the sequence link + assert.Len(t, ss.partials[0], 6) + assert.Len(t, ss.partials[1], 0) + require.True(t, ss.runSequence(e2)) + assert.Len(t, ss.partials[1], 1) + + require.Len(t, ss.matches, 2) + assert.Equal(t, uint32(859), ss.matches[0].PID) + assert.Equal(t, "WmiPrvSE.exe", ss.matches[0].PS.Parent.Name) + assert.Equal(t, uint32(859), ss.matches[1].PID) + assert.Equal(t, "C:\\Temp\\file.tmp", ss.matches[1].GetParamAsString(kparams.FilePath)) +} + +func TestSimpleSequenceDeadline(t *testing.T) { + log.SetLevel(log.DebugLevel) + + c := &config.FilterConfig{Name: "Command shell created a temp file"} + f := filter.New(` + sequence + maxspan 100ms + |kevt.name = 'CreateProcess' and ps.name = 'cmd.exe'| by ps.exe + |kevt.name = 'CreateFile' and file.path icontains 'temp'| by file.path + `, &config.Config{Kstream: config.KstreamConfig{EnableFileIOKevents: true}, Filters: &config.Filters{}}) + require.NoError(t, f.Compile()) + + ss := newSequenceState(f, c) + + e1 := &kevent.Kevent{ + Type: ktypes.CreateProcess, + Timestamp: time.Now(), + Name: "CreateProcess", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost-temp.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + require.False(t, ss.runSequence(e1)) + + e2 := &kevent.Kevent{ + Type: ktypes.CreateFile, + Timestamp: time.Now(), + Name: "CreateFile", + Tid: 2484, + PID: 859, + Category: ktypes.File, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost.exe", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Windows\\system32\\svchost-temp.exe"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + time.Sleep(time.Millisecond * 110) + require.False(t, ss.runSequence(e2)) + + require.Equal(t, sequenceInitialState, ss.currentState()) + assert.Len(t, ss.partials, 0) + + // now the state machine has transitioned + // to the initial state, which means we should + // be able to match the sequence if we reinsert + // the events + require.False(t, ss.runSequence(e1)) + require.True(t, ss.runSequence(e2)) + + ss.clearLocked() + require.Equal(t, sequenceInitialState, ss.currentState()) + assert.Len(t, ss.partials, 0) + + // assert the events again with the delay less than max span + require.False(t, ss.runSequence(e1)) + time.Sleep(time.Millisecond * 85) + require.True(t, ss.runSequence(e2)) +} + +func TestComplexSequence(t *testing.T) { + log.SetLevel(log.DebugLevel) + + c := &config.FilterConfig{Name: "Phishing dropper outbound communication"} + f := filter.New(` + sequence + maxspan 1h + |kevt.name = 'CreateProcess' and ps.child.name in ('firefox.exe', 'chrome.exe', 'edge.exe')| by ps.child.pid + |kevt.name = 'CreateFile' and file.operation = 'CREATE' and file.extension = '.exe'| by ps.pid + |kevt.name in ('Send', 'Connect')| by ps.pid + `, &config.Config{Kstream: config.KstreamConfig{EnableFileIOKevents: true}, Filters: &config.Filters{}}) + require.NoError(t, f.Compile()) + + ss := newSequenceState(f, c) + + e1 := &kevent.Kevent{ + Seq: 1, + Type: ktypes.CreateProcess, + Timestamp: time.Now(), + Category: ktypes.Process, + Name: "CreateProcess", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "explorer.exe", + Exe: "C:\\Windows\\system32\\explorer.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(2243)}, + kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "firefox.exe"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + require.False(t, ss.runSequence(e1)) + + e2 := &kevent.Kevent{ + Seq: 2, + Type: ktypes.CreateFile, + Timestamp: time.Now().Add(time.Millisecond * 250), + Name: "CreateFile", + Tid: 2484, + PID: 2243, + Category: ktypes.File, + PS: &pstypes.PS{ + Name: "firefox.exe", + Exe: "C:\\Program Files\\Mozilla Firefox\\firefox.exe", + Cmdline: "C:\\Program Files\\Mozilla Firefox\\firefox.exe\" -contentproc --channel=\"10464.7.539748228\\1366525930\" -childID 6 -isF", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper.exe"}, + kparams.FileOperation: {Name: kparams.FileOperation, Type: kparams.Enum, Value: uint32(2), Enum: fs.FileCreateDispositions}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + require.False(t, ss.runSequence(e2)) + + assert.Len(t, ss.partials[0], 1) + assert.Len(t, ss.partials[1], 1) + + e3 := &kevent.Kevent{ + Seq: 4, + Type: ktypes.ConnectTCPv4, + Timestamp: time.Now().Add(time.Second), + Category: ktypes.Net, + Name: "Connect", + Tid: 244, + PID: 2243, + PS: &pstypes.PS{ + Name: "firefox.exe", + Exe: "C:\\Program Files\\Mozilla Firefox\\firefox.exe", + Cmdline: "C:\\Program Files\\Mozilla Firefox\\firefox.exe\" -contentproc --channel=\"10464.7.539748228\\1366525930\" -childID 6 -isF", + }, + Kparams: kevent.Kparams{ + kparams.NetDIP: {Name: kparams.NetDIP, Type: kparams.IPv4, Value: net.ParseIP("10.0.2.3")}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + time.Sleep(time.Millisecond * 30) + require.True(t, ss.runSequence(e3)) + + time.Sleep(time.Millisecond * 50) + + ss.clearLocked() + + // FSM should transition from terminal to initial state + require.Equal(t, sequenceInitialState, ss.currentState()) + + require.False(t, ss.runSequence(e1)) + require.False(t, ss.runSequence(e2)) + time.Sleep(time.Millisecond * 15) + require.True(t, ss.runSequence(e3)) +} + +func TestSequenceOOO(t *testing.T) { + log.SetLevel(log.DebugLevel) + + c := &config.FilterConfig{Name: "LSASS memory dumping via legitimate or offensive tools"} + f := filter.New(` + sequence + maxspan 2m + |kevt.name = 'OpenProcess' and kevt.arg[exe] imatches '?:\\Windows\\System32\\lsass.exe'| by ps.uuid + |kevt.name = 'CreateFile' and file.operation = 'CREATE' and file.extension = '.dmp'| by ps.uuid + `, &config.Config{Kstream: config.KstreamConfig{EnableFileIOKevents: true}, Filters: &config.Filters{}}) + require.NoError(t, f.Compile()) + + ss := newSequenceState(f, c) + + e1 := &kevent.Kevent{ + Type: ktypes.CreateFile, + Timestamp: time.Now(), + Name: "CreateFile", + Tid: 2484, + PID: 859, + Category: ktypes.File, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\rundll32.exe", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\temp\\lsass.dmp"}, + kparams.FileOperation: {Name: kparams.FileOperation, Type: kparams.UnicodeString, Value: "CREATE"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + require.False(t, ss.runSequence(e1)) + require.Len(t, ss.partials[1], 1) + assert.True(t, ss.partials[1][0].ContainsMeta(kevent.RuleSequenceOOOKey)) + + e2 := &kevent.Kevent{ + Type: ktypes.OpenProcess, + Timestamp: time.Now(), + Name: "OpenProcess", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\rundll32.exe", + }, + Kparams: kevent.Kparams{ + kparams.Exe: {Name: kparams.Exe, Type: kparams.UnicodeString, Value: "C:\\Windows\\System32\\lsass.exe"}, + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(2243)}, + kparams.DesiredAccess: {Name: kparams.DesiredAccess, Type: kparams.Flags, Value: uint32(0x1400), Flags: kevent.PsAccessRightFlags}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + require.True(t, ss.runSequence(e2)) + assert.Len(t, ss.partials[0], 1) + assert.False(t, ss.partials[1][0].ContainsMeta(kevent.RuleSequenceOOOKey)) +} + +func TestSequenceGC(t *testing.T) { + log.SetLevel(log.DebugLevel) + + maxSequencePartialLifetime = time.Millisecond * 500 + + c := &config.FilterConfig{Name: "LSASS memory dumping via legitimate or offensive tools"} + f := filter.New(` + sequence + by ps.uuid + |kevt.name = 'OpenProcess' and kevt.arg[exe] imatches '?:\\Windows\\System32\\lsass.exe'| + |kevt.name = 'CreateFile' and file.operation = 'CREATE' and file.extension = '.dmp'| + `, &config.Config{Kstream: config.KstreamConfig{EnableFileIOKevents: true}, Filters: &config.Filters{}}) + require.NoError(t, f.Compile()) + + ss := newSequenceState(f, c) + + e := &kevent.Kevent{ + Type: ktypes.OpenProcess, + Timestamp: time.Now(), + Name: "OpenProcess", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\rundll32.exe", + }, + Kparams: kevent.Kparams{ + kparams.Exe: {Name: kparams.Exe, Type: kparams.UnicodeString, Value: "C:\\Windows\\System32\\lsass.exe"}, + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(2243)}, + kparams.DesiredAccess: {Name: kparams.DesiredAccess, Type: kparams.Flags, Value: uint32(0x1400), Flags: kevent.PsAccessRightFlags}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + require.False(t, ss.runSequence(e)) + assert.Len(t, ss.partials[0], 1) + + time.Sleep(time.Second) + + ss.gc() + + assert.Len(t, ss.partials[0], 0) +} + +func TestSequenceExpire(t *testing.T) { + log.SetLevel(log.DebugLevel) + + var tests = []struct { + c *config.FilterConfig + expr string + evts []*kevent.Kevent + wants bool + }{ + { + &config.FilterConfig{Name: "LSASS memory dumping via legitimate or offensive tools"}, + `sequence + maxspan 2m + |kevt.name = 'OpenProcess' and kevt.arg[exe] imatches '?:\\Windows\\System32\\lsass.exe'| by ps.uuid + |kevt.name = 'CreateFile' and file.operation = 'CREATE' and file.extension = '.dmp'| by ps.uuid + `, + []*kevent.Kevent{ + { + Type: ktypes.OpenProcess, + Timestamp: time.Now(), + Name: "OpenProcess", + Tid: 2484, + PID: 4143, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\rundll32.exe", + }, + Kparams: kevent.Kparams{ + kparams.Exe: {Name: kparams.Exe, Type: kparams.UnicodeString, Value: "C:\\Windows\\System32\\lsass.exe"}, + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(2243)}, + kparams.DesiredAccess: {Name: kparams.DesiredAccess, Type: kparams.Flags, Value: uint32(0x1400), Flags: kevent.PsAccessRightFlags}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + }, + { + Type: ktypes.TerminateProcess, + Name: "TerminateProcess", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(4143)}, + kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.AnsiString, Value: "powershell.exe"}, + }, + }, + }, + true, + }, + { + &config.FilterConfig{Name: "System Binary Proxy Execution via Rundll32"}, + `sequence + maxspan 2m + |kevt.name = 'CreateProcess' and ps.child.name = 'rundll32.exe'| by ps.child.pid + |kevt.name = 'CreateProcess' and ps.child.name = 'connhost.exe'| by ps.pid + `, + []*kevent.Kevent{ + { + Seq: 1, + Type: ktypes.CreateProcess, + Timestamp: time.Now(), + Category: ktypes.Process, + Name: "CreateProcess", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "explorer.exe", + Exe: "C:\\Windows\\system32\\explorer.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(2243)}, + kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "rundll32.exe"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + }, + { + Seq: 2, + Type: ktypes.CreateProcess, + Timestamp: time.Now(), + Category: ktypes.Process, + Name: "CreateProcess", + Tid: 2484, + PID: 2243, + PS: &pstypes.PS{ + Name: "explorer.exe", + Exe: "C:\\Windows\\system32\\explorer.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(12243)}, + kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.UnicodeString, Value: "connhost.exe"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + }, + { + Type: ktypes.TerminateProcess, + Name: "TerminateProcess", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(12243)}, + kparams.ProcessName: {Name: kparams.ProcessName, Type: kparams.AnsiString, Value: "powershell.exe"}, + }, + }, + }, + false, + }, + } + + for _, tt := range tests { + t.Run(tt.expr, func(t *testing.T) { + f := filter.New(tt.expr, &config.Config{Kstream: config.KstreamConfig{EnableFileIOKevents: true}, Filters: &config.Filters{}}) + require.NoError(t, f.Compile()) + + ss := newSequenceState(f, tt.c) + for _, evt := range tt.evts { + if evt.IsTerminateProcess() { + ss.expire(evt) + } else { + ss.runSequence(evt) + } + } + + require.Equal(t, tt.wants, ss.inExpired.Load()) + require.Len(t, ss.partials, 0) + ss.runSequence(tt.evts[0]) + require.False(t, ss.inExpired.Load()) + }) + } +} + +func TestSequenceBoundFields(t *testing.T) { + log.SetLevel(log.DebugLevel) + + maxSequencePartialLifetime = time.Millisecond * 500 + + c := &config.FilterConfig{Name: "Command shell created a temp file with network outbound"} + f := filter.New(` + sequence + maxspan 200ms + |kevt.name = 'CreateProcess' and ps.name = 'cmd.exe'| as e1 + |kevt.name = 'CreateFile' and file.path icontains 'temp' and $e1.ps.sid = ps.sid| as e2 + |kevt.name = 'Connect' and ps.sid != $e2.ps.sid and ps.sid = $e1.ps.sid| + `, &config.Config{Kstream: config.KstreamConfig{EnableFileIOKevents: true}, Filters: &config.Filters{}}) + require.NoError(t, f.Compile()) + + ss := newSequenceState(f, c) + + e1 := &kevent.Kevent{ + Type: ktypes.CreateProcess, + Timestamp: time.Now(), + Name: "CreateProcess", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost-temp.exe", + SID: "zinet", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + e2 := &kevent.Kevent{ + Type: ktypes.CreateProcess, + Timestamp: time.Now().Add(time.Millisecond * 20), + Name: "CreateProcess", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost-temp.exe", + SID: "nusret", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + e3 := &kevent.Kevent{ + Type: ktypes.CreateFile, + Timestamp: time.Now().Add(time.Second), + Name: "CreateFile", + Tid: 2484, + PID: 859, + Category: ktypes.File, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost.exe", + SID: "nusret", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Windows\\system32\\svchost-temp.exe"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + e4 := &kevent.Kevent{ + Type: ktypes.ConnectTCPv4, + Timestamp: time.Now().Add(time.Second * 3), + Name: "Connect", + Tid: 2484, + PID: 859, + Category: ktypes.File, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost.exe", + SID: "zinet", + }, + Kparams: kevent.Kparams{ + kparams.NetDport: {Name: kparams.NetDport, Type: kparams.Uint16, Value: uint16(80)}, + kparams.NetDIP: {Name: kparams.NetDIP, Type: kparams.IPv4, Value: net.ParseIP("172.1.2.3")}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + require.False(t, ss.runSequence(e1)) + require.False(t, ss.runSequence(e2)) + require.False(t, ss.runSequence(e3)) + require.True(t, ss.runSequence(e4)) +} + +func TestSequenceBoundFieldsWithFunctions(t *testing.T) { + log.SetLevel(log.DebugLevel) + + maxSequencePartialLifetime = time.Millisecond * 500 + + c := &config.FilterConfig{Name: "Command shell created a temp file with network outbound"} + f := filter.New(` + sequence + maxspan 5m + |kevt.name = 'CreateFile' and file.path imatches '?:\\Windows\\System32\\*.dll'| as e1 + |kevt.name = 'RegSetValue' and registry.path ~= 'HKEY_CURRENT_USER\\Volatile Environment\\Notification Packages' + and + get_reg_value(registry.path) iin (base($e1.file.path, false))| + `, &config.Config{Kstream: config.KstreamConfig{EnableFileIOKevents: true, EnableRegistryKevents: true}, Filters: &config.Filters{}}) + require.NoError(t, f.Compile()) + + ss := newSequenceState(f, c) + + e1 := &kevent.Kevent{ + Type: ktypes.CreateFile, + Name: "CreateFile", + Category: ktypes.File, + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\cmd.exe", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Windows\\System32\\passwdflt.dll"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + e2 := &kevent.Kevent{ + Type: ktypes.RegSetValue, + Name: "RegSetValue", + Category: ktypes.Registry, + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\cmd.exe", + }, + Kparams: kevent.Kparams{ + kparams.RegPath: {Name: kparams.RegPath, Type: kparams.UnicodeString, Value: "HKEY_CURRENT_USER\\Volatile Environment\\Notification Packages"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + key, err := registry.OpenKey(registry.CURRENT_USER, "Volatile Environment", registry.SET_VALUE) + require.NoError(t, err) + defer key.Close() + + defer func() { + _ = key.DeleteValue("Notification Packages") + }() + + require.NoError(t, key.SetStringsValue("Notification Packages", []string{"secli", "passwdflt"})) + + require.False(t, ss.runSequence(e1)) + require.True(t, ss.runSequence(e2)) +} + +func TestIsExpressionEvaluable(t *testing.T) { + log.SetLevel(log.DebugLevel) + + c := &config.FilterConfig{Name: "Command shell created a temp file"} + f := filter.New(` + sequence + maxspan 100ms + |kevt.name = 'CreateProcess' and ps.name = 'cmd.exe'| by ps.exe + |kevt.name = 'CreateFile' and file.path icontains 'temp'| by file.path + `, &config.Config{Kstream: config.KstreamConfig{EnableFileIOKevents: true}, Filters: &config.Filters{}}) + require.NoError(t, f.Compile()) + + ss := newSequenceState(f, c) + + e1 := &kevent.Kevent{ + Type: ktypes.CreateProcess, + Name: "CreateProcess", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost.exe", + }, + Kparams: kevent.Kparams{ + kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.Uint32, Value: uint32(4143)}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + e2 := &kevent.Kevent{ + Type: ktypes.RenameFile, + Name: "RenameFile", + Tid: 2484, + PID: 859, + PS: &pstypes.PS{ + Name: "cmd.exe", + Exe: "C:\\Windows\\system32\\svchost.exe", + }, + Kparams: kevent.Kparams{ + kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Temp\\dropper"}, + }, + Metadata: map[kevent.MetadataKey]any{"foo": "bar", "fooz": "barzz"}, + } + + assert.False(t, ss.filter.GetSequence().Expressions[0].IsEvaluable(e2)) + assert.True(t, ss.filter.GetSequence().Expressions[0].IsEvaluable(e1)) +}