Skip to content

Commit

Permalink
fuzz: Add validation of the action log before DeleteAll
Browse files Browse the repository at this point in the history
Signed-off-by: Jussi Maki <[email protected]>
  • Loading branch information
joamaki committed Apr 15, 2024
1 parent fda3186 commit b9ae279
Showing 1 changed file with 163 additions and 42 deletions.
205 changes: 163 additions & 42 deletions fuzz_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,10 +15,9 @@ import (
"sync"
"testing"

"github.com/stretchr/testify/require"

"github.com/cilium/statedb"
"github.com/cilium/statedb/index"
"golang.org/x/exp/maps"
)

// Run test with "--debug" for log output.
Expand All @@ -44,9 +43,10 @@ func newDebugLogger(worker int) *debugLogger {
}

const (
numUniqueIDs = 20
numWorkers = 50
numIterations = 1000
numUniqueIDs = 2000
numUniqueValues = 3000
numWorkers = 20
numIterations = 1000
)

type fuzzObj struct {
Expand All @@ -58,6 +58,10 @@ func mkID() uint64 {
return uint64(rand.Int63n(numUniqueIDs))
}

func mkValue() uint64 {
return uint64(rand.Int63n(numUniqueValues))
}

var idIndex = statedb.Index[fuzzObj, uint64]{
Name: "id",
FromObject: func(obj fuzzObj) index.KeySet {
Expand All @@ -77,13 +81,13 @@ var valueIndex = statedb.Index[fuzzObj, uint64]{
}

var (
tableFuzz1 = statedb.MustNewTable[fuzzObj]("fuzz1", idIndex)
tableFuzz2 = statedb.MustNewTable[fuzzObj]("fuzz2", idIndex)
tableFuzz1 = statedb.MustNewTable[fuzzObj]("fuzz1", idIndex, valueIndex)
tableFuzz2 = statedb.MustNewTable[fuzzObj]("fuzz2", idIndex, valueIndex)
tableFuzz3 = statedb.MustNewTable[fuzzObj]("fuzz3", idIndex, valueIndex)
tableFuzz4 = statedb.MustNewTable[fuzzObj]("fuzz4", idIndex, valueIndex)
fuzzTables = []statedb.TableMeta{tableFuzz1, tableFuzz2, tableFuzz3, tableFuzz4}
fuzzMetrics = statedb.NewExpVarMetrics(false)
fuzzDB, _ = statedb.NewDB(fuzzTables, fuzzMetrics)
fuzzDB *statedb.DB
)

func randomSubset[T any](xs []T) []T {
Expand All @@ -98,51 +102,50 @@ func randomSubset[T any](xs []T) []T {

type actionLog interface {
append(actionLogEntry)
validateTable(txn statedb.ReadTxn, table statedb.Table[fuzzObj]) error
}

type realActionLog struct {
sync.Mutex
log []actionLogEntry
log map[string][]actionLogEntry
}

func (a *realActionLog) append(e actionLogEntry) {
a.Lock()
a.log = append(a.log, e)
a.log[e.table.Name()] = append(a.log[e.table.Name()], e)
a.Unlock()
}

func (a *realActionLog) validate(db *statedb.DB, t *testing.T) {
func (a *realActionLog) validateTable(txn statedb.ReadTxn, table statedb.Table[fuzzObj]) error {
a.Lock()
defer a.Unlock()

// Collapse the log down to objects that are alive at the end.
alive := map[statedb.Table[fuzzObj]]map[uint64]struct{}{}
for _, e := range a.log {
aliveThis, ok := alive[e.table]
if !ok {
aliveThis = map[uint64]struct{}{}
alive[e.table] = aliveThis
}
alive := map[uint64]struct{}{}
for _, e := range a.log[table.Name()] {
switch e.act {
case actInsert:
aliveThis[e.id] = struct{}{}
alive[e.id] = struct{}{}
case actDelete:
delete(aliveThis, e.id)
delete(alive, e.id)
case actDeleteAll:
clear(aliveThis)
clear(alive)
}
}

for table, expected := range alive {
txn := db.ReadTxn()
iter, _ := table.All(txn)
actual := map[uint64]struct{}{}
for obj, _, ok := iter.Next(); ok; obj, _, ok = iter.Next() {
actual[obj.id] = struct{}{}
}
require.Equal(t, expected, actual, "validate failed, mismatching ids: %v",
setSymmetricDifference(actual, expected))
// Since everything was deleted we can clear the log entries for this table now
a.log[table.Name()] = nil

iter, _ := table.All(txn)
actual := map[uint64]struct{}{}
for obj, _, ok := iter.Next(); ok; obj, _, ok = iter.Next() {
actual[obj.id] = struct{}{}
}
diff := setSymmetricDifference(actual, alive)
if len(diff) != 0 {
return fmt.Errorf("validate failed, mismatching ids: %v", maps.Keys(diff))
}
return nil
}

func setSymmetricDifference[T comparable, M map[T]struct{}](s1, s2 M) M {
Expand All @@ -167,6 +170,10 @@ type nopActionLog struct {

func (nopActionLog) append(e actionLogEntry) {}

func (nopActionLog) validateTable(txn statedb.ReadTxn, table statedb.Table[fuzzObj]) error {
return nil
}

const (
actInsert = iota
actDelete
Expand All @@ -190,6 +197,7 @@ type txnActionLog struct {
}

type actionContext struct {
t *testing.T
log *debugLogger
actLog actionLog
txnLog *txnActionLog
Expand All @@ -201,7 +209,7 @@ type action func(ctx actionContext)

func insertAction(ctx actionContext) {
id := mkID()
value := rand.Uint64()
value := mkValue()
ctx.log.log("%s: Insert %d", ctx.table.Name(), id)
ctx.table.Insert(ctx.txn, fuzzObj{id, value})
e := actionLogEntry{ctx.table, actInsert, id, value}
Expand All @@ -220,20 +228,68 @@ func deleteAction(ctx actionContext) {

func deleteAllAction(ctx actionContext) {
ctx.log.log("%s: DeleteAll", ctx.table.Name())

// Validate the log before objects are wiped.
if err := ctx.actLog.validateTable(ctx.txn, ctx.table); err != nil {
panic(err)
}
ctx.table.DeleteAll(ctx.txn)
ctx.actLog.append(actionLogEntry{ctx.table, actDeleteAll, 0, 0})
clear(ctx.txnLog.latest)
}

func deleteManyAction(ctx actionContext) {
// Delete third of the objects using iteration to test that
// nothing bad happens when the iterator is used while deleting.
toDelete := ctx.table.NumObjects(ctx.txn) / 3

iter, _ := ctx.table.All(ctx.txn)
n := 0
for obj, _, ok := iter.Next(); ok; obj, _, ok = iter.Next() {
ctx.log.log("%s: DeleteMany %d (%d/%d)", ctx.table.Name(), obj.id, n+1, toDelete)
_, hadOld, _ := ctx.table.Delete(ctx.txn, obj)
if !hadOld {
panic("expected Delete of a known object to return the old object")
}
e := actionLogEntry{ctx.table, actDelete, obj.id, 0}
ctx.actLog.append(e)
ctx.txnLog.latest[tableAndID{ctx.table.Name(), obj.id}] = e

n++
if n >= toDelete {
break
}
}
}

func allAction(ctx actionContext) {
iter, _ := ctx.table.All(ctx.txn)
ctx.log.log("%s: All => %d found", ctx.table.Name(), len(statedb.Collect(iter)))
}

func getAction(ctx actionContext) {
id := mkID()
iter, _ := ctx.table.Get(ctx.txn, idIndex.Query(mkID()))
ctx.log.log("%s: Get(%d) => %d found", ctx.table.Name(), id, len(statedb.Collect(iter)))
value := mkValue()
iter, _ := ctx.table.Get(ctx.txn, valueIndex.Query(value))
ctx.log.log("%s: Get(%d)", ctx.table.Name(), value)
for obj, _, ok := iter.Next(); ok; obj, _, ok = iter.Next() {
if e, ok2 := ctx.txnLog.latest[tableAndID{ctx.table.Name(), obj.id}]; ok2 {
if e.act == actInsert {
if !ok {
panic("Get() returned not found, expected last inserted value")
}
if e.value != obj.value {
panic("Get() did not return the last write")
}
if obj.value != value {
panic(fmt.Sprintf("Get() returned object with wrong value, expected %d, got %d", value, obj.value))
}
} else if e.act == actDelete {
if ok {
panic("Get() returned value even though it was deleted")
}
}
}
}
}

func firstAction(ctx actionContext) {
Expand Down Expand Up @@ -270,24 +326,70 @@ func prefixAction(ctx actionContext) {
}

var actions = []action{
// Make inserts much more likely than deletions to build up larger tables.
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,
insertAction, insertAction, insertAction, insertAction, insertAction,

deleteAction, deleteAction, deleteAction,
deleteManyAction, deleteAllAction,

deleteAllAction,

firstAction, firstAction, firstAction, firstAction, firstAction,
allAction, lowerboundAction,
getAction, getAction, getAction,
prefixAction,
firstAction, firstAction, firstAction, firstAction, firstAction,
firstAction, firstAction, firstAction, firstAction, firstAction,
getAction, getAction, getAction, getAction, getAction,
allAction, allAction,
lowerboundAction, lowerboundAction, lowerboundAction,
prefixAction, prefixAction, prefixAction,
}

func randomAction() action {
return actions[rand.Intn(len(actions))]
}

func trackerWorker(stop <-chan struct{}) {
txn := fuzzDB.WriteTxn(tableFuzz1)
dt, err := tableFuzz1.DeleteTracker(txn, "tracker")
if err != nil {
panic(err)
}
txn.Commit()
defer dt.Close()

for {
watch := dt.Iterate(
fuzzDB.ReadTxn(),
func(obj fuzzObj, deleted bool, rev uint64) {
},
)
select {
case <-watch:
case <-stop:
return
}
}
}

func fuzzWorker(realActionLog *realActionLog, worker int, iterations int) {
log := newDebugLogger(worker)
for iterations > 0 {
Expand Down Expand Up @@ -337,21 +439,40 @@ func fuzzWorker(realActionLog *realActionLog, worker int, iterations int) {
func TestDB_Fuzz(t *testing.T) {
t.Parallel()

fuzzDB, _ = statedb.NewDB(fuzzTables, fuzzMetrics)

fuzzDB.Start(context.TODO())
defer fuzzDB.Stop(context.TODO())

var actionLog realActionLog
actionLog := &realActionLog{
log: map[string][]actionLogEntry{},
}

var wg sync.WaitGroup
wg.Add(numWorkers)
for i := 0; i < numWorkers; i++ {
i := i
go func() {
fuzzWorker(&actionLog, i, numIterations)
fuzzWorker(actionLog, i, numIterations)
wg.Done()
}()
}
stop := make(chan struct{})
var wg2 sync.WaitGroup
wg2.Add(1)
go func() {
trackerWorker(stop)
wg2.Done()
}()
wg.Wait()
actionLog.validate(fuzzDB, t)
close(stop)
wg2.Wait()

for _, table := range []statedb.Table[fuzzObj]{tableFuzz1, tableFuzz2, tableFuzz3, tableFuzz4} {
if err := actionLog.validateTable(fuzzDB.ReadTxn(), table); err != nil {
t.Fatal(err)
}
}

t.Logf("metrics:\n%s\n", fuzzMetrics.String())
}

0 comments on commit b9ae279

Please sign in to comment.