Skip to content

Commit

Permalink
add contexts to SaveBlock and LoadBlock
Browse files Browse the repository at this point in the history
  • Loading branch information
evan-forbes committed Jun 3, 2021
1 parent 61f2886 commit 04ce46b
Show file tree
Hide file tree
Showing 12 changed files with 82 additions and 54 deletions.
5 changes: 3 additions & 2 deletions blockchain/v0/reactor.go
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package v0

import (
"context"
"fmt"
"reflect"
"time"
Expand Down Expand Up @@ -178,7 +179,7 @@ func (bcR *BlockchainReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
func (bcR *BlockchainReactor) respondToPeer(msg *bcproto.BlockRequest,
src p2p.Peer) (queued bool) {

block, err := bcR.store.LoadBlock(msg.Height)
block, err := bcR.store.LoadBlock(context.TODO(), msg.Height)
if err != nil {
bcR.Logger.Error("failure to load block", "err", err)
return false
Expand Down Expand Up @@ -422,7 +423,7 @@ FOR_LOOP:
bcR.pool.PopRequest()

// TODO: batch saves so we dont persist to disk every block
err = bcR.store.SaveBlock(first, firstParts, second.LastCommit)
err = bcR.store.SaveBlock(context.TODO(), first, firstParts, second.LastCommit)
if err != nil {
panic(err)
}
Expand Down
7 changes: 4 additions & 3 deletions blockchain/v0/reactor_test.go
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package v0

import (
"context"
"crypto/sha256"
"fmt"
"os"
Expand Down Expand Up @@ -100,7 +101,7 @@ func newBlockchainReactor(
lastCommit := types.NewCommit(blockHeight-1, 0, types.BlockID{}, nil)
if blockHeight > 1 {
lastBlockMeta := blockStore.LoadBlockMeta(blockHeight - 1)
lastBlock, err := blockStore.LoadBlock(blockHeight - 1)
lastBlock, err := blockStore.LoadBlock(context.TODO(), blockHeight-1)
if err != nil {
panic(err)
}
Expand Down Expand Up @@ -130,7 +131,7 @@ func newBlockchainReactor(
panic(fmt.Errorf("error apply block: %w", err))
}

err = blockStore.SaveBlock(thisBlock, thisParts, lastCommit)
err = blockStore.SaveBlock(context.TODO(), thisBlock, thisParts, lastCommit)
if err != nil {
panic(err)
}
Expand Down Expand Up @@ -190,7 +191,7 @@ func TestNoBlockResponse(t *testing.T) {
assert.Equal(t, maxBlockHeight, reactorPairs[0].reactor.store.Height())

for _, tt := range tests {
block, err := reactorPairs[1].reactor.store.LoadBlock(tt.height)
block, err := reactorPairs[1].reactor.store.LoadBlock(context.TODO(), tt.height)
if tt.existent {
require.NoError(t, err)
assert.True(t, block != nil)
Expand Down
4 changes: 2 additions & 2 deletions consensus/replay.go
Original file line number Diff line number Diff line change
Expand Up @@ -463,7 +463,7 @@ func (h *Handshaker) replayBlocks(
}
for i := firstBlock; i <= finalBlock; i++ {
h.logger.Info("Applying block", "height", i)
block, err := h.store.LoadBlock(i)
block, err := h.store.LoadBlock(context.TODO(), i)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -495,7 +495,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, err := h.store.LoadBlock(height)
block, err := h.store.LoadBlock(context.TODO(), height)
if err != nil {
return sm.State{}, err
}
Expand Down
13 changes: 9 additions & 4 deletions consensus/replay_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -548,7 +548,7 @@ func TestSimulateValidatorsChange(t *testing.T) {
sim.Chain = make([]*types.Block, 0)
sim.Commits = make([]*types.Commit, 0)
for i := 1; i <= numBlocks; i++ {
block, err := css[0].blockStore.LoadBlock(int64(i))
block, err := css[0].blockStore.LoadBlock(context.TODO(), int64(i))
if err != nil {
panic(err)
}
Expand Down Expand Up @@ -1203,10 +1203,10 @@ func (bs *mockBlockStore) Height() int64 { return int64(len(bs.
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, error) {
func (bs *mockBlockStore) LoadBlock(ctx context.Context, height int64) (*types.Block, error) {
return bs.chain[height-1], nil
}
func (bs *mockBlockStore) LoadBlockByHash(hash []byte) (*types.Block, error) {
func (bs *mockBlockStore) LoadBlockByHash(ctx context.Context, hash []byte) (*types.Block, error) {
return bs.chain[int64(len(bs.chain))-1], nil
}
func (bs *mockBlockStore) LoadBlockMeta(height int64) *types.BlockMeta {
Expand All @@ -1217,7 +1217,12 @@ func (bs *mockBlockStore) LoadBlockMeta(height int64) *types.BlockMeta {
}
}
func (bs *mockBlockStore) LoadBlockPart(height int64, index int) *types.Part { return nil }
func (bs *mockBlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) error {
func (bs *mockBlockStore) SaveBlock(
ctx context.Context,
block *types.Block,
blockParts *types.PartSet,
seenCommit *types.Commit,
) error {
return nil
}
func (bs *mockBlockStore) LoadBlockCommit(height int64) *types.Commit {
Expand Down
2 changes: 1 addition & 1 deletion consensus/state.go
Original file line number Diff line number Diff line change
Expand Up @@ -1559,7 +1559,7 @@ func (cs *State) finalizeCommit(height int64) {
// but may differ from the LastCommit included in the next block
precommits := cs.Votes.Precommits(cs.CommitRound)
seenCommit := precommits.MakeCommit()
err := cs.blockStore.SaveBlock(block, blockParts, seenCommit)
err := cs.blockStore.SaveBlock(context.TODO(), block, blockParts, seenCommit)
if err != nil {
panic(err)
}
Expand Down
3 changes: 2 additions & 1 deletion evidence/pool_test.go
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package evidence_test

import (
"context"
"os"
"testing"
"time"
Expand Down Expand Up @@ -408,7 +409,7 @@ func initializeBlockStore(db dbm.DB, state sm.State, valAddr []byte) *store.Bloc
partSet := block.MakePartSet(parts)

seenCommit := makeCommit(i, valAddr)
err := blockStore.SaveBlock(block, partSet, seenCommit)
err := blockStore.SaveBlock(context.TODO(), block, partSet, seenCommit)
if err != nil {
panic(err)
}
Expand Down
7 changes: 4 additions & 3 deletions rpc/core/blocks.go
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package core

import (
"context"
"fmt"

tmmath "github.com/lazyledger/lazyledger-core/libs/math"
Expand Down Expand Up @@ -89,7 +90,7 @@ func Block(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultBlock, error)
return nil, err
}

block, err := env.BlockStore.LoadBlock(height)
block, err := env.BlockStore.LoadBlock(context.TODO(), height)
if err != nil {
return nil, err
}
Expand All @@ -103,7 +104,7 @@ func Block(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.ResultBlock, error)
// BlockByHash gets block by hash.
// More: https://docs.tendermint.com/master/rpc/#/Info/block_by_hash
func BlockByHash(ctx *rpctypes.Context, hash []byte) (*ctypes.ResultBlock, error) {
block, err := env.BlockStore.LoadBlockByHash(hash)
block, err := env.BlockStore.LoadBlockByHash(context.TODO(), hash)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -153,7 +154,7 @@ func DataAvailabilityHeader(ctx *rpctypes.Context, heightPtr *int64) (*ctypes.Re
// depends on either:
// - https://github.com/lazyledger/lazyledger-core/pull/312, or
// - https://github.com/lazyledger/lazyledger-core/pull/218
block, err := env.BlockStore.LoadBlock(height)
block, err := env.BlockStore.LoadBlock(context.TODO(), height)
if err != nil {
return nil, err
}
Expand Down
26 changes: 18 additions & 8 deletions rpc/core/blocks_test.go
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package core

import (
"context"
"fmt"
"testing"

Expand Down Expand Up @@ -118,17 +119,26 @@ type mockBlockStore struct {
height int64
}

func (mockBlockStore) Base() int64 { return 1 }
func (store mockBlockStore) Height() int64 { return store.height }
func (store mockBlockStore) Size() int64 { return store.height }
func (mockBlockStore) LoadBaseMeta() *types.BlockMeta { return nil }
func (mockBlockStore) LoadBlockMeta(height int64) *types.BlockMeta { return nil }
func (mockBlockStore) LoadBlock(height int64) (*types.Block, error) { return nil, nil }
func (mockBlockStore) LoadBlockByHash(hash []byte) (*types.Block, error) { return nil, nil }
func (mockBlockStore) Base() int64 { return 1 }
func (store mockBlockStore) Height() int64 { return store.height }
func (store mockBlockStore) Size() int64 { return store.height }
func (mockBlockStore) LoadBaseMeta() *types.BlockMeta { return nil }
func (mockBlockStore) LoadBlockMeta(height int64) *types.BlockMeta { return nil }
func (mockBlockStore) LoadBlock(ctx context.Context, height int64) (*types.Block, error) {
return nil, nil
}
func (mockBlockStore) LoadBlockByHash(ctx context.Context, hash []byte) (*types.Block, error) {
return nil, nil
}
func (mockBlockStore) LoadBlockPart(height int64, index int) *types.Part { return nil }
func (mockBlockStore) LoadBlockCommit(height int64) *types.Commit { return nil }
func (mockBlockStore) LoadSeenCommit(height int64) *types.Commit { return nil }
func (mockBlockStore) PruneBlocks(height int64) (uint64, error) { return 0, nil }
func (mockBlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) error {
func (mockBlockStore) SaveBlock(
ctx context.Context,
block *types.Block,
blockParts *types.PartSet,
seenCommit *types.Commit,
) error {
return nil
}
5 changes: 3 additions & 2 deletions rpc/core/tx.go
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package core

import (
"context"
"errors"
"fmt"
"sort"
Expand Down Expand Up @@ -37,7 +38,7 @@ func Tx(ctx *rpctypes.Context, hash []byte, prove bool) (*ctypes.ResultTx, error

var proof types.TxProof
if prove {
block, err := env.BlockStore.LoadBlock(height)
block, err := env.BlockStore.LoadBlock(context.TODO(), height)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -111,7 +112,7 @@ func TxSearch(ctx *rpctypes.Context, query string, prove bool, pagePtr, perPageP
var proof types.TxProof
if prove {
// TODO(evan): do something with this error
block, _ := env.BlockStore.LoadBlock(r.Height)
block, _ := env.BlockStore.LoadBlock(context.TODO(), r.Height)
proof = block.Data.Txs.Proof(int(r.Index)) // XXX: overflow on 32-bit machines
}

Expand Down
8 changes: 5 additions & 3 deletions state/services.go
Original file line number Diff line number Diff line change
@@ -1,6 +1,8 @@
package state

import (
"context"

"github.com/lazyledger/lazyledger-core/types"
)

Expand All @@ -20,13 +22,13 @@ type BlockStore interface {

LoadBaseMeta() *types.BlockMeta
LoadBlockMeta(height int64) *types.BlockMeta
LoadBlock(height int64) (*types.Block, error)
LoadBlock(ctx context.Context, height int64) (*types.Block, error)

SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) error
SaveBlock(ctx context.Context, block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) error

PruneBlocks(height int64) (uint64, error)

LoadBlockByHash(hash []byte) (*types.Block, error)
LoadBlockByHash(ctx context.Context, hash []byte) (*types.Block, error)
LoadBlockPart(height int64, index int) *types.Part

LoadBlockCommit(height int64) *types.Commit
Expand Down
17 changes: 11 additions & 6 deletions store/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -98,7 +98,7 @@ func (bs *BlockStore) LoadBaseMeta() *types.BlockMeta {

// LoadBlock returns the block with the given height.
// If no block is found for that height, it returns nil.
func (bs *BlockStore) LoadBlock(height int64) (*types.Block, error) {
func (bs *BlockStore) LoadBlock(ctx context.Context, height int64) (*types.Block, error) {
blockMeta := bs.LoadBlockMeta(height)
if blockMeta == nil {
// TODO(evan): return an error
Expand All @@ -107,7 +107,7 @@ func (bs *BlockStore) LoadBlock(height int64) (*types.Block, error) {

lastCommit := bs.LoadBlockCommit(height - 1)

data, err := ipld.RetrieveBlockData(context.TODO(), &blockMeta.DAHeader, bs.dag, rsmt2d.NewRSGF8Codec())
data, err := ipld.RetrieveBlockData(ctx, &blockMeta.DAHeader, bs.dag, rsmt2d.NewRSGF8Codec())
if err != nil {
return nil, err
}
Expand All @@ -125,7 +125,7 @@ func (bs *BlockStore) LoadBlock(height int64) (*types.Block, error) {
// LoadBlockByHash returns the block with the given hash.
// If no block is found for that hash, it returns nil.
// Panics if it fails to parse height associated with the given hash.
func (bs *BlockStore) LoadBlockByHash(hash []byte) (*types.Block, error) {
func (bs *BlockStore) LoadBlockByHash(ctx context.Context, hash []byte) (*types.Block, error) {
bz, err := bs.db.Get(calcBlockHashKey(hash))
if err != nil {
panic(err)
Expand All @@ -140,7 +140,7 @@ func (bs *BlockStore) LoadBlockByHash(hash []byte) (*types.Block, error) {
if err != nil {
panic(fmt.Sprintf("failed to extract height from %s: %v", s, err))
}
return bs.LoadBlock(height)
return bs.LoadBlock(ctx, height)
}

// LoadBlockPart returns the Part at the given index
Expand Down Expand Up @@ -328,7 +328,12 @@ func (bs *BlockStore) PruneBlocks(height int64) (uint64, error) {
// If all the nodes restart after committing a block,
// we need this to reload the precommits to catch-up nodes to the
// most recent height. Otherwise they'd stall at H-1.
func (bs *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) error {
func (bs *BlockStore) SaveBlock(
ctx context.Context,
block *types.Block,
blockParts *types.PartSet,
seenCommit *types.Commit,
) error {
if block == nil {
panic("BlockStore can only save a non-nil block")
}
Expand All @@ -352,7 +357,7 @@ func (bs *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, s
bs.saveBlockPart(height, i, part)
}

err := ipld.PutBlock(context.TODO(), bs.dag, block)
err := ipld.PutBlock(ctx, bs.dag, block)
if err != nil {
return err
}
Expand Down
Loading

0 comments on commit 04ce46b

Please sign in to comment.