Skip to content

Commit

Permalink
Make LoadBlock also return block metadata (tendermint#1557)
Browse files Browse the repository at this point in the history
* store: return metadata with LoadBlock

Change the signature of the LoadBlock and LoadBlockByHash
methods to also return block metadata.
Reason: the main implementation fetches block metadata in process of
retrieving the block from parts.
Other implementations should likely do the same.

* state: re-generate the BlockStore mock

* Update tests to LoadBlock changes

In mocks, have LoadBlock return also a `BlockMeta` value.

* Changelog entry for tendermint#1557

* inspect: fix up test expectations for LoadBlock

* consensus: fix off-by-one mistake in mock

Co-authored-by: Anton Kaliaev <anton.kalyaev@gmail.com>

---------

Co-authored-by: Andy Nogueira <me@andynogueira.dev>
Co-authored-by: Anton Kaliaev <anton.kalyaev@gmail.com>
  • Loading branch information
3 people authored Nov 8, 2023
1 parent cf55cff commit 83172f7
Show file tree
Hide file tree
Showing 17 changed files with 144 additions and 90 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,2 @@
- `[store]` Make the `LoadBlock` method also return block metadata
([\#1556](https://github.com/cometbft/cometbft/issues/1556))
2 changes: 1 addition & 1 deletion blocksync/reactor.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,7 +190,7 @@ func (bcR *Reactor) RemovePeer(peer p2p.Peer, _ interface{}) {
// respondToPeer loads a block and sends it to the requesting peer,
// if we have it. Otherwise, we'll respond saying we don't have it.
func (bcR *Reactor) respondToPeer(msg *bcproto.BlockRequest, src p2p.Peer) (queued bool) {
block := bcR.store.LoadBlock(msg.Height)
block, _ := bcR.store.LoadBlock(msg.Height)
if block == nil {
bcR.Logger.Info("Peer asking for a block we don't have", "src", src, "height", msg.Height)
return src.TrySend(p2p.Envelope{
Expand Down
2 changes: 1 addition & 1 deletion blocksync/reactor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -215,7 +215,7 @@ func TestNoBlockResponse(t *testing.T) {
assert.Equal(t, maxBlockHeight, reactorPairs[0].reactor.store.Height())

for _, tt := range tests {
block := reactorPairs[1].reactor.store.LoadBlock(tt.height)
block, _ := reactorPairs[1].reactor.store.LoadBlock(tt.height)
if tt.existent {
assert.True(t, block != nil)
} else {
Expand Down
2 changes: 1 addition & 1 deletion cmd/cometbft/commands/reindex_event.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,7 @@ func eventReIndex(cmd *cobra.Command, args eventReIndexArgs) error {
case <-cmd.Context().Done():
return fmt.Errorf("event re-index terminated at height %d: %w", height, cmd.Context().Err())
default:
block := args.blockStore.LoadBlock(height)
block, _ := args.blockStore.LoadBlock(height)
if block == nil {
return fmt.Errorf("not able to load block at height %d from the blockstore", height)
}
Expand Down
6 changes: 3 additions & 3 deletions cmd/cometbft/commands/reindex_event_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,9 +135,9 @@ func TestReIndexEvent(t *testing.T) {
mockBlockStore.
On("Base").Return(base).
On("Height").Return(height).
On("LoadBlock", base).Return(nil).Once().
On("LoadBlock", base).Return(&types.Block{Data: types.Data{Txs: types.Txs{make(types.Tx, 1)}}}).
On("LoadBlock", height).Return(&types.Block{Data: types.Data{Txs: types.Txs{make(types.Tx, 1)}}})
On("LoadBlock", base).Return(nil, nil).Once().
On("LoadBlock", base).Return(&types.Block{Data: types.Data{Txs: types.Txs{make(types.Tx, 1)}}}, &types.BlockMeta{}).
On("LoadBlock", height).Return(&types.Block{Data: types.Data{Txs: types.Txs{make(types.Tx, 1)}}}, &types.BlockMeta{})

abciResp := &abcitypes.ResponseFinalizeBlock{
TxResults: []*abcitypes.ExecTxResult{
Expand Down
17 changes: 6 additions & 11 deletions consensus/replay.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,6 @@ func (cs *State) readReplayMessage(msg *TimedWALMessage, newStepSub types.Subscr
// Replay only those messages since the last block. `timeoutRoutine` should
// run concurrently to read off tickChan.
func (cs *State) catchupReplay(csHeight int64) error {

// Set replayMode to true so we don't log signing errors.
cs.replayMode = true
defer func() { cs.replayMode = false }()
Expand Down Expand Up @@ -210,8 +209,8 @@ type Handshaker struct {
}

func NewHandshaker(stateStore sm.Store, state sm.State,
store sm.BlockStore, genDoc *types.GenesisDoc) *Handshaker {

store sm.BlockStore, genDoc *types.GenesisDoc,
) *Handshaker {
return &Handshaker{
stateStore: stateStore,
initialState: state,
Expand Down Expand Up @@ -240,7 +239,6 @@ func (h *Handshaker) NBlocks() int {

// TODO: retry the handshake/replay if it fails ?
func (h *Handshaker) Handshake(ctx context.Context, proxyApp proxy.AppConns) error {

// Handshake is done via ABCI Info on the query conn.
res, err := proxyApp.Query().Info(ctx, proxy.RequestInfo)
if err != nil {
Expand Down Expand Up @@ -393,13 +391,11 @@ func (h *Handshaker) ReplayBlocks(
if appBlockHeight < storeBlockHeight {
// the app is behind, so replay blocks, but no need to go through WAL (state is already synced to store)
return h.replayBlocks(ctx, state, proxyApp, appBlockHeight, storeBlockHeight, false)

} else if appBlockHeight == storeBlockHeight {
// We're good!
assertAppHashEqualsOneFromState(appHash, state)
return appHash, nil
}

} else if storeBlockHeight == stateBlockHeight+1 {
// We saved the block in the store but haven't updated the state,
// so we'll need to replay a block using the WAL.
Expand Down Expand Up @@ -436,7 +432,6 @@ func (h *Handshaker) ReplayBlocks(
state, err = h.replayBlock(state, storeBlockHeight, mockApp)
return state.AppHash, err
}

}

panic(fmt.Sprintf("uncovered case! appHeight: %d, storeHeight: %d, stateHeight: %d",
Expand All @@ -449,7 +444,8 @@ func (h *Handshaker) replayBlocks(
proxyApp proxy.AppConns,
appBlockHeight,
storeBlockHeight int64,
mutateState bool) ([]byte, error) {
mutateState bool,
) ([]byte, error) {
// App is further behind than it should be, so we need to replay blocks.
// We replay all blocks from appBlockHeight+1.
//
Expand Down Expand Up @@ -478,7 +474,7 @@ func (h *Handshaker) replayBlocks(
}

h.logger.Info("Applying block", "height", i)
block := h.store.LoadBlock(i)
block, _ := h.store.LoadBlock(i)
// Extra check to ensure the app was not changed in a way it shouldn't have.
if len(appHash) > 0 {
assertAppHashEqualsOneFromBlock(appHash, block)
Expand Down Expand Up @@ -507,8 +503,7 @@ func (h *Handshaker) replayBlocks(

// ApplyBlock on the proxyApp with the last block.
func (h *Handshaker) replayBlock(state sm.State, height int64, proxyApp proxy.AppConnConsensus) (sm.State, error) {
block := h.store.LoadBlock(height)
meta := h.store.LoadBlockMeta(height)
block, meta := h.store.LoadBlock(height)

// Use stubs for both mempool and evidence pool since no transactions nor
// evidence are needed here - block already exists.
Expand Down
21 changes: 13 additions & 8 deletions consensus/replay_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -554,7 +554,8 @@ func setupChainWithChangingValidators(t *testing.T, name string, nBlocks int) (*
chain := []*types.Block{}
extCommits := []*types.ExtendedCommit{}
for i := 1; i <= nBlocks; i++ {
chain = append(chain, css[0].blockStore.LoadBlock(int64(i)))
block, _ := css[0].blockStore.LoadBlock(int64(i))
chain = append(chain, block)
extCommits = append(extCommits, css[0].blockStore.LoadBlockExtendedCommit(int64(i)))
}
return config, chain, extCommits, genesisState
Expand Down Expand Up @@ -1183,13 +1184,17 @@ func newMockBlockStore(t *testing.T, config *cfg.Config, params types.ConsensusP
}
}

func (bs *mockBlockStore) Height() int64 { return int64(len(bs.chain)) }
func (bs *mockBlockStore) Base() int64 { return bs.base }
func (bs *mockBlockStore) Size() int64 { return bs.Height() - bs.Base() + 1 }
func (bs *mockBlockStore) LoadBaseMeta() *types.BlockMeta { return bs.LoadBlockMeta(bs.base) }
func (bs *mockBlockStore) LoadBlock(height int64) *types.Block { return bs.chain[height-1] }
func (bs *mockBlockStore) LoadBlockByHash([]byte) *types.Block {
return bs.chain[int64(len(bs.chain))-1]
func (bs *mockBlockStore) Height() int64 { return int64(len(bs.chain)) }
func (bs *mockBlockStore) Base() int64 { return bs.base }
func (bs *mockBlockStore) Size() int64 { return bs.Height() - bs.Base() + 1 }
func (bs *mockBlockStore) LoadBaseMeta() *types.BlockMeta { return bs.LoadBlockMeta(bs.base) }
func (bs *mockBlockStore) LoadBlock(height int64) (*types.Block, *types.BlockMeta) {
return bs.chain[height-1], bs.LoadBlockMeta(height)
}

func (bs *mockBlockStore) LoadBlockByHash([]byte) (*types.Block, *types.BlockMeta) {
height := int64(len(bs.chain))
return bs.chain[height-1], bs.LoadBlockMeta(height)
}
func (bs *mockBlockStore) LoadBlockMetaByHash([]byte) *types.BlockMeta { return nil }
func (bs *mockBlockStore) LoadBlockMeta(height int64) *types.BlockMeta {
Expand Down
11 changes: 4 additions & 7 deletions inspect/inspect_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,8 +67,7 @@ func TestBlock(t *testing.T) {
blockStoreMock := &statemocks.BlockStore{}
blockStoreMock.On("Height").Return(testHeight)
blockStoreMock.On("Base").Return(int64(0))
blockStoreMock.On("LoadBlockMeta", testHeight).Return(&types.BlockMeta{})
blockStoreMock.On("LoadBlock", testHeight).Return(testBlock)
blockStoreMock.On("LoadBlock", testHeight).Return(testBlock, &types.BlockMeta{})
blockStoreMock.On("Close").Return(nil)

txIndexerMock := &txindexmocks.TxIndexer{}
Expand Down Expand Up @@ -361,15 +360,14 @@ func TestBlockByHash(t *testing.T) {
stateStoreMock.On("Close").Return(nil)
blockStoreMock := &statemocks.BlockStore{}
blockStoreMock.On("Close").Return(nil)
blockStoreMock.On("LoadBlockMeta", testHeight).Return(&types.BlockMeta{
blockStoreMock.On("LoadBlockByHash", testHash).Return(testBlock, &types.BlockMeta{
BlockID: types.BlockID{
Hash: testHash,
},
Header: types.Header{
Height: testHeight,
},
}, nil)
blockStoreMock.On("LoadBlockByHash", testHash).Return(testBlock, nil)
txIndexerMock := &txindexmocks.TxIndexer{}
blkIdxMock := &indexermocks.BlockIndexer{}
rpcConfig := config.TestRPCConfig()
Expand Down Expand Up @@ -527,12 +525,11 @@ func TestBlockSearch(t *testing.T) {
Header: types.Header{
Height: testHeight,
},
}, nil)
blockStoreMock.On("LoadBlockMeta", testHeight).Return(&types.BlockMeta{
}, &types.BlockMeta{
BlockID: types.BlockID{
Hash: testBlockHash,
},
})
}, nil)
blkIdxMock.On("Search", mock.Anything,
mock.MatchedBy(func(q *query.Query) bool { return testQuery == q.String() })).
Return([]int64{testHeight}, nil)
Expand Down
24 changes: 9 additions & 15 deletions rpc/core/blocks.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,8 +127,7 @@ func (env *Environment) Block(_ *rpctypes.Context, heightPtr *int64) (*ctypes.Re
return nil, err
}

block := env.BlockStore.LoadBlock(height)
blockMeta := env.BlockStore.LoadBlockMeta(height)
block, blockMeta := env.BlockStore.LoadBlock(height)
if blockMeta == nil {
return &ctypes.ResultBlock{BlockID: types.BlockID{}, Block: block}, nil
}
Expand All @@ -138,12 +137,10 @@ func (env *Environment) Block(_ *rpctypes.Context, heightPtr *int64) (*ctypes.Re
// BlockByHash gets block by hash.
// More: https://docs.cometbft.com/main/rpc/#/Info/block_by_hash
func (env *Environment) BlockByHash(_ *rpctypes.Context, hash []byte) (*ctypes.ResultBlock, error) {
block := env.BlockStore.LoadBlockByHash(hash)
if block == nil {
block, blockMeta := env.BlockStore.LoadBlockByHash(hash)
if blockMeta == nil {
return &ctypes.ResultBlock{BlockID: types.BlockID{}, Block: nil}, nil
}
// If block is not nil, then blockMeta can't be nil.
blockMeta := env.BlockStore.LoadBlockMeta(block.Height)
return &ctypes.ResultBlock{BlockID: blockMeta.BlockID, Block: block}, nil
}

Expand Down Expand Up @@ -250,15 +247,12 @@ func (env *Environment) BlockSearch(

apiResults := make([]*ctypes.ResultBlock, 0, pageSize)
for i := skipCount; i < skipCount+pageSize; i++ {
block := env.BlockStore.LoadBlock(results[i])
if block != nil {
blockMeta := env.BlockStore.LoadBlockMeta(block.Height)
if blockMeta != nil {
apiResults = append(apiResults, &ctypes.ResultBlock{
Block: block,
BlockID: blockMeta.BlockID,
})
}
block, blockMeta := env.BlockStore.LoadBlock(results[i])
if blockMeta != nil {
apiResults = append(apiResults, &ctypes.ResultBlock{
Block: block,
BlockID: blockMeta.BlockID,
})
}
}

Expand Down
4 changes: 2 additions & 2 deletions rpc/core/tx.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ func (env *Environment) Tx(_ *rpctypes.Context, hash []byte, prove bool) (*ctype

var proof types.TxProof
if prove {
block := env.BlockStore.LoadBlock(r.Height)
block, _ := env.BlockStore.LoadBlock(r.Height)
proof = block.Data.Txs.Proof(int(r.Index))
}

Expand Down Expand Up @@ -113,7 +113,7 @@ func (env *Environment) TxSearch(

var proof types.TxProof
if prove {
block := env.BlockStore.LoadBlock(r.Height)
block, _ := env.BlockStore.LoadBlock(r.Height)
proof = block.Data.Txs.Proof(int(r.Index))
}

Expand Down
3 changes: 1 addition & 2 deletions rpc/grpc/server/services/blockservice/service.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ func (s *blockServiceServer) getBlock(height int64, logger log.Logger) (*ptypes.
return nil, nil, status.Error(codes.Internal, "Internal server error - see logs for details")
}

block := s.store.LoadBlock(height)
block, blockMeta := s.store.LoadBlock(height)
if block == nil {
return nil, nil, status.Errorf(codes.NotFound, fmt.Sprintf("Block not found for height %d", height))
}
Expand All @@ -85,7 +85,6 @@ func (s *blockServiceServer) getBlock(height int64, logger log.Logger) (*ptypes.
return nil, nil, status.Errorf(codes.Internal, fmt.Sprintf("Failed to load block from store (see logs for trace ID: %s)", traceID))
}

blockMeta := s.store.LoadBlockMeta(height)
if blockMeta == nil {
logger.Error("Failed to load block meta when block was successfully loaded", "height", height)
return nil, nil, status.Error(codes.Internal, "Internal server error - see logs for details")
Expand Down
32 changes: 28 additions & 4 deletions state/mocks/block_store.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

4 changes: 2 additions & 2 deletions state/services.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,14 +22,14 @@ type BlockStore interface {

LoadBaseMeta() *types.BlockMeta
LoadBlockMeta(height int64) *types.BlockMeta
LoadBlock(height int64) *types.Block
LoadBlock(height int64) (*types.Block, *types.BlockMeta)

SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit)
SaveBlockWithExtendedCommit(block *types.Block, blockParts *types.PartSet, seenCommit *types.ExtendedCommit)

PruneBlocks(height int64, state State) (uint64, int64, error)

LoadBlockByHash(hash []byte) *types.Block
LoadBlockByHash(hash []byte) (*types.Block, *types.BlockMeta)
LoadBlockMetaByHash(hash []byte) *types.BlockMeta
LoadBlockPart(height int64, index int) *types.Part

Expand Down
Loading

0 comments on commit 83172f7

Please sign in to comment.