diff --git a/internal/etw/consumer.go b/internal/etw/consumer.go index 78fed5590..f2ce29b4a 100644 --- a/internal/etw/consumer.go +++ b/internal/etw/consumer.go @@ -74,22 +74,21 @@ func (c *Consumer) ProcessEvent(ev *etw.EventRecord) error { if c.isClosing { return nil } - if event.IsCurrentProcDropped(ev.Header.ProcessID) { + + if !c.config.EventSource.EventExists(ev.ID()) { + eventsUnknown.Add(1) return nil } - if c.config.EventSource.ExcludeEvent(ev.Header.ProviderID, ev.HookID()) { - eventsExcluded.Add(1) + if event.IsCurrentProcDropped(ev.Header.ProcessID) { return nil } - - etype := event.NewFromEventRecord(ev) - if !etype.Exists() { - eventsUnknown.Add(1) + if c.config.EventSource.ExcludeEvent(ev.ID()) { + eventsExcluded.Add(1) return nil } eventsProcessed.Add(1) - evt := event.New(c.sequencer.Get(), etype, ev) + evt := event.New(c.sequencer.Get(), ev) // Dispatch each event to the processor chain. // Processors may further augment the event with diff --git a/internal/etw/source_test.go b/internal/etw/source_test.go index 3065f7a39..830dda6ad 100644 --- a/internal/etw/source_test.go +++ b/internal/etw/source_test.go @@ -123,6 +123,7 @@ func TestEventSourceStartTraces(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { + tt.cfg.EventSource.Init() evs := NewEventSource(psnap, hsnap, tt.cfg, nil) require.NoError(t, evs.Open(tt.cfg)) defer evs.Close() @@ -193,6 +194,7 @@ func TestEventSourceEnableFlagsDynamically(t *testing.T) { Filters: &config.Filters{}, } + cfg.EventSource.Init() evs := NewEventSource(psnap, hsnap, cfg, r) require.NoError(t, evs.Open(cfg)) defer evs.Close() @@ -277,6 +279,7 @@ func TestEventSourceEnableFlagsDynamicallyWithYaraEnabled(t *testing.T) { }, } + cfg.EventSource.Init() evs := NewEventSource(psnap, hsnap, cfg, r) require.NoError(t, evs.Open(cfg)) defer evs.Close() @@ -328,6 +331,7 @@ func TestEventSourceRundownEvents(t *testing.T) { Filters: &config.Filters{}, } + cfg.EventSource.Init() evs := NewEventSource(psnap, hsnap, cfg, nil) l := &MockListener{} @@ -741,6 +745,7 @@ func TestEventSourceAllEvents(t *testing.T) { StackEnrichment: false, } + evsConfig.Init() cfg := &config.Config{EventSource: evsConfig, Filters: &config.Filters{}} evs := NewEventSource(psnap, hsnap, cfg, nil) @@ -1226,6 +1231,8 @@ func testCallstackEnrichment(t *testing.T, hsnap handle.Snapshotter, psnap ps.Sn FlushTimer: 1, } + evsConfig.Init() + cfg := &config.Config{ EventSource: evsConfig, Filters: &config.Filters{}, diff --git a/internal/etw/stackext_test.go b/internal/etw/stackext_test.go index 480761b73..48b92b2b1 100644 --- a/internal/etw/stackext_test.go +++ b/internal/etw/stackext_test.go @@ -38,6 +38,9 @@ func TestStackExtensions(t *testing.T) { FlushTimer: time.Millisecond * 2300, }, } + + cfg.EventSource.Init() + exts := NewStackExtensions(cfg.EventSource) assert.Len(t, exts.EventIds(), 0) diff --git a/internal/etw/trace_test.go b/internal/etw/trace_test.go index 4b0fc4560..64b6a1929 100644 --- a/internal/etw/trace_test.go +++ b/internal/etw/trace_test.go @@ -36,6 +36,8 @@ func TestStartTrace(t *testing.T) { }, } + cfg.EventSource.Init() + trace := NewKernelTrace(cfg) require.NoError(t, trace.Start()) require.True(t, trace.IsStarted()) diff --git a/pkg/config/eventsource.go b/pkg/config/eventsource.go index 20fddfde2..60a316deb 100644 --- a/pkg/config/eventsource.go +++ b/pkg/config/eventsource.go @@ -23,7 +23,7 @@ package config import ( "github.com/rabbitstack/fibratus/pkg/event" - "golang.org/x/sys/windows" + "github.com/rabbitstack/fibratus/pkg/util/bitmask" "runtime" "time" @@ -102,7 +102,8 @@ type EventSourceConfig struct { // ExcludedImages are process image names that will be rejected if they generate a kernel event. ExcludedImages []string `json:"blacklist.images" yaml:"blacklist.images"` - dropMasks event.EventsetMasks + dropMasks *bitmask.Bitmask + allMasks *bitmask.Bitmask excludedImages map[string]bool } @@ -127,13 +128,21 @@ func (c *EventSourceConfig) initFromViper(v *viper.Viper) { c.ExcludedEvents = v.GetStringSlice(excludedEvents) c.ExcludedImages = v.GetStringSlice(excludedImages) + c.dropMasks = bitmask.New() + c.allMasks = bitmask.New() + c.excludedImages = make(map[string]bool) for _, name := range c.ExcludedEvents { if typ := event.NameToType(name); typ != event.UnknownType { - c.dropMasks.Set(typ) + c.dropMasks.Set(typ.ID()) } } + + for _, typ := range event.AllWithState() { + c.allMasks.Set(typ.ID()) + } + for _, name := range c.ExcludedImages { c.excludedImages[name] = true } @@ -142,35 +151,54 @@ func (c *EventSourceConfig) initFromViper(v *viper.Viper) { // Init is an exported method to allow initializing exclusion maps from external modules. func (c *EventSourceConfig) Init() { c.excludedImages = make(map[string]bool) + + if c.dropMasks == nil { + c.dropMasks = bitmask.New() + } for _, name := range c.ExcludedEvents { for _, typ := range event.NameToTypes(name) { if typ != event.UnknownType { - c.dropMasks.Set(typ) + c.dropMasks.Set(typ.ID()) } } } + for _, name := range c.ExcludedImages { c.excludedImages[name] = true } + + if c.allMasks == nil { + c.allMasks = bitmask.New() + } + for _, typ := range event.AllWithState() { + c.allMasks.Set(typ.ID()) + } } // SetDropMask inserts the event mask in the bitset to // instruct the given event type should be dropped from // the event stream. -func (c *EventSourceConfig) SetDropMask(Type event.Type) { - c.dropMasks.Set(Type) +func (c *EventSourceConfig) SetDropMask(typ event.Type) { + c.dropMasks.Set(typ.ID()) } // TestDropMask checks if the specified event type has // the drop mask in the bitset. -func (c *EventSourceConfig) TestDropMask(Type event.Type) bool { - return c.dropMasks.Test(Type.GUID(), Type.HookID()) +func (c *EventSourceConfig) TestDropMask(typ event.Type) bool { + return c.dropMasks.IsSet(typ.ID()) +} + +// ExcludeEvent determines whether the supplied short +// event ID exists in the bitset of excluded events. +func (c *EventSourceConfig) ExcludeEvent(id uint) bool { + return c.dropMasks.IsSet(id) } -// ExcludeEvent determines whether the supplied provider GUID -// and the hook identifier are in the bitset of excluded events. -func (c *EventSourceConfig) ExcludeEvent(guid windows.GUID, hookID uint16) bool { - return c.dropMasks.Test(guid, hookID) +// EventExists determines if the provided event ID exists +// in the internal event catalog by checking the event ID +// bitmask. +func (c *EventSourceConfig) EventExists(id uint) bool { + return c.allMasks.IsSet(id) } // ExcludeImage determines whether the process generating event is present in the diff --git a/pkg/config/eventsource_test.go b/pkg/config/eventsource_test.go index cd43354fb..614df5ef7 100644 --- a/pkg/config/eventsource_test.go +++ b/pkg/config/eventsource_test.go @@ -55,8 +55,8 @@ func TestEventSourceConfig(t *testing.T) { assert.False(t, c.EventSource.EnableImageEvents) assert.False(t, c.EventSource.EnableFileIOEvents) - assert.True(t, c.EventSource.ExcludeEvent(event.CloseHandle.GUID(), event.CloseHandle.HookID())) - assert.False(t, c.EventSource.ExcludeEvent(event.CreateProcess.GUID(), event.CreateProcess.HookID())) + assert.True(t, c.EventSource.ExcludeEvent(event.CloseHandle.ID())) + assert.False(t, c.EventSource.ExcludeEvent(event.CreateProcess.ID())) assert.True(t, c.EventSource.ExcludeImage(&pstypes.PS{Name: "svchost.exe"})) assert.False(t, c.EventSource.ExcludeImage(&pstypes.PS{Name: "explorer.exe"})) diff --git a/pkg/event/bitset.go b/pkg/event/bitset.go new file mode 100644 index 000000000..2a87e3a17 --- /dev/null +++ b/pkg/event/bitset.go @@ -0,0 +1,94 @@ +/* + * Copyright 2021-2022 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 event + +import ( + "github.com/bits-and-blooms/bitset" + "github.com/rabbitstack/fibratus/pkg/util/bitmask" +) + +// BitSetType defines the bitset type +type BitSetType uint8 + +const ( + // BitmaskBitSet designates the mask-based event id bitset + BitmaskBitSet BitSetType = iota + 1 + // TypeBitSet designates the uint16 number space event type bitset + TypeBitSet + // CategoryBitSet designates the event category bitset + CategoryBitSet +) + +// BitSets handles the group of category/event type bitsets +// and the bitmask for evaluating event ids bits. +type BitSets struct { + bitmask *bitmask.Bitmask + cats *bitset.BitSet + types *bitset.BitSet +} + +// SetBit sets the bit dictated by the bitset type. +func (b *BitSets) SetBit(bs BitSetType, typ Type) { + switch bs { + case BitmaskBitSet: + if b.bitmask == nil { + b.bitmask = bitmask.New() + } + b.bitmask.Set(typ.ID()) + + case TypeBitSet: + if b.types == nil { + b.types = bitset.New(uint(MaxTypeID() + 1)) + } + b.types.Set(uint(typ.HookID())) + + case CategoryBitSet: + if b.cats == nil { + b.cats = bitset.New(MaxCategoryIndex + 1) + } + b.cats.Set(uint(typ.Category().Index())) + } +} + +// SetCategoryBit toggles the category bit in the bitset. +func (b *BitSets) SetCategoryBit(c Category) { + if b.cats == nil { + b.cats = bitset.New(MaxCategoryIndex + 1) + } + b.cats.Set(uint(c.Index())) +} + +// IsBitSet checks if any of the populated bitsets +// contain the type, event ID, or category bit. +// This method evaluates first the event type bitset. +// The event type bitset should only be initialized +// if all event types pertain to the same category. +// Otherwise, event id bitset and last category bitset +// are tested for respective bits. +func (b *BitSets) IsBitSet(evt *Event) bool { + if b.types != nil && b.types.Test(uint(evt.Type.HookID())) { + return true + } + return (b.bitmask != nil && b.bitmask.IsSet(evt.Type.ID())) || + (b.cats != nil && b.cats.Test(uint(evt.Category.Index()))) +} + +func (b *BitSets) IsBitmaskInitialized() bool { return b.bitmask != nil } +func (b *BitSets) IsTypesInitialized() bool { return b.types != nil } +func (b *BitSets) IsCategoryInitialized() bool { return b.cats != nil } diff --git a/pkg/event/bitset_test.go b/pkg/event/bitset_test.go new file mode 100644 index 000000000..8aed8ef43 --- /dev/null +++ b/pkg/event/bitset_test.go @@ -0,0 +1,128 @@ +/* + * Copyright 2021-2022 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 event + +import ( + "github.com/rabbitstack/fibratus/pkg/util/bitmask" + "testing" + + "github.com/rabbitstack/fibratus/pkg/sys/etw" + "github.com/stretchr/testify/assert" +) + +func TestBitmask(t *testing.T) { + var tests = []struct { + typ Type + expected bool + }{ + {TerminateThread, true}, + {TerminateProcess, true}, + {CreateThread, true}, + {CreateFile, false}, + {WriteFile, false}, + {LoadImage, false}, + {MapFileRundown, true}, + {ProcessRundown, true}, + } + + b := bitmask.New() + for _, typ := range AllWithState() { + if typ == WriteFile || typ == LoadImage || typ == CreateFile { + continue + } + b.Set(typ.ID()) + } + + for _, tt := range tests { + t.Run(tt.typ.String(), func(t *testing.T) { + assert.Equal(t, tt.expected, b.IsSet(tt.typ.ID())) + }) + } +} + +func TestBitSets(t *testing.T) { + var tests = []struct { + evt *Event + expected bool + }{ + {&Event{Type: TerminateThread}, true}, + {&Event{Type: TerminateProcess}, true}, + {&Event{Type: CreateThread, Category: Thread}, true}, + {&Event{Type: CreateFile}, false}, + {&Event{Type: WriteFile}, false}, + {&Event{Type: LoadImage}, false}, + {&Event{Type: MapFileRundown}, true}, + {&Event{Type: ProcessRundown}, true}, + } + + var bitsets BitSets + + bitsets.SetBit(BitmaskBitSet, TerminateThread) + bitsets.SetBit(TypeBitSet, TerminateProcess) + bitsets.SetBit(CategoryBitSet, CreateThread) + bitsets.SetBit(TypeBitSet, MapFileRundown) + bitsets.SetBit(BitmaskBitSet, ProcessRundown) + + for _, tt := range tests { + t.Run(tt.evt.Type.String(), func(t *testing.T) { + assert.Equal(t, tt.expected, bitsets.IsBitSet(tt.evt)) + }) + } +} + +func BenchmarkBitmask(b *testing.B) { + b.ReportAllocs() + + bm := bitmask.New() + bm.Set(TerminateThread.ID()) + bm.Set(CreateThread.ID()) + bm.Set(TerminateProcess.ID()) + bm.Set(CreateFile.ID()) + + evt := &etw.EventRecord{Header: etw.EventHeader{ProviderID: ThreadEventGUID, EventDescriptor: etw.EventDescriptor{Opcode: 2}}} + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + if !bm.IsSet(evt.ID()) { + panic("mask should be present") + } + } +} + +func BenchmarkStdlibMap(b *testing.B) { + b.ReportAllocs() + + evts := make(map[Type]bool) + evts[TerminateThread] = true + evts[CreateThread] = true + evts[TerminateProcess] = true + evts[CreateFile] = true + + evt := etw.EventRecord{Header: etw.EventHeader{ProviderID: ThreadEventGUID, EventDescriptor: etw.EventDescriptor{Opcode: 2}}} + etype := NewTypeFromEventRecord(&evt) + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + if !evts[etype] { + panic("event should be present") + } + } +} diff --git a/pkg/event/category.go b/pkg/event/category.go index 119dd55bd..0f86043c7 100644 --- a/pkg/event/category.go +++ b/pkg/event/category.go @@ -19,7 +19,6 @@ package event import ( - "github.com/bits-and-blooms/bitset" "github.com/rabbitstack/fibratus/pkg/util/hashers" ) @@ -70,21 +69,6 @@ func (c Category) Hash() uint32 { return hashers.FnvUint32([]byte(c)) } -// CategoryMasks allows setting and checking the category bit mask. -type CategoryMasks struct { - bs bitset.BitSet -} - -// Set sets the category bit in the bit mask. -func (m *CategoryMasks) Set(c Category) { - m.bs.Set(uint(c.Index())) -} - -// Test checks if the given category bit is set. -func (m *CategoryMasks) Test(c Category) bool { - return m.bs.Test(uint(c.Index())) -} - // MaxCategoryIndex designates the maximum category index. const MaxCategoryIndex = 13 diff --git a/pkg/event/event_windows.go b/pkg/event/event_windows.go index 6269605ea..32e103863 100644 --- a/pkg/event/event_windows.go +++ b/pkg/event/event_windows.go @@ -47,13 +47,15 @@ var ( // New constructs a fresh event instance with basic fields and parameters // from the raw ETW event record. -func New(seq uint64, typ Type, evt *etw.EventRecord) *Event { +func New(seq uint64, evt *etw.EventRecord) *Event { var ( pid = evt.Header.ProcessID tid = evt.Header.ThreadID cpu = *(*uint8)(unsafe.Pointer(&evt.BufferContext.ProcessorIndex[0])) ts = filetime.ToEpoch(evt.Header.Timestamp) + typ = NewTypeFromEventRecord(evt) ) + e := &Event{ Seq: seq, PID: pid, @@ -68,8 +70,10 @@ func New(seq uint64, typ Type, evt *etw.EventRecord) *Event { Metadata: make(map[MetadataKey]any), Host: hostname.Get(), } + e.produceParams(evt) e.adjustPID() + return e } diff --git a/pkg/event/eventset.go b/pkg/event/eventset.go deleted file mode 100644 index 614029027..000000000 --- a/pkg/event/eventset.go +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Copyright 2021-2022 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 event - -import ( - "fmt" - "github.com/bits-and-blooms/bitset" - "golang.org/x/sys/windows" -) - -// EventsetMasks allows efficient testing -// of a group of bitsets containing event -// hook identifiers. For each provider -// represented by a GUID, a dedicated -// bitset is defined. -type EventsetMasks struct { - masks [ProvidersCount]bitset.BitSet -} - -// Set puts a new event type into the bitset. -func (e *EventsetMasks) Set(typ Type) { - g := typ.GUID() - i := e.bitsetIndex(g) - if i < 0 { - panic(fmt.Sprintf("invalid event bitset index: %s", g.String())) - } - e.masks[e.bitsetIndex(typ.GUID())].Set(uint(typ.HookID())) -} - -// Test checks if the given provider GUID and -// hook identifier are present in the bitset. -func (e *EventsetMasks) Test(guid windows.GUID, hookID uint16) bool { - i := e.bitsetIndex(guid) - if i < 0 { - return false - } - return e.masks[e.bitsetIndex(guid)].Test(uint(hookID)) -} - -// Clear clears the bitset for a given provider GUID. -func (e *EventsetMasks) Clear(guid windows.GUID) { - i := e.bitsetIndex(guid) - if i < 0 { - panic(fmt.Sprintf("invalid event bitset index: %s", guid.String())) - } - e.masks[e.bitsetIndex(guid)].ClearAll() -} - -func (e *EventsetMasks) bitsetIndex(guid windows.GUID) int { - switch guid { - case ProcessEventGUID: - return 0 - case ThreadEventGUID: - return 1 - case ImageEventGUID: - return 2 - case FileEventGUID: - return 3 - case RegistryEventGUID: - return 4 - case NetworkTCPEventGUID: - return 5 - case NetworkUDPEventGUID: - return 6 - case HandleEventGUID: - return 7 - case MemEventGUID: - return 8 - case AuditAPIEventGUID: - return 9 - case DNSEventGUID: - return 10 - case ThreadpoolGUID: - return 11 - default: - return -1 - } -} diff --git a/pkg/event/eventset_test.go b/pkg/event/eventset_test.go deleted file mode 100644 index 89d483523..000000000 --- a/pkg/event/eventset_test.go +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Copyright 2021-2022 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 event - -import ( - "github.com/rabbitstack/fibratus/pkg/sys/etw" - "github.com/stretchr/testify/require" - "testing" -) - -func TestEventsetMasks(t *testing.T) { - var masks EventsetMasks - masks.Set(TerminateThread) - masks.Set(CreateThread) - masks.Set(TerminateProcess) - - require.True(t, masks.Test(ThreadEventGUID, TerminateThread.HookID())) - require.True(t, masks.Test(ThreadEventGUID, CreateThread.HookID())) - require.False(t, masks.Test(ThreadEventGUID, ThreadRundown.HookID())) - require.True(t, masks.Test(ProcessEventGUID, TerminateProcess.HookID())) - require.False(t, masks.Test(ProcessEventGUID, CreateProcess.HookID())) - - masks.Clear(ThreadEventGUID) - - require.False(t, masks.Test(ThreadEventGUID, TerminateThread.HookID())) - require.False(t, masks.Test(ThreadEventGUID, CreateThread.HookID())) -} - -func BenchmarkEventsetMasks(b *testing.B) { - b.ReportAllocs() - - var masks EventsetMasks - masks.Set(TerminateThread) - masks.Set(CreateThread) - masks.Set(TerminateProcess) - masks.Set(CreateFile) - - evt := etw.EventRecord{Header: etw.EventHeader{ProviderID: ThreadEventGUID, EventDescriptor: etw.EventDescriptor{Opcode: 2}}} - - for i := 0; i < b.N; i++ { - if !masks.Test(evt.Header.ProviderID, uint16(evt.Header.EventDescriptor.Opcode)) { - panic("mask should be present") - } - } -} - -func BenchmarkStdlibMap(b *testing.B) { - b.ReportAllocs() - - evts := make(map[Type]bool) - evts[TerminateThread] = true - evts[CreateThread] = true - evts[TerminateProcess] = true - evts[CreateFile] = true - - evt := etw.EventRecord{Header: etw.EventHeader{ProviderID: ThreadEventGUID, EventDescriptor: etw.EventDescriptor{Opcode: 2}}} - kt := NewFromEventRecord(&evt) - - for i := 0; i < b.N; i++ { - if !evts[kt] { - panic("event should be present") - } - } -} diff --git a/pkg/event/metainfo_windows.go b/pkg/event/metainfo_windows.go index a052be00c..d400563eb 100644 --- a/pkg/event/metainfo_windows.go +++ b/pkg/event/metainfo_windows.go @@ -216,12 +216,42 @@ var indexedEvents = []Info{ // All returns all event types. func All() []Type { s := make([]Type, 0, len(types)) - for _, Type := range types { - s = append(s, Type) + for _, typ := range types { + s = append(s, typ) } return s } +// AllWithState returns all event types + +// event types used for state management. +func AllWithState() []Type { + s := All() + + s = append(s, ProcessRundown) + s = append(s, ThreadRundown) + s = append(s, ImageRundown) + s = append(s, FileRundown) + s = append(s, RegKCBRundown) + s = append(s, RegCreateKCB) + s = append(s, RegDeleteKCB) + s = append(s, FileOpEnd) + s = append(s, ReleaseFile) + s = append(s, MapFileRundown) + s = append(s, StackWalk) + + return s +} + +// MaxTypeID returns the maximum event type (hook id) value. +func MaxTypeID() uint16 { + types := AllWithState() + ids := make([]uint16, len(types)) + for i, t := range types { + ids[i] = t.HookID() + } + return slices.Max(ids) +} + // TypeToEventInfo maps the event type to the structure storing detailed information about the event. func TypeToEventInfo(typ Type) Info { if info, ok := events[typ]; ok { diff --git a/pkg/event/types_windows.go b/pkg/event/types_windows.go index f9658aa2d..125f19d3f 100644 --- a/pkg/event/types_windows.go +++ b/pkg/event/types_windows.go @@ -25,10 +25,6 @@ import ( "golang.org/x/sys/windows" ) -// ProvidersCount designates the number of interesting providers. -// Remember to increment if a new event source is introduced. -const ProvidersCount = 12 - // Source is the type that designates the provenance of the event type Source uint8 @@ -223,8 +219,8 @@ var ( UnknownType = pack(windows.GUID{}, 0) ) -// NewFromEventRecord creates a new event type from ETW event record. -func NewFromEventRecord(ev *etw.EventRecord) Type { +// NewTypeFromEventRecord creates a new event type from ETW event record. +func NewTypeFromEventRecord(ev *etw.EventRecord) Type { return pack(ev.Header.ProviderID, ev.HookID()) } @@ -573,6 +569,20 @@ func (t *Type) HookID() uint16 { return binary.BigEndian.Uint16(t[16:]) } +// ID is an unsigned integer that uniquely +// identifies the event. Handy for bitmask +// operations. +func (t Type) ID() uint { + id := uint(t[0])<<56 | + uint(t[1])<<48 | + uint(t[2])<<40 | + uint(t[3])<<32 | + uint(t[4])<<24 | + uint(t[5])<<16 | + uint(t.HookID()) + return id +} + // Source designates the provenance of this event type. func (t Type) Source() Source { switch t { diff --git a/pkg/event/types_windows_test.go b/pkg/event/types_windows_test.go index 588b580ae..371feca01 100644 --- a/pkg/event/types_windows_test.go +++ b/pkg/event/types_windows_test.go @@ -80,7 +80,7 @@ func TestEventTypeComparison(t *testing.T) { } func TestNewEventTypeFromEventRecord(t *testing.T) { - assert.Equal(t, CreateProcess, NewFromEventRecord(&etw.EventRecord{ + assert.Equal(t, CreateProcess, NewTypeFromEventRecord(&etw.EventRecord{ Header: etw.EventHeader{ ProviderID: windows.GUID{Data1: 0x3d6fa8d0, Data2: 0xfe05, Data3: 0x11d0, Data4: [8]byte{0x9d, 0xda, 0x0, 0xc0, 0x4f, 0xd7, 0xba, 0x7c}}, EventDescriptor: etw.EventDescriptor{ @@ -88,7 +88,7 @@ func TestNewEventTypeFromEventRecord(t *testing.T) { }, }, })) - assert.Equal(t, OpenProcess, NewFromEventRecord(&etw.EventRecord{ + assert.Equal(t, OpenProcess, NewTypeFromEventRecord(&etw.EventRecord{ Header: etw.EventHeader{ ProviderID: windows.GUID{Data1: 0xe02a841c, Data2: 0x75a3, Data3: 0x4fa7, Data4: [8]byte{0xaf, 0xc8, 0xae, 0x09, 0xcf, 0x9b, 0x7f, 0x23}}, EventDescriptor: etw.EventDescriptor{ @@ -103,6 +103,10 @@ func TestEventTypeExists(t *testing.T) { require.True(t, AcceptTCPv6.Exists()) } +func TestTypeID(t *testing.T) { + assert.Equal(t, uint(14439051552138264620), SetThreadpoolTimer.ID()) +} + func TestGUIDAndHookIDFromEventType(t *testing.T) { var tests = []struct { Type Type @@ -128,3 +132,19 @@ func TestGUIDAndHookIDFromEventType(t *testing.T) { }) } } + +func TestIDEquality(t *testing.T) { + evt := etw.EventRecord{Header: etw.EventHeader{ProviderID: ThreadEventGUID, EventDescriptor: etw.EventDescriptor{Opcode: 1}}} + typ := CreateThread + require.Equal(t, typ.ID(), evt.ID()) +} + +func TestEventTypeIDCollision(t *testing.T) { + ids := make(map[uint]Type) + for _, typ := range AllWithState() { + if etype, ok := ids[typ.ID()]; ok { + t.Fatalf("id collision for %s event type. Mapped event type: %s", typ.String(), etype.String()) + } + ids[typ.ID()] = typ + } +} diff --git a/pkg/filter/ql/literal.go b/pkg/filter/ql/literal.go index 8da866f36..d102d851e 100644 --- a/pkg/filter/ql/literal.go +++ b/pkg/filter/ql/literal.go @@ -278,10 +278,8 @@ type SequenceExpr struct { // Alias represents the sequence expression alias. Alias string - emasks event.EventsetMasks - cmasks event.CategoryMasks - - types []event.Type + bitsets event.BitSets + types []event.Type } func (e *SequenceExpr) init() { @@ -335,28 +333,47 @@ func (e *SequenceExpr) walk() { WalkFunc(e.Expr, walk) + uniqCats := make(map[event.Category]bool) + // initialize event type/category buckets for every such field for name, values := range stringFields { for _, v := range values { switch name { case fields.EvtName: for _, typ := range event.NameToTypes(v) { - e.emasks.Set(typ) + if typ == event.UnknownType { + continue + } e.types = append(e.types, typ) + uniqCats[event.TypeToEventInfo(typ).Category] = true } case fields.EvtCategory: - e.cmasks.Set(event.Category(v)) + e.bitsets.SetCategoryBit(event.Category(v)) } } } + + for _, t := range e.types { + switch len(uniqCats) { + case 0: + continue + case 1: + // happy path can use a single bitmask for all + // event types pertaining to the same category + e.bitsets.SetBit(event.TypeBitSet, t) + default: + // use map-backed bitmask for event identifiers + e.bitsets.SetBit(event.BitmaskBitSet, t) + } + } } // IsEvaluable determines if the expression should be evaluated by inspecting // the event type filter fields defined in the expression. We permit the expression -// to be evaluated when the incoming event type or category pertains to the one +// to be evaluated when the incoming event type, ID, or category pertains to the one // defined in the field literal. func (e *SequenceExpr) IsEvaluable(evt *event.Event) bool { - return e.emasks.Test(evt.Type.GUID(), evt.Type.HookID()) || e.cmasks.Test(evt.Category) + return e.bitsets.IsBitSet(evt) } // HasBoundFields determines if this sequence expression references any bound field. diff --git a/pkg/filter/ql/literal_test.go b/pkg/filter/ql/literal_test.go new file mode 100644 index 000000000..92269d2e9 --- /dev/null +++ b/pkg/filter/ql/literal_test.go @@ -0,0 +1,98 @@ +/* + * 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 ql + +import ( + "github.com/rabbitstack/fibratus/pkg/event" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "testing" +) + +func TestSequenceExprIsEvaluable(t *testing.T) { + var tests = []struct { + expr string + evt *event.Event + isEval bool + assertions func(t *testing.T, sexpr *SequenceExpr) + }{ + {"evt.name = 'CreateProcess'", &event.Event{Type: event.CreateProcess, Category: event.Process}, true, + func(t *testing.T, sexpr *SequenceExpr) { + assert.True(t, sexpr.bitsets.IsTypesInitialized()) + assert.False(t, sexpr.bitsets.IsBitmaskInitialized()) + assert.False(t, sexpr.bitsets.IsCategoryInitialized()) + }, + }, + {"evt.name = 'CreateProcess'", &event.Event{Type: event.TerminateProcess, Category: event.Process}, false, nil}, + {"evt.name = 'CreateProcess' or evt.name = 'TerminateThread'", &event.Event{Type: event.TerminateProcess, Category: event.Process}, false, nil}, + {"evt.name = 'CreateProcess' or evt.category = 'object'", &event.Event{Type: event.TerminateProcess, Category: event.Process}, false, nil}, + {"evt.name = 'CreateProcess' or evt.name = 'OpenProcess'", &event.Event{Type: event.OpenProcess, Category: event.Process}, true, + func(t *testing.T, sexpr *SequenceExpr) { + assert.True(t, sexpr.bitsets.IsTypesInitialized()) + assert.False(t, sexpr.bitsets.IsBitmaskInitialized()) + assert.False(t, sexpr.bitsets.IsCategoryInitialized()) + }, + }, + {"evt.name = 'CreateProcess' or evt.name = 'CreateThread'", &event.Event{Type: event.CreateThread, Category: event.Thread}, true, + func(t *testing.T, sexpr *SequenceExpr) { + assert.False(t, sexpr.bitsets.IsTypesInitialized()) + assert.True(t, sexpr.bitsets.IsBitmaskInitialized()) + assert.False(t, sexpr.bitsets.IsCategoryInitialized()) + }, + }, + {"evt.name = 'CreateProcess' or evt.category = 'registry'", &event.Event{Type: event.RegSetValue, Category: event.Registry}, true, + func(t *testing.T, sexpr *SequenceExpr) { + assert.True(t, sexpr.bitsets.IsTypesInitialized()) + assert.False(t, sexpr.bitsets.IsBitmaskInitialized()) + assert.True(t, sexpr.bitsets.IsCategoryInitialized()) + }, + }, + {"evt.name = 'CreateProcess' or evt.name = 'OpenProcess' or evt.category = 'registry'", &event.Event{Type: event.OpenProcess, Category: event.Process}, true, + func(t *testing.T, sexpr *SequenceExpr) { + assert.True(t, sexpr.bitsets.IsTypesInitialized()) + assert.False(t, sexpr.bitsets.IsBitmaskInitialized()) + assert.True(t, sexpr.bitsets.IsCategoryInitialized()) + }, + }, + {"evt.name = 'CreateProcess' or evt.name = 'SetThreadContext' or evt.category = 'registry'", &event.Event{Type: event.CreateProcess, Category: event.Process}, true, + func(t *testing.T, sexpr *SequenceExpr) { + assert.False(t, sexpr.bitsets.IsTypesInitialized()) + assert.True(t, sexpr.bitsets.IsBitmaskInitialized()) + assert.True(t, sexpr.bitsets.IsCategoryInitialized()) + }, + }, + } + + for _, tt := range tests { + t.Run(tt.expr, func(t *testing.T) { + p := NewParser(tt.expr) + expr, err := p.ParseExpr() + require.NoError(t, err) + + sexpr := &SequenceExpr{Expr: expr} + sexpr.init() + sexpr.walk() + + assert.Equal(t, tt.isEval, sexpr.IsEvaluable(tt.evt)) + if tt.assertions != nil { + tt.assertions(t, sexpr) + } + }) + } +} diff --git a/pkg/sys/etw/types.go b/pkg/sys/etw/types.go index 27c1a37e0..ffc6c881b 100644 --- a/pkg/sys/etw/types.go +++ b/pkg/sys/etw/types.go @@ -564,6 +564,24 @@ func (e *EventRecord) HookID() uint16 { return e.Header.EventDescriptor.ID } +// ID is an unsigned integer that uniquely +// identifies the event. Handy for bitmask +// operations. +func (e *EventRecord) ID() uint { + d1 := e.Header.ProviderID.Data1 + d2 := e.Header.ProviderID.Data2 + + id := uint(byte(d1>>24))<<56 | + uint(byte(d1>>16))<<48 | + uint(byte(d1>>8))<<40 | + uint(byte(d1))<<32 | + uint(byte(d2>>8))<<24 | + uint(byte(d2))<<16 | + uint(e.HookID()) + + return id +} + // ReadByte reads the byte from the buffer at the specified offset. func (e *EventRecord) ReadByte(offset uint16) byte { if offset > e.BufferLen { diff --git a/pkg/sys/etw/types_test.go b/pkg/sys/etw/types_test.go index f1b4bc087..da44ba3dd 100644 --- a/pkg/sys/etw/types_test.go +++ b/pkg/sys/etw/types_test.go @@ -98,3 +98,8 @@ func TestReadBuffer(t *testing.T) { tt.assertions(t, ev) } } + +func TestID(t *testing.T) { + ev := &EventRecord{Header: EventHeader{ProviderID: ThreadpoolGUID, EventDescriptor: EventDescriptor{ID: 44}}} + assert.Equal(t, uint(14439051552138264620), ev.ID()) +} diff --git a/pkg/event/category_test.go b/pkg/util/bitmask/bitmask.go similarity index 50% rename from pkg/event/category_test.go rename to pkg/util/bitmask/bitmask.go index 6b6a6eedb..20cc2a1f1 100644 --- a/pkg/event/category_test.go +++ b/pkg/util/bitmask/bitmask.go @@ -16,19 +16,42 @@ * limitations under the License. */ -package event +package bitmask -import ( - "github.com/stretchr/testify/assert" - "testing" -) +// Bitmask is the map-backed bitmask. +// Each bit index i is split into: +// +// wordIdx = i / 64 +// bitPos = i % 64 +// +// This allows a virtually unbounded sparse bitset over uint64. +type Bitmask struct { + words map[uint]uint +} + +func New() *Bitmask { + return &Bitmask{ + words: make(map[uint]uint), + } +} -func TestCategoryMasks(t *testing.T) { - var masks CategoryMasks - masks.Set(File) - masks.Set(Process) +func (b *Bitmask) Set(i uint) { + word := i / 64 + bit := i % 64 + b.words[word] |= 1 << bit +} + +func (b *Bitmask) Clear(i uint) { + word := i / 64 + bit := i % 64 + b.words[word] &^= 1 << bit + if b.words[word] == 0 { + delete(b.words, word) + } +} - assert.True(t, masks.Test(File)) - assert.True(t, masks.Test(Process)) - assert.False(t, masks.Test(Registry)) +func (b *Bitmask) IsSet(i uint) bool { + word := i / 64 + bit := i % 64 + return b.words[word]&(1<