Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
File renamed without changes.
17 changes: 16 additions & 1 deletion internal/etw/processors/fs_windows.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/rabbitstack/fibratus/pkg/sys"
"github.com/rabbitstack/fibratus/pkg/util/va"
"golang.org/x/sys/windows"
"golang.org/x/time/rate"
"sync"
"time"
)
Expand All @@ -43,6 +44,8 @@ var (
fileObjectMisses = expvar.NewInt("fs.file.objects.misses")
fileObjectHandleHits = expvar.NewInt("fs.file.object.handle.hits")
fileReleaseCount = expvar.NewInt("fs.file.releases")

fsFileCharacteristicsRateLimits = expvar.NewInt("fs.file.characteristics.rate.limits")
)

type fsProcessor struct {
Expand All @@ -65,6 +68,8 @@ type fsProcessor struct {
purger *time.Ticker

quit chan struct{}
// lim throttles the parsing of image characteristics
lim *rate.Limiter
}

// FileInfo stores file information obtained from event state.
Expand All @@ -91,6 +96,7 @@ func newFsProcessor(
buckets: make(map[uint64][]*kevent.Kevent),
purger: time.NewTicker(time.Second * 5),
quit: make(chan struct{}, 1),
lim: rate.NewLimiter(30, 40), // allow 30 parse ops per second or bursts of 40 ops
}

go f.purge()
Expand Down Expand Up @@ -239,10 +245,19 @@ func (f *fsProcessor) processEvent(e *kevent.Kevent) (*kevent.Kevent, error) {

// parse PE data for created files and append parameters
if ev.IsCreateDisposition() && ev.IsSuccess() {
err := parseImageFileCharacteristics(ev)
if !f.lim.Allow() {
fsFileCharacteristicsRateLimits.Add(1)
return ev, nil
}
path := ev.GetParamAsString(kparams.FilePath)
c, err := parseImageFileCharacteristics(path)
if err != nil {
return ev, nil
}
ev.AppendParam(kparams.FileIsDLL, kparams.Bool, c.isDLL)
ev.AppendParam(kparams.FileIsDriver, kparams.Bool, c.isDriver)
ev.AppendParam(kparams.FileIsExecutable, kparams.Bool, c.isExe)
ev.AppendParam(kparams.FileIsDotnet, kparams.Bool, c.isDotnet)
}

return ev, nil
Expand Down
39 changes: 37 additions & 2 deletions internal/etw/processors/fs_windows_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,11 +32,15 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
"os"
"reflect"
"testing"
)

func TestFsProcessor(t *testing.T) {
exe, err := os.Executable()
require.NoError(t, err)

var tests = []struct {
name string
e *kevent.Kevent
Expand Down Expand Up @@ -143,7 +147,7 @@ func TestFsProcessor(t *testing.T) {
Kparams: kevent.Kparams{
kparams.FileObject: {Name: kparams.FileObject, Type: kparams.Uint64, Value: uint64(12446738026482168384)},
kparams.FileCreateOptions: {Name: kparams.FileCreateOptions, Type: kparams.Uint32, Value: uint32(18874368)},
kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: "C:\\Windows\\temp\\idxx.exe"},
kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: exe},
kparams.FileShareMask: {Name: kparams.FileShareMask, Type: kparams.Uint32, Value: uint32(5)},
kparams.FileIrpPtr: {Name: kparams.FileIrpPtr, Type: kparams.Uint64, Value: uint64(1334543123112321)},
},
Expand All @@ -159,10 +163,14 @@ func TestFsProcessor(t *testing.T) {
assert.NotContains(t, fsProcessor.irps, uint64(1334543123112321))
assert.False(t, e.WaitEnqueue)
assert.Contains(t, fsProcessor.files, uint64(12446738026482168384))
assert.Equal(t, "C:\\Windows\\temp\\idxx.exe", fsProcessor.files[12446738026482168384].Name)
assert.Equal(t, exe, fsProcessor.files[12446738026482168384].Name)
assert.Equal(t, "Success", e.GetParamAsString(kparams.NTStatus))
assert.Equal(t, "File", e.GetParamAsString(kparams.FileType))
assert.Equal(t, "CREATE", e.GetParamAsString(kparams.FileOperation))
assert.True(t, e.Kparams.MustGetBool(kparams.FileIsExecutable))
assert.False(t, e.Kparams.MustGetBool(kparams.FileIsDotnet))
assert.False(t, e.Kparams.MustGetBool(kparams.FileIsDLL))
assert.False(t, e.Kparams.MustGetBool(kparams.FileIsDriver))
},
},
{
Expand All @@ -188,6 +196,33 @@ func TestFsProcessor(t *testing.T) {
assert.Empty(t, fsProcessor.files)
},
},
{
"parse created file characteristics",
&kevent.Kevent{
Type: ktypes.CreateFile,
Category: ktypes.File,
Kparams: kevent.Kparams{
kparams.FileObject: {Name: kparams.FileObject, Type: kparams.Uint64, Value: uint64(18446738026482168384)},
kparams.ThreadID: {Name: kparams.ThreadID, Type: kparams.Uint32, Value: uint32(1484)},
kparams.FileCreateOptions: {Name: kparams.FileCreateOptions, Type: kparams.Uint32, Value: uint32(1223456)},
kparams.FilePath: {Name: kparams.FilePath, Type: kparams.UnicodeString, Value: exe},
kparams.FileShareMask: {Name: kparams.FileShareMask, Type: kparams.Uint32, Value: uint32(5)},
kparams.FileIrpPtr: {Name: kparams.FileIrpPtr, Type: kparams.Uint64, Value: uint64(1234543123112321)},
kparams.FileOperation: {Name: kparams.FileOperation, Type: kparams.Uint64, Value: uint64(2)},
},
},
nil,
func() *handle.SnapshotterMock {
hsnap := new(handle.SnapshotterMock)
return hsnap
},
func(e *kevent.Kevent, t *testing.T, hsnap *handle.SnapshotterMock, p Processor) {
fsProcessor := p.(*fsProcessor)
assert.True(t, e.WaitEnqueue)
assert.Contains(t, fsProcessor.irps, uint64(1234543123112321))
assert.True(t, reflect.DeepEqual(e, fsProcessor.irps[1234543123112321]))
},
},
{
"unmap view file",
&kevent.Kevent{
Expand Down
78 changes: 70 additions & 8 deletions internal/etw/processors/image_windows.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,29 +19,71 @@
package processors

import (
"expvar"
"github.com/rabbitstack/fibratus/pkg/kevent"
"github.com/rabbitstack/fibratus/pkg/kevent/kparams"
"github.com/rabbitstack/fibratus/pkg/ps"
"sync"
"time"
)

var imageFileCharacteristicsCacheHits = expvar.NewInt("image.file.characteristics.cache.hits")

var modTTL = time.Minute * 10

type imageProcessor struct {
psnap ps.Snapshotter
psnap ps.Snapshotter
mods map[string]*imageFileCharacteristics
mu sync.Mutex
purger *time.Ticker
quit chan struct{}
}

func newImageProcessor(psnap ps.Snapshotter) Processor {
return &imageProcessor{psnap: psnap}
m := &imageProcessor{
psnap: psnap,
mods: make(map[string]*imageFileCharacteristics),
purger: time.NewTicker(time.Minute),
quit: make(chan struct{}, 1),
}

go m.purge()

return m
}

func (imageProcessor) Name() ProcessorType { return Image }
func (*imageProcessor) Name() ProcessorType { return Image }

func (m *imageProcessor) ProcessEvent(e *kevent.Kevent) (*kevent.Kevent, bool, error) {
if e.IsLoadImage() {
// parse PE image data
err := parseImageFileCharacteristics(e)
if err != nil {
return e, false, m.psnap.AddModule(e)
// is image characteristics data cached?
path := e.GetParamAsString(kparams.ImagePath)
key := path + e.GetParamAsString(kparams.ImageCheckSum)

m.mu.Lock()
defer m.mu.Unlock()
c, ok := m.mods[key]
if !ok {
// parse PE image data
var err error
c, err = parseImageFileCharacteristics(path)
if err != nil {
return e, false, m.psnap.AddModule(e)
}
c.keepalive()
m.mods[key] = c
} else {
imageFileCharacteristicsCacheHits.Add(1)
c.keepalive()
}

// append event parameters
e.AppendParam(kparams.FileIsDLL, kparams.Bool, c.isDLL)
e.AppendParam(kparams.FileIsDriver, kparams.Bool, c.isDriver)
e.AppendParam(kparams.FileIsExecutable, kparams.Bool, c.isExe)
e.AppendParam(kparams.FileIsDotnet, kparams.Bool, c.isDotnet)
}

if e.IsUnloadImage() {
pid := e.Kparams.MustGetPid()
addr := e.Kparams.TryGetAddress(kparams.ImageBase)
Expand All @@ -50,10 +92,30 @@ func (m *imageProcessor) ProcessEvent(e *kevent.Kevent) (*kevent.Kevent, bool, e
}
return e, false, m.psnap.RemoveModule(pid, addr)
}

if e.IsLoadImage() || e.IsImageRundown() {
return e, false, m.psnap.AddModule(e)
}
return e, true, nil
}

func (imageProcessor) Close() {}
func (m *imageProcessor) Close() {
m.quit <- struct{}{}
}

func (m *imageProcessor) purge() {
for {
select {
case <-m.purger.C:
m.mu.Lock()
for key, mod := range m.mods {
if time.Since(mod.accessed) > modTTL {
delete(m.mods, key)
}
}
m.mu.Unlock()
case <-m.quit:
return
}
}
}
27 changes: 27 additions & 0 deletions internal/etw/processors/image_windows_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,33 @@ func TestImageProcessor(t *testing.T) {
assert.Equal(t, "AUTHENTICODE", e.GetParamAsString(kparams.ImageSignatureLevel))
},
},
{
"parse image characteristics",
&kevent.Kevent{
Type: ktypes.LoadImage,
Kparams: kevent.Kparams{
kparams.ImagePath: {Name: kparams.ImagePath, Type: kparams.UnicodeString, Value: "../_fixtures/mscorlib.dll"},
kparams.ProcessID: {Name: kparams.ProcessID, Type: kparams.PID, Value: uint32(1023)},
kparams.ImageCheckSum: {Name: kparams.ImageCheckSum, Type: kparams.Uint32, Value: uint32(2323432)},
kparams.ImageBase: {Name: kparams.ImageBase, Type: kparams.Address, Value: uint64(0x7ffb313833a3)},
kparams.ImageSignatureType: {Name: kparams.ImageSignatureType, Type: kparams.Enum, Value: uint32(1), Enum: signature.Types},
kparams.ImageSignatureLevel: {Name: kparams.ImageSignatureLevel, Type: kparams.Enum, Value: uint32(4), Enum: signature.Levels},
},
},
func() *ps.SnapshotterMock {
psnap := new(ps.SnapshotterMock)
psnap.On("AddModule", mock.Anything).Return(nil)
return psnap
},
func(e *kevent.Kevent, t *testing.T, psnap *ps.SnapshotterMock) {
psnap.AssertNumberOfCalls(t, "AddModule", 1)
// should be enriched with image characteristics params
assert.True(t, e.Kparams.MustGetBool(kparams.FileIsDLL))
assert.True(t, e.Kparams.MustGetBool(kparams.FileIsDotnet))
assert.False(t, e.Kparams.MustGetBool(kparams.FileIsExecutable))
assert.False(t, e.Kparams.MustGetBool(kparams.FileIsDriver))
},
},
{
"unload image",
&kevent.Kevent{
Expand Down
44 changes: 29 additions & 15 deletions internal/etw/processors/processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@ package processors
import (
libntfs "github.com/rabbitstack/fibratus/pkg/fs/ntfs"
"github.com/rabbitstack/fibratus/pkg/kevent"
"github.com/rabbitstack/fibratus/pkg/kevent/kparams"
"github.com/rabbitstack/fibratus/pkg/pe"
"os"
"time"
)

// ProcessorType is an alias for the event processor type
Expand Down Expand Up @@ -83,6 +83,18 @@ func (typ ProcessorType) String() string {
}
}

type imageFileCharacteristics struct {
isExe bool
isDLL bool
isDriver bool
isDotnet bool
accessed time.Time
}

func (c *imageFileCharacteristics) keepalive() {
c.accessed = time.Now()
}

// parseImageFileCharacteristics parses the PE structure for the file path
// residing in the given event parameters. The preferred method for reading
// the PE metadata is by directly accessing the file.
Expand All @@ -91,40 +103,42 @@ func (typ ProcessorType) String() string {
// The given event is decorated with various parameters extracted from PE
// data. Most notably, parameters that indicate whether the file is a DLL,
// executable image, or a Windows driver.
func parseImageFileCharacteristics(e *kevent.Kevent) error {
func parseImageFileCharacteristics(path string) (*imageFileCharacteristics, error) {
var pefile *pe.PE
filename := e.GetParamAsString(kparams.FilePath)
f, err := os.Open(filename)

f, err := os.Open(path)
if err != nil {
// read file data blob from raw device
// if the regular file access fails
ntfs := libntfs.NewFS()
data, n, err := ntfs.Read(filename, 0, int64(os.Getpagesize()))
data, n, err := ntfs.Read(path, 0, int64(os.Getpagesize()))
defer ntfs.Close()
if err != nil {
return err
return nil, err
}
if n > 0 {
data = data[:n]
}
// parse PE file from byte slice
pefile, err = pe.ParseBytes(data, pe.WithSections(), pe.WithSymbols())
pefile, err = pe.ParseBytes(data, pe.WithSections(), pe.WithSymbols(), pe.WithCLR())
if err != nil {
return err
return nil, err
}
} else {
defer f.Close()
// parse PE file from on-disk file
pefile, err = pe.ParseFile(filename, pe.WithSections(), pe.WithSymbols())
pefile, err = pe.ParseFile(path, pe.WithSections(), pe.WithSymbols(), pe.WithCLR())
if err != nil {
return err
return nil, err
}
}

// append parameters
e.AppendParam(kparams.FileIsDLL, kparams.Bool, pefile.IsDLL)
e.AppendParam(kparams.FileIsDriver, kparams.Bool, pefile.IsDriver)
e.AppendParam(kparams.FileIsExecutable, kparams.Bool, pefile.IsExecutable)
c := &imageFileCharacteristics{
isExe: pefile.IsExecutable,
isDLL: pefile.IsDLL,
isDriver: pefile.IsDriver,
isDotnet: pefile.IsDotnet,
}

return nil
return c, nil
}
6 changes: 1 addition & 5 deletions pkg/filter/accessor_windows.go
Original file line number Diff line number Diff line change
Expand Up @@ -908,11 +908,7 @@ func (i *imageAccessor) Get(f Field, kevt *kevent.Kevent) (kparams.Value, error)
case fields.ImageIsExecutable:
return kevt.Kparams.GetBool(kparams.FileIsExecutable)
case fields.ImageIsDotnet:
p, err := pe.ParseFile(kevt.GetParamAsString(kparams.ImagePath), pe.WithCLR())
if err != nil {
return nil, err
}
return p.IsDotnet, nil
return kevt.Kparams.GetBool(kparams.FileIsDotnet)
}

return nil, nil
Expand Down
Loading