From 557c35965ce910cdf5bb9ba200ac8deb35276580 Mon Sep 17 00:00:00 2001 From: protolambda Date: Wed, 18 Sep 2024 14:10:10 -0600 Subject: [PATCH 1/4] op-supervisor: register each L2 block, refactor Signed-off-by: protolambda --- op-supervisor/supervisor/backend/backend.go | 40 +- op-supervisor/supervisor/backend/db/db.go | 125 +- .../supervisor/backend/db/db_test.go | 196 +--- .../supervisor/backend/db/entrydb/entry_db.go | 59 + op-supervisor/supervisor/backend/db/init.go | 34 - .../supervisor/backend/db/init_test.go | 91 -- .../supervisor/backend/db/logs/db.go | 641 +++++------ .../backend/db/logs/db_invariants_test.go | 96 +- .../supervisor/backend/db/logs/db_test.go | 1023 ++++++++++------- .../supervisor/backend/db/logs/entries.go | 154 +-- .../supervisor/backend/db/logs/iterator.go | 190 +-- .../supervisor/backend/db/logs/state.go | 407 +++++++ .../supervisor/backend/db/safety_checkers.go | 12 +- .../backend/db/safety_checkers_test.go | 19 +- .../supervisor/backend/source/chain.go | 9 +- .../backend/source/log_processor.go | 16 +- .../backend/source/log_processor_test.go | 96 +- 17 files changed, 1739 insertions(+), 1469 deletions(-) delete mode 100644 op-supervisor/supervisor/backend/db/init.go delete mode 100644 op-supervisor/supervisor/backend/db/init_test.go create mode 100644 op-supervisor/supervisor/backend/db/logs/state.go diff --git a/op-supervisor/supervisor/backend/backend.go b/op-supervisor/supervisor/backend/backend.go index 5be0b6a4c80d..86a862f5b4ac 100644 --- a/op-supervisor/supervisor/backend/backend.go +++ b/op-supervisor/supervisor/backend/backend.go @@ -4,6 +4,7 @@ import ( "context" "errors" "fmt" + "github.com/ethereum-optimism/optimism/op-service/eth" "io" "path/filepath" "sync/atomic" @@ -94,7 +95,7 @@ func (su *SupervisorBackend) addFromRPC(ctx context.Context, logger log.Logger, if err != nil { return fmt.Errorf("failed to create datadir for chain %v: %w", chainID, err) } - logDB, err := logs.NewFromFile(logger, cm, path) + logDB, err := logs.NewFromFile(logger, cm, path, true) if err != nil { return fmt.Errorf("failed to create logdb for chain %v at %v: %w", chainID, path, err) } @@ -133,8 +134,9 @@ func (su *SupervisorBackend) Start(ctx context.Context) error { if !su.started.CompareAndSwap(false, true) { return errors.New("already started") } - // initiate "Resume" on the chains db, which rewinds the database to the last block that is guaranteed to have been fully recorded - if err := su.db.Resume(); err != nil { + // initiate "ResumeFromLastSealedBlock" on the chains db, + // which rewinds the database to the last block that is guaranteed to have been fully recorded + if err := su.db.ResumeFromLastSealedBlock(); err != nil { return fmt.Errorf("failed to resume chains db: %w", err) } // start chain monitors @@ -144,8 +146,8 @@ func (su *SupervisorBackend) Start(ctx context.Context) error { } } // start db maintenance loop - maintinenceCtx, cancel := context.WithCancel(context.Background()) - su.db.StartCrossHeadMaintenance(maintinenceCtx) + maintenanceCtx, cancel := context.WithCancel(context.Background()) + su.db.StartCrossHeadMaintenance(maintenanceCtx) su.maintenanceCancel = cancel return nil } @@ -188,13 +190,16 @@ func (su *SupervisorBackend) CheckMessage(identifier types.Identifier, payloadHa chainID := identifier.ChainID blockNum := identifier.BlockNumber logIdx := identifier.LogIndex - ok, i, err := su.db.Check(chainID, blockNum, uint32(logIdx), backendTypes.TruncateHash(payloadHash)) - if err != nil { - return types.Invalid, fmt.Errorf("failed to check log: %w", err) + i, err := su.db.Check(chainID, blockNum, uint32(logIdx), backendTypes.TruncateHash(payloadHash)) + if errors.Is(err, logs.ErrFuture) { + return types.Unsafe, nil } - if !ok { + if errors.Is(err, logs.ErrConflict) { return types.Invalid, nil } + if err != nil { + return types.Invalid, fmt.Errorf("failed to check log: %w", err) + } safest := types.CrossUnsafe // at this point we have the log entry, and we can check if it is safe by various criteria for _, checker := range []db.SafetyChecker{ @@ -231,16 +236,19 @@ func (su *SupervisorBackend) CheckMessages( // The block is considered safe if all logs in the block are safe // this is decided by finding the last log in the block and func (su *SupervisorBackend) CheckBlock(chainID *hexutil.U256, blockHash common.Hash, blockNumber hexutil.Uint64) (types.SafetyLevel, error) { - // TODO(#11612): this function ignores blockHash and assumes that the block in the db is the one we are looking for - // In order to check block hash, the database must *always* insert a block hash checkpoint, which is not currently done safest := types.CrossUnsafe // find the last log index in the block - i, err := su.db.LastLogInBlock(types.ChainID(*chainID), uint64(blockNumber)) - // TODO(#11836) checking for EOF as a non-error case is a bit of a code smell - // and could potentially be incorrect if the given block number is intentionally far in the future - if err != nil && !errors.Is(err, io.EOF) { + id := eth.BlockID{Hash: blockHash, Number: uint64(blockNumber)} + i, err := su.db.FindSealedBlock(types.ChainID(*chainID), id) + if errors.Is(err, logs.ErrFuture) { + return types.Unsafe, nil + } + if errors.Is(err, logs.ErrConflict) { + return types.Invalid, nil + } + if err != nil { su.logger.Error("failed to scan block", "err", err) - return types.Invalid, fmt.Errorf("failed to scan block: %w", err) + return "", err } // at this point we have the extent of the block, and we can check if it is safe by various criteria for _, checker := range []db.SafetyChecker{ diff --git a/op-supervisor/supervisor/backend/db/db.go b/op-supervisor/supervisor/backend/db/db.go index c05bab400b7b..8d5954a04fb2 100644 --- a/op-supervisor/supervisor/backend/db/db.go +++ b/op-supervisor/supervisor/backend/db/db.go @@ -4,6 +4,7 @@ import ( "context" "errors" "fmt" + "github.com/ethereum/go-ethereum/common" "io" "time" @@ -22,16 +23,32 @@ var ( type LogStorage interface { io.Closer - AddLog(logHash backendTypes.TruncatedHash, block eth.BlockID, timestamp uint64, logIdx uint32, execMsg *backendTypes.ExecutingMessage) error + + AddLog(logHash backendTypes.TruncatedHash, parentBlock eth.BlockID, + logIdx uint32, execMsg *backendTypes.ExecutingMessage) error + + SealBlock(parentHash common.Hash, block eth.BlockID, timestamp uint64) error + Rewind(newHeadBlockNum uint64) error - LatestBlockNum() uint64 - ClosestBlockInfo(blockNum uint64) (uint64, backendTypes.TruncatedHash, error) - ClosestBlockIterator(blockNum uint64) (logs.Iterator, error) - Contains(blockNum uint64, logIdx uint32, loghash backendTypes.TruncatedHash) (bool, entrydb.EntryIdx, error) - LastCheckpointBehind(entrydb.EntryIdx) (logs.Iterator, error) - NextExecutingMessage(logs.Iterator) (backendTypes.ExecutingMessage, error) + + LatestSealedBlockNum() (n uint64, ok bool) + + // FindSealedBlock finds the requested block, to check if it exists, + // returning the next index after it where things continue from. + // returns ErrFuture if the block is too new to be able to tell + // returns ErrDifferent if the known block does not match + FindSealedBlock(block eth.BlockID) (nextEntry entrydb.EntryIdx, err error) + + IteratorStartingAt(i entrydb.EntryIdx) (logs.Iterator, error) + + // returns ErrConflict if the log does not match the canonical chain. + // returns ErrFuture if the log is out of reach. + // returns nil if the log is known and matches the canonical chain. + Contains(blockNum uint64, logIdx uint32, logHash backendTypes.TruncatedHash) (nextIndex entrydb.EntryIdx, err error) } +var _ LogStorage = (*logs.DB)(nil) + type HeadsStorage interface { Current() *heads.Heads Apply(op heads.Operation) error @@ -62,14 +79,20 @@ func (db *ChainsDB) AddLogDB(chain types.ChainID, logDB LogStorage) { db.logDBs[chain] = logDB } -// Resume prepares the chains db to resume recording events after a restart. -// It rewinds the database to the last block that is guaranteed to have been fully recorded to the database +// ResumeFromLastSealedBlock prepares the chains db to resume recording events after a restart. +// It rewinds the database to the last block that is guaranteed to have been fully recorded to the database, // to ensure it can resume recording from the first log of the next block. -// TODO(#11793): we can rename this to something more descriptive like "PrepareWithRollback" -func (db *ChainsDB) Resume() error { +func (db *ChainsDB) ResumeFromLastSealedBlock() error { for chain, logStore := range db.logDBs { - if err := Resume(logStore); err != nil { - return fmt.Errorf("failed to resume chain %v: %w", chain, err) + headNum, ok := logStore.LatestSealedBlockNum() + if ok { + // db must be empty, nothing to rewind to + db.logger.Info("Resuming, but found no DB contents", "chain", chain) + continue + } + db.logger.Info("Resuming, starting from last sealed block", "head", headNum) + if err := logStore.Rewind(headNum); err != nil { + return fmt.Errorf("failed to rewind chain %s to sealed block %d", chain, headNum) } } return nil @@ -101,10 +124,10 @@ func (db *ChainsDB) StartCrossHeadMaintenance(ctx context.Context) { } // Check calls the underlying logDB to determine if the given log entry is safe with respect to the checker's criteria. -func (db *ChainsDB) Check(chain types.ChainID, blockNum uint64, logIdx uint32, logHash backendTypes.TruncatedHash) (bool, entrydb.EntryIdx, error) { +func (db *ChainsDB) Check(chain types.ChainID, blockNum uint64, logIdx uint32, logHash backendTypes.TruncatedHash) (entrydb.EntryIdx, error) { logDB, ok := db.logDBs[chain] if !ok { - return false, 0, fmt.Errorf("%w: %v", ErrUnknownChain, chain) + return 0, fmt.Errorf("%w: %v", ErrUnknownChain, chain) } return logDB.Contains(blockNum, logIdx, logHash) } @@ -146,7 +169,7 @@ func (db *ChainsDB) UpdateCrossHeadsForChain(chainID types.ChainID, checker Safe // advance as far as the local head localHead := checker.LocalHeadForChain(chainID) // get an iterator for the last checkpoint behind the x-head - i, err := db.logDBs[chainID].LastCheckpointBehind(xHead) + iter, err := db.logDBs[chainID].IteratorStartingAt(xHead) if err != nil { return fmt.Errorf("failed to rewind cross-safe head for chain %v: %w", chainID, err) } @@ -158,16 +181,19 @@ func (db *ChainsDB) UpdateCrossHeadsForChain(chainID types.ChainID, checker Safe // - when we reach a message that is not safe // - if an error occurs for { - exec, err := db.logDBs[chainID].NextExecutingMessage(i) - if err == io.EOF { + if err := iter.NextExecMsg(); err == io.EOF { break } else if err != nil { return fmt.Errorf("failed to read next executing message for chain %v: %w", chainID, err) } - // if we are now beyond the local head, stop - if i.Index() > localHead { + // if we would exceed the local head, then abort + if iter.NextIndex() > localHead { break } + exec := iter.ExecMessage() + if exec == nil { + panic("expected executing message after traversing to one without error") + } // use the checker to determine if this message is safe safe := checker.Check( types.ChainIDFromUInt64(uint64(exec.Chain)), @@ -178,7 +204,7 @@ func (db *ChainsDB) UpdateCrossHeadsForChain(chainID types.ChainID, checker Safe break } // if all is well, prepare the x-head update to this point - xHead = i.Index() + xHead = iter.NextIndex() updated = true } @@ -210,62 +236,39 @@ func (db *ChainsDB) UpdateCrossHeads(checker SafetyChecker) error { return nil } -// LastLogInBlock scans through the logs of the given chain starting from the given block number, -// and returns the index of the last log entry in that block. -func (db *ChainsDB) LastLogInBlock(chain types.ChainID, blockNum uint64) (entrydb.EntryIdx, error) { +func (db *ChainsDB) FindSealedBlock(chain types.ChainID, block eth.BlockID) (nextEntry entrydb.EntryIdx, err error) { logDB, ok := db.logDBs[chain] if !ok { return 0, fmt.Errorf("%w: %v", ErrUnknownChain, chain) } - iter, err := logDB.ClosestBlockIterator(blockNum) - if err != nil { - return 0, fmt.Errorf("failed to get block iterator for chain %v: %w", chain, err) - } - ret := entrydb.EntryIdx(0) - // scan through using the iterator until the block number exceeds the target - for { - bn, index, _, err := iter.NextLog() - // if we have reached the end of the database, stop - if err == io.EOF { - break - } - // all other errors are fatal - if err != nil { - return 0, fmt.Errorf("failed to read next log entry for chain %v: %w", chain, err) - } - // if we are now beyond the target block, stop withour updating the return value - if bn > blockNum { - break - } - // only update the return value if the block number is the same - // it is possible the iterator started before the target block, or that the target block is not in the db - if bn == blockNum { - ret = entrydb.EntryIdx(index) - } - } - // if we never found the block, return an error - if ret == 0 { - return 0, fmt.Errorf("block %v not found in chain %v", blockNum, chain) - } - return ret, nil + return logDB.FindSealedBlock(block) } -// LatestBlockNum returns the latest block number that has been recorded to the logs db +// LatestBlockNum returns the latest fully-sealed block number that has been recorded to the logs db // for the given chain. It does not contain safety guarantees. -func (db *ChainsDB) LatestBlockNum(chain types.ChainID) uint64 { +// The block number might not be available (empty database, or non-existent chain). +func (db *ChainsDB) LatestBlockNum(chain types.ChainID) (num uint64, ok bool) { + logDB, knownChain := db.logDBs[chain] + if !knownChain { + return 0, false + } + return logDB.LatestSealedBlockNum() +} + +func (db *ChainsDB) SealBlock(chain types.ChainID, parentHash common.Hash, block eth.BlockID, timestamp uint64) error { logDB, ok := db.logDBs[chain] if !ok { - return 0 + return fmt.Errorf("%w: %v", ErrUnknownChain, chain) } - return logDB.LatestBlockNum() + return logDB.SealBlock(parentHash, block, timestamp) } -func (db *ChainsDB) AddLog(chain types.ChainID, logHash backendTypes.TruncatedHash, block eth.BlockID, timestamp uint64, logIdx uint32, execMsg *backendTypes.ExecutingMessage) error { +func (db *ChainsDB) AddLog(chain types.ChainID, logHash backendTypes.TruncatedHash, parentBlock eth.BlockID, logIdx uint32, execMsg *backendTypes.ExecutingMessage) error { logDB, ok := db.logDBs[chain] if !ok { return fmt.Errorf("%w: %v", ErrUnknownChain, chain) } - return logDB.AddLog(logHash, block, timestamp, logIdx, execMsg) + return logDB.AddLog(logHash, parentBlock, logIdx, execMsg) } func (db *ChainsDB) Rewind(chain types.ChainID, headBlockNum uint64) error { diff --git a/op-supervisor/supervisor/backend/db/db_test.go b/op-supervisor/supervisor/backend/db/db_test.go index 71a4c50cf239..cd3c55f3e7b3 100644 --- a/op-supervisor/supervisor/backend/db/db_test.go +++ b/op-supervisor/supervisor/backend/db/db_test.go @@ -2,9 +2,6 @@ package db import ( "fmt" - "io" - "testing" - "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/testlog" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/entrydb" @@ -12,14 +9,17 @@ import ( "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/logs" backendTypes "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/types" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/types" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/log" "github.com/stretchr/testify/require" + "io" + "testing" ) func TestChainsDB_AddLog(t *testing.T) { t.Run("UnknownChain", func(t *testing.T) { db := NewChainsDB(nil, &stubHeadStorage{}, testlog.Logger(t, log.LevelDebug)) - err := db.AddLog(types.ChainIDFromUInt64(2), backendTypes.TruncatedHash{}, eth.BlockID{}, 1234, 33, nil) + err := db.AddLog(types.ChainIDFromUInt64(2), backendTypes.TruncatedHash{}, eth.BlockID{}, 33, nil) require.ErrorIs(t, err, ErrUnknownChain) }) @@ -28,11 +28,14 @@ func TestChainsDB_AddLog(t *testing.T) { logDB := &stubLogDB{} db := NewChainsDB(map[types.ChainID]LogStorage{ chainID: logDB, - }, &stubHeadStorage{}, - testlog.Logger(t, log.LevelDebug)) - err := db.AddLog(chainID, backendTypes.TruncatedHash{}, eth.BlockID{}, 1234, 33, nil) + }, &stubHeadStorage{}, testlog.Logger(t, log.LevelDebug)) + bl10 := eth.BlockID{Hash: common.Hash{0x10}, Number: 10} + err := db.SealBlock(chainID, common.Hash{0x9}, bl10, 1234) + require.NoError(t, err, err) + err = db.AddLog(chainID, backendTypes.TruncatedHash{}, bl10, 0, nil) require.NoError(t, err, err) require.Equal(t, 1, logDB.addLogCalls) + require.Equal(t, 1, logDB.sealBlockCalls) }) } @@ -56,122 +59,6 @@ func TestChainsDB_Rewind(t *testing.T) { }) } -func TestChainsDB_LastLogInBlock(t *testing.T) { - // using a chainID of 1 for simplicity - chainID := types.ChainIDFromUInt64(1) - // get default stubbed components - logDB, _, h := setupStubbedForUpdateHeads(chainID) - logDB.nextLogs = []nextLogResponse{ - {10, 1, backendTypes.TruncatedHash{}, nil}, - {10, 2, backendTypes.TruncatedHash{}, nil}, - {10, 3, backendTypes.TruncatedHash{}, nil}, - {10, 4, backendTypes.TruncatedHash{}, nil}, - {11, 5, backendTypes.TruncatedHash{}, nil}, - } - - // The ChainsDB is real, but uses only stubbed components - db := NewChainsDB( - map[types.ChainID]LogStorage{ - chainID: logDB}, - &stubHeadStorage{h}, - testlog.Logger(t, log.LevelDebug)) - - // LastLogInBlock is expected to: - // 1. get a block iterator for block 10 (stubbed) - // 2. scan through the iterator until the block number exceeds the target (10) - // 3. return the index of the last log in the block (4) - index, err := db.LastLogInBlock(chainID, 10) - require.NoError(t, err) - require.Equal(t, entrydb.EntryIdx(4), index) -} - -func TestChainsDB_LastLogInBlockEOF(t *testing.T) { - // using a chainID of 1 for simplicity - chainID := types.ChainIDFromUInt64(1) - // get default stubbed components - logDB, _, h := setupStubbedForUpdateHeads(chainID) - logDB.nextLogs = []nextLogResponse{ - {10, 5, backendTypes.TruncatedHash{}, nil}, - {10, 6, backendTypes.TruncatedHash{}, nil}, - {10, 7, backendTypes.TruncatedHash{}, nil}, - {10, 8, backendTypes.TruncatedHash{}, nil}, - {10, 9, backendTypes.TruncatedHash{}, nil}, - {10, 10, backendTypes.TruncatedHash{}, nil}, - } - - // The ChainsDB is real, but uses only stubbed components - db := NewChainsDB( - map[types.ChainID]LogStorage{ - chainID: logDB}, - &stubHeadStorage{h}, - testlog.Logger(t, log.LevelDebug)) - - // LastLogInBlock is expected to: - // 1. get a block iterator for block 10 (stubbed) - // 2. scan through the iterator and never find the target block - // return an error - index, err := db.LastLogInBlock(chainID, 10) - require.NoError(t, err) - require.Equal(t, entrydb.EntryIdx(10), index) -} - -func TestChainsDB_LastLogInBlockNotFound(t *testing.T) { - // using a chainID of 1 for simplicity - chainID := types.ChainIDFromUInt64(1) - // get default stubbed components - logDB, _, h := setupStubbedForUpdateHeads(chainID) - logDB.nextLogs = []nextLogResponse{ - {100, 5, backendTypes.TruncatedHash{}, nil}, - {100, 6, backendTypes.TruncatedHash{}, nil}, - {100, 7, backendTypes.TruncatedHash{}, nil}, - {101, 8, backendTypes.TruncatedHash{}, nil}, - {101, 9, backendTypes.TruncatedHash{}, nil}, - {101, 10, backendTypes.TruncatedHash{}, nil}, - } - - // The ChainsDB is real, but uses only stubbed components - db := NewChainsDB( - map[types.ChainID]LogStorage{ - chainID: logDB}, - &stubHeadStorage{h}, - testlog.Logger(t, log.LevelDebug)) - - // LastLogInBlock is expected to: - // 1. get a block iterator for block 10 (stubbed) - // 2. scan through the iterator and never find the target block - // return an error - _, err := db.LastLogInBlock(chainID, 10) - require.ErrorContains(t, err, "block 10 not found") -} - -func TestChainsDB_LastLogInBlockError(t *testing.T) { - // using a chainID of 1 for simplicity - chainID := types.ChainIDFromUInt64(1) - // get default stubbed components - logDB, _, h := setupStubbedForUpdateHeads(chainID) - logDB.nextLogs = []nextLogResponse{ - {10, 1, backendTypes.TruncatedHash{}, nil}, - {10, 2, backendTypes.TruncatedHash{}, nil}, - {10, 3, backendTypes.TruncatedHash{}, nil}, - {0, 0, backendTypes.TruncatedHash{}, fmt.Errorf("some error")}, - {11, 5, backendTypes.TruncatedHash{}, nil}, - } - - // The ChainsDB is real, but uses only stubbed components - db := NewChainsDB( - map[types.ChainID]LogStorage{ - chainID: logDB}, - &stubHeadStorage{h}, - testlog.Logger(t, log.LevelDebug)) - - // LastLogInBlock is expected to: - // 1. get a block iterator for block 10 (stubbed) - // 2. scan through the iterator and encounter an error - // return an error - _, err := db.LastLogInBlock(chainID, 10) - require.ErrorContains(t, err, "some error") -} - func TestChainsDB_UpdateCrossHeads(t *testing.T) { // using a chainID of 1 for simplicity chainID := types.ChainIDFromUInt64(1) @@ -410,6 +297,7 @@ func (s *stubIterator) ExecMessage() (backendTypes.ExecutingMessage, error) { type stubLogDB struct { addLogCalls int + sealBlockCalls int headBlockNum uint64 emIndex int executingMessages []*backendTypes.ExecutingMessage @@ -420,51 +308,55 @@ type stubLogDB struct { containsResponse containsResponse } -// stubbed LastCheckpointBehind returns a stubbed iterator which was passed in to the struct -func (s *stubLogDB) LastCheckpointBehind(entrydb.EntryIdx) (logs.Iterator, error) { - return s.lastCheckpointBehind, nil +func (s *stubLogDB) AddLog(logHash backendTypes.TruncatedHash, parentBlock eth.BlockID, logIdx uint32, execMsg *backendTypes.ExecutingMessage) error { + s.addLogCalls++ + return nil } -func (s *stubLogDB) ClosestBlockIterator(blockNum uint64) (logs.Iterator, error) { - return &stubIterator{ - index: entrydb.EntryIdx(99), - nextLogs: s.nextLogs, - }, nil +func (s *stubLogDB) SealBlock(parentHash common.Hash, block eth.BlockID, timestamp uint64) error { + s.sealBlockCalls++ + return nil } -func (s *stubLogDB) NextExecutingMessage(i logs.Iterator) (backendTypes.ExecutingMessage, error) { - // if error overload is set, return it to simulate a failure condition - if s.errOverload != nil && s.emIndex >= s.errAfter { - return backendTypes.ExecutingMessage{}, s.errOverload - } - // increment the iterator to mark advancement - i.(*stubIterator).index += 1 - // return the next executing message - m := *s.executingMessages[s.emIndex] - // and increment to the next message for the next call - s.emIndex++ - return m, nil +func (s *stubLogDB) LatestSealedBlockNum() (n uint64, ok bool) { + return s.headBlockNum, true } -func (s *stubLogDB) ClosestBlockInfo(_ uint64) (uint64, backendTypes.TruncatedHash, error) { +func (s *stubLogDB) FindSealedBlock(block eth.BlockID) (nextEntry entrydb.EntryIdx, err error) { panic("not implemented") } -func (s *stubLogDB) AddLog(logHash backendTypes.TruncatedHash, block eth.BlockID, timestamp uint64, logIdx uint32, execMsg *backendTypes.ExecutingMessage) error { - s.addLogCalls++ - return nil +func (s *stubLogDB) IteratorStartingAt(i entrydb.EntryIdx) (logs.Iterator, error) { + //TODO implement me + panic("implement me") } +var _ LogStorage = (*stubLogDB)(nil) + +// +//func (s *stubLogDB) NextExecutingMessage(i logs.Iterator) (backendTypes.ExecutingMessage, error) { +// // if error overload is set, return it to simulate a failure condition +// if s.errOverload != nil && s.emIndex >= s.errAfter { +// return backendTypes.ExecutingMessage{}, s.errOverload +// } +// // increment the iterator to mark advancement +// i.(*stubIterator).index += 1 +// // return the next executing message +// m := *s.executingMessages[s.emIndex] +// // and increment to the next message for the next call +// s.emIndex++ +// return m, nil +//} + type containsResponse struct { - contains bool - index entrydb.EntryIdx - err error + index entrydb.EntryIdx + err error } // stubbed Contains records the arguments passed to it // it returns the response set in the struct, or an empty response -func (s *stubLogDB) Contains(blockNum uint64, logIdx uint32, loghash backendTypes.TruncatedHash) (bool, entrydb.EntryIdx, error) { - return s.containsResponse.contains, s.containsResponse.index, s.containsResponse.err +func (s *stubLogDB) Contains(blockNum uint64, logIdx uint32, logHash backendTypes.TruncatedHash) (nextIndex entrydb.EntryIdx, err error) { + return s.containsResponse.index, s.containsResponse.err } func (s *stubLogDB) Rewind(newHeadBlockNum uint64) error { diff --git a/op-supervisor/supervisor/backend/db/entrydb/entry_db.go b/op-supervisor/supervisor/backend/db/entrydb/entry_db.go index fb02c5be8151..446051821ce3 100644 --- a/op-supervisor/supervisor/backend/db/entrydb/entry_db.go +++ b/op-supervisor/supervisor/backend/db/entrydb/entry_db.go @@ -17,6 +17,65 @@ type EntryIdx int64 type Entry [EntrySize]byte +func (entry Entry) Type() EntryType { + return EntryType(entry[0]) +} + +type EntryTypeFlag uint8 + +const ( + FlagSearchCheckpoint EntryTypeFlag = 1 << TypeSearchCheckpoint + FlagCanonicalHash EntryTypeFlag = 1 << TypeCanonicalHash + FlagInitiatingEvent EntryTypeFlag = 1 << TypeInitiatingEvent + FlagExecutingLink EntryTypeFlag = 1 << TypeExecutingLink + FlagExecutingCheck EntryTypeFlag = 1 << TypeExecutingCheck + FlagPadding EntryTypeFlag = 1 << TypePadding + // for additional padding + FlagPadding2 EntryTypeFlag = FlagPadding << 1 +) + +func (ex EntryTypeFlag) Any(v EntryTypeFlag) bool { + return ex&v != 0 +} + +func (ex *EntryTypeFlag) Add(v EntryTypeFlag) { + *ex = *ex | v +} + +func (ex *EntryTypeFlag) Remove(v EntryTypeFlag) { + *ex = *ex &^ v +} + +type EntryType uint8 + +const ( + TypeSearchCheckpoint EntryType = iota + TypeCanonicalHash + TypeInitiatingEvent + TypeExecutingLink + TypeExecutingCheck + TypePadding +) + +func (d EntryType) String() string { + switch d { + case TypeSearchCheckpoint: + return "searchCheckpoint" + case TypeCanonicalHash: + return "canonicalHash" + case TypeInitiatingEvent: + return "initiatingEvent" + case TypeExecutingLink: + return "executingLink" + case TypeExecutingCheck: + return "executingCheck" + case TypePadding: + return "padding" + default: + return fmt.Sprintf("unknown-%d", uint8(d)) + } +} + // dataAccess defines a minimal API required to manipulate the actual stored data. // It is a subset of the os.File API but could (theoretically) be satisfied by an in-memory implementation for testing. type dataAccess interface { diff --git a/op-supervisor/supervisor/backend/db/init.go b/op-supervisor/supervisor/backend/db/init.go deleted file mode 100644 index fe6b51e5c21f..000000000000 --- a/op-supervisor/supervisor/backend/db/init.go +++ /dev/null @@ -1,34 +0,0 @@ -package db - -import ( - "errors" - "fmt" - "io" - "math" -) - -// Resume prepares the given LogStore to resume recording events. -// It returns the block number of the last block that is guaranteed to have been fully recorded to the database -// and rewinds the database to ensure it can resume recording from the first log of the next block. -func Resume(logDB LogStorage) error { - // Get the last checkpoint that was written then Rewind the db - // to the block prior to that block and start from there. - // Guarantees we will always roll back at least one block - // so we know we're always starting from a fully written block. - checkPointBlock, _, err := logDB.ClosestBlockInfo(math.MaxUint64) - if errors.Is(err, io.EOF) { - // No blocks recorded in the database, start from genesis - return nil - } else if err != nil { - return fmt.Errorf("failed to get block from checkpoint: %w", err) - } - if checkPointBlock == 0 { - return nil - } - block := checkPointBlock - 1 - err = logDB.Rewind(block) - if err != nil { - return fmt.Errorf("failed to rewind the database: %w", err) - } - return nil -} diff --git a/op-supervisor/supervisor/backend/db/init_test.go b/op-supervisor/supervisor/backend/db/init_test.go deleted file mode 100644 index 5ff44c524e5b..000000000000 --- a/op-supervisor/supervisor/backend/db/init_test.go +++ /dev/null @@ -1,91 +0,0 @@ -package db - -import ( - "fmt" - "io" - "testing" - - "github.com/ethereum-optimism/optimism/op-service/eth" - "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/entrydb" - "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/logs" - "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/types" - "github.com/stretchr/testify/require" -) - -func TestRecover(t *testing.T) { - tests := []struct { - name string - stubDB *stubLogStore - expectRewoundTo uint64 - }{ - { - name: "emptydb", - stubDB: &stubLogStore{closestBlockErr: fmt.Errorf("no entries: %w", io.EOF)}, - expectRewoundTo: 0, - }, - { - name: "genesis", - stubDB: &stubLogStore{}, - expectRewoundTo: 0, - }, - { - name: "with_blocks", - stubDB: &stubLogStore{closestBlockNumber: 15}, - expectRewoundTo: 14, - }, - } - for _, test := range tests { - test := test - t.Run(test.name, func(t *testing.T) { - err := Resume(test.stubDB) - require.NoError(t, err) - require.Equal(t, test.expectRewoundTo, test.stubDB.rewoundTo) - }) - } -} - -type stubLogStore struct { - closestBlockNumber uint64 - closestBlockErr error - rewoundTo uint64 -} - -func (s *stubLogStore) Contains(blockNum uint64, logIdx uint32, loghash types.TruncatedHash) (bool, entrydb.EntryIdx, error) { - panic("not supported") -} - -func (s *stubLogStore) ClosestBlockIterator(blockNum uint64) (logs.Iterator, error) { - panic("not supported") -} - -func (s *stubLogStore) LastCheckpointBehind(entrydb.EntryIdx) (logs.Iterator, error) { - panic("not supported") -} - -func (s *stubLogStore) ClosestBlockInfo(blockNum uint64) (uint64, types.TruncatedHash, error) { - if s.closestBlockErr != nil { - return 0, types.TruncatedHash{}, s.closestBlockErr - } - return s.closestBlockNumber, types.TruncatedHash{}, nil -} - -func (s *stubLogStore) NextExecutingMessage(logs.Iterator) (types.ExecutingMessage, error) { - panic("not supported") -} - -func (s *stubLogStore) Rewind(headBlockNum uint64) error { - s.rewoundTo = headBlockNum - return nil -} - -func (s *stubLogStore) AddLog(logHash types.TruncatedHash, block eth.BlockID, timestamp uint64, logIdx uint32, execMsg *types.ExecutingMessage) error { - panic("not supported") -} - -func (s *stubLogStore) LatestBlockNum() uint64 { - panic("not supported") -} - -func (s *stubLogStore) Close() error { - return nil -} diff --git a/op-supervisor/supervisor/backend/db/logs/db.go b/op-supervisor/supervisor/backend/db/logs/db.go index 0f757f5254cd..b788c5db5659 100644 --- a/op-supervisor/supervisor/backend/db/logs/db.go +++ b/op-supervisor/supervisor/backend/db/logs/db.go @@ -4,34 +4,35 @@ import ( "errors" "fmt" "io" - "math" "sync" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/common/hexutil" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/entrydb" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/types" - "github.com/ethereum/go-ethereum/log" ) const ( - searchCheckpointFrequency = 256 - - eventFlagIncrementLogIdx = byte(1) - eventFlagHasExecutingMessage = byte(1) << 1 -) - -const ( - typeSearchCheckpoint byte = iota - typeCanonicalHash - typeInitiatingEvent - typeExecutingLink - typeExecutingCheck + searchCheckpointFrequency = 256 + eventFlagHasExecutingMessage = byte(1) ) var ( - ErrLogOutOfOrder = errors.New("log out of order") + // ErrLogOutOfOrder happens when you try to add a log to the DB, + // but it does not actually fit onto the latest data (by being too old or new). + ErrLogOutOfOrder = errors.New("log out of order") + // ErrDataCorruption happens when the underlying DB has some I/O issue ErrDataCorruption = errors.New("data corruption") - ErrNotFound = errors.New("not found") + // ErrSkipped happens when we try to retrieve data that is not available (pruned) + // It may also happen if we erroneously skip data, that was not considered a conflict, if the DB is corrupted. + ErrSkipped = errors.New("skipped data") + // ErrFuture happens when data is just not yet available + ErrFuture = errors.New("future data") + // ErrConflict happens when we know for sure that there is different canonical data + ErrConflict = errors.New("conflicting data") ) type Metrics interface { @@ -39,11 +40,6 @@ type Metrics interface { RecordDBSearchEntriesRead(count int64) } -type logContext struct { - blockNum uint64 - logIdx uint32 -} - type EntryStore interface { Size() int64 LastEntryIdx() entrydb.EntryIdx @@ -60,28 +56,6 @@ type EntryStore interface { // Use a fixed 24 bytes per entry. // // Data is an append-only log, that can be binary searched for any necessary event data. -// -// Rules: -// if entry_index % 256 == 0: must be type 0. For easy binary search. -// type 1 always adjacent to type 0 -// type 2 "diff" values are offsets from type 0 values (always within 256 entries range) -// type 3 always after type 2 -// type 4 always after type 3 -// -// Types ( = 1 byte): -// type 0: "search checkpoint" = 20 bytes -// type 1: "canonical hash" = 21 bytes -// type 2: "initiating event" = 23 bytes -// type 3: "executing link" = 24 bytes -// type 4: "executing check" = 21 bytes -// other types: future compat. E.g. for linking to L1, registering block-headers as a kind of initiating-event, tracking safe-head progression, etc. -// -// Right-pad each entry that is not 24 bytes. -// -// event-flags: each bit represents a boolean value, currently only two are defined -// * event-flags & 0x01 - true if the log index should increment. Should only be false when the event is immediately after a search checkpoint and canonical hash -// * event-flags & 0x02 - true if the initiating event has an executing link that should follow. Allows detecting when the executing link failed to write. -// event-hash: H(origin, timestamp, payloadhash); enough to check identifier matches & payload matches. type DB struct { log log.Logger m Metrics @@ -91,21 +65,21 @@ type DB struct { lastEntryContext logContext } -func NewFromFile(logger log.Logger, m Metrics, path string) (*DB, error) { +func NewFromFile(logger log.Logger, m Metrics, path string, trimToLastSealed bool) (*DB, error) { store, err := entrydb.NewEntryDB(logger, path) if err != nil { return nil, fmt.Errorf("failed to open DB: %w", err) } - return NewFromEntryStore(logger, m, store) + return NewFromEntryStore(logger, m, store, trimToLastSealed) } -func NewFromEntryStore(logger log.Logger, m Metrics, store EntryStore) (*DB, error) { +func NewFromEntryStore(logger log.Logger, m Metrics, store EntryStore, trimToLastSealed bool) (*DB, error) { db := &DB{ log: logger, m: m, store: store, } - if err := db.init(); err != nil { + if err := db.init(trimToLastSealed); err != nil { return nil, fmt.Errorf("failed to init database: %w", err) } return db, nil @@ -115,59 +89,57 @@ func (db *DB) lastEntryIdx() entrydb.EntryIdx { return db.store.LastEntryIdx() } -func (db *DB) init() error { +func (db *DB) init(trimToLastSealed bool) error { defer db.updateEntryCountMetric() // Always update the entry count metric after init completes - if err := db.trimInvalidTrailingEntries(); err != nil { - return fmt.Errorf("failed to trim invalid trailing entries: %w", err) + if trimToLastSealed { + if err := db.trimToLastSealed(); err != nil { + return fmt.Errorf("failed to trim invalid trailing entries: %w", err) + } } if db.lastEntryIdx() < 0 { - // Database is empty so no context to load + // Database is empty. + // Make a state that is ready to apply the genesis block on top of as first entry. + // This will infer into a checkpoint (half of the block seal here) + // and is then followed up with canonical-hash entry of genesis. + db.lastEntryContext = logContext{ + nextEntryIndex: 0, + blockHash: types.TruncatedHash{}, + blockNum: 0, + timestamp: 0, + logsSince: 0, + logHash: types.TruncatedHash{}, + execMsg: nil, + out: nil, + } return nil } - + // start at the last checkpoint, + // and then apply any remaining changes on top, to hydrate the state. lastCheckpoint := (db.lastEntryIdx() / searchCheckpointFrequency) * searchCheckpointFrequency - i, err := db.newIterator(lastCheckpoint) - if err != nil { - return fmt.Errorf("failed to create iterator at last search checkpoint: %w", err) - } - // Read all entries until the end of the file - for { - _, _, _, err := i.NextLog() - if errors.Is(err, io.EOF) { - break - } else if err != nil { - return fmt.Errorf("failed to init from existing entries: %w", err) - } + i := db.newIterator(lastCheckpoint) + i.current.need.Add(entrydb.FlagCanonicalHash) + if err := i.End(); err != nil { + return fmt.Errorf("failed to init from remaining trailing data: %w", err) } db.lastEntryContext = i.current return nil } -func (db *DB) trimInvalidTrailingEntries() error { +func (db *DB) trimToLastSealed() error { i := db.lastEntryIdx() for ; i >= 0; i-- { entry, err := db.store.Read(i) if err != nil { return fmt.Errorf("failed to read %v to check for trailing entries: %w", i, err) } - if entry[0] == typeExecutingCheck { - // executing check is a valid final entry + if entry.Type() == entrydb.TypeCanonicalHash { + // only an executing hash, indicating a sealed block, is a valid point for restart break } - if entry[0] == typeInitiatingEvent { - evt, err := newInitiatingEventFromEntry(entry) - if err != nil { - // Entry is invalid, keep walking backwards - continue - } - if !evt.hasExecMsg { - // init event with no exec msg is a valid final entry - break - } - } } if i < db.lastEntryIdx() { db.log.Warn("Truncating unexpected trailing entries", "prev", db.lastEntryIdx(), "new", i) + // trim such that the last entry is the canonical-hash we identified return db.store.Truncate(i) } return nil @@ -177,383 +149,328 @@ func (db *DB) updateEntryCountMetric() { db.m.RecordDBEntryCount(db.store.Size()) } -func (db *DB) LatestBlockNum() uint64 { - return db.lastEntryContext.blockNum +func (db *DB) IteratorStartingAt(i entrydb.EntryIdx) (Iterator, error) { + db.rwLock.RLock() + defer db.rwLock.RUnlock() + if i > db.lastEntryContext.nextEntryIndex { + return nil, ErrFuture + } + // TODO this iterator is semi-broken; + // inferred entries will not be added if starting from an incomplete block or log + return db.newIterator(i), nil } -// ClosestBlockInfo returns the block number and hash of the highest recorded block at or before blockNum. -// Since block data is only recorded in search checkpoints, this may return an earlier block even if log data is -// recorded for the requested block. -func (db *DB) ClosestBlockInfo(blockNum uint64) (uint64, types.TruncatedHash, error) { +// FindSealedBlock finds the requested block, to check if it exists, +// returning the next index after it where things continue from. +// returns ErrFuture if the block is too new to be able to tell +// returns ErrDifferent if the known block does not match +func (db *DB) FindSealedBlock(block eth.BlockID) (nextEntry entrydb.EntryIdx, err error) { db.rwLock.RLock() defer db.rwLock.RUnlock() - checkpointIdx, err := db.searchCheckpoint(blockNum, math.MaxUint32) - if err != nil { - return 0, types.TruncatedHash{}, fmt.Errorf("no checkpoint at or before block %v found: %w", blockNum, err) + iter, err := db.newIteratorAt(block.Number, 0) + if errors.Is(err, ErrFuture) { + return 0, fmt.Errorf("block %d is not known yet: %w", block.Number, ErrFuture) + } else if err != nil { + return 0, fmt.Errorf("failed to find sealed block %d: %w", block.Number, err) } - checkpoint, err := db.readSearchCheckpoint(checkpointIdx) - if err != nil { - return 0, types.TruncatedHash{}, fmt.Errorf("failed to reach checkpoint: %w", err) + h, _, ok := iter.SealedBlock() + if !ok { + panic("expected block") } - entry, err := db.readCanonicalHash(checkpointIdx + 1) - if err != nil { - return 0, types.TruncatedHash{}, fmt.Errorf("failed to read canonical hash: %w", err) + if types.TruncateHash(block.Hash) != h { + return 0, fmt.Errorf("queried %s but got %s at number %d: %w", block.Hash, h, block.Number, ErrConflict) } - return checkpoint.blockNum, entry.hash, nil + return iter.NextIndex(), nil } -// ClosestBlockIterator returns an iterator for the block closest to the specified blockNum. -// The iterator will start at the search checkpoint for the block, or the first checkpoint before it. -func (db *DB) ClosestBlockIterator(blockNum uint64) (Iterator, error) { +// LatestSealedBlockNum returns the block number of the block that was last sealed, +// or ok=false if there is no sealed block (i.e. empty DB) +func (db *DB) LatestSealedBlockNum() (n uint64, ok bool) { db.rwLock.RLock() defer db.rwLock.RUnlock() - checkpointIdx, err := db.searchCheckpoint(blockNum, math.MaxUint32) - if err != nil { - return nil, fmt.Errorf("no checkpoint at or before block %v found: %w", blockNum, err) + if !db.lastEntryContext.hasCompleteBlock() { + if db.lastEntryContext.blockNum == 0 { + db.log.Debug("No DB contents yet") + } else { + db.log.Debug("New block is already in progress", "num", db.lastEntryContext.blockNum) + } } - return db.newIterator(checkpointIdx) + return db.lastEntryContext.blockNum, true } -// Get returns the truncated hash of the log at the specified blockNum and logIdx, -// or an error if the log is not found. -func (db *DB) Get(blockNum uint64, logiIdx uint32) (types.TruncatedHash, error) { +// Get returns the truncated hash of the log at the specified blockNum (of the sealed block) +// and logIdx (of the log after the block), or an error if the log is not found. +func (db *DB) Get(blockNum uint64, logIdx uint32) (types.TruncatedHash, error) { db.rwLock.RLock() defer db.rwLock.RUnlock() - hash, _, err := db.findLogInfo(blockNum, logiIdx) + hash, _, err := db.findLogInfo(blockNum, logIdx) return hash, err } -// Contains return true iff the specified logHash is recorded in the specified blockNum and logIdx. -// If the log is found, the entry index of the log is returned, too. +// Contains returns no error iff the specified logHash is recorded in the specified blockNum and logIdx. +// If the log is out of reach, then ErrFuture is returned. +// If the log is determined to conflict with the canonical chain, then ErrConflict is returned. // logIdx is the index of the log in the array of all logs in the block. // This can be used to check the validity of cross-chain interop events. -func (db *DB) Contains(blockNum uint64, logIdx uint32, logHash types.TruncatedHash) (bool, entrydb.EntryIdx, error) { +func (db *DB) Contains(blockNum uint64, logIdx uint32, logHash types.TruncatedHash) (entrydb.EntryIdx, error) { db.rwLock.RLock() defer db.rwLock.RUnlock() db.log.Trace("Checking for log", "blockNum", blockNum, "logIdx", logIdx, "hash", logHash) evtHash, iter, err := db.findLogInfo(blockNum, logIdx) - if errors.Is(err, ErrNotFound) { - // Did not find a log at blockNum and logIdx - return false, 0, nil - } else if err != nil { - return false, 0, err + if err != nil { + return 0, err // may be ErrConflict if the block does not have as many logs } db.log.Trace("Found initiatingEvent", "blockNum", blockNum, "logIdx", logIdx, "hash", evtHash) // Found the requested block and log index, check if the hash matches - if evtHash == logHash { - return true, iter.Index(), nil + if evtHash != logHash { + return 0, fmt.Errorf("payload hash mismatch: expected %s, got %s", logHash, evtHash) } - return false, 0, nil + return iter.NextIndex(), nil } -// Executes checks if the log identified by the specific block number and log index, has an ExecutingMessage associated -// with it that needs to be checked as part of interop validation. -// logIdx is the index of the log in the array of all logs in the block. -// Returns the ExecutingMessage if it exists, or ExecutingMessage{} if the log is found but has no ExecutingMessage. -// Returns ErrNotFound if the specified log does not exist in the database. -func (db *DB) Executes(blockNum uint64, logIdx uint32) (types.ExecutingMessage, error) { - db.rwLock.RLock() - defer db.rwLock.RUnlock() - _, iter, err := db.findLogInfo(blockNum, logIdx) - if err != nil { - return types.ExecutingMessage{}, err +func (db *DB) findLogInfo(blockNum uint64, logIdx uint32) (types.TruncatedHash, Iterator, error) { + if blockNum == 0 { + return types.TruncatedHash{}, nil, ErrConflict // no logs in block 0 + } + // blockNum-1, such that we find a log that came after the parent num-1 was sealed. + // logIdx, such that all entries before logIdx can be skipped, but logIdx itself is still readable. + iter, err := db.newIteratorAt(blockNum-1, logIdx) + if errors.Is(err, ErrFuture) { + db.log.Trace("Could not find log yet", "blockNum", blockNum, "logIdx", logIdx) + return types.TruncatedHash{}, nil, err + } else if err != nil { + db.log.Error("Failed searching for log", "blockNum", blockNum, "logIdx", logIdx) + return types.TruncatedHash{}, nil, err } - execMsg, err := iter.ExecMessage() - if err != nil { - return types.ExecutingMessage{}, fmt.Errorf("failed to read executing message: %w", err) + if err := iter.NextInitMsg(); err != nil { + return types.TruncatedHash{}, nil, fmt.Errorf("failed to read initiating message %d, on top of block %d: %w", logIdx, blockNum, err) + } + if _, x, ok := iter.SealedBlock(); !ok { + panic("expected block") + } else if x < blockNum-1 { + panic(fmt.Errorf("bug in newIteratorAt, expected to have found parent block %d but got %d", blockNum-1, x)) + } else if x > blockNum-1 { + return types.TruncatedHash{}, nil, fmt.Errorf("log does not exist, found next block already: %w", ErrConflict) + } + logHash, x, ok := iter.InitMessage() + if !ok { + panic("expected init message") + } else if x != logIdx { + panic(fmt.Errorf("bug in newIteratorAt, expected to have found log %d but got %d", logIdx, x)) } - return execMsg, nil + return logHash, iter, nil } -func (db *DB) findLogInfo(blockNum uint64, logIdx uint32) (types.TruncatedHash, Iterator, error) { - entryIdx, err := db.searchCheckpoint(blockNum, logIdx) +// newIteratorAt returns an iterator ready after the given sealed block number, +// and positioned such that the next log-read on the iterator return the log with logIndex, if any. +// It may return an ErrNotFound if the block number is unknown, +// or if there are just not that many seen log events after the block as requested. +func (db *DB) newIteratorAt(blockNum uint64, logIndex uint32) (*iterator, error) { + // find a checkpoint before or exactly when blockNum was sealed, + // and have processed up to but not including [logIndex] number of logs (i.e. all prior logs, if any). + searchCheckpointIndex, err := db.searchCheckpoint(blockNum, logIndex) if errors.Is(err, io.EOF) { // Did not find a checkpoint to start reading from so the log cannot be present. - return types.TruncatedHash{}, nil, ErrNotFound + return nil, ErrFuture } else if err != nil { - return types.TruncatedHash{}, nil, err + return nil, err } - - i, err := db.newIterator(entryIdx) + // The iterator did not consume the checkpoint yet, it's positioned right at it. + // So we can call NextBlock() and get the checkpoint itself as first entry. + iter := db.newIterator(searchCheckpointIndex) if err != nil { - return types.TruncatedHash{}, nil, fmt.Errorf("failed to create iterator: %w", err) + return nil, err } - db.log.Trace("Starting search", "entry", entryIdx, "blockNum", i.current.blockNum, "logIdx", i.current.logIdx) + iter.current.need.Add(entrydb.FlagCanonicalHash) defer func() { - db.m.RecordDBSearchEntriesRead(i.entriesRead) + db.m.RecordDBSearchEntriesRead(iter.entriesRead) }() + // First walk up to the block that we are sealed up to (incl.) for { - evtBlockNum, evtLogIdx, evtHash, err := i.NextLog() - if errors.Is(err, io.EOF) { - // Reached end of log without finding the event - return types.TruncatedHash{}, nil, ErrNotFound + if _, n, _ := iter.SealedBlock(); n == blockNum { // we may already have it exactly + break + } + if err := iter.NextBlock(); err == io.EOF { + db.log.Trace("ran out of data, could not find block", "nextIndex", iter.NextIndex(), "target", blockNum) + return nil, ErrFuture } else if err != nil { - return types.TruncatedHash{}, nil, fmt.Errorf("failed to read next log: %w", err) + db.log.Error("failed to read next block", "nextIndex", iter.NextIndex(), "target", blockNum) + return nil, err } - if evtBlockNum == blockNum && evtLogIdx == logIdx { - db.log.Trace("Found initiatingEvent", "blockNum", evtBlockNum, "logIdx", evtLogIdx, "hash", evtHash) - return evtHash, i, nil + h, num, ok := iter.SealedBlock() + if !ok { + panic("expected sealed block") } - if evtBlockNum > blockNum || (evtBlockNum == blockNum && evtLogIdx > logIdx) { - // Progressed past the requested log without finding it. - return types.TruncatedHash{}, nil, ErrNotFound + db.log.Trace("found sealed block", "num", num, "hash", h) + if num < blockNum { + continue } - } -} - -func (db *DB) newIterator(startCheckpointEntry entrydb.EntryIdx) (*iterator, error) { - checkpoint, err := db.readSearchCheckpoint(startCheckpointEntry) - if err != nil { - return nil, fmt.Errorf("failed to read search checkpoint entry %v: %w", startCheckpointEntry, err) - } - startIdx := startCheckpointEntry + 2 - firstEntry, err := db.store.Read(startIdx) - if errors.Is(err, io.EOF) { - // There should always be an entry after a checkpoint and canonical hash so an EOF here is data corruption - return nil, fmt.Errorf("%w: no entry after checkpoint and canonical hash at %v", ErrDataCorruption, startCheckpointEntry) - } else if err != nil { - return nil, fmt.Errorf("failed to read first entry to iterate %v: %w", startCheckpointEntry+2, err) - } - startLogCtx := logContext{ - blockNum: checkpoint.blockNum, - logIdx: checkpoint.logIdx, - } - // Handle starting from a checkpoint after initiating-event but before its executing-link or executing-check - if firstEntry[0] == typeExecutingLink || firstEntry[0] == typeExecutingCheck { - if firstEntry[0] == typeExecutingLink { - // The start checkpoint was between the initiating event and the executing link - // Step back to read the initiating event. The checkpoint block data will be for the initiating event - startIdx = startCheckpointEntry - 1 - } else { - // The start checkpoint was between the executing link and the executing check - // Step back to read the initiating event. The checkpoint block data will be for the initiating event - startIdx = startCheckpointEntry - 2 + if num != blockNum { // block does not contain + return nil, fmt.Errorf("looking for %d, but already at %d: %w", blockNum, num, ErrConflict) } - initEntry, err := db.store.Read(startIdx) - if err != nil { - return nil, fmt.Errorf("failed to read prior initiating event: %w", err) + break + } + // Now walk up to the number of seen logs that we want to have processed. + // E.g. logIndex == 2, need to have processed index 0 and 1, + // so two logs before quiting (and not 3 to then quit after). + for iter.current.logsSince < logIndex { + if err := iter.NextInitMsg(); err == io.EOF { + return nil, ErrFuture + } else if err != nil { + return nil, err } - initEvt, err := newInitiatingEventFromEntry(initEntry) - if err != nil { - return nil, fmt.Errorf("invalid initiating event at idx %v: %w", startIdx, err) + _, num, ok := iter.SealedBlock() + if !ok { + panic("expected sealed block") + } + if num > blockNum { + // we overshot, the block did not contain as many seen log events as requested + return nil, ErrConflict + } + _, idx, ok := iter.InitMessage() + if !ok { + panic("expected initializing message") } - startLogCtx = initEvt.preContext(startLogCtx) + if idx+1 < logIndex { + continue + } + if idx+1 == logIndex { + break // the NextInitMsg call will position the iterator at the re + } + return nil, fmt.Errorf("unexpected log-skip at block %d log %d", blockNum, idx) } - i := &iterator{ + return iter, nil +} + +// newIterator creates an iterator at the given index. +// None of the iterator attributes will be ready for reads, +// but the entry at the given index will be first read when using the iterator. +func (db *DB) newIterator(index entrydb.EntryIdx) *iterator { + return &iterator{ db: db, - // +2 to skip the initial search checkpoint and the canonical hash event after it - nextEntryIdx: startIdx, - current: startLogCtx, + current: logContext{ + nextEntryIndex: index, + }, } - return i, nil } -// searchCheckpoint performs a binary search of the searchCheckpoint entries to find the closest one at or before -// the requested log. -// Returns the index of the searchCheckpoint to begin reading from or an error -func (db *DB) searchCheckpoint(blockNum uint64, logIdx uint32) (entrydb.EntryIdx, error) { +// searchCheckpoint performs a binary search of the searchCheckpoint entries +// to find the closest one with an equal or lower block number and equal or lower amount of seen logs. +// Returns the index of the searchCheckpoint to begin reading from or an error. +func (db *DB) searchCheckpoint(sealedBlockNum uint64, logsSince uint32) (entrydb.EntryIdx, error) { n := (db.lastEntryIdx() / searchCheckpointFrequency) + 1 - // Define x[-1] < target and x[n] >= target. - // Invariant: x[i-1] < target, x[j] >= target. + // Define: x is the array of known checkpoints + // Invariant: x[i] <= target, x[j] > target. i, j := entrydb.EntryIdx(0), n - for i < j { - h := entrydb.EntryIdx(uint64(i+j) >> 1) // avoid overflow when computing h + for i+1 < j { // i is inclusive, j is exclusive. + // Get the checkpoint exactly in-between, + // bias towards a higher value if an even number of checkpoints. + // E.g. i=3 and j=4 would not run, since i + 1 < j + // E.g. i=3 and j=5 leaves checkpoints 3, 4, and we pick 4 as pivot + // E.g. i=3 and j=6 leaves checkpoints 3, 4, 5, and we pick 4 as pivot + // + // The following holds: i ≤ h < j + h := entrydb.EntryIdx((uint64(i) + uint64(j)) >> 1) checkpoint, err := db.readSearchCheckpoint(h * searchCheckpointFrequency) if err != nil { return 0, fmt.Errorf("failed to read entry %v: %w", h, err) } - // i ≤ h < j - if checkpoint.blockNum < blockNum || (checkpoint.blockNum == blockNum && checkpoint.logIdx < logIdx) { - i = h + 1 // preserves x[i-1] < target + if checkpoint.blockNum < sealedBlockNum || + (checkpoint.blockNum == sealedBlockNum && checkpoint.logsSince < logsSince) { + i = h } else { - j = h // preserves x[j] >= target + j = h } } - if i < n { - checkpoint, err := db.readSearchCheckpoint(i * searchCheckpointFrequency) - if err != nil { - return 0, fmt.Errorf("failed to read entry %v: %w", i, err) - } - if checkpoint.blockNum == blockNum && checkpoint.logIdx == logIdx { - // Found entry at requested block number and log index - return i * searchCheckpointFrequency, nil - } + if i+1 != j { + panic("expected to have 1 checkpoint left") + } + result := i * searchCheckpointFrequency + checkpoint, err := db.readSearchCheckpoint(result) + if err != nil { + return 0, fmt.Errorf("failed to read final search checkpoint result: %w", err) } - if i == 0 { - // There are no checkpoints before the requested blocks - return 0, io.EOF + if checkpoint.blockNum > sealedBlockNum || + (checkpoint.blockNum == sealedBlockNum && checkpoint.logsSince > logsSince) { + return 0, fmt.Errorf("missing data, earliest search checkpoint is %d with %d logs, cannot find something before or at %d with %d logs: %w", + checkpoint.blockNum, checkpoint.logsSince, sealedBlockNum, logsSince, ErrSkipped) } - // Not found, need to start reading from the entry prior - return (i - 1) * searchCheckpointFrequency, nil + return result, nil } -func (db *DB) AddLog(logHash types.TruncatedHash, block eth.BlockID, timestamp uint64, logIdx uint32, execMsg *types.ExecutingMessage) error { - db.rwLock.Lock() - defer db.rwLock.Unlock() - postState := logContext{ - blockNum: block.Number, - logIdx: logIdx, - } - if block.Number == 0 { - return fmt.Errorf("%w: should not have logs in block 0", ErrLogOutOfOrder) - } - if db.lastEntryContext.blockNum > block.Number { - return fmt.Errorf("%w: adding block %v, head block: %v", ErrLogOutOfOrder, block.Number, db.lastEntryContext.blockNum) - } - if db.lastEntryContext.blockNum == block.Number && db.lastEntryContext.logIdx+1 != logIdx { - return fmt.Errorf("%w: adding log %v in block %v, but currently at log %v", ErrLogOutOfOrder, logIdx, block.Number, db.lastEntryContext.logIdx) - } - if db.lastEntryContext.blockNum < block.Number && logIdx != 0 { - return fmt.Errorf("%w: adding log %v as first log in block %v", ErrLogOutOfOrder, logIdx, block.Number) +// debug util to log the last 10 entries of the chain +func (db *DB) debugTip() { + for x := 0; x < 10; x++ { + index := db.lastEntryIdx() - entrydb.EntryIdx(x) + if index < 0 { + continue + } + e, err := db.store.Read(index) + if err == nil { + db.log.Debug("tip", "index", index, "type", e.Type()) + } } - var entriesToAdd []entrydb.Entry - newContext := db.lastEntryContext - lastEntryIdx := db.lastEntryIdx() +} - addEntry := func(entry entrydb.Entry) { - entriesToAdd = append(entriesToAdd, entry) - lastEntryIdx++ +func (db *DB) flush() error { + for i, e := range db.lastEntryContext.out { + db.log.Trace("appending entry", "type", e.Type(), "entry", hexutil.Bytes(e[:]), + "next", int(db.lastEntryContext.nextEntryIndex)-len(db.lastEntryContext.out)+i) } - maybeAddCheckpoint := func() { - if (lastEntryIdx+1)%searchCheckpointFrequency == 0 { - addEntry(newSearchCheckpoint(block.Number, logIdx, timestamp).encode()) - addEntry(newCanonicalHash(types.TruncateHash(block.Hash)).encode()) - newContext = postState - } + if err := db.store.Append(db.lastEntryContext.out...); err != nil { + return fmt.Errorf("failed to append entries: %w", err) } - maybeAddCheckpoint() + db.lastEntryContext.out = db.lastEntryContext.out[:0] + db.updateEntryCountMetric() + return nil +} - evt, err := newInitiatingEvent(newContext, postState.blockNum, postState.logIdx, logHash, execMsg != nil) - if err != nil { - return fmt.Errorf("failed to create initiating event: %w", err) +func (db *DB) SealBlock(parentHash common.Hash, block eth.BlockID, timestamp uint64) error { + db.rwLock.Lock() + defer db.rwLock.Unlock() + + if err := db.lastEntryContext.SealBlock(parentHash, block, timestamp); err != nil { + return fmt.Errorf("failed to seal block: %w", err) } - addEntry(evt.encode()) + db.log.Trace("Sealed block", "parent", parentHash, "block", block, "timestamp", timestamp) + return db.flush() +} - if execMsg != nil { - maybeAddCheckpoint() - link, err := newExecutingLink(*execMsg) - if err != nil { - return fmt.Errorf("failed to create executing link: %w", err) - } - addEntry(link.encode()) +func (db *DB) AddLog(logHash types.TruncatedHash, parentBlock eth.BlockID, logIdx uint32, execMsg *types.ExecutingMessage) error { + db.rwLock.Lock() + defer db.rwLock.Unlock() - maybeAddCheckpoint() - addEntry(newExecutingCheck(execMsg.Hash).encode()) + if err := db.lastEntryContext.ApplyLog(parentBlock, logIdx, logHash, execMsg); err != nil { + return fmt.Errorf("failed to apply log: %w", err) } - if err := db.store.Append(entriesToAdd...); err != nil { - return fmt.Errorf("failed to append entries: %w", err) - } - db.lastEntryContext = postState - db.updateEntryCountMetric() - return nil + db.log.Trace("Applied log", "parentBlock", parentBlock, "logIndex", logIdx, "logHash", logHash, "executing", execMsg != nil) + return db.flush() } // Rewind the database to remove any blocks after headBlockNum // The block at headBlockNum itself is not removed. -func (db *DB) Rewind(headBlockNum uint64) error { +func (db *DB) Rewind(newHeadBlockNum uint64) error { db.rwLock.Lock() defer db.rwLock.Unlock() - if headBlockNum >= db.lastEntryContext.blockNum { - // Nothing to do - return nil - } - // Find the last checkpoint before the block to remove - idx, err := db.searchCheckpoint(headBlockNum+1, 0) - if errors.Is(err, io.EOF) { - // Requested a block prior to the first checkpoint - // Delete everything without scanning forward - idx = -1 - } else if err != nil { - return fmt.Errorf("failed to find checkpoint prior to block %v: %w", headBlockNum, err) - } else { - // Scan forward from the checkpoint to find the first entry about a block after headBlockNum - i, err := db.newIterator(idx) - if err != nil { - return fmt.Errorf("failed to create iterator when searching for rewind point: %w", err) - } - // If we don't find any useful logs after the checkpoint, we should delete the checkpoint itself - // So move our delete marker back to include it as a starting point - idx-- - for { - blockNum, _, _, err := i.NextLog() - if errors.Is(err, io.EOF) { - // Reached end of file, we need to keep everything - return nil - } else if err != nil { - return fmt.Errorf("failed to find rewind point: %w", err) - } - if blockNum > headBlockNum { - // Found the first entry we don't need, so stop searching and delete everything after idx - break - } - // Otherwise we need all of the entries the iterator just read - idx = i.nextEntryIdx - 1 - } + // Even if the last fully-processed block matches headBlockNum, + // we might still have trailing log events to get rid of. + iter, err := db.newIteratorAt(newHeadBlockNum, 0) + if err != nil { + return err } - // Truncate to contain idx+1 entries, since indices are 0 based, this deletes everything after idx - if err := db.store.Truncate(idx); err != nil { - return fmt.Errorf("failed to truncate to block %v: %w", headBlockNum, err) + // Truncate to contain idx+1 entries, since indices are 0 based, + // this deletes everything after idx + if err := db.store.Truncate(iter.NextIndex()); err != nil { + return fmt.Errorf("failed to truncate to block %v: %w", newHeadBlockNum, err) } // Use db.init() to find the log context for the new latest log entry - if err := db.init(); err != nil { + if err := db.init(true); err != nil { return fmt.Errorf("failed to find new last entry context: %w", err) } return nil } -// NextExecutingMessage returns the next executing message in the log database. -// it skips over any non-executing messages, and will return an error if encountered. -// the iterator is modified in the process. -func (db *DB) NextExecutingMessage(iter Iterator) (types.ExecutingMessage, error) { - db.rwLock.RLock() - defer db.rwLock.RUnlock() - // this for-loop will break: - // - when the iterator reaches the end of the log - // - when the iterator reaches an executing message - // - if an error occurs - for { - _, _, _, err := iter.NextLog() - if err != nil { - return types.ExecutingMessage{}, err - } - // if the log is not an executing message, both exec and err are empty - exec, err := iter.ExecMessage() - if err != nil { - return types.ExecutingMessage{}, fmt.Errorf("failed to get executing message: %w", err) - } - if exec != (types.ExecutingMessage{}) { - return exec, nil - } - } -} - -// LastCheckpointBehind returns an iterator for the last checkpoint behind the specified entry index. -// If the entry index is a search checkpoint, the iterator will start at that checkpoint. -// After searching back long enough (the searchCheckpointFrequency), an error is returned, -// as checkpoints are expected to be found within the frequency. -func (db *DB) LastCheckpointBehind(entryIdx entrydb.EntryIdx) (Iterator, error) { - for attempts := 0; attempts < searchCheckpointFrequency; attempts++ { - // attempt to read the index entry as a search checkpoint - _, err := db.readSearchCheckpoint(entryIdx) - if err == nil { - return db.newIterator(entryIdx) - } - // ErrDataCorruption is the return value if the entry is not a search checkpoint - // if it's not that type of error, we should return it instead of continuing - if !errors.Is(err, ErrDataCorruption) { - return nil, err - } - // don't attempt to read behind the start of the data - if entryIdx == 0 { - break - } - // reverse if we haven't found it yet - entryIdx -= 1 - } - return nil, fmt.Errorf("failed to find a search checkpoint in the last %v entries", searchCheckpointFrequency) -} - func (db *DB) readSearchCheckpoint(entryIdx entrydb.EntryIdx) (searchCheckpoint, error) { data, err := db.store.Read(entryIdx) if err != nil { diff --git a/op-supervisor/supervisor/backend/db/logs/db_invariants_test.go b/op-supervisor/supervisor/backend/db/logs/db_invariants_test.go index e0837e8c57bf..04c004f3d096 100644 --- a/op-supervisor/supervisor/backend/db/logs/db_invariants_test.go +++ b/op-supervisor/supervisor/backend/db/logs/db_invariants_test.go @@ -38,16 +38,13 @@ func checkDBInvariants(t *testing.T, dbPath string, m *stubMetrics) { } entryInvariants := []entryInvariant{ - invariantSearchCheckpointOnlyAtFrequency, invariantSearchCheckpointAtEverySearchCheckpointFrequency, - invariantCanonicalHashAfterEverySearchCheckpoint, + invariantCanonicalHashOrCheckpointAfterEverySearchCheckpoint, invariantSearchCheckpointBeforeEveryCanonicalHash, - invariantIncrementLogIdxIfNotImmediatelyAfterCanonicalHash, invariantExecLinkAfterInitEventWithFlagSet, invariantExecLinkOnlyAfterInitiatingEventWithFlagSet, invariantExecCheckAfterExecLink, invariantExecCheckOnlyAfterExecLink, - invariantValidLastEntry, } for i, entry := range entries { for _, invariant := range entryInvariants { @@ -83,81 +80,47 @@ func invariantFileSizeMatchesEntryCountMetric(stat os.FileInfo, m *stubMetrics) return nil } -func invariantSearchCheckpointOnlyAtFrequency(entryIdx int, entry entrydb.Entry, entries []entrydb.Entry, m *stubMetrics) error { - if entry[0] != typeSearchCheckpoint { - return nil - } - if entryIdx%searchCheckpointFrequency != 0 { - return fmt.Errorf("should only have search checkpoints every %v entries but found at entry %v", searchCheckpointFrequency, entryIdx) - } - return nil -} - func invariantSearchCheckpointAtEverySearchCheckpointFrequency(entryIdx int, entry entrydb.Entry, entries []entrydb.Entry, m *stubMetrics) error { - if entryIdx%searchCheckpointFrequency == 0 && entry[0] != typeSearchCheckpoint { + if entryIdx%searchCheckpointFrequency == 0 && entry.Type() != entrydb.TypeSearchCheckpoint { return fmt.Errorf("should have search checkpoints every %v entries but entry %v was %x", searchCheckpointFrequency, entryIdx, entry) } return nil } -func invariantCanonicalHashAfterEverySearchCheckpoint(entryIdx int, entry entrydb.Entry, entries []entrydb.Entry, m *stubMetrics) error { - if entry[0] != typeSearchCheckpoint { +func invariantCanonicalHashOrCheckpointAfterEverySearchCheckpoint(entryIdx int, entry entrydb.Entry, entries []entrydb.Entry, m *stubMetrics) error { + if entry.Type() != entrydb.TypeSearchCheckpoint { return nil } if entryIdx+1 >= len(entries) { - return fmt.Errorf("expected canonical hash after search checkpoint at entry %v but no further entries found", entryIdx) + return fmt.Errorf("expected canonical hash or checkpoint after search checkpoint at entry %v but no further entries found", entryIdx) } nextEntry := entries[entryIdx+1] - if nextEntry[0] != typeCanonicalHash { - return fmt.Errorf("expected canonical hash after search checkpoint at entry %v but got %x", entryIdx, nextEntry) + if nextEntry.Type() != entrydb.TypeCanonicalHash && nextEntry.Type() != entrydb.TypeSearchCheckpoint { + return fmt.Errorf("expected canonical hash or checkpoint after search checkpoint at entry %v but got %x", entryIdx, nextEntry) } return nil } // invariantSearchCheckpointBeforeEveryCanonicalHash ensures we don't have extra canonical-hash entries func invariantSearchCheckpointBeforeEveryCanonicalHash(entryIdx int, entry entrydb.Entry, entries []entrydb.Entry, m *stubMetrics) error { - if entry[0] != typeCanonicalHash { + if entry.Type() != entrydb.TypeCanonicalHash { return nil } if entryIdx == 0 { return fmt.Errorf("expected search checkpoint before canonical hash at entry %v but no previous entries present", entryIdx) } prevEntry := entries[entryIdx-1] - if prevEntry[0] != typeSearchCheckpoint { + if prevEntry.Type() != entrydb.TypeSearchCheckpoint { return fmt.Errorf("expected search checkpoint before canonical hash at entry %v but got %x", entryIdx, prevEntry) } return nil } -func invariantIncrementLogIdxIfNotImmediatelyAfterCanonicalHash(entryIdx int, entry entrydb.Entry, entries []entrydb.Entry, m *stubMetrics) error { - if entry[0] != typeInitiatingEvent { - return nil - } - if entryIdx == 0 { - return fmt.Errorf("found initiating event at index %v before any search checkpoint", entryIdx) - } - blockDiff := entry[1] - flags := entry[2] - incrementsLogIdx := flags&eventFlagIncrementLogIdx != 0 - prevEntry := entries[entryIdx-1] - prevEntryIsCanonicalHash := prevEntry[0] == typeCanonicalHash - if incrementsLogIdx && prevEntryIsCanonicalHash { - return fmt.Errorf("initiating event at index %v increments logIdx despite being immediately after canonical hash (prev entry %x)", entryIdx, prevEntry) - } - if incrementsLogIdx && blockDiff > 0 { - return fmt.Errorf("initiating event at index %v increments logIdx despite starting a new block", entryIdx) - } - if !incrementsLogIdx && !prevEntryIsCanonicalHash && blockDiff == 0 { - return fmt.Errorf("initiating event at index %v does not increment logIdx when block unchanged and not after canonical hash (prev entry %x)", entryIdx, prevEntry) - } - return nil -} - func invariantExecLinkAfterInitEventWithFlagSet(entryIdx int, entry entrydb.Entry, entries []entrydb.Entry, m *stubMetrics) error { - if entry[0] != typeInitiatingEvent { + if entry.Type() != entrydb.TypeInitiatingEvent { return nil } - hasExecMessage := entry[2]&eventFlagHasExecutingMessage != 0 + hasExecMessage := entry[1]&eventFlagHasExecutingMessage != 0 if !hasExecMessage { return nil } @@ -168,14 +131,14 @@ func invariantExecLinkAfterInitEventWithFlagSet(entryIdx int, entry entrydb.Entr if len(entries) <= linkIdx { return fmt.Errorf("expected executing link after initiating event with exec msg flag set at entry %v but there were no more events", entryIdx) } - if entries[linkIdx][0] != typeExecutingLink { + if entries[linkIdx].Type() != entrydb.TypeExecutingLink { return fmt.Errorf("expected executing link at idx %v after initiating event with exec msg flag set at entry %v but got type %v", linkIdx, entryIdx, entries[linkIdx][0]) } return nil } func invariantExecLinkOnlyAfterInitiatingEventWithFlagSet(entryIdx int, entry entrydb.Entry, entries []entrydb.Entry, m *stubMetrics) error { - if entry[0] != typeExecutingLink { + if entry.Type() != entrydb.TypeExecutingLink { return nil } if entryIdx == 0 { @@ -189,10 +152,10 @@ func invariantExecLinkOnlyAfterInitiatingEventWithFlagSet(entryIdx int, entry en return fmt.Errorf("found executing link without a preceding initiating event at entry %v", entryIdx) } initEntry := entries[initIdx] - if initEntry[0] != typeInitiatingEvent { + if initEntry.Type() != entrydb.TypeInitiatingEvent { return fmt.Errorf("expected initiating event at entry %v prior to executing link at %v but got %x", initIdx, entryIdx, initEntry[0]) } - flags := initEntry[2] + flags := initEntry[1] if flags&eventFlagHasExecutingMessage == 0 { return fmt.Errorf("initiating event at %v prior to executing link at %v does not have flag set to indicate needing a executing event: %x", initIdx, entryIdx, initEntry) } @@ -200,7 +163,7 @@ func invariantExecLinkOnlyAfterInitiatingEventWithFlagSet(entryIdx int, entry en } func invariantExecCheckAfterExecLink(entryIdx int, entry entrydb.Entry, entries []entrydb.Entry, m *stubMetrics) error { - if entry[0] != typeExecutingLink { + if entry.Type() != entrydb.TypeExecutingLink { return nil } checkIdx := entryIdx + 1 @@ -211,14 +174,14 @@ func invariantExecCheckAfterExecLink(entryIdx int, entry entrydb.Entry, entries return fmt.Errorf("expected executing link at %v to be followed by executing check at %v but ran out of entries", entryIdx, checkIdx) } checkEntry := entries[checkIdx] - if checkEntry[0] != typeExecutingCheck { + if checkEntry.Type() != entrydb.TypeExecutingCheck { return fmt.Errorf("expected executing link at %v to be followed by executing check at %v but got type %v", entryIdx, checkIdx, checkEntry[0]) } return nil } func invariantExecCheckOnlyAfterExecLink(entryIdx int, entry entrydb.Entry, entries []entrydb.Entry, m *stubMetrics) error { - if entry[0] != typeExecutingCheck { + if entry.Type() != entrydb.TypeExecutingCheck { return nil } if entryIdx == 0 { @@ -232,29 +195,8 @@ func invariantExecCheckOnlyAfterExecLink(entryIdx int, entry entrydb.Entry, entr return fmt.Errorf("found executing link without a preceding initiating event at entry %v", entryIdx) } linkEntry := entries[linkIdx] - if linkEntry[0] != typeExecutingLink { + if linkEntry.Type() != entrydb.TypeExecutingLink { return fmt.Errorf("expected executing link at entry %v prior to executing check at %v but got %x", linkIdx, entryIdx, linkEntry[0]) } return nil } - -// invariantValidLastEntry checks that the last entry is either a executing check or initiating event with no exec message -func invariantValidLastEntry(entryIdx int, entry entrydb.Entry, entries []entrydb.Entry, m *stubMetrics) error { - if entryIdx+1 < len(entries) { - return nil - } - if entry[0] == typeExecutingCheck { - return nil - } - if entry[0] != typeInitiatingEvent { - return fmt.Errorf("invalid final event type: %v", entry[0]) - } - evt, err := newInitiatingEventFromEntry(entry) - if err != nil { - return fmt.Errorf("final event was invalid: %w", err) - } - if evt.hasExecMsg { - return errors.New("ends with init event that should have exec msg but no exec msg follows") - } - return nil -} diff --git a/op-supervisor/supervisor/backend/db/logs/db_test.go b/op-supervisor/supervisor/backend/db/logs/db_test.go index 91caa2fe3e5d..6bde3ef2c04a 100644 --- a/op-supervisor/supervisor/backend/db/logs/db_test.go +++ b/op-supervisor/supervisor/backend/db/logs/db_test.go @@ -1,21 +1,23 @@ package logs import ( - "bytes" - "fmt" + "encoding/binary" "io" "io/fs" "os" "path/filepath" "testing" + "github.com/stretchr/testify/require" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/testlog" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/entrydb" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/types" - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/log" - "github.com/stretchr/testify/require" ) func createTruncatedHash(i int) types.TruncatedHash { @@ -23,22 +25,27 @@ func createTruncatedHash(i int) types.TruncatedHash { } func createHash(i int) common.Hash { - data := bytes.Repeat([]byte{byte(i)}, common.HashLength) - return common.BytesToHash(data) + if i == -1 { // parent-hash of genesis is zero + return common.Hash{} + } + var data [9]byte + data[0] = 0xff + binary.BigEndian.PutUint64(data[1:], uint64(i)) + return crypto.Keccak256Hash(data[:]) } func TestErrorOpeningDatabase(t *testing.T) { dir := t.TempDir() - _, err := NewFromFile(testlog.Logger(t, log.LvlInfo), &stubMetrics{}, filepath.Join(dir, "missing-dir", "file.db")) + _, err := NewFromFile(testlog.Logger(t, log.LvlInfo), &stubMetrics{}, filepath.Join(dir, "missing-dir", "file.db"), false) require.ErrorIs(t, err, os.ErrNotExist) } func runDBTest(t *testing.T, setup func(t *testing.T, db *DB, m *stubMetrics), assert func(t *testing.T, db *DB, m *stubMetrics)) { createDb := func(t *testing.T, dir string) (*DB, *stubMetrics, string) { - logger := testlog.Logger(t, log.LvlInfo) + logger := testlog.Logger(t, log.LvlTrace) path := filepath.Join(dir, "test.db") m := &stubMetrics{} - db, err := NewFromFile(logger, m, path) + db, err := NewFromFile(logger, m, path, false) require.NoError(t, err, "Failed to create database") t.Cleanup(func() { err := db.Close() @@ -73,8 +80,8 @@ func TestEmptyDbDoesNotFindEntry(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) {}, func(t *testing.T, db *DB, m *stubMetrics) { - requireNotContains(t, db, 0, 0, createHash(1)) - requireNotContains(t, db, 0, 0, common.Hash{}) + requireFuture(t, db, 1, 0, createHash(1)) + requireFuture(t, db, 1, 0, common.Hash{}) }) } @@ -84,111 +91,145 @@ func TestAddLog(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) {}, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 0}, 5000, 0, nil) + genesis := eth.BlockID{Hash: createHash(15), Number: 0} + err := db.AddLog(createTruncatedHash(1), genesis, 0, nil) require.ErrorIs(t, err, ErrLogOutOfOrder) }) }) - t.Run("FirstEntry", func(t *testing.T) { + t.Run("FirstEntries", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 0, nil) - require.NoError(t, err) + genesis := eth.BlockID{Hash: createHash(15), Number: 15} + require.NoError(t, db.SealBlock(common.Hash{}, genesis, 5000), "seal genesis") + err := db.AddLog(createTruncatedHash(1), genesis, 0, nil) + require.NoError(t, err, "first log after genesis") + require.NoError(t, db.SealBlock(genesis.Hash, eth.BlockID{Hash: createHash(16), Number: 16}, 5001)) }, func(t *testing.T, db *DB, m *stubMetrics) { - requireContains(t, db, 15, 0, createHash(1)) + requireContains(t, db, 16, 0, createHash(1)) }) }) t.Run("MultipleEntriesFromSameBlock", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 0, nil) + // create 15 empty blocks + for i := 0; i <= 15; i++ { + bl := eth.BlockID{Hash: createHash(i), Number: uint64(i)} + require.NoError(t, db.SealBlock(createHash(i-1), bl, 5000+uint64(i)), "seal blocks") + } + // Now apply 3 logs on top of that, contents for block 16 + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.AddLog(createTruncatedHash(1), bl15, 0, nil) + require.NoError(t, err) + err = db.AddLog(createTruncatedHash(2), bl15, 1, nil) require.NoError(t, err) - err = db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 1, nil) + err = db.AddLog(createTruncatedHash(3), bl15, 2, nil) require.NoError(t, err) - err = db.AddLog(createTruncatedHash(3), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 2, nil) + // Now seal block 16 + bl16 := eth.BlockID{Hash: createHash(16), Number: 16} + err = db.SealBlock(bl15.Hash, bl16, 5016) require.NoError(t, err) }, func(t *testing.T, db *DB, m *stubMetrics) { - require.EqualValues(t, 5, m.entryCount, "should not output new searchCheckpoint for every log") - requireContains(t, db, 15, 0, createHash(1)) - requireContains(t, db, 15, 1, createHash(2)) - requireContains(t, db, 15, 2, createHash(3)) + require.EqualValues(t, 16*2+3+2, m.entryCount, "empty blocks have logs") + requireContains(t, db, 16, 0, createHash(1)) + requireContains(t, db, 16, 1, createHash(2)) + requireContains(t, db, 16, 2, createHash(3)) }) }) t.Run("MultipleEntriesFromMultipleBlocks", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 0, nil) + bl14 := eth.BlockID{Hash: createHash(14), Number: 14} + err := db.SealBlock(createHash(13), bl14, 5000) require.NoError(t, err) - err = db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 1, nil) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err = db.SealBlock(createHash(14), bl15, 5001) require.NoError(t, err) - err = db.AddLog(createTruncatedHash(3), eth.BlockID{Hash: createHash(16), Number: 16}, 5002, 0, nil) + err = db.AddLog(createTruncatedHash(1), bl15, 0, nil) require.NoError(t, err) - err = db.AddLog(createTruncatedHash(4), eth.BlockID{Hash: createHash(16), Number: 16}, 5002, 1, nil) + err = db.AddLog(createTruncatedHash(2), bl15, 1, nil) require.NoError(t, err) + bl16 := eth.BlockID{Hash: createHash(16), Number: 16} + err = db.SealBlock(bl15.Hash, bl16, 5003) + require.NoError(t, err) + err = db.AddLog(createTruncatedHash(3), bl16, 0, nil) + require.NoError(t, err) + err = db.AddLog(createTruncatedHash(4), bl16, 1, nil) + require.NoError(t, err) + bl17 := eth.BlockID{Hash: createHash(17), Number: 17} + err = db.SealBlock(bl16.Hash, bl17, 5003) }, func(t *testing.T, db *DB, m *stubMetrics) { - require.EqualValues(t, 6, m.entryCount, "should not output new searchCheckpoint for every block") - requireContains(t, db, 15, 0, createHash(1)) - requireContains(t, db, 15, 1, createHash(2)) - requireContains(t, db, 16, 0, createHash(3)) - requireContains(t, db, 16, 1, createHash(4)) + require.EqualValues(t, 2+2+1+1+2+1+1+2, m.entryCount, "should not output new searchCheckpoint for every block") + requireContains(t, db, 16, 0, createHash(1)) + requireContains(t, db, 16, 1, createHash(2)) + requireContains(t, db, 17, 0, createHash(3)) + requireContains(t, db, 17, 1, createHash(4)) }) }) t.Run("ErrorWhenBeforeCurrentBlock", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 0, nil) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.SealBlock(common.Hash{}, bl15, 5001) require.NoError(t, err) }, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(14), Number: 14}, 4998, 0, nil) - require.ErrorIs(t, err, ErrLogOutOfOrder) + bl14 := eth.BlockID{Hash: createHash(14), Number: 14} + err := db.SealBlock(createHash(13), bl14, 5000) + require.ErrorIs(t, err, ErrConflict) }) }) t.Run("ErrorWhenBeforeCurrentBlockButAfterLastCheckpoint", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(13), Number: 13}, 5000, 0, nil) + err := db.lastEntryContext.forceBlock(eth.BlockID{Hash: createHash(13), Number: 13}, 5000) + require.NoError(t, err) + err = db.SealBlock(createHash(13), eth.BlockID{Hash: createHash(14), Number: 14}, 5001) require.NoError(t, err) - err = db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 0, nil) + err = db.SealBlock(createHash(14), eth.BlockID{Hash: createHash(15), Number: 15}, 5002) require.NoError(t, err) }, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(14), Number: 14}, 4998, 0, nil) - require.ErrorIs(t, err, ErrLogOutOfOrder) + onto := eth.BlockID{Hash: createHash(14), Number: 14} + err := db.AddLog(createTruncatedHash(1), onto, 0, nil) + require.ErrorIs(t, err, ErrLogOutOfOrder, "cannot build logs on 14 when 15 is already sealed") }) }) t.Run("ErrorWhenBeforeCurrentLogEvent", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 1, nil)) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.lastEntryContext.forceBlock(bl15, 5000) + require.NoError(t, err) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl15, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl15, 1, nil)) }, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(14), Number: 15}, 4998, 0, nil) - require.ErrorIs(t, err, ErrLogOutOfOrder) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.AddLog(createTruncatedHash(1), bl15, 0, nil) + require.ErrorIs(t, err, ErrLogOutOfOrder, "already at log index 2") }) }) - t.Run("ErrorWhenBeforeCurrentLogEventButAfterLastCheckpoint", func(t *testing.T) { + t.Run("ErrorWhenBeforeBlockSeal", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 0, nil) - require.NoError(t, err) - err = db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 1, nil) - require.NoError(t, err) - err = db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 2, nil) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.lastEntryContext.forceBlock(bl15, 5000) require.NoError(t, err) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl15, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl15, 1, nil)) }, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(14), Number: 15}, 4998, 1, nil) + err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(16), Number: 16}, 0, nil) require.ErrorIs(t, err, ErrLogOutOfOrder) }) }) @@ -196,24 +237,31 @@ func TestAddLog(t *testing.T) { t.Run("ErrorWhenAtCurrentLogEvent", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 1, nil)) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.lastEntryContext.forceBlock(bl15, 5000) + require.NoError(t, err) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl15, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl15, 1, nil)) }, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 4998, 1, nil) - require.ErrorIs(t, err, ErrLogOutOfOrder) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.AddLog(createTruncatedHash(1), bl15, 1, nil) + require.ErrorIs(t, err, ErrLogOutOfOrder, "already at log index 2") }) }) t.Run("ErrorWhenAtCurrentLogEventButAfterLastCheckpoint", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 1, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 2, nil)) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.lastEntryContext.forceBlock(bl15, 5000) + require.NoError(t, err) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl15, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl15, 1, nil)) }, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(14), Number: 15}, 4998, 2, nil) + bl15 := eth.BlockID{Hash: createHash(16), Number: 16} + err := db.AddLog(createTruncatedHash(1), bl15, 2, nil) require.ErrorIs(t, err, ErrLogOutOfOrder) }) }) @@ -221,19 +269,27 @@ func TestAddLog(t *testing.T) { t.Run("ErrorWhenSkippingLogEvent", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 0, nil) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.lastEntryContext.forceBlock(bl15, 5000) require.NoError(t, err) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl15, 0, nil)) }, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 4998, 2, nil) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.AddLog(createTruncatedHash(1), bl15, 2, nil) require.ErrorIs(t, err, ErrLogOutOfOrder) }) }) t.Run("ErrorWhenFirstLogIsNotLogIdxZero", func(t *testing.T) { - runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) {}, + runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.lastEntryContext.forceBlock(bl15, 5000) + require.NoError(t, err) + }, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 4998, 5, nil) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.AddLog(createTruncatedHash(1), bl15, 5, nil) require.ErrorIs(t, err, ErrLogOutOfOrder) }) }) @@ -241,54 +297,102 @@ func TestAddLog(t *testing.T) { t.Run("ErrorWhenFirstLogOfNewBlockIsNotLogIdxZero", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(14), Number: 14}, 4996, 0, nil)) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.lastEntryContext.forceBlock(bl15, 5000) + require.NoError(t, err) + err = db.AddLog(createTruncatedHash(1), bl15, 0, nil) + require.NoError(t, err) }, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 4998, 1, nil) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + err := db.AddLog(createTruncatedHash(1), bl15, 1, nil) + require.NoError(t, err) + bl16 := eth.BlockID{Hash: createHash(16), Number: 16} + err = db.SealBlock(bl15.Hash, bl16, 5001) + require.NoError(t, err) + err = db.AddLog(createTruncatedHash(1), bl16, 1, nil) require.ErrorIs(t, err, ErrLogOutOfOrder) }) }) t.Run("MultipleSearchCheckpoints", func(t *testing.T) { + block0 := eth.BlockID{Hash: createHash(10), Number: 10} block1 := eth.BlockID{Hash: createHash(11), Number: 11} block2 := eth.BlockID{Hash: createHash(12), Number: 12} - block3 := eth.BlockID{Hash: createHash(15), Number: 15} - block4 := eth.BlockID{Hash: createHash(16), Number: 16} - // First checkpoint is at entry idx 0 - // Block 1 logs don't reach the second checkpoint + block3 := eth.BlockID{Hash: createHash(13), Number: 13} + block4 := eth.BlockID{Hash: createHash(14), Number: 14} + // Ignoring seal-checkpoints in checkpoint counting comments here; + // First search-checkpoint is at entry idx 0 + // Block 1 logs don't reach the second search-checkpoint block1LogCount := searchCheckpointFrequency - 10 - // Block 2 logs extend to just after the third checkpoint - block2LogCount := searchCheckpointFrequency + 20 - // Block 3 logs extend to immediately before the fourth checkpoint - block3LogCount := searchCheckpointFrequency - 16 + // Block 2 logs extend to just after the third search-checkpoint + block2LogCount := searchCheckpointFrequency + 16 + // Block 3 logs extend to immediately before the fourth search-checkpoint + block3LogCount := searchCheckpointFrequency - 19 block4LogCount := 2 runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - for i := 0; i < block1LogCount; i++ { - err := db.AddLog(createTruncatedHash(i), block1, 3000, uint32(i), nil) - require.NoErrorf(t, err, "failed to add log %v of block 1", i) + // force in block 0 + require.NoError(t, db.lastEntryContext.forceBlock(block0, 3000)) + expectedIndex := entrydb.EntryIdx(2) + t.Logf("block 0 complete, at entry %d", db.lastEntryContext.NextIndex()) + require.Equal(t, expectedIndex, db.lastEntryContext.NextIndex()) + { // create block 1 + for i := 0; i < block1LogCount; i++ { + err := db.AddLog(createTruncatedHash(i), block0, uint32(i), nil) + require.NoError(t, err) + } + err := db.SealBlock(block0.Hash, block1, 3001) // second seal-checkpoint + require.NoError(t, err) } - for i := 0; i < block2LogCount; i++ { - err := db.AddLog(createTruncatedHash(i), block2, 3002, uint32(i), nil) - require.NoErrorf(t, err, "failed to add log %v of block 2", i) + expectedIndex += entrydb.EntryIdx(block1LogCount) + 2 + t.Logf("block 1 complete, at entry %d", db.lastEntryContext.NextIndex()) + require.Equal(t, expectedIndex, db.lastEntryContext.NextIndex(), "added logs and a seal checkpoint") + { // create block 2 + for i := 0; i < block2LogCount; i++ { + // two of these imply a search checkpoint, the second and third search-checkpoint + err := db.AddLog(createTruncatedHash(i), block1, uint32(i), nil) + require.NoError(t, err) + } + err := db.SealBlock(block1.Hash, block2, 3002) // third seal-checkpoint + require.NoError(t, err) } - for i := 0; i < block3LogCount; i++ { - err := db.AddLog(createTruncatedHash(i), block3, 3004, uint32(i), nil) - require.NoErrorf(t, err, "failed to add log %v of block 3", i) + expectedIndex += entrydb.EntryIdx(block2LogCount) + 2 + 2 + 2 + t.Logf("block 2 complete, at entry %d", db.lastEntryContext.NextIndex()) + require.Equal(t, expectedIndex, db.lastEntryContext.NextIndex(), "added logs, two search checkpoints, and a seal checkpoint") + { // create block 3 + for i := 0; i < block3LogCount; i++ { + err := db.AddLog(createTruncatedHash(i), block2, uint32(i), nil) + require.NoError(t, err) + } + err := db.SealBlock(block2.Hash, block3, 3003) + require.NoError(t, err) } - // Verify that we're right before the fourth checkpoint will be written. + expectedIndex += entrydb.EntryIdx(block3LogCount) + 2 + t.Logf("block 3 complete, at entry %d", db.lastEntryContext.NextIndex()) + require.Equal(t, expectedIndex, db.lastEntryContext.NextIndex(), "added logs, and a seal checkpoint") + + // Verify that we're right before the fourth search-checkpoint will be written. // entryCount is the number of entries, so given 0 based indexing is the index of the next entry // the first checkpoint is at entry 0, the second at entry searchCheckpointFrequency etc - // so the fourth is at entry 3*searchCheckpointFrequency - require.EqualValues(t, 3*searchCheckpointFrequency, m.entryCount) - for i := 0; i < block4LogCount; i++ { - err := db.AddLog(createTruncatedHash(i), block4, 3006, uint32(i), nil) - require.NoErrorf(t, err, "failed to add log %v of block 4", i) + // so the fourth is at entry 3*searchCheckpointFrequency. + require.EqualValues(t, 3*searchCheckpointFrequency-1, m.entryCount) + { // create block 4 + for i := 0; i < block4LogCount; i++ { + // includes a fourth search checkpoint + err := db.AddLog(createTruncatedHash(i), block3, uint32(i), nil) + require.NoError(t, err) + } + err := db.SealBlock(block3.Hash, block4, 3003) // fourth seal checkpoint + require.NoError(t, err) } + expectedIndex += entrydb.EntryIdx(block4LogCount) + 2 + 2 + require.Equal(t, expectedIndex, db.lastEntryContext.NextIndex(), "added logs, a search checkpoint, and a seal checkpoint") + t.Logf("block 4 complete, at entry %d", db.lastEntryContext.NextIndex()) }, func(t *testing.T, db *DB, m *stubMetrics) { - // Check that we wrote additional search checkpoints - expectedCheckpointCount := 4 + // Check that we wrote additional search checkpoints and seal checkpoints + expectedCheckpointCount := 4 + 4 expectedEntryCount := block1LogCount + block2LogCount + block3LogCount + block4LogCount + (2 * expectedCheckpointCount) require.EqualValues(t, expectedEntryCount, m.entryCount) // Check we can find all the logs. @@ -322,68 +426,93 @@ func TestAddDependentLog(t *testing.T) { t.Run("FirstEntry", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 0, &execMsg) - require.NoError(t, err) - }, - func(t *testing.T, db *DB, m *stubMetrics) { - requireContains(t, db, 15, 0, createHash(1), execMsg) - }) - }) - - t.Run("CheckpointBetweenInitEventAndExecLink", func(t *testing.T) { - runDBTest(t, - func(t *testing.T, db *DB, m *stubMetrics) { - for i := uint32(0); m.entryCount < searchCheckpointFrequency-1; i++ { - require.NoError(t, db.AddLog(createTruncatedHash(9), eth.BlockID{Hash: createHash(9), Number: 1}, 500, i, nil)) - } - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 0, &execMsg) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + require.NoError(t, db.lastEntryContext.forceBlock(bl15, 5000)) + err := db.AddLog(createTruncatedHash(1), bl15, 0, &execMsg) require.NoError(t, err) }, func(t *testing.T, db *DB, m *stubMetrics) { - requireContains(t, db, 15, 0, createHash(1), execMsg) + requireContains(t, db, 16, 0, createHash(1), execMsg) }) }) - t.Run("CheckpointBetweenInitEventAndExecLinkNotIncrementingBlock", func(t *testing.T) { + t.Run("BlockSealSearchCheckpointOverlap", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + require.NoError(t, db.lastEntryContext.forceBlock(bl15, 5000)) for i := uint32(0); m.entryCount < searchCheckpointFrequency-1; i++ { - require.NoError(t, db.AddLog(createTruncatedHash(9), eth.BlockID{Hash: createHash(9), Number: 1}, 500, i, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(9), bl15, i, nil)) } - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 1}, 5000, 253, &execMsg) + bl16 := eth.BlockID{Hash: createHash(16), Number: 16} + require.NoError(t, db.SealBlock(bl15.Hash, bl16, 5001)) + // added 3 entries: seal-checkpoint, then a search-checkpoint, then the canonical hash + require.Equal(t, m.entryCount, int64(searchCheckpointFrequency+2)) + err := db.AddLog(createTruncatedHash(1), bl16, 0, &execMsg) require.NoError(t, err) }, func(t *testing.T, db *DB, m *stubMetrics) { - requireContains(t, db, 1, 253, createHash(1), execMsg) + requireContains(t, db, 16, 0, createHash(9)) + requireContains(t, db, 17, 0, createHash(1), execMsg) }) }) - t.Run("CheckpointBetweenExecLinkAndExecCheck", func(t *testing.T) { + t.Run("AvoidCheckpointOverlapWithExecutingCheck", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - for i := uint32(0); m.entryCount < searchCheckpointFrequency-2; i++ { - require.NoError(t, db.AddLog(createTruncatedHash(9), eth.BlockID{Hash: createHash(9), Number: 1}, 500, i, nil)) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + require.NoError(t, db.lastEntryContext.forceBlock(bl15, 5000)) + // we add 256 - 2 (start) - 2 (init msg, exec link) = 252 entries + for i := uint32(0); i < 252; i++ { + require.NoError(t, db.AddLog(createTruncatedHash(9), bl15, i, nil)) } - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 15}, 5000, 0, &execMsg) + // add an executing message + err := db.AddLog(createTruncatedHash(1), bl15, 252, &execMsg) require.NoError(t, err) + // 0,1: start + // 2..252+2: initiating logs without exec message + // 254 = inferred padding - 3 entries for exec msg would overlap with checkpoint + // 255 = inferred padding + // 256 = search checkpoint - what would be the exec check without padding + // 257 = canonical hash + // 258 = initiating message + // 259 = executing message link + // 260 = executing message check + require.Equal(t, int64(261), m.entryCount) + db.debugTip() }, func(t *testing.T, db *DB, m *stubMetrics) { - requireContains(t, db, 15, 0, createHash(1), execMsg) + requireContains(t, db, 16, 251, createHash(9)) + requireContains(t, db, 16, 252, createHash(1), execMsg) }) }) - t.Run("CheckpointBetweenExecLinkAndExecCheckNotIncrementingBlock", func(t *testing.T) { + t.Run("AvoidCheckpointOverlapWithExecutingLink", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - for i := uint32(0); m.entryCount < searchCheckpointFrequency-2; i++ { - require.NoError(t, db.AddLog(createTruncatedHash(9), eth.BlockID{Hash: createHash(9), Number: 1}, 500, i, nil)) + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + require.NoError(t, db.lastEntryContext.forceBlock(bl15, 5000)) + // we add 256 - 2 (start) - 1 (init msg) = 253 entries + for i := uint32(0); i < 253; i++ { + require.NoError(t, db.AddLog(createTruncatedHash(9), bl15, i, nil)) } - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(15), Number: 1}, 5000, 252, &execMsg) + // add an executing message + err := db.AddLog(createTruncatedHash(1), bl15, 253, &execMsg) require.NoError(t, err) + // 0,1: start + // 2..253+2: initiating logs without exec message + // 255 = inferred padding - 3 entries for exec msg would overlap with checkpoint + // 256 = search checkpoint - what would be the exec link without padding + // 257 = canonical hash + // 258 = initiating message + // 259 = executing message link + // 260 = executing message check + db.debugTip() + require.Equal(t, int64(261), m.entryCount) }, func(t *testing.T, db *DB, m *stubMetrics) { - requireContains(t, db, 1, 252, createHash(1), execMsg) + requireContains(t, db, 16, 252, createHash(9)) + requireContains(t, db, 16, 253, createHash(1), execMsg) }) }) } @@ -391,35 +520,36 @@ func TestAddDependentLog(t *testing.T) { func TestContains(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(3), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 1, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 2, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(52), Number: 52}, 500, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(3), eth.BlockID{Hash: createHash(52), Number: 52}, 500, 1, nil)) + bl50 := eth.BlockID{Hash: createHash(50), Number: 50} + require.NoError(t, db.lastEntryContext.forceBlock(bl50, 5000)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl50, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(3), bl50, 1, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(2), bl50, 2, nil)) + bl51 := eth.BlockID{Hash: createHash(51), Number: 51} + require.NoError(t, db.SealBlock(bl50.Hash, bl51, 5001)) + bl52 := eth.BlockID{Hash: createHash(52), Number: 52} + require.NoError(t, db.SealBlock(bl51.Hash, bl52, 5001)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl52, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(3), bl52, 1, nil)) }, func(t *testing.T, db *DB, m *stubMetrics) { // Should find added logs - requireContains(t, db, 50, 0, createHash(1)) - requireContains(t, db, 50, 1, createHash(3)) - requireContains(t, db, 50, 2, createHash(2)) - requireContains(t, db, 52, 0, createHash(1)) - requireContains(t, db, 52, 1, createHash(3)) - - // Should not find log when block number too low - requireNotContains(t, db, 49, 0, createHash(1)) - - // Should not find log when block number too high - requireNotContains(t, db, 51, 0, createHash(1)) - - // Should not find log when requested log after end of database - requireNotContains(t, db, 52, 2, createHash(3)) - requireNotContains(t, db, 53, 0, createHash(3)) - - // Should not find log when log index too high - requireNotContains(t, db, 50, 3, createHash(2)) - - // Should not find log when hash doesn't match log at block number and index - requireWrongHash(t, db, 50, 0, createHash(5), types.ExecutingMessage{}) + requireContains(t, db, 51, 0, createHash(1)) + requireContains(t, db, 51, 1, createHash(3)) + requireContains(t, db, 51, 2, createHash(2)) + requireContains(t, db, 53, 0, createHash(1)) + requireContains(t, db, 53, 1, createHash(3)) + + // 52 was sealed as empty + requireConflicts(t, db, 52, 0, createHash(1)) + + // 53 only contained 2 logs, not 3, and is not sealed yet + requireFuture(t, db, 53, 2, createHash(3)) + // 54 doesn't exist yet + requireFuture(t, db, 54, 0, createHash(3)) + + // 51 only contained 3 logs, not 4 + requireConflicts(t, db, 51, 3, createHash(2)) }) } @@ -447,54 +577,63 @@ func TestExecutes(t *testing.T) { } runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(3), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 1, &execMsg1)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 2, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(52), Number: 52}, 500, 0, &execMsg2)) - require.NoError(t, db.AddLog(createTruncatedHash(3), eth.BlockID{Hash: createHash(52), Number: 52}, 500, 1, &execMsg3)) + bl50 := eth.BlockID{Hash: createHash(50), Number: 50} + require.NoError(t, db.lastEntryContext.forceBlock(bl50, 500)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl50, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(3), bl50, 1, &execMsg1)) + require.NoError(t, db.AddLog(createTruncatedHash(2), bl50, 2, nil)) + bl51 := eth.BlockID{Hash: createHash(51), Number: 51} + require.NoError(t, db.SealBlock(bl50.Hash, bl51, 5001)) + bl52 := eth.BlockID{Hash: createHash(52), Number: 52} + require.NoError(t, db.SealBlock(bl51.Hash, bl52, 5001)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl52, 0, &execMsg2)) + require.NoError(t, db.AddLog(createTruncatedHash(3), bl52, 1, &execMsg3)) }, func(t *testing.T, db *DB, m *stubMetrics) { // Should find added logs - requireExecutingMessage(t, db, 50, 0, types.ExecutingMessage{}) - requireExecutingMessage(t, db, 50, 1, execMsg1) - requireExecutingMessage(t, db, 50, 2, types.ExecutingMessage{}) - requireExecutingMessage(t, db, 52, 0, execMsg2) - requireExecutingMessage(t, db, 52, 1, execMsg3) - - // Should not find log when block number too low - requireNotContains(t, db, 49, 0, createHash(1)) - - // Should not find log when block number too high - requireNotContains(t, db, 51, 0, createHash(1)) - - // Should not find log when requested log after end of database - requireNotContains(t, db, 52, 2, createHash(3)) - requireNotContains(t, db, 53, 0, createHash(3)) - - // Should not find log when log index too high - requireNotContains(t, db, 50, 3, createHash(2)) + requireExecutingMessage(t, db, 51, 0, types.ExecutingMessage{}) + requireExecutingMessage(t, db, 51, 1, execMsg1) + requireExecutingMessage(t, db, 51, 2, types.ExecutingMessage{}) + requireExecutingMessage(t, db, 53, 0, execMsg2) + requireExecutingMessage(t, db, 53, 1, execMsg3) + + // 52 was sealed without logs + requireConflicts(t, db, 52, 0, createHash(1)) + + // 53 only contained 2 logs, not 3, and is not sealed yet + requireFuture(t, db, 53, 2, createHash(3)) + // 54 doesn't exist yet + requireFuture(t, db, 54, 0, createHash(3)) + + // 51 only contained 3 logs, not 4 + requireConflicts(t, db, 51, 3, createHash(2)) }) } func TestGetBlockInfo(t *testing.T) { - t.Run("ReturnsEOFWhenEmpty", func(t *testing.T) { + t.Run("ReturnsErrFutureWhenEmpty", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) {}, func(t *testing.T, db *DB, m *stubMetrics) { - _, _, err := db.ClosestBlockInfo(10) - require.ErrorIs(t, err, io.EOF) + bl10 := eth.BlockID{Hash: createHash(10), Number: 10} + _, err := db.FindSealedBlock(bl10) + require.ErrorIs(t, err, ErrFuture) }) }) - t.Run("ReturnsEOFWhenRequestedBlockBeforeFirstSearchCheckpoint", func(t *testing.T) { + t.Run("ReturnsErrFutureWhenRequestedBlockBeforeFirstSearchCheckpoint", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(11), Number: 11}, 500, 0, nil) + bl11 := eth.BlockID{Hash: createHash(11), Number: 11} + require.NoError(t, db.lastEntryContext.forceBlock(bl11, 500)) + err := db.AddLog(createTruncatedHash(1), bl11, 0, nil) require.NoError(t, err) }, func(t *testing.T, db *DB, m *stubMetrics) { - _, _, err := db.ClosestBlockInfo(10) - require.ErrorIs(t, err, io.EOF) + // if the DB starts at 11, then shouldn't find 10 + bl10 := eth.BlockID{Hash: createHash(10), Number: 10} + _, err := db.FindSealedBlock(bl10) + require.ErrorIs(t, err, ErrSkipped) }) }) @@ -502,56 +641,24 @@ func TestGetBlockInfo(t *testing.T) { block := eth.BlockID{Hash: createHash(11), Number: 11} runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(1), block, 500, 0, nil) - require.NoError(t, err) - }, - func(t *testing.T, db *DB, m *stubMetrics) { - requireClosestBlockInfo(t, db, 11, block.Number, block.Hash) - requireClosestBlockInfo(t, db, 12, block.Number, block.Hash) - requireClosestBlockInfo(t, db, 200, block.Number, block.Hash) - }) - }) - - t.Run("ReturnClosestCheckpointBlockInfo", func(t *testing.T) { - runDBTest(t, - func(t *testing.T, db *DB, m *stubMetrics) { - for i := 1; i < searchCheckpointFrequency+3; i++ { - block := eth.BlockID{Hash: createHash(i), Number: uint64(i)} - err := db.AddLog(createTruncatedHash(i), block, uint64(i)*2, 0, nil) - require.NoError(t, err) - } + require.NoError(t, db.SealBlock(common.Hash{}, block, 500)) }, func(t *testing.T, db *DB, m *stubMetrics) { - // Expect block from the first checkpoint - requireClosestBlockInfo(t, db, 1, 1, createHash(1)) - requireClosestBlockInfo(t, db, 10, 1, createHash(1)) - requireClosestBlockInfo(t, db, searchCheckpointFrequency-3, 1, createHash(1)) - - // Expect block from the second checkpoint - // 2 entries used for initial checkpoint but we start at block 1 - secondCheckpointBlockNum := searchCheckpointFrequency - 1 - requireClosestBlockInfo(t, db, uint64(secondCheckpointBlockNum), uint64(secondCheckpointBlockNum), createHash(secondCheckpointBlockNum)) - requireClosestBlockInfo(t, db, uint64(secondCheckpointBlockNum)+1, uint64(secondCheckpointBlockNum), createHash(secondCheckpointBlockNum)) - requireClosestBlockInfo(t, db, uint64(secondCheckpointBlockNum)+2, uint64(secondCheckpointBlockNum), createHash(secondCheckpointBlockNum)) + index, err := db.FindSealedBlock(block) + require.NoError(t, err) + require.Equal(t, entrydb.EntryIdx(2), index, + "expecting to continue after search checkpoint that declared the block") }) }) } -func requireClosestBlockInfo(t *testing.T, db *DB, searchFor uint64, expectedBlockNum uint64, expectedHash common.Hash) { - blockNum, hash, err := db.ClosestBlockInfo(searchFor) - require.NoError(t, err) - require.Equal(t, expectedBlockNum, blockNum) - require.Equal(t, types.TruncateHash(expectedHash), hash) -} - func requireContains(t *testing.T, db *DB, blockNum uint64, logIdx uint32, logHash common.Hash, execMsg ...types.ExecutingMessage) { require.LessOrEqual(t, len(execMsg), 1, "cannot have multiple executing messages for a single log") m, ok := db.m.(*stubMetrics) require.True(t, ok, "Did not get the expected metrics type") - result, _, err := db.Contains(blockNum, logIdx, types.TruncateHash(logHash)) + _, err := db.Contains(blockNum, logIdx, types.TruncateHash(logHash)) require.NoErrorf(t, err, "Error searching for log %v in block %v", logIdx, blockNum) - require.Truef(t, result, "Did not find log %v in block %v with hash %v", logIdx, blockNum, logHash) - require.LessOrEqual(t, m.entriesReadForSearch, int64(searchCheckpointFrequency), "Should not need to read more than between two checkpoints") + require.LessOrEqual(t, m.entriesReadForSearch, int64(searchCheckpointFrequency*2), "Should not need to read more than between two checkpoints") require.NotZero(t, m.entriesReadForSearch, "Must read at least some entries to find the log") var expectedExecMsg types.ExecutingMessage @@ -561,318 +668,354 @@ func requireContains(t *testing.T, db *DB, blockNum uint64, logIdx uint32, logHa requireExecutingMessage(t, db, blockNum, logIdx, expectedExecMsg) } -func requireNotContains(t *testing.T, db *DB, blockNum uint64, logIdx uint32, logHash common.Hash) { +func requireConflicts(t *testing.T, db *DB, blockNum uint64, logIdx uint32, logHash common.Hash) { m, ok := db.m.(*stubMetrics) require.True(t, ok, "Did not get the expected metrics type") - result, _, err := db.Contains(blockNum, logIdx, types.TruncateHash(logHash)) - require.NoErrorf(t, err, "Error searching for log %v in block %v", logIdx, blockNum) - require.Falsef(t, result, "Found unexpected log %v in block %v with hash %v", logIdx, blockNum, logHash) - require.LessOrEqual(t, m.entriesReadForSearch, int64(searchCheckpointFrequency), "Should not need to read more than between two checkpoints") - - _, err = db.Executes(blockNum, logIdx) - require.ErrorIs(t, err, ErrNotFound, "Found unexpected log when getting executing message") - require.LessOrEqual(t, m.entriesReadForSearch, int64(searchCheckpointFrequency), "Should not need to read more than between two checkpoints") + _, err := db.Contains(blockNum, logIdx, types.TruncateHash(logHash)) + require.ErrorIs(t, err, ErrConflict, "canonical chain must not include this log") + require.LessOrEqual(t, m.entriesReadForSearch, int64(searchCheckpointFrequency*2), "Should not need to read more than between two checkpoints") } -func requireExecutingMessage(t *testing.T, db *DB, blockNum uint64, logIdx uint32, execMsg types.ExecutingMessage) { +func requireFuture(t *testing.T, db *DB, blockNum uint64, logIdx uint32, logHash common.Hash) { m, ok := db.m.(*stubMetrics) require.True(t, ok, "Did not get the expected metrics type") - actualExecMsg, err := db.Executes(blockNum, logIdx) - require.NoError(t, err, "Error when searching for executing message") - require.Equal(t, execMsg, actualExecMsg, "Should return matching executing message") - require.LessOrEqual(t, m.entriesReadForSearch, int64(searchCheckpointFrequency), "Should not need to read more than between two checkpoints") - require.NotZero(t, m.entriesReadForSearch, "Must read at least some entries to find the log") + _, err := db.Contains(blockNum, logIdx, types.TruncateHash(logHash)) + require.ErrorIs(t, err, ErrFuture, "canonical chain does not yet include this log") + require.LessOrEqual(t, m.entriesReadForSearch, int64(searchCheckpointFrequency*2), "Should not need to read more than between two checkpoints") } -func requireWrongHash(t *testing.T, db *DB, blockNum uint64, logIdx uint32, logHash common.Hash, _ types.ExecutingMessage) { +func requireExecutingMessage(t *testing.T, db *DB, blockNum uint64, logIdx uint32, execMsg types.ExecutingMessage) { m, ok := db.m.(*stubMetrics) require.True(t, ok, "Did not get the expected metrics type") - result, _, err := db.Contains(blockNum, logIdx, types.TruncateHash(logHash)) - require.NoErrorf(t, err, "Error searching for log %v in block %v", logIdx, blockNum) - require.Falsef(t, result, "Found unexpected log %v in block %v with hash %v", logIdx, blockNum, logHash) - - _, err = db.Executes(blockNum, logIdx) + _, iter, err := db.findLogInfo(blockNum, logIdx) require.NoError(t, err, "Error when searching for executing message") - require.LessOrEqual(t, m.entriesReadForSearch, int64(searchCheckpointFrequency), "Should not need to read more than between two checkpoints") + actualExecMsg := iter.ExecMessage() // non-nil if not just an initiating message, but also an executing message + if execMsg == (types.ExecutingMessage{}) { + require.Nil(t, actualExecMsg) + } else { + require.NotNil(t, actualExecMsg) + require.Equal(t, execMsg, *actualExecMsg, "Should return matching executing message") + } + require.LessOrEqual(t, m.entriesReadForSearch, int64(searchCheckpointFrequency*2), "Should not need to read more than between two checkpoints") + require.NotZero(t, m.entriesReadForSearch, "Must read at least some entries to find the log") } func TestRecoverOnCreate(t *testing.T) { createDb := func(t *testing.T, store *stubEntryStore) (*DB, *stubMetrics, error) { logger := testlog.Logger(t, log.LvlInfo) m := &stubMetrics{} - db, err := NewFromEntryStore(logger, m, store) + db, err := NewFromEntryStore(logger, m, store, true) return db, m, err } - validInitEvent, err := newInitiatingEvent(logContext{blockNum: 1, logIdx: 0}, 1, 0, createTruncatedHash(1), false) - require.NoError(t, err) - validEventSequence := []entrydb.Entry{ - newSearchCheckpoint(1, 0, 100).encode(), - newCanonicalHash(createTruncatedHash(344)).encode(), - validInitEvent.encode(), + storeWithEvents := func(evts ...entrydb.Entry) *stubEntryStore { + store := &stubEntryStore{} + store.entries = append(store.entries, evts...) + return store } - var emptyEventSequence []entrydb.Entry - - for _, prefixEvents := range [][]entrydb.Entry{emptyEventSequence, validEventSequence} { - prefixEvents := prefixEvents - storeWithEvents := func(evts ...entrydb.Entry) *stubEntryStore { - store := &stubEntryStore{} - store.entries = append(store.entries, prefixEvents...) - store.entries = append(store.entries, evts...) - return store + t.Run("NoTruncateWhenLastEntryIsLogWithNoExecMessageSealed", func(t *testing.T) { + store := storeWithEvents( + // seal 0, 1, 2, 3 + newSearchCheckpoint(0, 0, 100).encode(), + newCanonicalHash(createTruncatedHash(300)).encode(), + newSearchCheckpoint(1, 0, 101).encode(), + newCanonicalHash(createTruncatedHash(301)).encode(), + newSearchCheckpoint(2, 0, 102).encode(), + newCanonicalHash(createTruncatedHash(302)).encode(), + newSearchCheckpoint(3, 0, 103).encode(), + newCanonicalHash(createTruncatedHash(303)).encode(), + // open and seal 4 + newInitiatingEvent(createTruncatedHash(1), false).encode(), + newSearchCheckpoint(4, 0, 104).encode(), + newCanonicalHash(createTruncatedHash(304)).encode(), + ) + db, m, err := createDb(t, store) + require.NoError(t, err) + require.EqualValues(t, int64(4*2+3), m.entryCount) + requireContains(t, db, 4, 0, createHash(1)) + }) + + t.Run("NoTruncateWhenLastEntryIsExecutingCheckSealed", func(t *testing.T) { + execMsg := types.ExecutingMessage{ + Chain: 4, + BlockNum: 10, + LogIdx: 4, + Timestamp: 1288, + Hash: createTruncatedHash(4), } - t.Run(fmt.Sprintf("PrefixEvents-%v", len(prefixEvents)), func(t *testing.T) { - t.Run("NoTruncateWhenLastEntryIsLogWithNoExecMessage", func(t *testing.T) { - initEvent, err := newInitiatingEvent(logContext{blockNum: 3, logIdx: 0}, 3, 0, createTruncatedHash(1), false) - require.NoError(t, err) - store := storeWithEvents( - newSearchCheckpoint(3, 0, 100).encode(), - newCanonicalHash(createTruncatedHash(344)).encode(), - initEvent.encode(), - ) - db, m, err := createDb(t, store) - require.NoError(t, err) - require.EqualValues(t, len(prefixEvents)+3, m.entryCount) - requireContains(t, db, 3, 0, createHash(1)) - }) + linkEvt, err := newExecutingLink(execMsg) + require.NoError(t, err) + store := storeWithEvents( + newSearchCheckpoint(0, 0, 100).encode(), + newCanonicalHash(createTruncatedHash(300)).encode(), + newSearchCheckpoint(1, 0, 101).encode(), + newCanonicalHash(createTruncatedHash(301)).encode(), + newSearchCheckpoint(2, 0, 102).encode(), + newCanonicalHash(createTruncatedHash(302)).encode(), + newInitiatingEvent(createTruncatedHash(1111), true).encode(), + linkEvt.encode(), + newExecutingCheck(execMsg.Hash).encode(), + newSearchCheckpoint(3, 0, 103).encode(), + newCanonicalHash(createTruncatedHash(303)).encode(), + ) + db, m, err := createDb(t, store) + require.NoError(t, err) + require.EqualValues(t, int64(3*2+5), m.entryCount) + requireContains(t, db, 3, 0, createHash(1111), execMsg) + }) - t.Run("NoTruncateWhenLastEntryIsExecutingCheck", func(t *testing.T) { - initEvent, err := newInitiatingEvent(logContext{blockNum: 3, logIdx: 0}, 3, 0, createTruncatedHash(1), true) - execMsg := types.ExecutingMessage{ - Chain: 4, - BlockNum: 10, - LogIdx: 4, - Timestamp: 1288, - Hash: createTruncatedHash(4), - } - require.NoError(t, err) - linkEvt, err := newExecutingLink(execMsg) - require.NoError(t, err) - store := storeWithEvents( - newSearchCheckpoint(3, 0, 100).encode(), - newCanonicalHash(createTruncatedHash(344)).encode(), - initEvent.encode(), - linkEvt.encode(), - newExecutingCheck(execMsg.Hash).encode(), - ) - db, m, err := createDb(t, store) - require.NoError(t, err) - require.EqualValues(t, len(prefixEvents)+5, m.entryCount) - requireContains(t, db, 3, 0, createHash(1), execMsg) - }) + t.Run("TruncateWhenLastEntrySearchCheckpoint", func(t *testing.T) { + // A checkpoint, without a canonical blockhash, is useless, and thus truncated. + store := storeWithEvents( + newSearchCheckpoint(0, 0, 100).encode()) + _, m, err := createDb(t, store) + require.NoError(t, err) + require.EqualValues(t, int64(0), m.entryCount) + }) - t.Run("TruncateWhenLastEntrySearchCheckpoint", func(t *testing.T) { - store := storeWithEvents(newSearchCheckpoint(3, 0, 100).encode()) - _, m, err := createDb(t, store) - require.NoError(t, err) - require.EqualValues(t, len(prefixEvents), m.entryCount) - }) + t.Run("NoTruncateWhenLastEntryCanonicalHash", func(t *testing.T) { + // A completed seal is fine to have as last entry. + store := storeWithEvents( + newSearchCheckpoint(0, 0, 100).encode(), + newCanonicalHash(createTruncatedHash(344)).encode(), + ) + _, m, err := createDb(t, store) + require.NoError(t, err) + require.EqualValues(t, int64(2), m.entryCount) + }) - t.Run("TruncateWhenLastEntryCanonicalHash", func(t *testing.T) { - store := storeWithEvents( - newSearchCheckpoint(3, 0, 100).encode(), - newCanonicalHash(createTruncatedHash(344)).encode(), - ) - _, m, err := createDb(t, store) - require.NoError(t, err) - require.EqualValues(t, len(prefixEvents), m.entryCount) - }) + t.Run("TruncateWhenLastEntryInitEventWithExecMsg", func(t *testing.T) { + // An initiating event that claims an executing message, + // without said executing message, is dropped. + store := storeWithEvents( + newSearchCheckpoint(0, 0, 100).encode(), + newCanonicalHash(createTruncatedHash(344)).encode(), + // both pruned because we go back to a seal + newInitiatingEvent(createTruncatedHash(0), false).encode(), + newInitiatingEvent(createTruncatedHash(1), true).encode(), + ) + _, m, err := createDb(t, store) + require.NoError(t, err) + require.EqualValues(t, int64(2), m.entryCount) + }) - t.Run("TruncateWhenLastEntryInitEventWithExecMsg", func(t *testing.T) { - initEvent, err := newInitiatingEvent(logContext{blockNum: 3, logIdx: 0}, 3, 0, createTruncatedHash(1), true) - require.NoError(t, err) - store := storeWithEvents( - newSearchCheckpoint(3, 0, 100).encode(), - newCanonicalHash(createTruncatedHash(344)).encode(), - initEvent.encode(), - ) - _, m, err := createDb(t, store) - require.NoError(t, err) - require.EqualValues(t, len(prefixEvents), m.entryCount) - }) + t.Run("NoTruncateWhenLastEntrySealed", func(t *testing.T) { + // An initiating event that claims an executing message, + // without said executing message, is dropped. + store := storeWithEvents( + newSearchCheckpoint(0, 0, 100).encode(), + newCanonicalHash(createTruncatedHash(300)).encode(), + // pruned because we go back to a seal + newInitiatingEvent(createTruncatedHash(0), false).encode(), + newSearchCheckpoint(1, 0, 100).encode(), + newCanonicalHash(createTruncatedHash(301)).encode(), + ) + _, m, err := createDb(t, store) + require.NoError(t, err) + require.EqualValues(t, int64(5), m.entryCount) + }) - t.Run("TruncateWhenLastEntryInitEventWithExecLink", func(t *testing.T) { - initEvent, err := newInitiatingEvent(logContext{blockNum: 3, logIdx: 0}, 3, 0, createTruncatedHash(1), true) - require.NoError(t, err) - execMsg := types.ExecutingMessage{ - Chain: 4, - BlockNum: 10, - LogIdx: 4, - Timestamp: 1288, - Hash: createTruncatedHash(4), - } - require.NoError(t, err) - linkEvt, err := newExecutingLink(execMsg) - require.NoError(t, err) - store := storeWithEvents( - newSearchCheckpoint(3, 0, 100).encode(), - newCanonicalHash(createTruncatedHash(344)).encode(), - initEvent.encode(), - linkEvt.encode(), - ) - _, m, err := createDb(t, store) - require.NoError(t, err) - require.EqualValues(t, len(prefixEvents), m.entryCount) - }) - }) - } + t.Run("TruncateWhenLastEntryInitEventWithExecLink", func(t *testing.T) { + execMsg := types.ExecutingMessage{ + Chain: 4, + BlockNum: 10, + LogIdx: 4, + Timestamp: 1288, + Hash: createTruncatedHash(4), + } + linkEvt, err := newExecutingLink(execMsg) + require.NoError(t, err) + store := storeWithEvents( + newSearchCheckpoint(3, 0, 100).encode(), + newCanonicalHash(createTruncatedHash(344)).encode(), + newInitiatingEvent(createTruncatedHash(1), true).encode(), + linkEvt.encode(), + ) + _, m, err := createDb(t, store) + require.NoError(t, err) + require.EqualValues(t, int64(2), m.entryCount) + }) } func TestRewind(t *testing.T) { t.Run("WhenEmpty", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) {}, func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.Rewind(100)) - require.NoError(t, db.Rewind(0)) + require.ErrorIs(t, db.Rewind(100), ErrFuture) + // Genesis is a block to, not present in an empty DB + require.ErrorIs(t, db.Rewind(0), ErrFuture) }) }) t.Run("AfterLastBlock", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 1, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(3), eth.BlockID{Hash: createHash(51), Number: 51}, 502, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(4), eth.BlockID{Hash: createHash(74), Number: 74}, 700, 0, nil)) - require.NoError(t, db.Rewind(75)) + bl50 := eth.BlockID{Hash: createHash(50), Number: 50} + require.NoError(t, db.SealBlock(createHash(49), bl50, 500)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl50, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(2), bl50, 1, nil)) + bl51 := eth.BlockID{Hash: createHash(51), Number: 51} + require.NoError(t, db.SealBlock(bl50.Hash, bl51, 502)) + require.NoError(t, db.AddLog(createTruncatedHash(3), bl51, 0, nil)) + bl52 := eth.BlockID{Hash: createHash(52), Number: 52} + require.NoError(t, db.SealBlock(bl51.Hash, bl52, 504)) + require.NoError(t, db.AddLog(createTruncatedHash(4), bl52, 0, nil)) + // cannot rewind to a block that is not sealed yet + require.ErrorIs(t, db.Rewind(53), ErrFuture) }, func(t *testing.T, db *DB, m *stubMetrics) { - requireContains(t, db, 50, 0, createHash(1)) - requireContains(t, db, 50, 1, createHash(2)) - requireContains(t, db, 51, 0, createHash(3)) - requireContains(t, db, 74, 0, createHash(4)) + requireContains(t, db, 51, 0, createHash(1)) + requireContains(t, db, 51, 1, createHash(2)) + requireContains(t, db, 52, 0, createHash(3)) + // Still have the pending log of unsealed block if the rewind to unknown sealed block fails + requireContains(t, db, 53, 0, createHash(4)) }) }) t.Run("BeforeFirstBlock", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 1, nil)) - require.NoError(t, db.Rewind(25)) + bl50 := eth.BlockID{Hash: createHash(50), Number: 50} + require.NoError(t, db.SealBlock(createHash(49), bl50, 500)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl50, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(2), bl50, 1, nil)) + // cannot go back to an unknown block + require.ErrorIs(t, db.Rewind(25), ErrSkipped) }, func(t *testing.T, db *DB, m *stubMetrics) { - requireNotContains(t, db, 50, 0, createHash(1)) - requireNotContains(t, db, 50, 0, createHash(1)) - require.Zero(t, m.entryCount) + requireContains(t, db, 51, 0, createHash(1)) + requireContains(t, db, 51, 0, createHash(1)) }) }) t.Run("AtFirstBlock", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 1, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(51), Number: 51}, 502, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(51), Number: 51}, 502, 1, nil)) - require.NoError(t, db.Rewind(50)) + bl50 := eth.BlockID{Hash: createHash(50), Number: 50} + require.NoError(t, db.SealBlock(createHash(49), bl50, 500)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl50, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(2), bl50, 1, nil)) + bl51 := eth.BlockID{Hash: createHash(51), Number: 51} + require.NoError(t, db.SealBlock(bl50.Hash, bl51, 502)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl51, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(2), bl51, 1, nil)) + bl52 := eth.BlockID{Hash: createHash(52), Number: 52} + require.NoError(t, db.SealBlock(bl51.Hash, bl52, 504)) + require.NoError(t, db.Rewind(51)) }, func(t *testing.T, db *DB, m *stubMetrics) { - requireContains(t, db, 50, 0, createHash(1)) - requireContains(t, db, 50, 1, createHash(2)) - requireNotContains(t, db, 51, 0, createHash(1)) - requireNotContains(t, db, 51, 1, createHash(2)) + requireContains(t, db, 51, 0, createHash(1)) + requireContains(t, db, 51, 1, createHash(2)) + requireFuture(t, db, 52, 0, createHash(1)) + requireFuture(t, db, 52, 1, createHash(2)) }) }) - t.Run("AtSecondCheckpoint", func(t *testing.T) { + t.Run("AfterSecondCheckpoint", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { + bl50 := eth.BlockID{Hash: createHash(50), Number: 50} + require.NoError(t, db.SealBlock(createHash(49), bl50, 500)) for i := uint32(0); m.entryCount < searchCheckpointFrequency; i++ { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(50), Number: 50}, 500, i, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl50, i, nil)) } - require.EqualValues(t, searchCheckpointFrequency, m.entryCount) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(51), Number: 51}, 502, 0, nil)) - require.EqualValues(t, searchCheckpointFrequency+3, m.entryCount, "Should have inserted new checkpoint and extra log") - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(51), Number: 51}, 502, 1, nil)) - require.NoError(t, db.Rewind(50)) + // The checkpoint is added automatically, + // it will be there as soon as it reaches 255 with log events. + // Thus add 2 for the checkpoint. + require.EqualValues(t, searchCheckpointFrequency+2, m.entryCount) + bl51 := eth.BlockID{Hash: createHash(51), Number: 51} + require.NoError(t, db.SealBlock(bl50.Hash, bl51, 502)) + require.NoError(t, db.AddLog(createTruncatedHash(1), bl51, 0, nil)) + require.EqualValues(t, searchCheckpointFrequency+2+3, m.entryCount, "Should have inserted new checkpoint and extra log") + require.NoError(t, db.AddLog(createTruncatedHash(2), bl51, 1, nil)) + bl52 := eth.BlockID{Hash: createHash(52), Number: 52} + require.NoError(t, db.SealBlock(bl51.Hash, bl52, 504)) + require.NoError(t, db.Rewind(51)) }, func(t *testing.T, db *DB, m *stubMetrics) { - require.EqualValues(t, searchCheckpointFrequency, m.entryCount, "Should have deleted second checkpoint") - requireContains(t, db, 50, 0, createHash(1)) - requireContains(t, db, 50, 1, createHash(1)) - requireNotContains(t, db, 51, 0, createHash(1)) - requireNotContains(t, db, 51, 1, createHash(2)) + require.EqualValues(t, searchCheckpointFrequency+2+2, m.entryCount, "Should have deleted second checkpoint") + requireContains(t, db, 51, 0, createHash(1)) + requireContains(t, db, 51, 1, createHash(1)) + requireFuture(t, db, 52, 0, createHash(1)) + requireFuture(t, db, 52, 1, createHash(2)) }) }) - t.Run("BetweenLogEntries", func(t *testing.T) { + t.Run("BetweenBlockEntries", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 1, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(60), Number: 60}, 502, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(60), Number: 60}, 502, 1, nil)) - require.NoError(t, db.Rewind(55)) - }, - func(t *testing.T, db *DB, m *stubMetrics) { - requireContains(t, db, 50, 0, createHash(1)) - requireContains(t, db, 50, 1, createHash(2)) - requireNotContains(t, db, 60, 0, createHash(1)) - requireNotContains(t, db, 60, 1, createHash(2)) - }) - }) - - t.Run("AtExistingLogEntry", func(t *testing.T) { - runDBTest(t, - func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(59), Number: 59}, 500, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(59), Number: 59}, 500, 1, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(60), Number: 60}, 502, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(60), Number: 60}, 502, 1, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(61), Number: 61}, 502, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(61), Number: 61}, 502, 1, nil)) - require.NoError(t, db.Rewind(60)) + // create many blocks, and all the odd blocks get 2 logs + for i := uint32(0); i < 30; i++ { + bl := eth.BlockID{Hash: createHash(int(i)), Number: uint64(i)} + require.NoError(t, db.SealBlock(createHash(int(i)-1), bl, 500+uint64(i))) + if i%2 == 0 { + require.NoError(t, db.AddLog(createTruncatedHash(1), bl, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(2), bl, 1, nil)) + } + } + require.NoError(t, db.Rewind(15)) }, func(t *testing.T, db *DB, m *stubMetrics) { - requireContains(t, db, 59, 0, createHash(1)) - requireContains(t, db, 59, 1, createHash(2)) - requireContains(t, db, 60, 0, createHash(1)) - requireContains(t, db, 60, 1, createHash(2)) - requireNotContains(t, db, 61, 0, createHash(1)) - requireNotContains(t, db, 61, 1, createHash(2)) + requireContains(t, db, 15, 0, createHash(1)) + requireContains(t, db, 15, 1, createHash(2)) + requireFuture(t, db, 16, 0, createHash(1)) + requireFuture(t, db, 16, 1, createHash(2)) }) }) t.Run("AtLastEntry", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(50), Number: 50}, 500, 1, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(60), Number: 60}, 502, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(60), Number: 60}, 502, 1, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(70), Number: 70}, 502, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(70), Number: 70}, 502, 1, nil)) - require.NoError(t, db.Rewind(70)) + // create many blocks, and all the even blocks get 2 logs + for i := uint32(0); i <= 30; i++ { + bl := eth.BlockID{Hash: createHash(int(i)), Number: uint64(i)} + require.NoError(t, db.SealBlock(createHash(int(i)-1), bl, 500+uint64(i))) + if i%2 == 1 { + require.NoError(t, db.AddLog(createTruncatedHash(1), bl, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(2), bl, 1, nil)) + } + } + // We ended at 30, and sealed it, nothing left to prune + require.NoError(t, db.Rewind(30)) }, func(t *testing.T, db *DB, m *stubMetrics) { - requireContains(t, db, 50, 0, createHash(1)) - requireContains(t, db, 50, 1, createHash(2)) - requireContains(t, db, 60, 0, createHash(1)) - requireContains(t, db, 60, 1, createHash(2)) - requireContains(t, db, 70, 0, createHash(1)) - requireContains(t, db, 70, 1, createHash(2)) + requireContains(t, db, 20, 0, createHash(1)) + requireContains(t, db, 20, 1, createHash(2)) + // built on top of 29, these are in sealed block 30, still around + requireContains(t, db, 30, 0, createHash(1)) + requireContains(t, db, 30, 1, createHash(2)) }) }) - t.Run("ReaddDeletedBlocks", func(t *testing.T) { + t.Run("ReadDeletedBlocks", func(t *testing.T) { runDBTest(t, func(t *testing.T, db *DB, m *stubMetrics) { - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(59), Number: 59}, 500, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(59), Number: 59}, 500, 1, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(60), Number: 60}, 502, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(60), Number: 60}, 502, 1, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(61), Number: 61}, 502, 0, nil)) - require.NoError(t, db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(61), Number: 61}, 502, 1, nil)) - require.NoError(t, db.Rewind(60)) + // create many blocks, and all the odd blocks get 2 logs + for i := uint32(0); i < 30; i++ { + bl := eth.BlockID{Hash: createHash(int(i)), Number: uint64(i)} + require.NoError(t, db.SealBlock(createHash(int(i)-1), bl, 500+uint64(i))) + if i%2 == 0 { + require.NoError(t, db.AddLog(createTruncatedHash(1), bl, 0, nil)) + require.NoError(t, db.AddLog(createTruncatedHash(2), bl, 1, nil)) + } + } + require.NoError(t, db.Rewind(16)) }, func(t *testing.T, db *DB, m *stubMetrics) { - err := db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(59), Number: 59}, 500, 1, nil) - require.ErrorIs(t, err, ErrLogOutOfOrder, "Cannot add block before rewound head") - err = db.AddLog(createTruncatedHash(2), eth.BlockID{Hash: createHash(60), Number: 60}, 502, 1, nil) - require.ErrorIs(t, err, ErrLogOutOfOrder, "Cannot add block that was rewound to") - err = db.AddLog(createTruncatedHash(1), eth.BlockID{Hash: createHash(60), Number: 61}, 502, 0, nil) - require.NoError(t, err, "Can re-add deleted block") + bl29 := eth.BlockID{Hash: createHash(29), Number: 29} + // 29 was deleted + err := db.AddLog(createTruncatedHash(2), bl29, 1, nil) + require.ErrorIs(t, err, ErrLogOutOfOrder, "Cannot add log on removed block") + // 15 is older, we have up to 16 + bl15 := eth.BlockID{Hash: createHash(15), Number: 15} + // try to add a third log to 15 + err = db.AddLog(createTruncatedHash(10), bl15, 2, nil) + require.ErrorIs(t, err, ErrLogOutOfOrder) + bl16 := eth.BlockID{Hash: createHash(16), Number: 16} + // try to add a log to 17, on top of 16 + err = db.AddLog(createTruncatedHash(42), bl16, 0, nil) + require.NoError(t, err) + requireContains(t, db, 17, 0, createHash(42)) }) }) } diff --git a/op-supervisor/supervisor/backend/db/logs/entries.go b/op-supervisor/supervisor/backend/db/logs/entries.go index 8816474cdd2f..5dbc1e3b4816 100644 --- a/op-supervisor/supervisor/backend/db/logs/entries.go +++ b/op-supervisor/supervisor/backend/db/logs/entries.go @@ -3,44 +3,46 @@ package logs import ( "encoding/binary" "fmt" - "math" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/entrydb" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/types" ) +// searchCheckpoint is both a checkpoint for searching, as well as a checkpoint for sealing blocks. type searchCheckpoint struct { - blockNum uint64 - logIdx uint32 + blockNum uint64 + // seen logs *after* the seal of the mentioned block, i.e. not part of this block, but building on top of it. + // There is at least one checkpoint per L2 block with logsSince == 0, i.e. the exact block boundary. + logsSince uint32 timestamp uint64 } -func newSearchCheckpoint(blockNum uint64, logIdx uint32, timestamp uint64) searchCheckpoint { +func newSearchCheckpoint(blockNum uint64, logsSince uint32, timestamp uint64) searchCheckpoint { return searchCheckpoint{ blockNum: blockNum, - logIdx: logIdx, + logsSince: logsSince, timestamp: timestamp, } } func newSearchCheckpointFromEntry(data entrydb.Entry) (searchCheckpoint, error) { - if data[0] != typeSearchCheckpoint { - return searchCheckpoint{}, fmt.Errorf("%w: attempting to decode search checkpoint but was type %v", ErrDataCorruption, data[0]) + if data.Type() != entrydb.TypeSearchCheckpoint { + return searchCheckpoint{}, fmt.Errorf("%w: attempting to decode search checkpoint but was type %s", ErrDataCorruption, data.Type()) } return searchCheckpoint{ blockNum: binary.LittleEndian.Uint64(data[1:9]), - logIdx: binary.LittleEndian.Uint32(data[9:13]), + logsSince: binary.LittleEndian.Uint32(data[9:13]), timestamp: binary.LittleEndian.Uint64(data[13:21]), }, nil } -// encode creates a search checkpoint entry -// type 0: "search checkpoint" = 20 bytes +// encode creates a checkpoint entry +// type 0: "search checkpoint" = 21 bytes func (s searchCheckpoint) encode() entrydb.Entry { var data entrydb.Entry - data[0] = typeSearchCheckpoint + data[0] = uint8(entrydb.TypeSearchCheckpoint) binary.LittleEndian.PutUint64(data[1:9], s.blockNum) - binary.LittleEndian.PutUint32(data[9:13], s.logIdx) + binary.LittleEndian.PutUint32(data[9:13], s.logsSince) binary.LittleEndian.PutUint64(data[13:21], s.timestamp) return data } @@ -54,8 +56,8 @@ func newCanonicalHash(hash types.TruncatedHash) canonicalHash { } func newCanonicalHashFromEntry(data entrydb.Entry) (canonicalHash, error) { - if data[0] != typeCanonicalHash { - return canonicalHash{}, fmt.Errorf("%w: attempting to decode canonical hash but was type %v", ErrDataCorruption, data[0]) + if data.Type() != entrydb.TypeCanonicalHash { + return canonicalHash{}, fmt.Errorf("%w: attempting to decode canonical hash but was type %s", ErrDataCorruption, data.Type()) } var truncated types.TruncatedHash copy(truncated[:], data[1:21]) @@ -64,100 +66,48 @@ func newCanonicalHashFromEntry(data entrydb.Entry) (canonicalHash, error) { func (c canonicalHash) encode() entrydb.Entry { var entry entrydb.Entry - entry[0] = typeCanonicalHash + entry[0] = uint8(entrydb.TypeCanonicalHash) copy(entry[1:21], c.hash[:]) return entry } type initiatingEvent struct { - blockDiff uint8 - incrementLogIdx bool - hasExecMsg bool - logHash types.TruncatedHash + hasExecMsg bool + logHash types.TruncatedHash } func newInitiatingEventFromEntry(data entrydb.Entry) (initiatingEvent, error) { - if data[0] != typeInitiatingEvent { - return initiatingEvent{}, fmt.Errorf("%w: attempting to decode initiating event but was type %v", ErrDataCorruption, data[0]) + if data.Type() != entrydb.TypeInitiatingEvent { + return initiatingEvent{}, fmt.Errorf("%w: attempting to decode initiating event but was type %s", ErrDataCorruption, data.Type()) } - blockNumDiff := data[1] - flags := data[2] + flags := data[1] return initiatingEvent{ - blockDiff: blockNumDiff, - incrementLogIdx: flags&eventFlagIncrementLogIdx != 0, - hasExecMsg: flags&eventFlagHasExecutingMessage != 0, - logHash: types.TruncatedHash(data[3:23]), + hasExecMsg: flags&eventFlagHasExecutingMessage != 0, + logHash: types.TruncatedHash(data[2:22]), }, nil } -func newInitiatingEvent(pre logContext, blockNum uint64, logIdx uint32, logHash types.TruncatedHash, hasExecMsg bool) (initiatingEvent, error) { - blockDiff := blockNum - pre.blockNum - if blockDiff > math.MaxUint8 { - // TODO(optimism#11091): Need to find a way to support this. - return initiatingEvent{}, fmt.Errorf("too many block skipped between %v and %v", pre.blockNum, blockNum) - } - - currLogIdx := pre.logIdx - if blockDiff > 0 { - currLogIdx = 0 - } - logDiff := logIdx - currLogIdx - if logDiff > 1 { - return initiatingEvent{}, fmt.Errorf("skipped logs between %v and %v", currLogIdx, logIdx) - } - +func newInitiatingEvent(logHash types.TruncatedHash, hasExecMsg bool) initiatingEvent { return initiatingEvent{ - blockDiff: uint8(blockDiff), - incrementLogIdx: logDiff > 0, - hasExecMsg: hasExecMsg, - logHash: logHash, - }, nil + hasExecMsg: hasExecMsg, + logHash: logHash, + } } // encode creates an initiating event entry -// type 2: "initiating event" = 23 bytes +// type 2: "initiating event" = 22 bytes func (i initiatingEvent) encode() entrydb.Entry { var data entrydb.Entry - data[0] = typeInitiatingEvent - data[1] = i.blockDiff + data[0] = uint8(entrydb.TypeInitiatingEvent) flags := byte(0) - if i.incrementLogIdx { - // Set flag to indicate log idx needs to be incremented (ie we're not directly after a checkpoint) - flags = flags | eventFlagIncrementLogIdx - } if i.hasExecMsg { flags = flags | eventFlagHasExecutingMessage } - data[2] = flags - copy(data[3:23], i.logHash[:]) + data[1] = flags + copy(data[2:22], i.logHash[:]) return data } -func (i initiatingEvent) postContext(pre logContext) logContext { - post := logContext{ - blockNum: pre.blockNum + uint64(i.blockDiff), - logIdx: pre.logIdx, - } - if i.blockDiff > 0 { - post.logIdx = 0 - } - if i.incrementLogIdx { - post.logIdx++ - } - return post -} - -// preContext is the reverse of postContext and calculates the logContext required as input to get the specified post -// context after applying this init event. -func (i initiatingEvent) preContext(post logContext) logContext { - pre := post - pre.blockNum = post.blockNum - uint64(i.blockDiff) - if i.incrementLogIdx { - pre.logIdx-- - } - return pre -} - type executingLink struct { chain uint32 blockNum uint64 @@ -178,8 +128,8 @@ func newExecutingLink(msg types.ExecutingMessage) (executingLink, error) { } func newExecutingLinkFromEntry(data entrydb.Entry) (executingLink, error) { - if data[0] != typeExecutingLink { - return executingLink{}, fmt.Errorf("%w: attempting to decode executing link but was type %v", ErrDataCorruption, data[0]) + if data.Type() != entrydb.TypeExecutingLink { + return executingLink{}, fmt.Errorf("%w: attempting to decode executing link but was type %s", ErrDataCorruption, data.Type()) } timestamp := binary.LittleEndian.Uint64(data[16:24]) return executingLink{ @@ -194,7 +144,7 @@ func newExecutingLinkFromEntry(data entrydb.Entry) (executingLink, error) { // type 3: "executing link" = 24 bytes func (e executingLink) encode() entrydb.Entry { var entry entrydb.Entry - entry[0] = typeExecutingLink + entry[0] = uint8(entrydb.TypeExecutingLink) binary.LittleEndian.PutUint32(entry[1:5], e.chain) binary.LittleEndian.PutUint64(entry[5:13], e.blockNum) @@ -214,12 +164,12 @@ func newExecutingCheck(hash types.TruncatedHash) executingCheck { return executingCheck{hash: hash} } -func newExecutingCheckFromEntry(entry entrydb.Entry) (executingCheck, error) { - if entry[0] != typeExecutingCheck { - return executingCheck{}, fmt.Errorf("%w: attempting to decode executing check but was type %v", ErrDataCorruption, entry[0]) +func newExecutingCheckFromEntry(data entrydb.Entry) (executingCheck, error) { + if data.Type() != entrydb.TypeExecutingCheck { + return executingCheck{}, fmt.Errorf("%w: attempting to decode executing check but was type %s", ErrDataCorruption, data.Type()) } var hash types.TruncatedHash - copy(hash[:], entry[1:21]) + copy(hash[:], data[1:21]) return newExecutingCheck(hash), nil } @@ -227,25 +177,17 @@ func newExecutingCheckFromEntry(entry entrydb.Entry) (executingCheck, error) { // type 4: "executing check" = 21 bytes func (e executingCheck) encode() entrydb.Entry { var entry entrydb.Entry - entry[0] = typeExecutingCheck + entry[0] = uint8(entrydb.TypeExecutingCheck) copy(entry[1:21], e.hash[:]) return entry } -func newExecutingMessageFromEntries(linkEntry entrydb.Entry, checkEntry entrydb.Entry) (types.ExecutingMessage, error) { - link, err := newExecutingLinkFromEntry(linkEntry) - if err != nil { - return types.ExecutingMessage{}, fmt.Errorf("invalid executing link: %w", err) - } - check, err := newExecutingCheckFromEntry(checkEntry) - if err != nil { - return types.ExecutingMessage{}, fmt.Errorf("invalid executing check: %w", err) - } - return types.ExecutingMessage{ - Chain: link.chain, - BlockNum: link.blockNum, - LogIdx: link.logIdx, - Timestamp: link.timestamp, - Hash: check.hash, - }, nil +type paddingEntry struct{} + +// encoding of the padding entry +// type 5: "padding" = 24 bytes +func (e paddingEntry) encode() entrydb.Entry { + var entry entrydb.Entry + entry[0] = uint8(entrydb.TypePadding) + return entry } diff --git a/op-supervisor/supervisor/backend/db/logs/iterator.go b/op-supervisor/supervisor/backend/db/logs/iterator.go index 7312966b2455..29b47245e71c 100644 --- a/op-supervisor/supervisor/backend/db/logs/iterator.go +++ b/op-supervisor/supervisor/backend/db/logs/iterator.go @@ -9,106 +9,134 @@ import ( "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/types" ) +type IteratorState interface { + NextIndex() entrydb.EntryIdx + SealedBlock() (hash types.TruncatedHash, num uint64, ok bool) + InitMessage() (hash types.TruncatedHash, logIndex uint32, ok bool) + ExecMessage() *types.ExecutingMessage +} + type Iterator interface { - NextLog() (blockNum uint64, logIdx uint32, evtHash types.TruncatedHash, outErr error) - Index() entrydb.EntryIdx - ExecMessage() (types.ExecutingMessage, error) + End() error + NextInitMsg() error + NextExecMsg() error + NextBlock() error + IteratorState } type iterator struct { - db *DB - nextEntryIdx entrydb.EntryIdx - - current logContext - hasExecMsg bool - + db *DB + current logContext entriesRead int64 } -// NextLog returns the next log in the iterator. -// It scans forward until it finds an initiating event, returning the block number, log index, and event hash. -func (i *iterator) NextLog() (blockNum uint64, logIdx uint32, evtHash types.TruncatedHash, outErr error) { - for i.nextEntryIdx <= i.db.lastEntryIdx() { - entryIdx := i.nextEntryIdx - entry, err := i.db.store.Read(entryIdx) +// End traverses the iterator to the end of the DB. +// It does not return io.EOF or ErrFuture. +func (i *iterator) End() error { + for { + _, err := i.next() + if errors.Is(err, ErrFuture) { + return nil + } else if err != nil { + return err + } + } +} + +// NextInitMsg returns the next initiating message in the iterator. +// It scans forward until it finds and fully reads an initiating event, skipping any blocks. +func (i *iterator) NextInitMsg() error { + seenLog := false + for { + typ, err := i.next() if err != nil { - outErr = fmt.Errorf("failed to read entry %v: %w", i, err) - return + return err + } + if typ == entrydb.TypeInitiatingEvent { + seenLog = true } - i.nextEntryIdx++ - i.entriesRead++ - i.hasExecMsg = false - switch entry[0] { - case typeSearchCheckpoint: - current, err := newSearchCheckpointFromEntry(entry) - if err != nil { - outErr = fmt.Errorf("failed to parse search checkpoint at idx %v: %w", entryIdx, err) - return - } - i.current.blockNum = current.blockNum - i.current.logIdx = current.logIdx - case typeInitiatingEvent: - evt, err := newInitiatingEventFromEntry(entry) - if err != nil { - outErr = fmt.Errorf("failed to parse initiating event at idx %v: %w", entryIdx, err) - return - } - i.current = evt.postContext(i.current) - blockNum = i.current.blockNum - logIdx = i.current.logIdx - evtHash = evt.logHash - i.hasExecMsg = evt.hasExecMsg - return - case typeCanonicalHash: // Skip - case typeExecutingCheck: // Skip - case typeExecutingLink: // Skip - default: - outErr = fmt.Errorf("unknown entry type at idx %v %v", entryIdx, entry[0]) - return + if !i.current.hasCompleteBlock() { + continue // must know the block we're building on top of + } + if i.current.hasIncompleteLog() { + continue // didn't finish processing the log yet + } + if seenLog { + return nil } } - outErr = io.EOF - return } -func (i *iterator) Index() entrydb.EntryIdx { - return i.nextEntryIdx - 1 +// NextExecMsg returns the next executing message in the iterator. +// It scans forward until it finds and fully reads an initiating event, skipping any blocks. +// This does not stay at the executing message of the current initiating message, if there is any. +func (i *iterator) NextExecMsg() error { + for { + err := i.NextInitMsg() + if err != nil { + return err + } + if i.current.execMsg != nil { + return nil // found a new executing message! + } + } } -func (i *iterator) ExecMessage() (types.ExecutingMessage, error) { - if !i.hasExecMsg { - return types.ExecutingMessage{}, nil - } - // Look ahead to find the exec message info - logEntryIdx := i.nextEntryIdx - 1 - execMsg, err := i.readExecMessage(logEntryIdx) - if err != nil { - return types.ExecutingMessage{}, fmt.Errorf("failed to read exec message for initiating event at %v: %w", logEntryIdx, err) +// NextBlock returns the next block in the iterator. +// It scans forward until it finds and fully reads a block, skipping any events. +func (i *iterator) NextBlock() error { + seenBlock := false + for { + typ, err := i.next() + if err != nil { + return err + } + if typ == entrydb.TypeSearchCheckpoint { + seenBlock = true + } + if !i.current.hasCompleteBlock() { + continue // need the full block content + } + if seenBlock { + return nil + } } - return execMsg, nil } -func (i *iterator) readExecMessage(initEntryIdx entrydb.EntryIdx) (types.ExecutingMessage, error) { - linkIdx := initEntryIdx + 1 - if linkIdx%searchCheckpointFrequency == 0 { - linkIdx += 2 // skip the search checkpoint and canonical hash entries +// Read and apply the next entry. +func (i *iterator) next() (entrydb.EntryType, error) { + index := i.current.nextEntryIndex + entry, err := i.db.store.Read(index) + if err != nil { + if errors.Is(err, io.EOF) { + return 0, ErrFuture + } + return 0, fmt.Errorf("failed to read entry %d: %w", index, err) } - linkEntry, err := i.db.store.Read(linkIdx) - if errors.Is(err, io.EOF) { - return types.ExecutingMessage{}, fmt.Errorf("%w: missing expected executing link event at idx %v", ErrDataCorruption, linkIdx) - } else if err != nil { - return types.ExecutingMessage{}, fmt.Errorf("failed to read executing link event at idx %v: %w", linkIdx, err) + if err := i.current.ApplyEntry(entry); err != nil { + return entry.Type(), fmt.Errorf("failed to apply entry %d to iterator state: %w", index, err) } - checkIdx := linkIdx + 1 - if checkIdx%searchCheckpointFrequency == 0 { - checkIdx += 2 // skip the search checkpoint and canonical hash entries - } - checkEntry, err := i.db.store.Read(checkIdx) - if errors.Is(err, io.EOF) { - return types.ExecutingMessage{}, fmt.Errorf("%w: missing expected executing check event at idx %v", ErrDataCorruption, checkIdx) - } else if err != nil { - return types.ExecutingMessage{}, fmt.Errorf("failed to read executing check event at idx %v: %w", checkIdx, err) - } - return newExecutingMessageFromEntries(linkEntry, checkEntry) + i.entriesRead++ + return entry.Type(), nil +} + +func (i *iterator) NextIndex() entrydb.EntryIdx { + return i.current.NextIndex() +} + +// SealedBlock returns the sealed block that we are appending logs after, if any is available. +// I.e. the block is the parent block of the block containing the logs that are currently appending to it. +func (i *iterator) SealedBlock() (hash types.TruncatedHash, num uint64, ok bool) { + return i.current.SealedBlock() +} + +// InitMessage returns the current initiating message, if any is available. +func (i *iterator) InitMessage() (hash types.TruncatedHash, logIndex uint32, ok bool) { + return i.current.InitMessage() +} + +// ExecMessage returns the current executing message, if any is available. +func (i *iterator) ExecMessage() *types.ExecutingMessage { + return i.current.ExecMessage() } diff --git a/op-supervisor/supervisor/backend/db/logs/state.go b/op-supervisor/supervisor/backend/db/logs/state.go new file mode 100644 index 000000000000..083e07f97083 --- /dev/null +++ b/op-supervisor/supervisor/backend/db/logs/state.go @@ -0,0 +1,407 @@ +package logs + +import ( + "errors" + "fmt" + "io" + + "github.com/ethereum/go-ethereum/common" + + "github.com/ethereum-optimism/optimism/op-service/eth" + "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/entrydb" + "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/types" +) + +// logContext is a buffer on top of the DB, +// where blocks and logs can be applied to. +// +// Rules: +// +// if entry_index % 256 == 0: must be type 0. For easy binary search. +// else if end_of_block: also type 0. +// else: +// after type 0: type 1 +// after type 1: type 2 iff any event and space, otherwise type 0 +// after type 2: type 3 iff executing, otherwise type 2 or 0 +// after type 3: type 4 +// after type 4: type 2 iff any event and space, otherwise type 0 +// after type 5: any +// +// Type 0 can repeat: seal the block, then start a search checkpoint, then a single canonical hash. +// Type 0 may also be used as padding: type 2 only starts when it will not be interrupted by a search checkpoint. +// +// Types ( = 1 byte): +// type 0: "checkpoint" = 21 bytes +// type 1: "canonical hash" = 21 bytes +// type 2: "initiating event" = 22 bytes +// type 3: "executing link" = 24 bytes +// type 4: "executing check" = 21 bytes +// type 5: "padding" = 24 bytes +// other types: future compat. E.g. for linking to L1, registering block-headers as a kind of initiating-event, tracking safe-head progression, etc. +// +// Right-pad each entry that is not 24 bytes. +// +// We insert a checkpoint for every search interval and block sealing event, +// and these may overlap as the same thing. +// Such seal has logsSince == 0, i.e. wrapping up the last block and starting a fresh list of logs. +// +// event-flags: each bit represents a boolean value, currently only two are defined +// * event-flags & 0x01 - true if the initiating event has an executing link that should follow. Allows detecting when the executing link failed to write. +// event-hash: H(origin, timestamp, payloadhash); enough to check identifier matches & payload matches. +type logContext struct { + // next entry index, including the contents of `out` + nextEntryIndex entrydb.EntryIdx + + // blockHash of the last sealed block. + // A block is not considered sealed until we know its block hash. + // While we process logs we keep the parent-block of said logs around as sealed block. + blockHash types.TruncatedHash + // blockNum of the last sealed block + blockNum uint64 + // timestamp of the last sealed block + timestamp uint64 + + // number of logs since the last sealed block + logsSince uint32 + + // payload-hash of the log-event that was last processed. (may not be fully processed, see doneLog) + logHash types.TruncatedHash + + // executing message that might exist for the current log event. + // Might be incomplete; if !logDone while we already processed the initiating event, + // then we know an executing message is still coming. + execMsg *types.ExecutingMessage + + need entrydb.EntryTypeFlag + + // buffer of entries not yet in the DB. + // This is generated as objects are applied. + // E.g. you can build multiple hypothetical blocks with log events on top of the state, + // before flushing the entries to a DB. + // However, no entries can be read from the DB while objects are being applied. + out []entrydb.Entry +} + +type EntryObj interface { + encode() entrydb.Entry +} + +func (l *logContext) NextIndex() entrydb.EntryIdx { + return l.nextEntryIndex +} + +// SealedBlock returns the block that we are building on top of, and if it is sealed. +func (l *logContext) SealedBlock() (hash types.TruncatedHash, num uint64, ok bool) { + if !l.hasCompleteBlock() { + return types.TruncatedHash{}, 0, false + } + return l.blockHash, l.blockNum, true +} + +func (l *logContext) hasCompleteBlock() bool { + return !l.need.Any(entrydb.FlagCanonicalHash) +} + +func (l *logContext) hasIncompleteLog() bool { + return l.need.Any(entrydb.FlagInitiatingEvent | entrydb.FlagExecutingLink | entrydb.FlagExecutingCheck) +} + +func (l *logContext) hasReadableLog() bool { + return l.logsSince > 0 && !l.hasIncompleteLog() +} + +// InitMessage returns the current initiating message, if any is available. +func (l *logContext) InitMessage() (hash types.TruncatedHash, logIndex uint32, ok bool) { + if !l.hasReadableLog() { + return types.TruncatedHash{}, 0, false + } + return l.logHash, l.logsSince - 1, true +} + +// ExecMessage returns the current executing message, if any is available. +func (l *logContext) ExecMessage() *types.ExecutingMessage { + if l.hasCompleteBlock() && l.hasReadableLog() && l.execMsg != nil { + return l.execMsg + } + return nil +} + +// ApplyEntry applies an entry on top of the current state. +func (l *logContext) ApplyEntry(entry entrydb.Entry) error { + // Wrap processEntry to add common useful error message info + err := l.processEntry(entry) + if err != nil { + return fmt.Errorf("failed to process type %s entry at idx %d (%x): %w", entry.Type().String(), l.nextEntryIndex, entry[:], err) + } + return nil +} + +// processEntry decodes and applies an entry to the state. +// Entries may not be applied if we are in the process of generating entries from objects. +// These outputs need to be flushed before inputs can be accepted. +func (l *logContext) processEntry(entry entrydb.Entry) error { + if len(l.out) != 0 { + panic("can only apply without appending if the state is still empty") + } + switch entry.Type() { + case entrydb.TypeSearchCheckpoint: + current, err := newSearchCheckpointFromEntry(entry) + if err != nil { + return err + } + l.blockNum = current.blockNum + l.blockHash = types.TruncatedHash{} + l.logsSince = current.logsSince // TODO this is bumping the logsSince? + l.timestamp = current.timestamp + l.need.Add(entrydb.FlagCanonicalHash) + // Log data after the block we are sealing remains to be seen + if l.logsSince == 0 { + l.logHash = types.TruncatedHash{} + l.execMsg = nil + } + case entrydb.TypeCanonicalHash: + if !l.need.Any(entrydb.FlagCanonicalHash) { + return errors.New("not ready for canonical hash entry, already sealed the last block") + } + canonHash, err := newCanonicalHashFromEntry(entry) + if err != nil { + return err + } + l.blockHash = canonHash.hash + l.need.Remove(entrydb.FlagCanonicalHash) + case entrydb.TypeInitiatingEvent: + if !l.hasCompleteBlock() { + return errors.New("did not complete block seal, cannot add log") + } + if l.hasIncompleteLog() { + return errors.New("cannot process log before last log completes") + } + evt, err := newInitiatingEventFromEntry(entry) + if err != nil { + return err + } + l.execMsg = nil // clear the old state + l.logHash = evt.logHash + if evt.hasExecMsg { + l.need.Add(entrydb.FlagExecutingLink | entrydb.FlagExecutingCheck) + } else { + l.logsSince += 1 + } + l.need.Remove(entrydb.FlagInitiatingEvent) + case entrydb.TypeExecutingLink: + if !l.need.Any(entrydb.FlagExecutingLink) { + return errors.New("unexpected executing-link") + } + link, err := newExecutingLinkFromEntry(entry) + if err != nil { + return err + } + l.execMsg = &types.ExecutingMessage{ + Chain: link.chain, + BlockNum: link.blockNum, + LogIdx: link.logIdx, + Timestamp: link.timestamp, + Hash: types.TruncatedHash{}, // not known yet + } + l.need.Remove(entrydb.FlagExecutingLink) + l.need.Add(entrydb.FlagExecutingCheck) + case entrydb.TypeExecutingCheck: + if l.need.Any(entrydb.FlagExecutingLink) { + return errors.New("need executing link to be applied before the check part") + } + if !l.need.Any(entrydb.FlagExecutingCheck) { + return errors.New("unexpected executing check") + } + link, err := newExecutingCheckFromEntry(entry) + if err != nil { + return err + } + l.execMsg.Hash = link.hash + l.need.Remove(entrydb.FlagExecutingCheck) + l.logsSince += 1 + case entrydb.TypePadding: + if l.need.Any(entrydb.FlagPadding) { + l.need.Remove(entrydb.FlagPadding) + } else { + l.need.Remove(entrydb.FlagPadding2) + } + default: + return fmt.Errorf("unknown entry type: %s", entry.Type()) + } + l.nextEntryIndex += 1 + return nil +} + +// appendEntry add the entry to the output-buffer, +// and registers it as last processed entry type, and increments the next entry-index. +func (l *logContext) appendEntry(obj EntryObj) { + entry := obj.encode() + l.out = append(l.out, entry) + l.nextEntryIndex += 1 +} + +// infer advances the logContext in cases where multiple entries are to be appended implicitly +// depending on the last type of entry, a new entry is appended, +// or when the searchCheckpoint should be inserted. +// This can be done repeatedly until there is no more implied data to extend. +func (l *logContext) infer() error { + // We force-insert a checkpoint whenever we hit the known fixed interval. + if l.nextEntryIndex%searchCheckpointFrequency == 0 { + l.need.Add(entrydb.FlagSearchCheckpoint) + } + if l.need.Any(entrydb.FlagSearchCheckpoint) { + l.appendEntry(newSearchCheckpoint(l.blockNum, l.logsSince, l.timestamp)) + l.need.Add(entrydb.FlagCanonicalHash) // always follow with a canonical hash + l.need.Remove(entrydb.FlagSearchCheckpoint) + return nil + } + if l.need.Any(entrydb.FlagCanonicalHash) { + l.appendEntry(newCanonicalHash(l.blockHash)) + l.need.Remove(entrydb.FlagCanonicalHash) + return nil + } + if l.need.Any(entrydb.FlagPadding) { + l.appendEntry(paddingEntry{}) + l.need.Remove(entrydb.FlagPadding) + return nil + } + if l.need.Any(entrydb.FlagPadding2) { + l.appendEntry(paddingEntry{}) + l.need.Remove(entrydb.FlagPadding2) + return nil + } + if l.need.Any(entrydb.FlagInitiatingEvent) { + // If we are running out of space for log-event data, + // write some checkpoints as padding, to pass the checkpoint. + if l.execMsg != nil { // takes 3 total. Need to avoid the checkpoint. + switch l.nextEntryIndex % searchCheckpointFrequency { + case searchCheckpointFrequency - 1: + l.need.Add(entrydb.FlagPadding) + return nil + case searchCheckpointFrequency - 2: + l.need.Add(entrydb.FlagPadding | entrydb.FlagPadding2) + return nil + } + } + evt := newInitiatingEvent(l.logHash, l.execMsg != nil) + l.appendEntry(evt) + l.need.Remove(entrydb.FlagInitiatingEvent) + if l.execMsg == nil { + l.logsSince += 1 + } + return nil + } + if l.need.Any(entrydb.FlagExecutingLink) { + link, err := newExecutingLink(*l.execMsg) + if err != nil { + return fmt.Errorf("failed to create executing link: %w", err) + } + l.appendEntry(link) + l.need.Remove(entrydb.FlagExecutingLink) + return nil + } + if l.need.Any(entrydb.FlagExecutingCheck) { + l.appendEntry(newExecutingCheck(l.execMsg.Hash)) + l.need.Remove(entrydb.FlagExecutingCheck) + l.logsSince += 1 + return nil + } + return io.EOF +} + +// inferFull advances the queued entries held by the log context repeatedly +// until no more implied entries can be added +func (l *logContext) inferFull() error { + for i := 0; i < 10; i++ { + err := l.infer() + if err == nil { + continue + } + if err == io.EOF { // wrapped io.EOF does not count. + return nil + } else { + return err + } + } + panic("hit sanity limit") +} + +// forceBlock force-overwrites the state, to match the given sealed block as starting point (excl) +func (l *logContext) forceBlock(upd eth.BlockID, timestamp uint64) error { + if l.nextEntryIndex != 0 { + return errors.New("can only bootstrap on top of an empty state") + } + l.blockHash = types.TruncateHash(upd.Hash) + l.blockNum = upd.Number + l.timestamp = timestamp + l.logsSince = 0 + l.execMsg = nil + l.logHash = types.TruncatedHash{} + l.need = 0 + l.out = nil + return l.inferFull() // apply to the state as much as possible +} + +// SealBlock applies a block header on top of the current state. +// This seals the state; no further logs of this block may be added with ApplyLog. +func (l *logContext) SealBlock(parent common.Hash, upd eth.BlockID, timestamp uint64) error { + // If we don't have any entries yet, allow any block to start things off + if l.nextEntryIndex != 0 { + if err := l.inferFull(); err != nil { // ensure we can start applying + return err + } + if l.blockHash != types.TruncateHash(parent) { + return fmt.Errorf("%w: cannot apply block %s (parent %s) on top of %s", ErrConflict, upd, parent, l.blockHash) + } + if l.blockHash != (types.TruncatedHash{}) && l.blockNum+1 != upd.Number { + return fmt.Errorf("%w: cannot apply block %d on top of %d", ErrConflict, upd.Number, l.blockNum) + } + if l.timestamp > timestamp { + return fmt.Errorf("%w: block timestamp %d must be equal or larger than current timestamp %d", ErrConflict, timestamp, l.timestamp) + } + } + l.blockHash = types.TruncateHash(upd.Hash) + l.blockNum = upd.Number + l.timestamp = timestamp + l.logsSince = 0 + l.execMsg = nil + l.logHash = types.TruncatedHash{} + l.need.Add(entrydb.FlagSearchCheckpoint) + return l.inferFull() // apply to the state as much as possible +} + +// ApplyLog applies a log on top of the current state. +// The parent-block that the log comes after must be applied with ApplyBlock first. +func (l *logContext) ApplyLog(parentBlock eth.BlockID, logIdx uint32, logHash types.TruncatedHash, execMsg *types.ExecutingMessage) error { + if parentBlock == (eth.BlockID{}) { + return fmt.Errorf("genesis does not have logs: %w", ErrLogOutOfOrder) + } + if err := l.inferFull(); err != nil { // ensure we can start applying + return err + } + if !l.hasCompleteBlock() { + if l.blockNum == 0 { + return fmt.Errorf("%w: should not have logs in block 0", ErrLogOutOfOrder) + } else { + return errors.New("cannot append log before last known block is sealed") + } + } + // check parent block + if l.blockHash != types.TruncateHash(parentBlock.Hash) { + return fmt.Errorf("%w: log builds on top of block %s, but have block %s", ErrLogOutOfOrder, parentBlock, l.blockHash) + } + if l.blockNum != parentBlock.Number { + return fmt.Errorf("%w: log builds on top of block %d, but have block %d", ErrLogOutOfOrder, parentBlock.Number, l.blockNum) + } + // check if log fits on top. The length so far == the index of the next log. + if logIdx != l.logsSince { + return fmt.Errorf("%w: expected event index %d, cannot append %d", ErrLogOutOfOrder, l.logsSince, logIdx) + } + l.logHash = logHash + l.execMsg = execMsg + l.need.Add(entrydb.FlagInitiatingEvent) + if execMsg != nil { + l.need.Add(entrydb.FlagExecutingLink | entrydb.FlagExecutingCheck) + } + return l.inferFull() // apply to the state as much as possible +} diff --git a/op-supervisor/supervisor/backend/db/safety_checkers.go b/op-supervisor/supervisor/backend/db/safety_checkers.go index 3c92c1e808df..94f2925fe366 100644 --- a/op-supervisor/supervisor/backend/db/safety_checkers.go +++ b/op-supervisor/supervisor/backend/db/safety_checkers.go @@ -1,8 +1,10 @@ package db import ( + "errors" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/entrydb" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/heads" + "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/logs" backendTypes "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/types" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/types" ) @@ -132,11 +134,17 @@ func check( // exist at the blockNum and logIdx // have a hash that matches the provided hash (implicit in the Contains call), and // be less than or equal to the local head for the chain - exists, index, err := chainsDB.logDBs[chain].Contains(blockNum, logIdx, logHash) + index, err := chainsDB.logDBs[chain].Contains(blockNum, logIdx, logHash) if err != nil { + if errors.Is(err, logs.ErrFuture) { + return false // TODO + } + if errors.Is(err, logs.ErrConflict) { + return false // TODO + } return false } - return exists && index <= localHead + return index <= localHead } // Check checks if the log entry is safe, provided a local head for the chain diff --git a/op-supervisor/supervisor/backend/db/safety_checkers_test.go b/op-supervisor/supervisor/backend/db/safety_checkers_test.go index 0e815bdaccb5..d424b8d1645e 100644 --- a/op-supervisor/supervisor/backend/db/safety_checkers_test.go +++ b/op-supervisor/supervisor/backend/db/safety_checkers_test.go @@ -1,7 +1,8 @@ package db import ( - "fmt" + "errors" + "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/logs" "testing" "github.com/ethereum-optimism/optimism/op-service/testlog" @@ -115,7 +116,7 @@ func TestCheck(t *testing.T) { 1, 1, backendTypes.TruncatedHash{1, 2, 3}, - containsResponse{true, entrydb.EntryIdx(6), nil}, + containsResponse{entrydb.EntryIdx(6), nil}, true, }, { @@ -126,7 +127,7 @@ func TestCheck(t *testing.T) { 1, 1, backendTypes.TruncatedHash{1, 2, 3}, - containsResponse{true, entrydb.EntryIdx(3), nil}, + containsResponse{entrydb.EntryIdx(3), nil}, true, }, { @@ -137,7 +138,7 @@ func TestCheck(t *testing.T) { 1, 1, backendTypes.TruncatedHash{1, 2, 3}, - containsResponse{true, entrydb.EntryIdx(1), nil}, + containsResponse{entrydb.EntryIdx(1), nil}, true, }, { @@ -148,7 +149,7 @@ func TestCheck(t *testing.T) { 1, 1, backendTypes.TruncatedHash{1, 2, 3}, - containsResponse{false, entrydb.EntryIdx(1), nil}, + containsResponse{entrydb.EntryIdx(1), logs.ErrConflict}, false, }, { @@ -159,7 +160,7 @@ func TestCheck(t *testing.T) { 1, 1, backendTypes.TruncatedHash{1, 2, 3}, - containsResponse{true, entrydb.EntryIdx(100), nil}, + containsResponse{entrydb.EntryIdx(100), nil}, false, }, { @@ -170,7 +171,7 @@ func TestCheck(t *testing.T) { 1, 1, backendTypes.TruncatedHash{1, 2, 3}, - containsResponse{true, entrydb.EntryIdx(5), nil}, + containsResponse{entrydb.EntryIdx(5), nil}, false, }, { @@ -181,7 +182,7 @@ func TestCheck(t *testing.T) { 1, 1, backendTypes.TruncatedHash{1, 2, 3}, - containsResponse{true, entrydb.EntryIdx(3), nil}, + containsResponse{entrydb.EntryIdx(3), nil}, false, }, { @@ -192,7 +193,7 @@ func TestCheck(t *testing.T) { 1, 1, backendTypes.TruncatedHash{1, 2, 3}, - containsResponse{false, entrydb.EntryIdx(0), fmt.Errorf("error")}, + containsResponse{entrydb.EntryIdx(0), errors.New("error")}, false, }, } diff --git a/op-supervisor/supervisor/backend/source/chain.go b/op-supervisor/supervisor/backend/source/chain.go index f7fd31b202dc..c8fef89f8b83 100644 --- a/op-supervisor/supervisor/backend/source/chain.go +++ b/op-supervisor/supervisor/backend/source/chain.go @@ -26,7 +26,7 @@ type Metrics interface { type Storage interface { LogStorage DatabaseRewinder - LatestBlockNum(chainID types.ChainID) uint64 + LatestBlockNum(chainID types.ChainID) (num uint64, ok bool) } // ChainMonitor monitors a source L2 chain, retrieving the data required to populate the database and perform @@ -43,8 +43,13 @@ func NewChainMonitor(ctx context.Context, logger log.Logger, m Metrics, chainID return nil, err } + latest, ok := store.LatestBlockNum(chainID) + if !ok { + logger.Warn("") + } + startingHead := eth.L1BlockRef{ - Number: store.LatestBlockNum(chainID), + Number: latest, } processLogs := newLogProcessor(chainID, store) diff --git a/op-supervisor/supervisor/backend/source/log_processor.go b/op-supervisor/supervisor/backend/source/log_processor.go index 3fd96476d41f..1a23d149216a 100644 --- a/op-supervisor/supervisor/backend/source/log_processor.go +++ b/op-supervisor/supervisor/backend/source/log_processor.go @@ -5,17 +5,19 @@ import ( "errors" "fmt" + "github.com/ethereum/go-ethereum/common" + ethTypes "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/source/contracts" backendTypes "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/types" supTypes "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/types" - "github.com/ethereum/go-ethereum/common" - ethTypes "github.com/ethereum/go-ethereum/core/types" - "github.com/ethereum/go-ethereum/crypto" ) type LogStorage interface { - AddLog(chain supTypes.ChainID, logHash backendTypes.TruncatedHash, block eth.BlockID, timestamp uint64, logIdx uint32, execMsg *backendTypes.ExecutingMessage) error + SealBlock(chain supTypes.ChainID, parentHash common.Hash, block eth.BlockID, timestamp uint64) error + AddLog(chain supTypes.ChainID, logHash backendTypes.TruncatedHash, parentBlock eth.BlockID, logIdx uint32, execMsg *backendTypes.ExecutingMessage) error } type EventDecoder interface { @@ -53,13 +55,15 @@ func (p *logProcessor) ProcessLogs(_ context.Context, block eth.L1BlockRef, rcpt } // executing messages have multiple entries in the database // they should start with the initiating message and then include the execution - fmt.Println("p.chain", p.chain) - err = p.logStore.AddLog(p.chain, logHash, block.ID(), block.Time, uint32(l.Index), execMsg) + err = p.logStore.AddLog(p.chain, logHash, block.ParentID(), uint32(l.Index), execMsg) if err != nil { return fmt.Errorf("failed to add log %d from block %v: %w", l.Index, block.ID(), err) } } } + if err := p.logStore.SealBlock(p.chain, block.ParentHash, block.ID(), block.Time); err != nil { + return fmt.Errorf("failed to seal block %s: %w", block.ID(), err) + } return nil } diff --git a/op-supervisor/supervisor/backend/source/log_processor_test.go b/op-supervisor/supervisor/backend/source/log_processor_test.go index 5c65973ab4e4..01d274aa57ee 100644 --- a/op-supervisor/supervisor/backend/source/log_processor_test.go +++ b/op-supervisor/supervisor/backend/source/log_processor_test.go @@ -18,7 +18,12 @@ var logProcessorChainID = supTypes.ChainIDFromUInt64(4) func TestLogProcessor(t *testing.T) { ctx := context.Background() - block1 := eth.L1BlockRef{Number: 100, Hash: common.Hash{0x11}, Time: 1111} + block1 := eth.L1BlockRef{ + ParentHash: common.Hash{0x42}, + Number: 100, + Hash: common.Hash{0x11}, + Time: 1111, + } t.Run("NoOutputWhenLogsAreEmpty", func(t *testing.T) { store := &stubLogStorage{} processor := newLogProcessor(logProcessorChainID, store) @@ -59,30 +64,36 @@ func TestLogProcessor(t *testing.T) { err := processor.ProcessLogs(ctx, block1, rcpts) require.NoError(t, err) - expected := []storedLog{ + expectedLogs := []storedLog{ { - block: block1.ID(), - timestamp: block1.Time, - logIdx: 0, - logHash: logToLogHash(rcpts[0].Logs[0]), - execMsg: nil, + parent: block1.ParentID(), + logIdx: 0, + logHash: logToLogHash(rcpts[0].Logs[0]), + execMsg: nil, }, { - block: block1.ID(), - timestamp: block1.Time, - logIdx: 0, - logHash: logToLogHash(rcpts[0].Logs[1]), - execMsg: nil, + parent: block1.ParentID(), + logIdx: 0, + logHash: logToLogHash(rcpts[0].Logs[1]), + execMsg: nil, }, { + parent: block1.ParentID(), + logIdx: 0, + logHash: logToLogHash(rcpts[1].Logs[0]), + execMsg: nil, + }, + } + require.Equal(t, expectedLogs, store.logs) + + expectedBlocks := []storedSeal{ + { + parent: block1.ParentHash, block: block1.ID(), timestamp: block1.Time, - logIdx: 0, - logHash: logToLogHash(rcpts[1].Logs[0]), - execMsg: nil, }, } - require.Equal(t, expected, store.logs) + require.Equal(t, expectedBlocks, store.seals) }) t.Run("IncludeExecutingMessage", func(t *testing.T) { @@ -115,14 +126,22 @@ func TestLogProcessor(t *testing.T) { require.NoError(t, err) expected := []storedLog{ { + parent: block1.ParentID(), + logIdx: 0, + logHash: logToLogHash(rcpts[0].Logs[0]), + execMsg: &execMsg, + }, + } + require.Equal(t, expected, store.logs) + + expectedBlocks := []storedSeal{ + { + parent: block1.ParentHash, block: block1.ID(), timestamp: block1.Time, - logIdx: 0, - logHash: logToLogHash(rcpts[0].Logs[0]), - execMsg: &execMsg, }, } - require.Equal(t, expected, store.logs) + require.Equal(t, expectedBlocks, store.seals) }) } @@ -183,29 +202,46 @@ func TestToLogHash(t *testing.T) { } type stubLogStorage struct { - logs []storedLog + logs []storedLog + seals []storedSeal } -func (s *stubLogStorage) AddLog(chainID supTypes.ChainID, logHash backendTypes.TruncatedHash, block eth.BlockID, timestamp uint64, logIdx uint32, execMsg *backendTypes.ExecutingMessage) error { +func (s *stubLogStorage) SealBlock(chainID supTypes.ChainID, parentHash common.Hash, block eth.BlockID, timestamp uint64) error { if logProcessorChainID != chainID { return fmt.Errorf("chain id mismatch, expected %v but got %v", logProcessorChainID, chainID) } - s.logs = append(s.logs, storedLog{ + s.seals = append(s.seals, storedSeal{ + parent: parentHash, block: block, timestamp: timestamp, - logIdx: logIdx, - logHash: logHash, - execMsg: execMsg, }) return nil } -type storedLog struct { +func (s *stubLogStorage) AddLog(chainID supTypes.ChainID, logHash backendTypes.TruncatedHash, parentBlock eth.BlockID, logIdx uint32, execMsg *backendTypes.ExecutingMessage) error { + if logProcessorChainID != chainID { + return fmt.Errorf("chain id mismatch, expected %v but got %v", logProcessorChainID, chainID) + } + s.logs = append(s.logs, storedLog{ + parent: parentBlock, + logIdx: logIdx, + logHash: logHash, + execMsg: execMsg, + }) + return nil +} + +type storedSeal struct { + parent common.Hash block eth.BlockID timestamp uint64 - logIdx uint32 - logHash backendTypes.TruncatedHash - execMsg *backendTypes.ExecutingMessage +} + +type storedLog struct { + parent eth.BlockID + logIdx uint32 + logHash backendTypes.TruncatedHash + execMsg *backendTypes.ExecutingMessage } type EventDecoderFn func(*ethTypes.Log) (backendTypes.ExecutingMessage, error) From 6181703702fa5e66f844fb37db79d5d66f9018a8 Mon Sep 17 00:00:00 2001 From: protolambda Date: Fri, 20 Sep 2024 16:22:13 -0600 Subject: [PATCH 2/4] op-supervisor: fix lint and tests --- op-supervisor/supervisor/backend/backend.go | 9 +- op-supervisor/supervisor/backend/db/db.go | 6 +- .../supervisor/backend/db/db_test.go | 225 ++++++++++++------ .../supervisor/backend/db/logs/db.go | 33 ++- .../supervisor/backend/db/logs/db_test.go | 1 + .../supervisor/backend/db/safety_checkers.go | 1 + .../backend/db/safety_checkers_test.go | 8 +- 7 files changed, 189 insertions(+), 94 deletions(-) diff --git a/op-supervisor/supervisor/backend/backend.go b/op-supervisor/supervisor/backend/backend.go index 86a862f5b4ac..57b17e591059 100644 --- a/op-supervisor/supervisor/backend/backend.go +++ b/op-supervisor/supervisor/backend/backend.go @@ -4,14 +4,18 @@ import ( "context" "errors" "fmt" - "github.com/ethereum-optimism/optimism/op-service/eth" "io" "path/filepath" "sync/atomic" "time" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/common/hexutil" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum-optimism/optimism/op-service/client" "github.com/ethereum-optimism/optimism/op-service/dial" + "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-supervisor/config" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/heads" @@ -20,9 +24,6 @@ import ( backendTypes "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/types" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/frontend" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/types" - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/common/hexutil" - "github.com/ethereum/go-ethereum/log" ) type SupervisorBackend struct { diff --git a/op-supervisor/supervisor/backend/db/db.go b/op-supervisor/supervisor/backend/db/db.go index 8d5954a04fb2..9124a975279a 100644 --- a/op-supervisor/supervisor/backend/db/db.go +++ b/op-supervisor/supervisor/backend/db/db.go @@ -4,17 +4,18 @@ import ( "context" "errors" "fmt" - "github.com/ethereum/go-ethereum/common" "io" "time" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/entrydb" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/heads" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/logs" backendTypes "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/types" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/types" - "github.com/ethereum/go-ethereum/log" ) var ( @@ -188,6 +189,7 @@ func (db *ChainsDB) UpdateCrossHeadsForChain(chainID types.ChainID, checker Safe } // if we would exceed the local head, then abort if iter.NextIndex() > localHead { + xHead = localHead // clip to local head break } exec := iter.ExecMessage() diff --git a/op-supervisor/supervisor/backend/db/db_test.go b/op-supervisor/supervisor/backend/db/db_test.go index cd3c55f3e7b3..e3c4b75ad058 100644 --- a/op-supervisor/supervisor/backend/db/db_test.go +++ b/op-supervisor/supervisor/backend/db/db_test.go @@ -1,7 +1,16 @@ package db import ( - "fmt" + "errors" + "io" + "math/rand" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum-optimism/optimism/op-service/eth" "github.com/ethereum-optimism/optimism/op-service/testlog" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/entrydb" @@ -9,11 +18,6 @@ import ( "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/logs" backendTypes "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/types" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/types" - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/log" - "github.com/stretchr/testify/require" - "io" - "testing" ) func TestChainsDB_AddLog(t *testing.T) { @@ -65,6 +69,9 @@ func TestChainsDB_UpdateCrossHeads(t *testing.T) { // get default stubbed components logDB, checker, h := setupStubbedForUpdateHeads(chainID) + checker.numSafe = 1 + xSafe := checker.crossHeadForChain + // The ChainsDB is real, but uses only stubbed components db := NewChainsDB( map[types.ChainID]LogStorage{ @@ -72,14 +79,10 @@ func TestChainsDB_UpdateCrossHeads(t *testing.T) { &stubHeadStorage{h}, testlog.Logger(t, log.LevelDebug)) - // Update cross-heads is expected to: - // 1. get a last checkpoint iterator from the logDB (stubbed to be at 15) - // 2. progress the iterator to the next log (16) because the first safety check will pass - // 3. fail the second safety check - // 4. update the cross-heads to the last successful safety check (16) err := db.UpdateCrossHeads(checker) require.NoError(t, err) - require.Equal(t, entrydb.EntryIdx(16), checker.updated) + // found a safe executing message, and no new initiating messages + require.Equal(t, xSafe+1, checker.updated) } func TestChainsDB_UpdateCrossHeadsBeyondLocal(t *testing.T) { @@ -87,9 +90,11 @@ func TestChainsDB_UpdateCrossHeadsBeyondLocal(t *testing.T) { chainID := types.ChainIDFromUInt64(1) // get default stubbed components logDB, checker, h := setupStubbedForUpdateHeads(chainID) - // set the safety checker to pass 99 times, effeciively allowing all messages to be safe + // set the safety checker to pass 99 times, effectively allowing all messages to be safe checker.numSafe = 99 + startLocalSafe := checker.localHeadForChain + // The ChainsDB is real, but uses only stubbed components db := NewChainsDB( map[types.ChainID]LogStorage{ @@ -103,7 +108,7 @@ func TestChainsDB_UpdateCrossHeadsBeyondLocal(t *testing.T) { // 3. exceed the local head, and update the cross-head to the local head (40) err := db.UpdateCrossHeads(checker) require.NoError(t, err) - require.Equal(t, entrydb.EntryIdx(40), checker.updated) + require.Equal(t, startLocalSafe, checker.updated) } func TestChainsDB_UpdateCrossHeadsEOF(t *testing.T) { @@ -112,9 +117,10 @@ func TestChainsDB_UpdateCrossHeadsEOF(t *testing.T) { // get default stubbed components logDB, checker, h := setupStubbedForUpdateHeads(chainID) // set the log DB to return an EOF error when trying to get the next executing message - // after processing 10 messages as safe (with more messages available to be safe) - logDB.errOverload = io.EOF - logDB.errAfter = 10 + // after processing 10 message (with more messages available to be safe) + logDB.nextLogs = logDB.nextLogs[:checker.crossHeadForChain+11] + // This is a legacy test, the local head is further than the DB content... + checker.numSafe = 99 // The ChainsDB is real, but uses only stubbed components @@ -125,12 +131,11 @@ func TestChainsDB_UpdateCrossHeadsEOF(t *testing.T) { testlog.Logger(t, log.LevelDebug)) // Update cross-heads is expected to: - // 1. get a last checkpoint iterator from the logDB (stubbed to be at 15) - // 2. after processing 10 messages as safe, fail to find any executing messages (EOF) - // 3. update to the last successful safety check (25) without returning an error + // - process 10 logs as safe, 5 of which execute something + // - update cross-safe to what was there err := db.UpdateCrossHeads(checker) require.NoError(t, err) - require.Equal(t, entrydb.EntryIdx(25), checker.updated) + require.Equal(t, checker.crossHeadForChain+11, checker.updated) } func TestChainsDB_UpdateCrossHeadsError(t *testing.T) { @@ -140,8 +145,18 @@ func TestChainsDB_UpdateCrossHeadsError(t *testing.T) { logDB, checker, h := setupStubbedForUpdateHeads(chainID) // set the log DB to return an error when trying to get the next executing message // after processing 3 messages as safe (with more messages available to be safe) - logDB.errOverload = fmt.Errorf("some error") - logDB.errAfter = 3 + + executed := 0 + for i, e := range logDB.nextLogs { + if executed == 3 { + logDB.nextLogs[i].err = errors.New("some error") + } + if entrydb.EntryIdx(i) > checker.crossHeadForChain && e.execIdx >= 0 { + executed++ + } + } + + // everything is safe until error checker.numSafe = 99 // The ChainsDB is real, but uses only stubbed components @@ -167,8 +182,6 @@ func TestChainsDB_UpdateCrossHeadsError(t *testing.T) { // this isn't an issue for now, as all tests can modify the stubbed components directly after calling this function. // but readability and maintainability would be improved by making this function more configurable. func setupStubbedForUpdateHeads(chainID types.ChainID) (*stubLogDB, *stubChecker, *heads.Heads) { - // the checkpoint starts somewhere behind the last known cross-safe head - checkpoint := entrydb.EntryIdx(15) // the last known cross-safe head is at 20 cross := entrydb.EntryIdx(20) // the local head (the limit of the update) is at 40 @@ -177,15 +190,10 @@ func setupStubbedForUpdateHeads(chainID types.ChainID) (*stubLogDB, *stubChecker numExecutingMessages := 30 // number of safety checks that will pass before returning false numSafe := 1 - // number of calls to nextExecutingMessage before potentially returning an error - errAfter := 4 // set up stubbed logDB logDB := &stubLogDB{} - // the log DB will start the iterator at the checkpoint index - logDB.lastCheckpointBehind = &stubIterator{checkpoint, 0, nil} - // rig the log DB to return an error after a certain number of calls to NextExecutingMessage - logDB.errAfter = errAfter + // set up stubbed executing messages that the ChainsDB can pass to the checker logDB.executingMessages = []*backendTypes.ExecutingMessage{} for i := 0; i < numExecutingMessages; i++ { @@ -197,6 +205,31 @@ func setupStubbedForUpdateHeads(chainID types.ChainID) (*stubLogDB, *stubChecker }) } + rng := rand.New(rand.NewSource(123)) + blockNum := uint64(100) + logIndex := uint32(0) + executedCount := 0 + for i := entrydb.EntryIdx(0); i <= local; i++ { + var logHash backendTypes.TruncatedHash + rng.Read(logHash[:]) + + execIndex := -1 + // All the even messages have an executing message + if i%2 == 0 { + execIndex = rng.Intn(len(logDB.executingMessages)) + executedCount += 1 + } + var msgErr error + + logDB.nextLogs = append(logDB.nextLogs, nextLogResponse{ + blockNum: blockNum, + logIdx: logIndex, + evtHash: logHash, + err: msgErr, + execIdx: execIndex, + }) + } + // set up stubbed checker checker := &stubChecker{ localHeadForChain: local, @@ -269,43 +302,100 @@ func (s *stubHeadStorage) Current() *heads.Heads { type nextLogResponse struct { blockNum uint64 - logIdx uint32 - evtHash backendTypes.TruncatedHash - err error + + logIdx uint32 + + evtHash backendTypes.TruncatedHash + + err error + + // -1 if not executing + execIdx int } + type stubIterator struct { - index entrydb.EntryIdx - nextLogIndex int - nextLogs []nextLogResponse + index entrydb.EntryIdx + + db *stubLogDB +} + +func (s *stubIterator) End() error { + return nil // only used for DB-loading. The stub is already loaded } -func (s *stubIterator) NextLog() (uint64, uint32, backendTypes.TruncatedHash, error) { - if s.nextLogIndex >= len(s.nextLogs) { - return 0, 0, backendTypes.TruncatedHash{}, io.EOF +func (s *stubIterator) NextInitMsg() error { + s.index += 1 + if s.index >= entrydb.EntryIdx(len(s.db.nextLogs)) { + return io.EOF + } + e := s.db.nextLogs[s.index] + return e.err +} + +func (s *stubIterator) NextExecMsg() error { + for { + s.index += 1 + if s.index >= entrydb.EntryIdx(len(s.db.nextLogs)) { + return io.EOF + } + e := s.db.nextLogs[s.index] + if e.err != nil { + return e.err + } + if e.execIdx >= 0 { + return nil + } } - r := s.nextLogs[s.nextLogIndex] - s.nextLogIndex++ - return r.blockNum, r.logIdx, r.evtHash, r.err } -func (s *stubIterator) Index() entrydb.EntryIdx { - return s.index +func (s *stubIterator) NextBlock() error { + panic("not yet supported") } -func (s *stubIterator) ExecMessage() (backendTypes.ExecutingMessage, error) { - panic("not implemented") + +func (s *stubIterator) NextIndex() entrydb.EntryIdx { + return s.index + 1 +} + +func (s *stubIterator) SealedBlock() (hash backendTypes.TruncatedHash, num uint64, ok bool) { + panic("not yet supported") +} + +func (s *stubIterator) InitMessage() (hash backendTypes.TruncatedHash, logIndex uint32, ok bool) { + if s.index < 0 { + return backendTypes.TruncatedHash{}, 0, false + } + if s.index >= entrydb.EntryIdx(len(s.db.nextLogs)) { + return backendTypes.TruncatedHash{}, 0, false + } + e := s.db.nextLogs[s.index] + return e.evtHash, e.logIdx, true +} + +func (s *stubIterator) ExecMessage() *backendTypes.ExecutingMessage { + if s.index < 0 { + return nil + } + if s.index >= entrydb.EntryIdx(len(s.db.nextLogs)) { + return nil + } + e := s.db.nextLogs[s.index] + if e.execIdx < 0 { + return nil + } + return s.db.executingMessages[e.execIdx] } +var _ logs.Iterator = (*stubIterator)(nil) + type stubLogDB struct { - addLogCalls int - sealBlockCalls int - headBlockNum uint64 - emIndex int - executingMessages []*backendTypes.ExecutingMessage - nextLogs []nextLogResponse - lastCheckpointBehind *stubIterator - errOverload error - errAfter int - containsResponse containsResponse + addLogCalls int + sealBlockCalls int + headBlockNum uint64 + + executingMessages []*backendTypes.ExecutingMessage + nextLogs []nextLogResponse + + containsResponse containsResponse } func (s *stubLogDB) AddLog(logHash backendTypes.TruncatedHash, parentBlock eth.BlockID, logIdx uint32, execMsg *backendTypes.ExecutingMessage) error { @@ -327,27 +417,14 @@ func (s *stubLogDB) FindSealedBlock(block eth.BlockID) (nextEntry entrydb.EntryI } func (s *stubLogDB) IteratorStartingAt(i entrydb.EntryIdx) (logs.Iterator, error) { - //TODO implement me - panic("implement me") + return &stubIterator{ + index: i - 1, + db: s, + }, nil } var _ LogStorage = (*stubLogDB)(nil) -// -//func (s *stubLogDB) NextExecutingMessage(i logs.Iterator) (backendTypes.ExecutingMessage, error) { -// // if error overload is set, return it to simulate a failure condition -// if s.errOverload != nil && s.emIndex >= s.errAfter { -// return backendTypes.ExecutingMessage{}, s.errOverload -// } -// // increment the iterator to mark advancement -// i.(*stubIterator).index += 1 -// // return the next executing message -// m := *s.executingMessages[s.emIndex] -// // and increment to the next message for the next call -// s.emIndex++ -// return m, nil -//} - type containsResponse struct { index entrydb.EntryIdx err error diff --git a/op-supervisor/supervisor/backend/db/logs/db.go b/op-supervisor/supervisor/backend/db/logs/db.go index b788c5db5659..8e3dfac1df20 100644 --- a/op-supervisor/supervisor/backend/db/logs/db.go +++ b/op-supervisor/supervisor/backend/db/logs/db.go @@ -155,9 +155,28 @@ func (db *DB) IteratorStartingAt(i entrydb.EntryIdx) (Iterator, error) { if i > db.lastEntryContext.nextEntryIndex { return nil, ErrFuture } - // TODO this iterator is semi-broken; - // inferred entries will not be added if starting from an incomplete block or log - return db.newIterator(i), nil + // TODO(#12031): Workaround while we not have IteratorStartingAt(heads.HeadPointer): + // scroll back from the index, to find block info. + idx := i + for ; idx >= 0; i-- { + entry, err := db.store.Read(idx) + if err != nil { + return nil, err + } + if entry.Type() == entrydb.TypeSearchCheckpoint { + break + } + if idx == 0 { + return nil, fmt.Errorf("empty DB, no block entry, cannot start at %d", i) + } + } + iter := db.newIterator(idx) + for iter.NextIndex() < i { + if _, err := iter.next(); err != nil { + return nil, errors.New("failed to process back up to the head pointer") + } + } + return iter, nil } // FindSealedBlock finds the requested block, to check if it exists, @@ -479,14 +498,6 @@ func (db *DB) readSearchCheckpoint(entryIdx entrydb.EntryIdx) (searchCheckpoint, return newSearchCheckpointFromEntry(data) } -func (db *DB) readCanonicalHash(entryIdx entrydb.EntryIdx) (canonicalHash, error) { - data, err := db.store.Read(entryIdx) - if err != nil { - return canonicalHash{}, fmt.Errorf("failed to read entry %v: %w", entryIdx, err) - } - return newCanonicalHashFromEntry(data) -} - func (db *DB) Close() error { return db.store.Close() } diff --git a/op-supervisor/supervisor/backend/db/logs/db_test.go b/op-supervisor/supervisor/backend/db/logs/db_test.go index 6bde3ef2c04a..d2dbced9f393 100644 --- a/op-supervisor/supervisor/backend/db/logs/db_test.go +++ b/op-supervisor/supervisor/backend/db/logs/db_test.go @@ -162,6 +162,7 @@ func TestAddLog(t *testing.T) { require.NoError(t, err) bl17 := eth.BlockID{Hash: createHash(17), Number: 17} err = db.SealBlock(bl16.Hash, bl17, 5003) + require.NoError(t, err) }, func(t *testing.T, db *DB, m *stubMetrics) { require.EqualValues(t, 2+2+1+1+2+1+1+2, m.entryCount, "should not output new searchCheckpoint for every block") diff --git a/op-supervisor/supervisor/backend/db/safety_checkers.go b/op-supervisor/supervisor/backend/db/safety_checkers.go index 94f2925fe366..1c1b53db1216 100644 --- a/op-supervisor/supervisor/backend/db/safety_checkers.go +++ b/op-supervisor/supervisor/backend/db/safety_checkers.go @@ -2,6 +2,7 @@ package db import ( "errors" + "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/entrydb" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/heads" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/logs" diff --git a/op-supervisor/supervisor/backend/db/safety_checkers_test.go b/op-supervisor/supervisor/backend/db/safety_checkers_test.go index d424b8d1645e..667cd8d46607 100644 --- a/op-supervisor/supervisor/backend/db/safety_checkers_test.go +++ b/op-supervisor/supervisor/backend/db/safety_checkers_test.go @@ -2,16 +2,18 @@ package db import ( "errors" - "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/logs" "testing" + "github.com/stretchr/testify/require" + + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum-optimism/optimism/op-service/testlog" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/entrydb" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/heads" + "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/db/logs" backendTypes "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/backend/types" "github.com/ethereum-optimism/optimism/op-supervisor/supervisor/types" - "github.com/ethereum/go-ethereum/log" - "github.com/stretchr/testify/require" ) // TestHeadsForChain tests the heads for a chain, From e18186e81fca5846aca4852e96ebc409c3c49286 Mon Sep 17 00:00:00 2001 From: protolambda Date: Fri, 20 Sep 2024 16:40:11 -0600 Subject: [PATCH 3/4] op-supervisor: minor fixes, logging --- op-e2e/interop/supersystem.go | 3 ++- op-supervisor/supervisor/backend/db/db.go | 6 ++++-- op-supervisor/supervisor/backend/db/logs/db.go | 7 +++++-- op-supervisor/supervisor/backend/source/chain_processor.go | 2 +- 4 files changed, 12 insertions(+), 6 deletions(-) diff --git a/op-e2e/interop/supersystem.go b/op-e2e/interop/supersystem.go index 58c0d4f4370f..ffa91bef97f3 100644 --- a/op-e2e/interop/supersystem.go +++ b/op-e2e/interop/supersystem.go @@ -416,7 +416,8 @@ func (s *interopE2ESystem) newL2(id string, l2Out *interopgen.L2Output) l2Set { // prepareSupervisor creates a new supervisor for the system func (s *interopE2ESystem) prepareSupervisor() *supervisor.SupervisorService { - logger := s.logger.New("role", "supervisor") + // Be verbose with op-supervisor, it's in early test phase + logger := testlog.Logger(s.t, log.LevelDebug).New("role", "supervisor") cfg := supervisorConfig.Config{ MetricsConfig: metrics.CLIConfig{ Enabled: false, diff --git a/op-supervisor/supervisor/backend/db/db.go b/op-supervisor/supervisor/backend/db/db.go index 9124a975279a..ac8c5f506ca5 100644 --- a/op-supervisor/supervisor/backend/db/db.go +++ b/op-supervisor/supervisor/backend/db/db.go @@ -190,6 +190,7 @@ func (db *ChainsDB) UpdateCrossHeadsForChain(chainID types.ChainID, checker Safe // if we would exceed the local head, then abort if iter.NextIndex() > localHead { xHead = localHead // clip to local head + updated = localHead != xHead break } exec := iter.ExecMessage() @@ -209,7 +210,6 @@ func (db *ChainsDB) UpdateCrossHeadsForChain(chainID types.ChainID, checker Safe xHead = iter.NextIndex() updated = true } - // have the checker create an update to the x-head in question, and apply that update err = db.heads.Apply(checker.Update(chainID, xHead)) if err != nil { @@ -219,8 +219,10 @@ func (db *ChainsDB) UpdateCrossHeadsForChain(chainID types.ChainID, checker Safe // this allows for the maintenance loop to handle cascading updates // instead of waiting for the next scheduled update if updated { - db.logger.Debug("heads were updated, requesting maintenance") + db.logger.Info("Promoting cross-head", "head", xHead, "safety-level", checker.SafetyLevel()) db.RequestMaintenance() + } else { + db.logger.Info("No cross-head update", "head", xHead, "safety-level", checker.SafetyLevel()) } return nil } diff --git a/op-supervisor/supervisor/backend/db/logs/db.go b/op-supervisor/supervisor/backend/db/logs/db.go index 8e3dfac1df20..1a2d8c8adfea 100644 --- a/op-supervisor/supervisor/backend/db/logs/db.go +++ b/op-supervisor/supervisor/backend/db/logs/db.go @@ -161,6 +161,9 @@ func (db *DB) IteratorStartingAt(i entrydb.EntryIdx) (Iterator, error) { for ; idx >= 0; i-- { entry, err := db.store.Read(idx) if err != nil { + if errors.Is(err, io.EOF) { + continue // traverse to when we did have blocks + } return nil, err } if entry.Type() == entrydb.TypeSearchCheckpoint { @@ -310,11 +313,11 @@ func (db *DB) newIteratorAt(blockNum uint64, logIndex uint32) (*iterator, error) if _, n, _ := iter.SealedBlock(); n == blockNum { // we may already have it exactly break } - if err := iter.NextBlock(); err == io.EOF { + if err := iter.NextBlock(); errors.Is(err, ErrFuture) { db.log.Trace("ran out of data, could not find block", "nextIndex", iter.NextIndex(), "target", blockNum) return nil, ErrFuture } else if err != nil { - db.log.Error("failed to read next block", "nextIndex", iter.NextIndex(), "target", blockNum) + db.log.Error("failed to read next block", "nextIndex", iter.NextIndex(), "target", blockNum, "err", err) return nil, err } h, num, ok := iter.SealedBlock() diff --git a/op-supervisor/supervisor/backend/source/chain_processor.go b/op-supervisor/supervisor/backend/source/chain_processor.go index 714d9f2e4a6d..0a42da1556a0 100644 --- a/op-supervisor/supervisor/backend/source/chain_processor.go +++ b/op-supervisor/supervisor/backend/source/chain_processor.go @@ -49,7 +49,7 @@ func NewChainProcessor(log log.Logger, client BlockByNumberSource, chain types.C } func (s *ChainProcessor) OnNewHead(ctx context.Context, head eth.L1BlockRef) { - s.log.Debug("Processing chain", "chain", s.chain, "head", head) + s.log.Debug("Processing chain", "chain", s.chain, "head", head, "last", s.lastBlock) if head.Number <= s.lastBlock.Number { s.log.Info("head is not newer than last processed block", "head", head, "lastBlock", s.lastBlock) return From 702b610ca961e7832c9b5fc575cfa953f4eb9aca Mon Sep 17 00:00:00 2001 From: protolambda Date: Fri, 20 Sep 2024 17:21:17 -0600 Subject: [PATCH 4/4] op-supervisor: fix semgrep --- op-supervisor/supervisor/backend/db/db_test.go | 2 +- op-supervisor/supervisor/backend/db/safety_checkers.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/op-supervisor/supervisor/backend/db/db_test.go b/op-supervisor/supervisor/backend/db/db_test.go index e3c4b75ad058..eb10afd7f641 100644 --- a/op-supervisor/supervisor/backend/db/db_test.go +++ b/op-supervisor/supervisor/backend/db/db_test.go @@ -3,7 +3,7 @@ package db import ( "errors" "io" - "math/rand" + "math/rand" // nosemgrep "testing" "github.com/stretchr/testify/require" diff --git a/op-supervisor/supervisor/backend/db/safety_checkers.go b/op-supervisor/supervisor/backend/db/safety_checkers.go index 1c1b53db1216..3ed297a60c3e 100644 --- a/op-supervisor/supervisor/backend/db/safety_checkers.go +++ b/op-supervisor/supervisor/backend/db/safety_checkers.go @@ -138,10 +138,10 @@ func check( index, err := chainsDB.logDBs[chain].Contains(blockNum, logIdx, logHash) if err != nil { if errors.Is(err, logs.ErrFuture) { - return false // TODO + return false // TODO(#12031) } if errors.Is(err, logs.ErrConflict) { - return false // TODO + return false // TODO(#12031) } return false }