mirror of
https://github.com/trufflesecurity/trufflehog.git
synced 2024-11-10 07:04:24 +00:00
Polite Verification (#2356)
* draft reverify chunks * remove * remove * reduce dupe map cap * do not verify chunk * cli arg and use val for dupe lut * remove counter * skipp empty results] * working on test and normalizing val for comparison * forgot to save file * optimize normalize * reuse map * remove print * use levenshtein distance to check dupes * forgot to leave in emptying map * use slice * small tweak * comment * use bytes * praise * use ctx logger * add len check * add comments * use 8x concurrency for reverifier workers * revert worker count * use more workers * process result directly for any collisions * continue after decoder match for reverifying * use map * use map * otimization and fix the bug. * revert worker count * better option naming * handle identical secrets in chunks * update comment * update comment * fix test * use DetecotrKey * rm out of scope tests and testdata * rename all reverification elements * don't re-write map entry * use correct key * rename worker, remove log val * test likelydupe, add eq detector check in loop * add test * add comment * add test * Set verification error * Update tests --------- Co-authored-by: Zachary Rice <zachary.rice@trufflesec.com> Co-authored-by: Dustin Decker <dustin@trufflesec.com>
This commit is contained in:
parent
c2ae31d060
commit
b2074ad05d
9 changed files with 530 additions and 34 deletions
1
go.mod
1
go.mod
|
@ -11,6 +11,7 @@ require (
|
||||||
github.com/AzureAD/microsoft-authentication-library-for-go v1.2.1
|
github.com/AzureAD/microsoft-authentication-library-for-go v1.2.1
|
||||||
github.com/BobuSumisu/aho-corasick v1.0.3
|
github.com/BobuSumisu/aho-corasick v1.0.3
|
||||||
github.com/TheZeroSlave/zapsentry v1.19.0
|
github.com/TheZeroSlave/zapsentry v1.19.0
|
||||||
|
github.com/adrg/strutil v0.3.1
|
||||||
github.com/alecthomas/kingpin/v2 v2.4.0
|
github.com/alecthomas/kingpin/v2 v2.4.0
|
||||||
github.com/aws/aws-sdk-go v1.50.0
|
github.com/aws/aws-sdk-go v1.50.0
|
||||||
github.com/aymanbagabas/go-osc52 v1.2.2
|
github.com/aymanbagabas/go-osc52 v1.2.2
|
||||||
|
|
2
go.sum
2
go.sum
|
@ -82,6 +82,8 @@ github.com/ProtonMail/go-crypto v0.0.0-20230828082145-3c4c8a2d2371 h1:kkhsdkhsCv
|
||||||
github.com/ProtonMail/go-crypto v0.0.0-20230828082145-3c4c8a2d2371/go.mod h1:EjAoLdwvbIOoOQr3ihjnSoLZRtE8azugULFRteWMNc0=
|
github.com/ProtonMail/go-crypto v0.0.0-20230828082145-3c4c8a2d2371/go.mod h1:EjAoLdwvbIOoOQr3ihjnSoLZRtE8azugULFRteWMNc0=
|
||||||
github.com/TheZeroSlave/zapsentry v1.19.0 h1:/FVdMrq/w7bYt98m49ImZgmCTybXWbGc8/hOT0nLmyc=
|
github.com/TheZeroSlave/zapsentry v1.19.0 h1:/FVdMrq/w7bYt98m49ImZgmCTybXWbGc8/hOT0nLmyc=
|
||||||
github.com/TheZeroSlave/zapsentry v1.19.0/go.mod h1:D1YMfSuu6xnkhwFXxrronesmsiyDhIqo+86I3Ok+r64=
|
github.com/TheZeroSlave/zapsentry v1.19.0/go.mod h1:D1YMfSuu6xnkhwFXxrronesmsiyDhIqo+86I3Ok+r64=
|
||||||
|
github.com/adrg/strutil v0.3.1 h1:OLvSS7CSJO8lBii4YmBt8jiK9QOtB9CzCzwl4Ic/Fz4=
|
||||||
|
github.com/adrg/strutil v0.3.1/go.mod h1:8h90y18QLrs11IBffcGX3NW/GFBXCMcNg4M7H6MspPA=
|
||||||
github.com/alecthomas/chroma v0.10.0 h1:7XDcGkCQopCNKjZHfYrNLraA+M7e0fMiJ/Mfikbfjek=
|
github.com/alecthomas/chroma v0.10.0 h1:7XDcGkCQopCNKjZHfYrNLraA+M7e0fMiJ/Mfikbfjek=
|
||||||
github.com/alecthomas/chroma v0.10.0/go.mod h1:jtJATyUxlIORhUOFNA9NZDWGAQ8wpxQQqNSB4rjA/1s=
|
github.com/alecthomas/chroma v0.10.0/go.mod h1:jtJATyUxlIORhUOFNA9NZDWGAQ8wpxQQqNSB4rjA/1s=
|
||||||
github.com/alecthomas/kingpin/v2 v2.4.0 h1:f48lwail6p8zpO1bC4TxtqACaGqHYA22qkHjHpqDjYY=
|
github.com/alecthomas/kingpin/v2 v2.4.0 h1:f48lwail6p8zpO1bC4TxtqACaGqHYA22qkHjHpqDjYY=
|
||||||
|
|
2
main.go
2
main.go
|
@ -49,6 +49,7 @@ var (
|
||||||
concurrency = cli.Flag("concurrency", "Number of concurrent workers.").Default(strconv.Itoa(runtime.NumCPU())).Int()
|
concurrency = cli.Flag("concurrency", "Number of concurrent workers.").Default(strconv.Itoa(runtime.NumCPU())).Int()
|
||||||
noVerification = cli.Flag("no-verification", "Don't verify the results.").Bool()
|
noVerification = cli.Flag("no-verification", "Don't verify the results.").Bool()
|
||||||
onlyVerified = cli.Flag("only-verified", "Only output verified results.").Bool()
|
onlyVerified = cli.Flag("only-verified", "Only output verified results.").Bool()
|
||||||
|
allowVerificationOverlap = cli.Flag("allow-verification-overlap", "Allow verification of similar credentials across detectors").Bool()
|
||||||
filterUnverified = cli.Flag("filter-unverified", "Only output first unverified result per chunk per detector if there are more than one results.").Bool()
|
filterUnverified = cli.Flag("filter-unverified", "Only output first unverified result per chunk per detector if there are more than one results.").Bool()
|
||||||
filterEntropy = cli.Flag("filter-entropy", "Filter unverified results with Shannon entropy. Start with 3.0.").Float64()
|
filterEntropy = cli.Flag("filter-entropy", "Filter unverified results with Shannon entropy. Start with 3.0.").Float64()
|
||||||
configFilename = cli.Flag("config", "Path to configuration file.").ExistingFile()
|
configFilename = cli.Flag("config", "Path to configuration file.").ExistingFile()
|
||||||
|
@ -411,6 +412,7 @@ func run(state overseer.State) {
|
||||||
engine.WithPrintAvgDetectorTime(*printAvgDetectorTime),
|
engine.WithPrintAvgDetectorTime(*printAvgDetectorTime),
|
||||||
engine.WithPrinter(printer),
|
engine.WithPrinter(printer),
|
||||||
engine.WithFilterEntropy(*filterEntropy),
|
engine.WithFilterEntropy(*filterEntropy),
|
||||||
|
engine.WithVerificationOverlap(*allowVerificationOverlap),
|
||||||
)
|
)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
logFatal(err, "error initializing engine")
|
logFatal(err, "error initializing engine")
|
||||||
|
|
|
@ -47,7 +47,7 @@ func NewAhoCorasickCore(allDetectors []detectors.Detector) *AhoCorasickCore {
|
||||||
detectorsByKey := make(map[DetectorKey]detectors.Detector, len(allDetectors))
|
detectorsByKey := make(map[DetectorKey]detectors.Detector, len(allDetectors))
|
||||||
var keywords []string
|
var keywords []string
|
||||||
for _, d := range allDetectors {
|
for _, d := range allDetectors {
|
||||||
key := createDetectorKey(d)
|
key := CreateDetectorKey(d)
|
||||||
detectorsByKey[key] = d
|
detectorsByKey[key] = d
|
||||||
for _, kw := range d.Keywords() {
|
for _, kw := range d.Keywords() {
|
||||||
kwLower := strings.ToLower(kw)
|
kwLower := strings.ToLower(kw)
|
||||||
|
@ -63,20 +63,46 @@ func NewAhoCorasickCore(allDetectors []detectors.Detector) *AhoCorasickCore {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// DetectorInfo represents a detected pattern's metadata in a data chunk.
|
||||||
|
// It encapsulates the key identifying a specific detector and the detector instance itself.
|
||||||
|
type DetectorInfo struct {
|
||||||
|
Key DetectorKey
|
||||||
|
detectors.Detector
|
||||||
|
}
|
||||||
|
|
||||||
// PopulateMatchingDetectors populates the given detector slice with all the detectors matching the
|
// PopulateMatchingDetectors populates the given detector slice with all the detectors matching the
|
||||||
// provided input. This method populates an existing map rather than allocating a new one because
|
// provided input. This method populates an existing map rather than allocating a new one because
|
||||||
// it will be called once per chunk and that many allocations has a noticeable performance cost.
|
// it will be called once per chunk and that many allocations has a noticeable performance cost.
|
||||||
func (ac *AhoCorasickCore) PopulateMatchingDetectors(chunkData string, detectors map[DetectorKey]detectors.Detector) {
|
// It returns a slice of unique 'DetectorInfo' corresponding to the matched detectors. This slice is
|
||||||
for _, m := range ac.prefilter.MatchString(strings.ToLower(chunkData)) {
|
// constructed to prevent duplications by utilizing an internal map to track already processed detectors.
|
||||||
|
func (ac *AhoCorasickCore) PopulateMatchingDetectors(chunkData string, dts map[DetectorKey]detectors.Detector) []DetectorInfo {
|
||||||
|
matches := ac.prefilter.MatchString(strings.ToLower(chunkData))
|
||||||
|
|
||||||
|
// Use a map to avoid adding duplicate detectors to the slice.
|
||||||
|
addedDetectors := make(map[DetectorKey]struct{})
|
||||||
|
uniqueDetectors := make([]DetectorInfo, 0, len(matches))
|
||||||
|
|
||||||
|
for _, m := range matches {
|
||||||
for _, k := range ac.keywordsToDetectors[m.MatchString()] {
|
for _, k := range ac.keywordsToDetectors[m.MatchString()] {
|
||||||
detectors[k] = ac.detectorsByKey[k]
|
if _, exists := addedDetectors[k]; exists {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
// Add to the map to track already added detectors.
|
||||||
|
addedDetectors[k] = struct{}{}
|
||||||
|
|
||||||
|
// Add the detector to the map and slice.
|
||||||
|
detector := ac.detectorsByKey[k]
|
||||||
|
dts[k] = detector
|
||||||
|
uniqueDetectors = append(uniqueDetectors, DetectorInfo{Key: k, Detector: detector})
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
return uniqueDetectors
|
||||||
}
|
}
|
||||||
|
|
||||||
// createDetectorKey creates a unique key for each detector from its type, version, and, for
|
// CreateDetectorKey creates a unique key for each detector from its type, version, and, for
|
||||||
// custom regex detectors, its name.
|
// custom regex detectors, its name.
|
||||||
func createDetectorKey(d detectors.Detector) DetectorKey {
|
func CreateDetectorKey(d detectors.Detector) DetectorKey {
|
||||||
detectorType := d.Type()
|
detectorType := d.Type()
|
||||||
var version int
|
var version int
|
||||||
if v, ok := d.(detectors.Versioner); ok {
|
if v, ok := d.(detectors.Versioner); ok {
|
||||||
|
|
|
@ -2,12 +2,15 @@ package engine
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
"runtime"
|
"runtime"
|
||||||
"sync"
|
"sync"
|
||||||
"sync/atomic"
|
"sync/atomic"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/adrg/strutil"
|
||||||
|
"github.com/adrg/strutil/metrics"
|
||||||
lru "github.com/hashicorp/golang-lru"
|
lru "github.com/hashicorp/golang-lru"
|
||||||
"google.golang.org/protobuf/proto"
|
"google.golang.org/protobuf/proto"
|
||||||
|
|
||||||
|
@ -26,6 +29,8 @@ import (
|
||||||
"github.com/trufflesecurity/trufflehog/v3/pkg/sources"
|
"github.com/trufflesecurity/trufflehog/v3/pkg/sources"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
var overlapError = errors.New("More than one detector has found this result. For your safety, verification has been disabled. You can override this behavior by using the --allow-verification-overlap flag.")
|
||||||
|
|
||||||
// Metrics for the scan engine for external consumption.
|
// Metrics for the scan engine for external consumption.
|
||||||
type Metrics struct {
|
type Metrics struct {
|
||||||
BytesScanned uint64
|
BytesScanned uint64
|
||||||
|
@ -63,6 +68,7 @@ type Engine struct {
|
||||||
// entropyFilter is used to filter out unverified results using Shannon entropy.
|
// entropyFilter is used to filter out unverified results using Shannon entropy.
|
||||||
filterEntropy *float64
|
filterEntropy *float64
|
||||||
onlyVerified bool
|
onlyVerified bool
|
||||||
|
verificationOverlap bool
|
||||||
printAvgDetectorTime bool
|
printAvgDetectorTime bool
|
||||||
|
|
||||||
// ahoCorasickHandler manages the Aho-Corasick trie and related keyword lookups.
|
// ahoCorasickHandler manages the Aho-Corasick trie and related keyword lookups.
|
||||||
|
@ -72,7 +78,9 @@ type Engine struct {
|
||||||
sourceManager *sources.SourceManager
|
sourceManager *sources.SourceManager
|
||||||
results chan detectors.ResultWithMetadata
|
results chan detectors.ResultWithMetadata
|
||||||
detectableChunksChan chan detectableChunk
|
detectableChunksChan chan detectableChunk
|
||||||
|
verificationOverlapChunksChan chan verificationOverlapChunk
|
||||||
workersWg sync.WaitGroup
|
workersWg sync.WaitGroup
|
||||||
|
verificationOverlapWg sync.WaitGroup
|
||||||
wgDetectorWorkers sync.WaitGroup
|
wgDetectorWorkers sync.WaitGroup
|
||||||
WgNotifier sync.WaitGroup
|
WgNotifier sync.WaitGroup
|
||||||
|
|
||||||
|
@ -92,6 +100,20 @@ type Engine struct {
|
||||||
|
|
||||||
// verify determines whether the scanner will attempt to verify candidate secrets
|
// verify determines whether the scanner will attempt to verify candidate secrets
|
||||||
verify bool
|
verify bool
|
||||||
|
|
||||||
|
// Note: bad hack only used for testing
|
||||||
|
verificationOverlapTracker *verificationOverlapTracker
|
||||||
|
}
|
||||||
|
|
||||||
|
type verificationOverlapTracker struct {
|
||||||
|
verificationOverlapDuplicateCount int
|
||||||
|
mu sync.Mutex
|
||||||
|
}
|
||||||
|
|
||||||
|
func (r *verificationOverlapTracker) increment() {
|
||||||
|
r.mu.Lock()
|
||||||
|
r.verificationOverlapDuplicateCount++
|
||||||
|
r.mu.Unlock()
|
||||||
}
|
}
|
||||||
|
|
||||||
// Option is used to configure the engine during initialization using functional options.
|
// Option is used to configure the engine during initialization using functional options.
|
||||||
|
@ -182,6 +204,21 @@ func WithVerify(verify bool) Option {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func withVerificationOverlapTracking() Option {
|
||||||
|
return func(e *Engine) {
|
||||||
|
e.verificationOverlapTracker = &verificationOverlapTracker{
|
||||||
|
verificationOverlapDuplicateCount: 0,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// WithVerificationOverlap
|
||||||
|
func WithVerificationOverlap(verificationOverlap bool) Option {
|
||||||
|
return func(e *Engine) {
|
||||||
|
e.verificationOverlap = verificationOverlap
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
func filterDetectors(filterFunc func(detectors.Detector) bool, input []detectors.Detector) []detectors.Detector {
|
func filterDetectors(filterFunc func(detectors.Detector) bool, input []detectors.Detector) []detectors.Detector {
|
||||||
var out []detectors.Detector
|
var out []detectors.Detector
|
||||||
for _, detector := range input {
|
for _, detector := range input {
|
||||||
|
@ -287,7 +324,7 @@ func Start(ctx context.Context, options ...Option) (*Engine, error) {
|
||||||
return e, nil
|
return e, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
const defaultChannelBuffer = 1
|
var defaultChannelBuffer = runtime.NumCPU()
|
||||||
|
|
||||||
// initialize prepares the engine's internal structures. The LRU cache optimizes
|
// initialize prepares the engine's internal structures. The LRU cache optimizes
|
||||||
// deduplication efforts, allowing the engine to quickly check if a chunk has
|
// deduplication efforts, allowing the engine to quickly check if a chunk has
|
||||||
|
@ -300,10 +337,25 @@ func (e *Engine) initialize(ctx context.Context, options ...Option) error {
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return fmt.Errorf("failed to initialize LRU cache: %w", err)
|
return fmt.Errorf("failed to initialize LRU cache: %w", err)
|
||||||
}
|
}
|
||||||
|
const (
|
||||||
|
// detectableChunksChanMultiplier is set to accommodate a high number of concurrent worker goroutines.
|
||||||
|
// This multiplier ensures that the detectableChunksChan channel has sufficient buffer capacity
|
||||||
|
// to hold messages from multiple worker groups (detector workers/ verificationOverlap workers) without blocking.
|
||||||
|
// A large buffer helps accommodate for the fact workers are producing data at a faster rate
|
||||||
|
// than it can be consumed.
|
||||||
|
detectableChunksChanMultiplier = 50
|
||||||
|
// verificationOverlapChunksChanMultiplier uses a smaller buffer compared to detectableChunksChanMultiplier.
|
||||||
|
// This reflects the anticipated lower volume of data that needs re-verification.
|
||||||
|
// The buffer size is a trade-off between memory usage and the need to prevent blocking.
|
||||||
|
verificationOverlapChunksChanMultiplier = 25
|
||||||
|
)
|
||||||
|
|
||||||
// Channels are used for communication between different parts of the engine,
|
// Channels are used for communication between different parts of the engine,
|
||||||
// ensuring that data flows smoothly without race conditions.
|
// ensuring that data flows smoothly without race conditions.
|
||||||
e.detectableChunksChan = make(chan detectableChunk, defaultChannelBuffer)
|
// The buffer sizes for these channels are set to multiples of defaultChannelBuffer,
|
||||||
|
// considering the expected concurrency and workload in the system.
|
||||||
|
e.detectableChunksChan = make(chan detectableChunk, defaultChannelBuffer*detectableChunksChanMultiplier)
|
||||||
|
e.verificationOverlapChunksChan = make(chan verificationOverlapChunk, defaultChannelBuffer*verificationOverlapChunksChanMultiplier)
|
||||||
e.results = make(chan detectors.ResultWithMetadata, defaultChannelBuffer)
|
e.results = make(chan detectors.ResultWithMetadata, defaultChannelBuffer)
|
||||||
e.dedupeCache = cache
|
e.dedupeCache = cache
|
||||||
e.printer = new(output.PlainPrinter)
|
e.printer = new(output.PlainPrinter)
|
||||||
|
@ -393,6 +445,20 @@ func (e *Engine) startWorkers(ctx context.Context) {
|
||||||
}()
|
}()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// verificationOverlap workers handle verification of chunks that have been detected by multiple detectors.
|
||||||
|
// They ensure that verification is disabled for any secrets that have been detected by multiple detectors.
|
||||||
|
const verificationOverlapWorkerMultiplier = detectorWorkerMultiplier
|
||||||
|
ctx.Logger().V(2).Info("starting verificationOverlap workers", "count", e.concurrency)
|
||||||
|
for worker := uint64(0); worker < uint64(e.concurrency*verificationOverlapWorkerMultiplier); worker++ {
|
||||||
|
e.verificationOverlapWg.Add(1)
|
||||||
|
go func() {
|
||||||
|
ctx := context.WithValue(ctx, "verification_overlap_worker_id", common.RandomID(5))
|
||||||
|
defer common.Recover(ctx)
|
||||||
|
defer e.verificationOverlapWg.Done()
|
||||||
|
e.verificationOverlapWorker(ctx)
|
||||||
|
}()
|
||||||
|
}
|
||||||
|
|
||||||
// Notifier workers communicate detected issues to the user or any downstream systems.
|
// Notifier workers communicate detected issues to the user or any downstream systems.
|
||||||
// We want 1/4th of the notifier workers as the number of scanner workers.
|
// We want 1/4th of the notifier workers as the number of scanner workers.
|
||||||
const notifierWorkerRatio = 4
|
const notifierWorkerRatio = 4
|
||||||
|
@ -421,6 +487,10 @@ func (e *Engine) Finish(ctx context.Context) error {
|
||||||
err := e.sourceManager.Wait()
|
err := e.sourceManager.Wait()
|
||||||
|
|
||||||
e.workersWg.Wait() // Wait for the workers to finish scanning chunks.
|
e.workersWg.Wait() // Wait for the workers to finish scanning chunks.
|
||||||
|
|
||||||
|
close(e.verificationOverlapChunksChan)
|
||||||
|
e.verificationOverlapWg.Wait()
|
||||||
|
|
||||||
close(e.detectableChunksChan)
|
close(e.detectableChunksChan)
|
||||||
e.wgDetectorWorkers.Wait() // Wait for the detector workers to finish detecting chunks.
|
e.wgDetectorWorkers.Wait() // Wait for the detector workers to finish detecting chunks.
|
||||||
|
|
||||||
|
@ -459,11 +529,22 @@ type detectableChunk struct {
|
||||||
wgDoneFn func()
|
wgDoneFn func()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// verificationOverlapChunk is a decoded chunk that has multiple detectors that match it.
|
||||||
|
// It will be initially processed with verification disabled, and then reprocessed with verification
|
||||||
|
// enabled if the same secret was not found by multiple detectors.
|
||||||
|
type verificationOverlapChunk struct {
|
||||||
|
chunk sources.Chunk
|
||||||
|
decoder detectorspb.DecoderType
|
||||||
|
detectors []ahocorasick.DetectorInfo
|
||||||
|
verificationOverlapWgDoneFn func()
|
||||||
|
}
|
||||||
|
|
||||||
func (e *Engine) detectorWorker(ctx context.Context) {
|
func (e *Engine) detectorWorker(ctx context.Context) {
|
||||||
var wgDetect sync.WaitGroup
|
var wgDetect sync.WaitGroup
|
||||||
|
var wgVerificationOverlap sync.WaitGroup
|
||||||
|
|
||||||
// Reuse the same map to avoid allocations.
|
// Reuse the same map to avoid allocations.
|
||||||
const avgDetectorsPerChunk = 2
|
const avgDetectorsPerChunk = 8
|
||||||
chunkSpecificDetectors := make(map[ahocorasick.DetectorKey]detectors.Detector, avgDetectorsPerChunk)
|
chunkSpecificDetectors := make(map[ahocorasick.DetectorKey]detectors.Detector, avgDetectorsPerChunk)
|
||||||
for originalChunk := range e.ChunksChan() {
|
for originalChunk := range e.ChunksChan() {
|
||||||
for chunk := range sources.Chunker(originalChunk) {
|
for chunk := range sources.Chunker(originalChunk) {
|
||||||
|
@ -475,7 +556,21 @@ func (e *Engine) detectorWorker(ctx context.Context) {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
e.ahoCorasickCore.PopulateMatchingDetectors(string(decoded.Chunk.Data), chunkSpecificDetectors)
|
matchingDetectors := e.ahoCorasickCore.PopulateMatchingDetectors(string(decoded.Chunk.Data), chunkSpecificDetectors)
|
||||||
|
if len(chunkSpecificDetectors) > 1 && !e.verificationOverlap {
|
||||||
|
wgVerificationOverlap.Add(1)
|
||||||
|
e.verificationOverlapChunksChan <- verificationOverlapChunk{
|
||||||
|
chunk: *decoded.Chunk,
|
||||||
|
detectors: matchingDetectors,
|
||||||
|
decoder: decoded.DecoderType,
|
||||||
|
verificationOverlapWgDoneFn: wgVerificationOverlap.Done,
|
||||||
|
}
|
||||||
|
// Empty the map.
|
||||||
|
for k := range chunkSpecificDetectors {
|
||||||
|
delete(chunkSpecificDetectors, k)
|
||||||
|
}
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
for k, detector := range chunkSpecificDetectors {
|
for k, detector := range chunkSpecificDetectors {
|
||||||
decoded.Chunk.Verify = e.verify
|
decoded.Chunk.Verify = e.verify
|
||||||
|
@ -492,10 +587,145 @@ func (e *Engine) detectorWorker(ctx context.Context) {
|
||||||
}
|
}
|
||||||
atomic.AddUint64(&e.metrics.ChunksScanned, 1)
|
atomic.AddUint64(&e.metrics.ChunksScanned, 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
wgVerificationOverlap.Wait()
|
||||||
wgDetect.Wait()
|
wgDetect.Wait()
|
||||||
ctx.Logger().V(4).Info("finished scanning chunks")
|
ctx.Logger().V(4).Info("finished scanning chunks")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// chunkSecretKey ties secrets to the specific detector that found them. This allows identifying identical
|
||||||
|
// credentials extracted by multiple different detectors processing the same chunk. Or duplicates found
|
||||||
|
// by the same detector in the chunk. Exact matches on lookup indicate a duplicate secret for a detector
|
||||||
|
// in that chunk - which is expected and not malicious. Those intra-detector dupes are still verified.
|
||||||
|
type chunkSecretKey struct {
|
||||||
|
secret string
|
||||||
|
detectorInfo ahocorasick.DetectorInfo
|
||||||
|
}
|
||||||
|
|
||||||
|
func likelyDuplicate(ctx context.Context, val chunkSecretKey, dupes map[chunkSecretKey]struct{}) bool {
|
||||||
|
const similarityThreshold = 0.9
|
||||||
|
|
||||||
|
valStr := val.secret
|
||||||
|
for dupeKey := range dupes {
|
||||||
|
dupe := dupeKey.secret
|
||||||
|
// Avoid comparing strings of vastly different lengths.
|
||||||
|
if len(dupe)*10 < len(valStr)*9 || len(dupe)*10 > len(valStr)*11 {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
|
// If the detector type is the same, we don't need to compare the strings.
|
||||||
|
// These are not duplicates, and should be verified.
|
||||||
|
if val.detectorInfo.Type() == dupeKey.detectorInfo.Type() {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
|
if valStr == dupe {
|
||||||
|
ctx.Logger().V(2).Info(
|
||||||
|
"found exact duplicate",
|
||||||
|
)
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
|
similarity := strutil.Similarity(valStr, dupe, metrics.NewLevenshtein())
|
||||||
|
|
||||||
|
// close enough
|
||||||
|
if similarity > similarityThreshold {
|
||||||
|
ctx.Logger().V(2).Info(
|
||||||
|
"found similar duplicate",
|
||||||
|
)
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
|
||||||
|
func (e *Engine) verificationOverlapWorker(ctx context.Context) {
|
||||||
|
var wgDetect sync.WaitGroup
|
||||||
|
|
||||||
|
// Reuse the same map and slice to avoid allocations.
|
||||||
|
const avgSecretsPerDetector = 8
|
||||||
|
detectorsWithResult := make(map[ahocorasick.DetectorInfo]struct{}, avgSecretsPerDetector)
|
||||||
|
chunkSecrets := make(map[chunkSecretKey]struct{}, avgSecretsPerDetector)
|
||||||
|
|
||||||
|
for chunk := range e.verificationOverlapChunksChan {
|
||||||
|
for _, detector := range chunk.detectors {
|
||||||
|
// DO NOT VERIFY at this stage of the pipeline.
|
||||||
|
results, err := detector.FromData(ctx, false, chunk.chunk.Data)
|
||||||
|
if err != nil {
|
||||||
|
ctx.Logger().Error(err, "error verifying chunk")
|
||||||
|
}
|
||||||
|
|
||||||
|
if len(results) == 0 {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
if _, ok := detectorsWithResult[detector]; !ok {
|
||||||
|
detectorsWithResult[detector] = struct{}{}
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, res := range results {
|
||||||
|
var val []byte
|
||||||
|
if res.RawV2 != nil {
|
||||||
|
val = res.RawV2
|
||||||
|
} else {
|
||||||
|
val = res.Raw
|
||||||
|
}
|
||||||
|
|
||||||
|
// Use levenstein distance to determine if the secret is likely the same.
|
||||||
|
// Ex:
|
||||||
|
// - postman api key: PMAK-qnwfsLyRSyfCwfpHaQP1UzDhrgpWvHjbYzjpRCMshjt417zWcrzyHUArs7r
|
||||||
|
// - malicious detector "api key": qnwfsLyRSyfCwfpHaQP1UzDhrgpWvHjbYzjpRCMshjt417zWcrzyHUArs7r
|
||||||
|
key := chunkSecretKey{secret: string(val), detectorInfo: detector}
|
||||||
|
if _, ok := chunkSecrets[key]; ok {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
|
if likelyDuplicate(ctx, key, chunkSecrets) {
|
||||||
|
// This indicates that the same secret was found by multiple detectors.
|
||||||
|
// We should NOT VERIFY this chunk's data.
|
||||||
|
if e.verificationOverlapTracker != nil {
|
||||||
|
e.verificationOverlapTracker.increment()
|
||||||
|
}
|
||||||
|
res.SetVerificationError(overlapError)
|
||||||
|
e.processResult(ctx, detectableChunk{
|
||||||
|
chunk: chunk.chunk,
|
||||||
|
detector: detector,
|
||||||
|
decoder: chunk.decoder,
|
||||||
|
wgDoneFn: wgDetect.Done,
|
||||||
|
}, res)
|
||||||
|
|
||||||
|
// Remove the detector from the list of detectors with results.
|
||||||
|
delete(detectorsWithResult, detector)
|
||||||
|
}
|
||||||
|
chunkSecrets[key] = struct{}{}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
for detector := range detectorsWithResult {
|
||||||
|
wgDetect.Add(1)
|
||||||
|
chunk.chunk.Verify = e.verify
|
||||||
|
e.detectableChunksChan <- detectableChunk{
|
||||||
|
chunk: chunk.chunk,
|
||||||
|
detector: detector,
|
||||||
|
decoder: chunk.decoder,
|
||||||
|
wgDoneFn: wgDetect.Done,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Empty the dupes and detectors slice
|
||||||
|
for k := range chunkSecrets {
|
||||||
|
delete(chunkSecrets, k)
|
||||||
|
}
|
||||||
|
for k := range detectorsWithResult {
|
||||||
|
delete(detectorsWithResult, k)
|
||||||
|
}
|
||||||
|
|
||||||
|
chunk.verificationOverlapWgDoneFn()
|
||||||
|
}
|
||||||
|
|
||||||
|
wgDetect.Wait()
|
||||||
|
ctx.Logger().V(4).Info("finished verificationOverlap chunks")
|
||||||
|
}
|
||||||
|
|
||||||
func (e *Engine) detectChunks(ctx context.Context) {
|
func (e *Engine) detectChunks(ctx context.Context) {
|
||||||
for data := range e.detectableChunksChan {
|
for data := range e.detectableChunksChan {
|
||||||
e.detectChunk(ctx, data)
|
e.detectChunk(ctx, data)
|
||||||
|
|
|
@ -2,15 +2,23 @@ package engine
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"net/http"
|
||||||
|
"net/http/httptest"
|
||||||
|
"os"
|
||||||
"path/filepath"
|
"path/filepath"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
|
|
||||||
|
"github.com/trufflesecurity/trufflehog/v3/pkg/common"
|
||||||
|
"github.com/trufflesecurity/trufflehog/v3/pkg/config"
|
||||||
"github.com/trufflesecurity/trufflehog/v3/pkg/context"
|
"github.com/trufflesecurity/trufflehog/v3/pkg/context"
|
||||||
|
"github.com/trufflesecurity/trufflehog/v3/pkg/custom_detectors"
|
||||||
"github.com/trufflesecurity/trufflehog/v3/pkg/decoders"
|
"github.com/trufflesecurity/trufflehog/v3/pkg/decoders"
|
||||||
"github.com/trufflesecurity/trufflehog/v3/pkg/detectors"
|
"github.com/trufflesecurity/trufflehog/v3/pkg/detectors"
|
||||||
|
"github.com/trufflesecurity/trufflehog/v3/pkg/engine/ahocorasick"
|
||||||
|
"github.com/trufflesecurity/trufflehog/v3/pkg/pb/custom_detectorspb"
|
||||||
"github.com/trufflesecurity/trufflehog/v3/pkg/pb/source_metadatapb"
|
"github.com/trufflesecurity/trufflehog/v3/pkg/pb/source_metadatapb"
|
||||||
"github.com/trufflesecurity/trufflehog/v3/pkg/pb/sourcespb"
|
"github.com/trufflesecurity/trufflehog/v3/pkg/pb/sourcespb"
|
||||||
"github.com/trufflesecurity/trufflehog/v3/pkg/sources"
|
"github.com/trufflesecurity/trufflehog/v3/pkg/sources"
|
||||||
|
@ -188,7 +196,7 @@ func BenchmarkSupportsLineNumbersLoop(b *testing.B) {
|
||||||
func TestEngine_DuplicatSecrets(t *testing.T) {
|
func TestEngine_DuplicatSecrets(t *testing.T) {
|
||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
|
|
||||||
absPath, err := filepath.Abs("./testdata")
|
absPath, err := filepath.Abs("./testdata/secrets.txt")
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
|
||||||
ctx, cancel := context.WithTimeout(ctx, 10*time.Second)
|
ctx, cancel := context.WithTimeout(ctx, 10*time.Second)
|
||||||
|
@ -198,7 +206,7 @@ func TestEngine_DuplicatSecrets(t *testing.T) {
|
||||||
WithConcurrency(1),
|
WithConcurrency(1),
|
||||||
WithDecoders(decoders.DefaultDecoders()...),
|
WithDecoders(decoders.DefaultDecoders()...),
|
||||||
WithDetectors(DefaultDetectors()...),
|
WithDetectors(DefaultDetectors()...),
|
||||||
WithVerify(true),
|
WithVerify(false),
|
||||||
WithPrinter(new(discardPrinter)),
|
WithPrinter(new(discardPrinter)),
|
||||||
)
|
)
|
||||||
assert.Nil(t, err)
|
assert.Nil(t, err)
|
||||||
|
@ -214,6 +222,140 @@ func TestEngine_DuplicatSecrets(t *testing.T) {
|
||||||
assert.Equal(t, want, e.GetMetrics().UnverifiedSecretsFound)
|
assert.Equal(t, want, e.GetMetrics().UnverifiedSecretsFound)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// TestEngine_VersionedDetectorsVerifiedSecrets is a test that detects ALL verified secrets across
|
||||||
|
// versioned detectors.
|
||||||
|
func TestEngine_VersionedDetectorsVerifiedSecrets(t *testing.T) {
|
||||||
|
ctx, cancel := context.WithTimeout(context.Background(), time.Second*10)
|
||||||
|
defer cancel()
|
||||||
|
testSecrets, err := common.GetSecret(ctx, "trufflehog-testing", "detectors4")
|
||||||
|
assert.NoError(t, err)
|
||||||
|
secretV2 := testSecrets.MustGetField("GITLABV2")
|
||||||
|
secretV1 := testSecrets.MustGetField("GITLAB")
|
||||||
|
|
||||||
|
tmpFile, err := os.CreateTemp("", "testfile")
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer tmpFile.Close()
|
||||||
|
defer os.Remove(tmpFile.Name())
|
||||||
|
|
||||||
|
_, err = tmpFile.WriteString(fmt.Sprintf("You can find a gitlab secrets %s and another gitlab secret %s within", secretV2, secretV1))
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
e, err := Start(ctx,
|
||||||
|
WithConcurrency(1),
|
||||||
|
WithDecoders(decoders.DefaultDecoders()...),
|
||||||
|
WithDetectors(DefaultDetectors()...),
|
||||||
|
WithVerify(true),
|
||||||
|
WithPrinter(new(discardPrinter)),
|
||||||
|
)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
cfg := sources.FilesystemConfig{Paths: []string{tmpFile.Name()}}
|
||||||
|
if err := e.ScanFileSystem(ctx, cfg); err != nil {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
assert.Nil(t, e.Finish(ctx))
|
||||||
|
want := uint64(2)
|
||||||
|
assert.Equal(t, want, e.GetMetrics().VerifiedSecretsFound)
|
||||||
|
}
|
||||||
|
|
||||||
|
// TestEngine_CustomDetectorsDetectorsVerifiedSecrets is a test that covers an edge case where there are
|
||||||
|
// multiple detectors with the same type, keywords and regex that match the same secret.
|
||||||
|
// This ensures that those secrets get verified.
|
||||||
|
func TestEngine_CustomDetectorsDetectorsVerifiedSecrets(t *testing.T) {
|
||||||
|
tmpFile, err := os.CreateTemp("", "testfile")
|
||||||
|
assert.Nil(t, err)
|
||||||
|
defer tmpFile.Close()
|
||||||
|
defer os.Remove(tmpFile.Name())
|
||||||
|
|
||||||
|
_, err = tmpFile.WriteString("test stuff")
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||||
|
w.WriteHeader(http.StatusOK)
|
||||||
|
}))
|
||||||
|
defer ts.Close()
|
||||||
|
|
||||||
|
customDetector1, err := custom_detectors.NewWebhookCustomRegex(&custom_detectorspb.CustomRegex{
|
||||||
|
Name: "custom detector 1",
|
||||||
|
Keywords: []string{"test"},
|
||||||
|
Regex: map[string]string{"test": "\\w+"},
|
||||||
|
Verify: []*custom_detectorspb.VerifierConfig{{Endpoint: ts.URL, Unsafe: true, SuccessRanges: []string{"200"}}},
|
||||||
|
})
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
customDetector2, err := custom_detectors.NewWebhookCustomRegex(&custom_detectorspb.CustomRegex{
|
||||||
|
Name: "custom detector 2",
|
||||||
|
Keywords: []string{"test"},
|
||||||
|
Regex: map[string]string{"test": "\\w+"},
|
||||||
|
Verify: []*custom_detectorspb.VerifierConfig{{Endpoint: ts.URL, Unsafe: true, SuccessRanges: []string{"200"}}},
|
||||||
|
})
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
allDetectors := []detectors.Detector{customDetector1, customDetector2}
|
||||||
|
|
||||||
|
ctx, cancel := context.WithTimeout(context.Background(), time.Second*5)
|
||||||
|
defer cancel()
|
||||||
|
e, err := Start(ctx,
|
||||||
|
WithConcurrency(1),
|
||||||
|
WithDecoders(decoders.DefaultDecoders()...),
|
||||||
|
WithDetectors(allDetectors...),
|
||||||
|
WithVerify(true),
|
||||||
|
WithPrinter(new(discardPrinter)),
|
||||||
|
)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
cfg := sources.FilesystemConfig{Paths: []string{tmpFile.Name()}}
|
||||||
|
if err := e.ScanFileSystem(ctx, cfg); err != nil {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
assert.Nil(t, e.Finish(ctx))
|
||||||
|
// We should have 4 verified secrets, 2 for each custom detector.
|
||||||
|
want := uint64(4)
|
||||||
|
assert.Equal(t, want, e.GetMetrics().VerifiedSecretsFound)
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestVerificationOverlapChunk(t *testing.T) {
|
||||||
|
ctx := context.Background()
|
||||||
|
|
||||||
|
absPath, err := filepath.Abs("./testdata/verificationoverlap_secrets.txt")
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
ctx, cancel := context.WithTimeout(ctx, 10*time.Second)
|
||||||
|
defer cancel()
|
||||||
|
|
||||||
|
confPath, err := filepath.Abs("./testdata/verificationoverlap_detectors.yaml")
|
||||||
|
assert.Nil(t, err)
|
||||||
|
conf, err := config.Read(confPath)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
e, err := Start(ctx,
|
||||||
|
WithConcurrency(1),
|
||||||
|
WithDecoders(decoders.DefaultDecoders()...),
|
||||||
|
WithDetectors(conf.Detectors...),
|
||||||
|
WithVerify(false),
|
||||||
|
WithPrinter(new(discardPrinter)),
|
||||||
|
withVerificationOverlapTracking(),
|
||||||
|
)
|
||||||
|
assert.Nil(t, err)
|
||||||
|
|
||||||
|
cfg := sources.FilesystemConfig{Paths: []string{absPath}}
|
||||||
|
if err := e.ScanFileSystem(ctx, cfg); err != nil {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
// Wait for all the chunks to be processed.
|
||||||
|
assert.Nil(t, e.Finish(ctx))
|
||||||
|
// We want TWO secrets that match both the custom regexes.
|
||||||
|
want := uint64(2)
|
||||||
|
assert.Equal(t, want, e.GetMetrics().UnverifiedSecretsFound)
|
||||||
|
|
||||||
|
// We want 0 because these are custom detectors and verification should still occur.
|
||||||
|
wantDupe := 0
|
||||||
|
assert.Equal(t, wantDupe, e.verificationOverlapTracker.verificationOverlapDuplicateCount)
|
||||||
|
}
|
||||||
|
|
||||||
func TestFragmentFirstLineAndLink(t *testing.T) {
|
func TestFragmentFirstLineAndLink(t *testing.T) {
|
||||||
tests := []struct {
|
tests := []struct {
|
||||||
name string
|
name string
|
||||||
|
@ -398,3 +540,81 @@ func TestSetLink(t *testing.T) {
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestLikelyDuplicate(t *testing.T) {
|
||||||
|
// Initialize detectors
|
||||||
|
// (not actually calling detector FromData or anything, just using detector struct for key creation)
|
||||||
|
detectorA := ahocorasick.DetectorInfo{
|
||||||
|
Key: ahocorasick.CreateDetectorKey(DefaultDetectors()[0]),
|
||||||
|
Detector: DefaultDetectors()[0],
|
||||||
|
}
|
||||||
|
detectorB := ahocorasick.DetectorInfo{
|
||||||
|
Key: ahocorasick.CreateDetectorKey(DefaultDetectors()[1]),
|
||||||
|
Detector: DefaultDetectors()[1],
|
||||||
|
}
|
||||||
|
|
||||||
|
// Define test cases
|
||||||
|
tests := []struct {
|
||||||
|
name string
|
||||||
|
val chunkSecretKey
|
||||||
|
dupes map[chunkSecretKey]struct{}
|
||||||
|
expected bool
|
||||||
|
}{
|
||||||
|
{
|
||||||
|
name: "exact duplicate different detector",
|
||||||
|
val: chunkSecretKey{"PMAK-qnwfsLyRSyfCwfpHaQP1UzDhrgpWvHjbYzjpRCMshjt417zWcrzyHUArs7r", detectorA},
|
||||||
|
dupes: map[chunkSecretKey]struct{}{
|
||||||
|
{"PMAK-qnwfsLyRSyfCwfpHaQP1UzDhrgpWvHjbYzjpRCMshjt417zWcrzyHUArs7r", detectorB}: {},
|
||||||
|
},
|
||||||
|
expected: true,
|
||||||
|
},
|
||||||
|
{
|
||||||
|
name: "non-duplicate length outside range",
|
||||||
|
val: chunkSecretKey{"short", detectorA},
|
||||||
|
dupes: map[chunkSecretKey]struct{}{
|
||||||
|
{"muchlongerthanthevalstring", detectorB}: {},
|
||||||
|
},
|
||||||
|
expected: false,
|
||||||
|
},
|
||||||
|
{
|
||||||
|
name: "similar within threshold",
|
||||||
|
val: chunkSecretKey{"PMAK-qnwfsLyRSyfCwfpHaQP1UzDhrgpWvHjbYzjpRCMshjt417zWcrzyHUArs7r", detectorA},
|
||||||
|
dupes: map[chunkSecretKey]struct{}{
|
||||||
|
{"qnwfsLyRSyfCwfpHaQP1UzDhrgpWvHjbYzjpRCMshjt417zWcrzyHUArs7r", detectorB}: {},
|
||||||
|
},
|
||||||
|
expected: true,
|
||||||
|
},
|
||||||
|
{
|
||||||
|
name: "similar outside threshold",
|
||||||
|
val: chunkSecretKey{"anotherkey", detectorA},
|
||||||
|
dupes: map[chunkSecretKey]struct{}{
|
||||||
|
{"completelydifferent", detectorB}: {},
|
||||||
|
},
|
||||||
|
expected: false,
|
||||||
|
},
|
||||||
|
{
|
||||||
|
name: "empty strings",
|
||||||
|
val: chunkSecretKey{"", detectorA},
|
||||||
|
dupes: map[chunkSecretKey]struct{}{{"", detectorB}: {}},
|
||||||
|
expected: true,
|
||||||
|
},
|
||||||
|
{
|
||||||
|
name: "similar within threshold same detector",
|
||||||
|
val: chunkSecretKey{"PMAK-qnwfsLyRSyfCwfpHaQP1UzDhrgpWvHjbYzjpRCMshjt417zWcrzyHUArs7r", detectorA},
|
||||||
|
dupes: map[chunkSecretKey]struct{}{
|
||||||
|
{"qnwfsLyRSyfCwfpHaQP1UzDhrgpWvHjbYzjpRCMshjt417zWcrzyHUArs7r", detectorA}: {},
|
||||||
|
},
|
||||||
|
expected: false,
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, tc := range tests {
|
||||||
|
t.Run(tc.name, func(t *testing.T) {
|
||||||
|
ctx := context.Background()
|
||||||
|
result := likelyDuplicate(ctx, tc.val, tc.dupes)
|
||||||
|
if result != tc.expected {
|
||||||
|
t.Errorf("expected %v, got %v", tc.expected, result)
|
||||||
|
}
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
13
pkg/engine/testdata/verificationoverlap_detectors.yaml
vendored
Normal file
13
pkg/engine/testdata/verificationoverlap_detectors.yaml
vendored
Normal file
|
@ -0,0 +1,13 @@
|
||||||
|
# config.yaml
|
||||||
|
detectors:
|
||||||
|
- name: detector1
|
||||||
|
keywords:
|
||||||
|
- PMAK
|
||||||
|
regex:
|
||||||
|
api_key: \b(PMAK-[a-zA-Z-0-9]{59})\b
|
||||||
|
|
||||||
|
- name: detector2
|
||||||
|
keywords:
|
||||||
|
- ost
|
||||||
|
regex:
|
||||||
|
api_key: \b([a-zA-Z-0-9]{59})\b
|
2
pkg/engine/testdata/verificationoverlap_secrets.txt
vendored
Normal file
2
pkg/engine/testdata/verificationoverlap_secrets.txt
vendored
Normal file
|
@ -0,0 +1,2 @@
|
||||||
|
|
||||||
|
POSTMAN_API_KEY="PMAK-qnwfsLyRSyfCwfpHaQP1UzDhrgpWvHjbYzjpRCMshjt417zWcrzyHUArs7r"
|
Loading…
Reference in a new issue