Merge PR #4748: Write-Through Inter-Block Cache

This commit is contained in:
Alexander Bezobchuk 2019-09-04 13:33:32 -04:00 committed by GitHub
parent d55a90e4d1
commit f010d2c6f1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
31 changed files with 1054 additions and 741 deletions

View File

@ -37,9 +37,11 @@ Ref: https://keepachangelog.com/en/1.0.0/
## [Unreleased]
### API Breaking Changes
* (store) [\#4748](https://github.com/cosmos/cosmos-sdk/pull/4748) The `CommitMultiStore` interface
now requires a `SetInterBlockCache` method. Applications that do not wish to support this can simply
have this method perform a no-op.
* (modules) [\#4665](https://github.com/cosmos/cosmos-sdk/issues/4665) Refactored `x/gov` module structure and dev-UX:
* Prepare for module spec integration
* Update gov keys to use big endian encoding instead of little endian
@ -48,7 +50,6 @@ Ref: https://keepachangelog.com/en/1.0.0/
* (rest) [\#4783](https://github.com/cosmos/cosmos-sdk/issues/4783) The balance field in the DelegationResponse type is now sdk.Coin instead of sdk.Int
### Features
* (cli) [\#4973](https://github.com/cosmos/cosmos-sdk/pull/4973) Enable application CPU profiling
@ -80,6 +81,10 @@ longer panics if the store to load contains substores that we didn't explicitly
* (simulation) [\#4906](https://github.com/cosmos/cosmos-sdk/issues/4906) Add simulation `Config` struct that wraps simulation flags
* (store) [\#4792](https://github.com/cosmos/cosmos-sdk/issues/4792) panic on non-registered store
* (types) [\#4821](https://github.com/cosmos/cosmos-sdk/issues/4821) types/errors package added with support for stacktraces. It is meant as a more feature-rich replacement for sdk.Errors in the mid-term.
* (store) [\#1947](https://github.com/cosmos/cosmos-sdk/issues/1947) Implement inter-block (persistent)
caching through `CommitKVStoreCacheManager`. Any application wishing to utilize an inter-block cache
must set it in their app via a `BaseApp` option. The `BaseApp` docs have been drastically improved
to detail this new feature and how state transitions occur.
### Bug Fixes

433
baseapp/abci.go Normal file
View File

@ -0,0 +1,433 @@
package baseapp
import (
"fmt"
"os"
"sort"
"strings"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/cosmos/cosmos-sdk/codec"
sdk "github.com/cosmos/cosmos-sdk/types"
)
// InitChain implements the ABCI interface. It runs the initialization logic
// directly on the CommitMultiStore.
func (app *BaseApp) InitChain(req abci.RequestInitChain) (res abci.ResponseInitChain) {
// stash the consensus params in the cms main store and memoize
if req.ConsensusParams != nil {
app.setConsensusParams(req.ConsensusParams)
app.storeConsensusParams(req.ConsensusParams)
}
initHeader := abci.Header{ChainID: req.ChainId, Time: req.Time}
// initialize the deliver state and check state with a correct header
app.setDeliverState(initHeader)
app.setCheckState(initHeader)
if app.initChainer == nil {
return
}
// reset the inter-block cache in case successive InitChain calls are made
if app.interBlockCache != nil {
app.interBlockCache.Reset()
}
// add block gas meter for any genesis transactions (allow infinite gas)
app.deliverState.ctx = app.deliverState.ctx.WithBlockGasMeter(sdk.NewInfiniteGasMeter())
res = app.initChainer(app.deliverState.ctx, req)
// sanity check
if len(req.Validators) > 0 {
if len(req.Validators) != len(res.Validators) {
panic(
fmt.Errorf(
"len(RequestInitChain.Validators) != len(GenesisValidators) (%d != %d)",
len(req.Validators), len(res.Validators),
),
)
}
sort.Sort(abci.ValidatorUpdates(req.Validators))
sort.Sort(abci.ValidatorUpdates(res.Validators))
for i, val := range res.Validators {
if !val.Equal(req.Validators[i]) {
panic(fmt.Errorf("genesisValidators[%d] != req.Validators[%d] ", i, i))
}
}
}
// NOTE: We don't commit, but BeginBlock for block 1 starts from this
// deliverState.
return
}
// Info implements the ABCI interface.
func (app *BaseApp) Info(req abci.RequestInfo) abci.ResponseInfo {
lastCommitID := app.cms.LastCommitID()
return abci.ResponseInfo{
Data: app.name,
LastBlockHeight: lastCommitID.Version,
LastBlockAppHash: lastCommitID.Hash,
}
}
// SetOption implements the ABCI interface.
func (app *BaseApp) SetOption(req abci.RequestSetOption) (res abci.ResponseSetOption) {
// TODO: Implement!
return
}
// FilterPeerByAddrPort filters peers by address/port.
func (app *BaseApp) FilterPeerByAddrPort(info string) abci.ResponseQuery {
if app.addrPeerFilter != nil {
return app.addrPeerFilter(info)
}
return abci.ResponseQuery{}
}
// FilterPeerByIDfilters peers by node ID.
func (app *BaseApp) FilterPeerByID(info string) abci.ResponseQuery {
if app.idPeerFilter != nil {
return app.idPeerFilter(info)
}
return abci.ResponseQuery{}
}
// BeginBlock implements the ABCI application interface.
func (app *BaseApp) BeginBlock(req abci.RequestBeginBlock) (res abci.ResponseBeginBlock) {
if app.cms.TracingEnabled() {
app.cms.SetTracingContext(sdk.TraceContext(
map[string]interface{}{"blockHeight": req.Header.Height},
))
}
if err := app.validateHeight(req); err != nil {
panic(err)
}
// Initialize the DeliverTx state. If this is the first block, it should
// already be initialized in InitChain. Otherwise app.deliverState will be
// nil, since it is reset on Commit.
if app.deliverState == nil {
app.setDeliverState(req.Header)
} else {
// In the first block, app.deliverState.ctx will already be initialized
// by InitChain. Context is now updated with Header information.
app.deliverState.ctx = app.deliverState.ctx.
WithBlockHeader(req.Header).
WithBlockHeight(req.Header.Height)
}
// add block gas meter
var gasMeter sdk.GasMeter
if maxGas := app.getMaximumBlockGas(); maxGas > 0 {
gasMeter = sdk.NewGasMeter(maxGas)
} else {
gasMeter = sdk.NewInfiniteGasMeter()
}
app.deliverState.ctx = app.deliverState.ctx.WithBlockGasMeter(gasMeter)
if app.beginBlocker != nil {
res = app.beginBlocker(app.deliverState.ctx, req)
}
// set the signed validators for addition to context in deliverTx
app.voteInfos = req.LastCommitInfo.GetVotes()
return
}
// EndBlock implements the ABCI interface.
func (app *BaseApp) EndBlock(req abci.RequestEndBlock) (res abci.ResponseEndBlock) {
if app.deliverState.ms.TracingEnabled() {
app.deliverState.ms = app.deliverState.ms.SetTracingContext(nil).(sdk.CacheMultiStore)
}
if app.endBlocker != nil {
res = app.endBlocker(app.deliverState.ctx, req)
}
return
}
// CheckTx implements the ABCI interface. It runs the "basic checks" to see
// whether or not a transaction can possibly be executed, first decoding and then
// the ante handler (which checks signatures/fees/ValidateBasic).
//
// NOTE:CheckTx does not run the actual Msg handler function(s).
func (app *BaseApp) CheckTx(req abci.RequestCheckTx) (res abci.ResponseCheckTx) {
var result sdk.Result
tx, err := app.txDecoder(req.Tx)
if err != nil {
result = err.Result()
} else {
result = app.runTx(runTxModeCheck, req.Tx, tx)
}
return abci.ResponseCheckTx{
Code: uint32(result.Code),
Data: result.Data,
Log: result.Log,
GasWanted: int64(result.GasWanted), // TODO: Should type accept unsigned ints?
GasUsed: int64(result.GasUsed), // TODO: Should type accept unsigned ints?
Events: result.Events.ToABCIEvents(),
}
}
// DeliverTx implements the ABCI interface.
func (app *BaseApp) DeliverTx(req abci.RequestDeliverTx) (res abci.ResponseDeliverTx) {
var result sdk.Result
tx, err := app.txDecoder(req.Tx)
if err != nil {
result = err.Result()
} else {
result = app.runTx(runTxModeDeliver, req.Tx, tx)
}
return abci.ResponseDeliverTx{
Code: uint32(result.Code),
Codespace: string(result.Codespace),
Data: result.Data,
Log: result.Log,
GasWanted: int64(result.GasWanted), // TODO: Should type accept unsigned ints?
GasUsed: int64(result.GasUsed), // TODO: Should type accept unsigned ints?
Events: result.Events.ToABCIEvents(),
}
}
// Commit implements the ABCI interface. It will commit all state that exists in
// the deliver state's multi-store and includes the resulting commit ID in the
// returned abci.ResponseCommit. Commit will set the check state based on the
// latest header and reset the deliver state. Also, if a non-zero halt height is
// defined in config, Commit will execute a deferred function call to check
// against that height and gracefully halt if it matches the latest committed
// height.
func (app *BaseApp) Commit() (res abci.ResponseCommit) {
header := app.deliverState.ctx.BlockHeader()
// Write the DeliverTx state which is cache-wrapped and commit the MultiStore.
// The write to the DeliverTx state writes all state transitions to the root
// MultiStore (app.cms) so when Commit() is called is persists those values.
app.deliverState.ms.Write()
commitID := app.cms.Commit()
app.logger.Debug("Commit synced", "commit", fmt.Sprintf("%X", commitID))
// Reset the Check state to the latest committed.
//
// NOTE: This is safe because Tendermint holds a lock on the mempool for
// Commit. Use the header from this latest block.
app.setCheckState(header)
// empty/reset the deliver state
app.deliverState = nil
defer func() {
if app.haltHeight > 0 && uint64(header.Height) == app.haltHeight {
app.logger.Info("halting node per configuration", "height", app.haltHeight)
os.Exit(0)
}
}()
return abci.ResponseCommit{
Data: commitID.Hash,
}
}
// Query implements the ABCI interface. It delegates to CommitMultiStore if it
// implements Queryable.
func (app *BaseApp) Query(req abci.RequestQuery) (res abci.ResponseQuery) {
path := splitPath(req.Path)
if len(path) == 0 {
msg := "no query path provided"
return sdk.ErrUnknownRequest(msg).QueryResult()
}
switch path[0] {
// "/app" prefix for special application queries
case "app":
return handleQueryApp(app, path, req)
case "store":
return handleQueryStore(app, path, req)
case "p2p":
return handleQueryP2P(app, path, req)
case "custom":
return handleQueryCustom(app, path, req)
}
msg := "unknown query path"
return sdk.ErrUnknownRequest(msg).QueryResult()
}
func handleQueryApp(app *BaseApp, path []string, req abci.RequestQuery) (res abci.ResponseQuery) {
if len(path) >= 2 {
var result sdk.Result
switch path[1] {
case "simulate":
txBytes := req.Data
tx, err := app.txDecoder(txBytes)
if err != nil {
result = err.Result()
} else {
result = app.Simulate(txBytes, tx)
}
case "version":
return abci.ResponseQuery{
Code: uint32(sdk.CodeOK),
Codespace: string(sdk.CodespaceRoot),
Height: req.Height,
Value: []byte(app.appVersion),
}
default:
result = sdk.ErrUnknownRequest(fmt.Sprintf("unknown query: %s", path)).Result()
}
value := codec.Cdc.MustMarshalBinaryLengthPrefixed(result)
return abci.ResponseQuery{
Code: uint32(sdk.CodeOK),
Codespace: string(sdk.CodespaceRoot),
Height: req.Height,
Value: value,
}
}
msg := "expected second parameter to be either 'simulate' or 'version', neither was present"
return sdk.ErrUnknownRequest(msg).QueryResult()
}
func handleQueryStore(app *BaseApp, path []string, req abci.RequestQuery) abci.ResponseQuery {
// "/store" prefix for store queries
queryable, ok := app.cms.(sdk.Queryable)
if !ok {
msg := "multistore doesn't support queries"
return sdk.ErrUnknownRequest(msg).QueryResult()
}
req.Path = "/" + strings.Join(path[1:], "/")
// when a client did not provide a query height, manually inject the latest
if req.Height == 0 {
req.Height = app.LastBlockHeight()
}
if req.Height <= 1 && req.Prove {
return sdk.ErrInternal("cannot query with proof when height <= 1; please provide a valid height").QueryResult()
}
resp := queryable.Query(req)
resp.Height = req.Height
return resp
}
func handleQueryP2P(app *BaseApp, path []string, _ abci.RequestQuery) (res abci.ResponseQuery) {
// "/p2p" prefix for p2p queries
if len(path) >= 4 {
cmd, typ, arg := path[1], path[2], path[3]
switch cmd {
case "filter":
switch typ {
case "addr":
return app.FilterPeerByAddrPort(arg)
case "id":
return app.FilterPeerByID(arg)
}
default:
msg := "expected second parameter to be 'filter'"
return sdk.ErrUnknownRequest(msg).QueryResult()
}
}
msg := "Expected path is p2p filter <addr|id> <parameter>"
return sdk.ErrUnknownRequest(msg).QueryResult()
}
func handleQueryCustom(app *BaseApp, path []string, req abci.RequestQuery) (res abci.ResponseQuery) {
// path[0] should be "custom" because "/custom" prefix is required for keeper
// queries.
//
// The queryRouter routes using path[1]. For example, in the path
// "custom/gov/proposal", queryRouter routes using "gov".
if len(path) < 2 || path[1] == "" {
return sdk.ErrUnknownRequest("No route for custom query specified").QueryResult()
}
querier := app.queryRouter.Route(path[1])
if querier == nil {
return sdk.ErrUnknownRequest(fmt.Sprintf("no custom querier found for route %s", path[1])).QueryResult()
}
// when a client did not provide a query height, manually inject the latest
if req.Height == 0 {
req.Height = app.LastBlockHeight()
}
if req.Height <= 1 && req.Prove {
return sdk.ErrInternal("cannot query with proof when height <= 1; please provide a valid height").QueryResult()
}
cacheMS, err := app.cms.CacheMultiStoreWithVersion(req.Height)
if err != nil {
return sdk.ErrInternal(
fmt.Sprintf(
"failed to load state at height %d; %s (latest height: %d)",
req.Height, err, app.LastBlockHeight(),
),
).QueryResult()
}
// cache wrap the commit-multistore for safety
ctx := sdk.NewContext(
cacheMS, app.checkState.ctx.BlockHeader(), true, app.logger,
).WithMinGasPrices(app.minGasPrices)
// Passes the rest of the path as an argument to the querier.
//
// For example, in the path "custom/gov/proposal/test", the gov querier gets
// []string{"proposal", "test"} as the path.
resBytes, queryErr := querier(ctx, path[2:], req)
if queryErr != nil {
return abci.ResponseQuery{
Code: uint32(queryErr.Code()),
Codespace: string(queryErr.Codespace()),
Height: req.Height,
Log: queryErr.ABCILog(),
}
}
return abci.ResponseQuery{
Code: uint32(sdk.CodeOK),
Height: req.Height,
Value: resBytes,
}
}
// splitPath splits a string path using the delimiter '/'.
//
// e.g. "this/is/funny" becomes []string{"this", "is", "funny"}
func splitPath(requestPath string) (path []string) {
path = strings.Split(requestPath, "/")
// first element is empty string
if len(path) > 0 && path[0] == "" {
path = path[1:]
}
return path
}

View File

@ -2,18 +2,15 @@ package baseapp
import (
"encoding/json"
"errors"
"fmt"
"io/ioutil"
"os"
"reflect"
"runtime/debug"
"sort"
"strings"
"errors"
"github.com/gogo/protobuf/proto"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/tmhash"
"github.com/tendermint/tendermint/libs/log"
@ -25,12 +22,6 @@ import (
sdk "github.com/cosmos/cosmos-sdk/types"
)
// Key to store the consensus params in the main store.
var mainConsensusParamsKey = []byte("consensus_params")
// Enum mode for app.runTx
type runTxMode uint8
const (
// Check a transaction
runTxModeCheck runTxMode = iota
@ -43,11 +34,24 @@ const (
MainStoreKey = "main"
)
// StoreLoader defines a customizable function to control how we load the CommitMultiStore
// from disk. This is useful for state migration, when loading a datastore written with
// an older version of the software. In particular, if a module changed the substore key name
// (or removed a substore) between two versions of the software.
type StoreLoader func(ms sdk.CommitMultiStore) error
var (
_ abci.Application = (*BaseApp)(nil)
// mainConsensusParamsKey defines a key to store the consensus params in the
// main store.
mainConsensusParamsKey = []byte("consensus_params")
)
type (
// Enum mode for app.runTx
runTxMode uint8
// StoreLoader defines a customizable function to control how we load the CommitMultiStore
// from disk. This is useful for state migration, when loading a datastore written with
// an older version of the software. In particular, if a module changed the substore key name
// (or removed a substore) between two versions of the software.
StoreLoader func(ms sdk.CommitMultiStore) error
)
// BaseApp reflects the ABCI application implementation.
type BaseApp struct {
@ -72,14 +76,18 @@ type BaseApp struct {
idPeerFilter sdk.PeerFilter // filter peers by node ID
fauxMerkleMode bool // if true, IAVL MountStores uses MountStoresDB for simulation speed.
// --------------------
// Volatile state
// checkState is set on initialization and reset on Commit.
// deliverState is set in InitChain and BeginBlock and cleared on Commit.
// See methods setCheckState and setDeliverState.
checkState *state // for CheckTx
deliverState *state // for DeliverTx
voteInfos []abci.VoteInfo // absent validators from begin block
// volatile states:
//
// checkState is set on InitChain and reset on Commit
// deliverState is set on InitChain and BeginBlock and set to nil on Commit
checkState *state // for CheckTx
deliverState *state // for DeliverTx
// an inter-block write-through cache provided to the context during deliverState
interBlockCache sdk.MultiStorePersistentCache
// absent validators from begin block
voteInfos []abci.VoteInfo
// consensus params
// TODO: Move this in the future to baseapp param store on main store.
@ -99,8 +107,6 @@ type BaseApp struct {
appVersion string
}
var _ abci.Application = (*BaseApp)(nil)
// NewBaseApp returns a reference to an initialized BaseApp. It accepts a
// variadic number of option functions, which act on the BaseApp to set
// configuration choices.
@ -125,6 +131,10 @@ func NewBaseApp(
option(app)
}
if app.interBlockCache != nil {
app.cms.SetInterBlockCache(app.interBlockCache)
}
return app
}
@ -156,8 +166,10 @@ func (app *BaseApp) MountStores(keys ...sdk.StoreKey) {
// retain history, but it's useful for faster simulation.
app.MountStore(key, sdk.StoreTypeDB)
}
case *sdk.TransientStoreKey:
app.MountStore(key, sdk.StoreTypeTransient)
default:
panic("Unrecognized store key type " + reflect.TypeOf(key).Name())
}
@ -333,6 +345,10 @@ func (app *BaseApp) setHaltHeight(height uint64) {
app.haltHeight = height
}
func (app *BaseApp) setInterBlockCache(cache sdk.MultiStorePersistentCache) {
app.interBlockCache = cache
}
// Router returns the router of the BaseApp.
func (app *BaseApp) Router() sdk.Router {
if app.sealed {
@ -352,9 +368,10 @@ func (app *BaseApp) Seal() { app.sealed = true }
// IsSealed returns true if the BaseApp is sealed and false otherwise.
func (app *BaseApp) IsSealed() bool { return app.sealed }
// setCheckState sets checkState with the cached multistore and
// the context wrapping it.
// It is called by InitChain() and Commit()
// setCheckState sets the BaseApp's checkState with a cache-wrapped multi-store
// (i.e. a CacheMultiStore) and a new Context with the cache-wrapped multi-store,
// provided header, and minimum gas prices set. It is set on InitChain and reset
// on Commit.
func (app *BaseApp) setCheckState(header abci.Header) {
ms := app.cms.CacheMultiStore()
app.checkState = &state{
@ -363,10 +380,10 @@ func (app *BaseApp) setCheckState(header abci.Header) {
}
}
// setCheckState sets checkState with the cached multistore and
// the context wrapping it.
// It is called by InitChain() and BeginBlock(),
// and deliverState is set nil on Commit().
// setDeliverState sets the BaseApp's deliverState with a cache-wrapped multi-store
// (i.e. a CacheMultiStore) and a new Context with the cache-wrapped multi-store,
// and provided header. It is set on InitChain and BeginBlock and set to nil on
// Commit.
func (app *BaseApp) setDeliverState(header abci.Header) {
ms := app.cms.CacheMultiStore()
app.deliverState = &state{
@ -411,275 +428,6 @@ func (app *BaseApp) getMaximumBlockGas() uint64 {
}
}
// ----------------------------------------------------------------------------
// ABCI
// Info implements the ABCI interface.
func (app *BaseApp) Info(req abci.RequestInfo) abci.ResponseInfo {
lastCommitID := app.cms.LastCommitID()
return abci.ResponseInfo{
Data: app.name,
LastBlockHeight: lastCommitID.Version,
LastBlockAppHash: lastCommitID.Hash,
}
}
// SetOption implements the ABCI interface.
func (app *BaseApp) SetOption(req abci.RequestSetOption) (res abci.ResponseSetOption) {
// TODO: Implement!
return
}
// InitChain implements the ABCI interface. It runs the initialization logic
// directly on the CommitMultiStore.
func (app *BaseApp) InitChain(req abci.RequestInitChain) (res abci.ResponseInitChain) {
// stash the consensus params in the cms main store and memoize
if req.ConsensusParams != nil {
app.setConsensusParams(req.ConsensusParams)
app.storeConsensusParams(req.ConsensusParams)
}
initHeader := abci.Header{ChainID: req.ChainId, Time: req.Time}
// initialize the deliver state and check state with a correct header
app.setDeliverState(initHeader)
app.setCheckState(initHeader)
if app.initChainer == nil {
return
}
// add block gas meter for any genesis transactions (allow infinite gas)
app.deliverState.ctx = app.deliverState.ctx.
WithBlockGasMeter(sdk.NewInfiniteGasMeter())
res = app.initChainer(app.deliverState.ctx, req)
// sanity check
if len(req.Validators) > 0 {
if len(req.Validators) != len(res.Validators) {
panic(fmt.Errorf(
"len(RequestInitChain.Validators) != len(validators) (%d != %d)",
len(req.Validators), len(res.Validators)))
}
sort.Sort(abci.ValidatorUpdates(req.Validators))
sort.Sort(abci.ValidatorUpdates(res.Validators))
for i, val := range res.Validators {
if !val.Equal(req.Validators[i]) {
panic(fmt.Errorf("validators[%d] != req.Validators[%d] ", i, i))
}
}
}
// NOTE: We don't commit, but BeginBlock for block 1 starts from this
// deliverState.
return
}
// FilterPeerByAddrPort filters peers by address/port.
func (app *BaseApp) FilterPeerByAddrPort(info string) abci.ResponseQuery {
if app.addrPeerFilter != nil {
return app.addrPeerFilter(info)
}
return abci.ResponseQuery{}
}
// FilterPeerByIDfilters peers by node ID.
func (app *BaseApp) FilterPeerByID(info string) abci.ResponseQuery {
if app.idPeerFilter != nil {
return app.idPeerFilter(info)
}
return abci.ResponseQuery{}
}
// Splits a string path using the delimiter '/'.
// e.g. "this/is/funny" becomes []string{"this", "is", "funny"}
func splitPath(requestPath string) (path []string) {
path = strings.Split(requestPath, "/")
// first element is empty string
if len(path) > 0 && path[0] == "" {
path = path[1:]
}
return path
}
// Query implements the ABCI interface. It delegates to CommitMultiStore if it
// implements Queryable.
func (app *BaseApp) Query(req abci.RequestQuery) (res abci.ResponseQuery) {
path := splitPath(req.Path)
if len(path) == 0 {
msg := "no query path provided"
return sdk.ErrUnknownRequest(msg).QueryResult()
}
switch path[0] {
// "/app" prefix for special application queries
case "app":
return handleQueryApp(app, path, req)
case "store":
return handleQueryStore(app, path, req)
case "p2p":
return handleQueryP2P(app, path, req)
case "custom":
return handleQueryCustom(app, path, req)
}
msg := "unknown query path"
return sdk.ErrUnknownRequest(msg).QueryResult()
}
func handleQueryApp(app *BaseApp, path []string, req abci.RequestQuery) (res abci.ResponseQuery) {
if len(path) >= 2 {
var result sdk.Result
switch path[1] {
case "simulate":
txBytes := req.Data
tx, err := app.txDecoder(txBytes)
if err != nil {
result = err.Result()
} else {
result = app.Simulate(txBytes, tx)
}
case "version":
return abci.ResponseQuery{
Code: uint32(sdk.CodeOK),
Codespace: string(sdk.CodespaceRoot),
Height: req.Height,
Value: []byte(app.appVersion),
}
default:
result = sdk.ErrUnknownRequest(fmt.Sprintf("Unknown query: %s", path)).Result()
}
value := codec.Cdc.MustMarshalBinaryLengthPrefixed(result)
return abci.ResponseQuery{
Code: uint32(sdk.CodeOK),
Codespace: string(sdk.CodespaceRoot),
Height: req.Height,
Value: value,
}
}
msg := "Expected second parameter to be either simulate or version, neither was present"
return sdk.ErrUnknownRequest(msg).QueryResult()
}
func handleQueryStore(app *BaseApp, path []string, req abci.RequestQuery) abci.ResponseQuery {
// "/store" prefix for store queries
queryable, ok := app.cms.(sdk.Queryable)
if !ok {
msg := "multistore doesn't support queries"
return sdk.ErrUnknownRequest(msg).QueryResult()
}
req.Path = "/" + strings.Join(path[1:], "/")
// when a client did not provide a query height, manually inject the latest
if req.Height == 0 {
req.Height = app.LastBlockHeight()
}
if req.Height <= 1 && req.Prove {
return sdk.ErrInternal("cannot query with proof when height <= 1; please provide a valid height").QueryResult()
}
resp := queryable.Query(req)
resp.Height = req.Height
return resp
}
func handleQueryP2P(app *BaseApp, path []string, _ abci.RequestQuery) (res abci.ResponseQuery) {
// "/p2p" prefix for p2p queries
if len(path) >= 4 {
cmd, typ, arg := path[1], path[2], path[3]
switch cmd {
case "filter":
switch typ {
case "addr":
return app.FilterPeerByAddrPort(arg)
case "id":
return app.FilterPeerByID(arg)
}
default:
msg := "Expected second parameter to be filter"
return sdk.ErrUnknownRequest(msg).QueryResult()
}
}
msg := "Expected path is p2p filter <addr|id> <parameter>"
return sdk.ErrUnknownRequest(msg).QueryResult()
}
func handleQueryCustom(app *BaseApp, path []string, req abci.RequestQuery) (res abci.ResponseQuery) {
// path[0] should be "custom" because "/custom" prefix is required for keeper
// queries.
//
// The queryRouter routes using path[1]. For example, in the path
// "custom/gov/proposal", queryRouter routes using "gov".
if len(path) < 2 || path[1] == "" {
return sdk.ErrUnknownRequest("No route for custom query specified").QueryResult()
}
querier := app.queryRouter.Route(path[1])
if querier == nil {
return sdk.ErrUnknownRequest(fmt.Sprintf("no custom querier found for route %s", path[1])).QueryResult()
}
// when a client did not provide a query height, manually inject the latest
if req.Height == 0 {
req.Height = app.LastBlockHeight()
}
if req.Height <= 1 && req.Prove {
return sdk.ErrInternal("cannot query with proof when height <= 1; please provide a valid height").QueryResult()
}
cacheMS, err := app.cms.CacheMultiStoreWithVersion(req.Height)
if err != nil {
return sdk.ErrInternal(
fmt.Sprintf(
"failed to load state at height %d; %s (latest height: %d)",
req.Height, err, app.LastBlockHeight(),
),
).QueryResult()
}
// cache wrap the commit-multistore for safety
ctx := sdk.NewContext(
cacheMS, app.checkState.ctx.BlockHeader(), true, app.logger,
).WithMinGasPrices(app.minGasPrices)
// Passes the rest of the path as an argument to the querier.
//
// For example, in the path "custom/gov/proposal/test", the gov querier gets
// []string{"proposal", "test"} as the path.
resBytes, queryErr := querier(ctx, path[2:], req)
if queryErr != nil {
return abci.ResponseQuery{
Code: uint32(queryErr.Code()),
Codespace: string(queryErr.Codespace()),
Height: req.Height,
Log: queryErr.ABCILog(),
}
}
return abci.ResponseQuery{
Code: uint32(sdk.CodeOK),
Height: req.Height,
Value: resBytes,
}
}
func (app *BaseApp) validateHeight(req abci.RequestBeginBlock) error {
if req.Header.Height < 1 {
return fmt.Errorf("invalid height: %d", req.Header.Height)
@ -693,97 +441,6 @@ func (app *BaseApp) validateHeight(req abci.RequestBeginBlock) error {
return nil
}
// BeginBlock implements the ABCI application interface.
func (app *BaseApp) BeginBlock(req abci.RequestBeginBlock) (res abci.ResponseBeginBlock) {
if app.cms.TracingEnabled() {
app.cms.SetTracingContext(sdk.TraceContext(
map[string]interface{}{"blockHeight": req.Header.Height},
))
}
if err := app.validateHeight(req); err != nil {
panic(err)
}
// Initialize the DeliverTx state. If this is the first block, it should
// already be initialized in InitChain. Otherwise app.deliverState will be
// nil, since it is reset on Commit.
if app.deliverState == nil {
app.setDeliverState(req.Header)
} else {
// In the first block, app.deliverState.ctx will already be initialized
// by InitChain. Context is now updated with Header information.
app.deliverState.ctx = app.deliverState.ctx.
WithBlockHeader(req.Header).
WithBlockHeight(req.Header.Height)
}
// add block gas meter
var gasMeter sdk.GasMeter
if maxGas := app.getMaximumBlockGas(); maxGas > 0 {
gasMeter = sdk.NewGasMeter(maxGas)
} else {
gasMeter = sdk.NewInfiniteGasMeter()
}
app.deliverState.ctx = app.deliverState.ctx.WithBlockGasMeter(gasMeter)
if app.beginBlocker != nil {
res = app.beginBlocker(app.deliverState.ctx, req)
}
// set the signed validators for addition to context in deliverTx
app.voteInfos = req.LastCommitInfo.GetVotes()
return
}
// CheckTx implements the ABCI interface. It runs the "basic checks" to see
// whether or not a transaction can possibly be executed, first decoding and then
// the ante handler (which checks signatures/fees/ValidateBasic).
//
// NOTE:CheckTx does not run the actual Msg handler function(s).
func (app *BaseApp) CheckTx(req abci.RequestCheckTx) (res abci.ResponseCheckTx) {
var result sdk.Result
tx, err := app.txDecoder(req.Tx)
if err != nil {
result = err.Result()
} else {
result = app.runTx(runTxModeCheck, req.Tx, tx)
}
return abci.ResponseCheckTx{
Code: uint32(result.Code),
Data: result.Data,
Log: result.Log,
GasWanted: int64(result.GasWanted), // TODO: Should type accept unsigned ints?
GasUsed: int64(result.GasUsed), // TODO: Should type accept unsigned ints?
Events: result.Events.ToABCIEvents(),
}
}
// DeliverTx implements the ABCI interface.
func (app *BaseApp) DeliverTx(req abci.RequestDeliverTx) (res abci.ResponseDeliverTx) {
var result sdk.Result
tx, err := app.txDecoder(req.Tx)
if err != nil {
result = err.Result()
} else {
result = app.runTx(runTxModeDeliver, req.Tx, tx)
}
return abci.ResponseDeliverTx{
Code: uint32(result.Code),
Codespace: string(result.Codespace),
Data: result.Data,
Log: result.Log,
GasWanted: int64(result.GasWanted), // TODO: Should type accept unsigned ints?
GasUsed: int64(result.GasUsed), // TODO: Should type accept unsigned ints?
Events: result.Events.ToABCIEvents(),
}
}
// validateBasicTxMsgs executes basic validator calls for messages.
func validateBasicTxMsgs(msgs []sdk.Msg) sdk.Error {
if len(msgs) == 0 {
@ -801,6 +458,16 @@ func validateBasicTxMsgs(msgs []sdk.Msg) sdk.Error {
return nil
}
// Returns the applications's deliverState if app is in runTxModeDeliver,
// otherwise it returns the application's checkstate.
func (app *BaseApp) getState(mode runTxMode) *state {
if mode == runTxModeCheck || mode == runTxModeSimulate {
return app.checkState
}
return app.deliverState
}
// retrieve the context for the tx w/ txBytes and other memoized values.
func (app *BaseApp) getContextForTx(mode runTxMode, txBytes []byte) (ctx sdk.Context) {
ctx = app.getState(mode).ctx.
@ -815,87 +482,9 @@ func (app *BaseApp) getContextForTx(mode runTxMode, txBytes []byte) (ctx sdk.Con
return
}
// runMsgs iterates through all the messages and executes them.
// nolint: gocyclo
func (app *BaseApp) runMsgs(ctx sdk.Context, msgs []sdk.Msg, mode runTxMode) (result sdk.Result) {
idxLogs := make([]sdk.ABCIMessageLog, 0, len(msgs)) // a list of JSON-encoded logs with msg index
var (
data []byte
code sdk.CodeType
codespace sdk.CodespaceType
)
events := sdk.EmptyEvents()
// NOTE: GasWanted is determined by ante handler and GasUsed by the GasMeter.
for msgIdx, msg := range msgs {
// match message route
msgRoute := msg.Route()
handler := app.router.Route(msgRoute)
if handler == nil {
return sdk.ErrUnknownRequest("Unrecognized Msg type: " + msgRoute).Result()
}
var msgResult sdk.Result
// skip actual execution for CheckTx mode
if mode != runTxModeCheck {
msgResult = handler(ctx, msg)
}
// Each message result's Data must be length prefixed in order to separate
// each result.
data = append(data, msgResult.Data...)
// append events from the message's execution and a message action event
events = events.AppendEvent(sdk.NewEvent(sdk.EventTypeMessage, sdk.NewAttribute(sdk.AttributeKeyAction, msg.Type())))
events = events.AppendEvents(msgResult.Events)
idxLog := sdk.ABCIMessageLog{MsgIndex: uint16(msgIdx), Log: msgResult.Log}
// stop execution and return on first failed message
if !msgResult.IsOK() {
idxLog.Success = false
idxLogs = append(idxLogs, idxLog)
code = msgResult.Code
codespace = msgResult.Codespace
break
}
idxLog.Success = true
idxLogs = append(idxLogs, idxLog)
}
logJSON := codec.Cdc.MustMarshalJSON(idxLogs)
result = sdk.Result{
Code: code,
Codespace: codespace,
Data: data,
Log: strings.TrimSpace(string(logJSON)),
GasUsed: ctx.GasMeter().GasConsumed(),
Events: events,
}
return result
}
// Returns the applications's deliverState if app is in runTxModeDeliver,
// otherwise it returns the application's checkstate.
func (app *BaseApp) getState(mode runTxMode) *state {
if mode == runTxModeCheck || mode == runTxModeSimulate {
return app.checkState
}
return app.deliverState
}
// cacheTxContext returns a new context based off of the provided context with
// a cache wrapped multi-store.
func (app *BaseApp) cacheTxContext(ctx sdk.Context, txBytes []byte) (
sdk.Context, sdk.CacheMultiStore) {
func (app *BaseApp) cacheTxContext(ctx sdk.Context, txBytes []byte) (sdk.Context, sdk.CacheMultiStore) {
ms := ctx.MultiStore()
// TODO: https://github.com/cosmos/cosmos-sdk/issues/2824
msCache := ms.CacheMultiStore()
@ -1011,8 +600,9 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte, tx sdk.Tx) (result sdk
msCache.Write()
}
// Create a new context based off of the existing context with a cache wrapped
// multi-store in case message processing fails.
// Create a new Context based off of the existing Context with a cache-wrapped
// MultiStore in case message processing fails. At this point, the MultiStore
// is doubly cached-wrapped.
runMsgCtx, msCache := app.cacheTxContext(ctx, txBytes)
result = app.runMsgs(runMsgCtx, msgs, mode)
result.GasWanted = gasWanted
@ -1030,67 +620,68 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte, tx sdk.Tx) (result sdk
return result
}
// EndBlock implements the ABCI interface.
func (app *BaseApp) EndBlock(req abci.RequestEndBlock) (res abci.ResponseEndBlock) {
if app.deliverState.ms.TracingEnabled() {
app.deliverState.ms = app.deliverState.ms.SetTracingContext(nil).(sdk.CacheMultiStore)
}
// runMsgs iterates through all the messages and executes them.
// nolint: gocyclo
func (app *BaseApp) runMsgs(ctx sdk.Context, msgs []sdk.Msg, mode runTxMode) (result sdk.Result) {
idxLogs := make([]sdk.ABCIMessageLog, 0, len(msgs)) // a list of JSON-encoded logs with msg index
if app.endBlocker != nil {
res = app.endBlocker(app.deliverState.ctx, req)
}
var (
data []byte
code sdk.CodeType
codespace sdk.CodespaceType
)
return
}
events := sdk.EmptyEvents()
// Commit implements the ABCI interface. It will commit all state that exists in
// the deliver state's multi-store and includes the resulting commit ID in the
// returned abci.ResponseCommit. Commit will set the check state based on the
// latest header and reset the deliver state. Also, if a non-zero halt height is
// defined in config, Commit will execute a deferred function call to check
// against that height and gracefully halt if it matches the latest committed
// height.
func (app *BaseApp) Commit() (res abci.ResponseCommit) {
header := app.deliverState.ctx.BlockHeader()
// write the Deliver state and commit the MultiStore
app.deliverState.ms.Write()
commitID := app.cms.Commit()
app.logger.Debug("Commit synced", "commit", fmt.Sprintf("%X", commitID))
// Reset the Check state to the latest committed.
//
// NOTE: This is safe because Tendermint holds a lock on the mempool for
// Commit. Use the header from this latest block.
app.setCheckState(header)
// empty/reset the deliver state
app.deliverState = nil
defer func() {
if app.haltHeight > 0 && uint64(header.Height) == app.haltHeight {
app.logger.Info("halting node per configuration", "height", app.haltHeight)
os.Exit(0)
// NOTE: GasWanted is determined by ante handler and GasUsed by the GasMeter.
for i, msg := range msgs {
// match message route
msgRoute := msg.Route()
handler := app.router.Route(msgRoute)
if handler == nil {
return sdk.ErrUnknownRequest("unrecognized Msg type: " + msgRoute).Result()
}
}()
return abci.ResponseCommit{
Data: commitID.Hash,
var msgResult sdk.Result
// skip actual execution for CheckTx mode
if mode != runTxModeCheck {
msgResult = handler(ctx, msg)
}
// Each message result's Data must be length prefixed in order to separate
// each result.
data = append(data, msgResult.Data...)
// append events from the message's execution and a message action event
events = events.AppendEvent(sdk.NewEvent(sdk.EventTypeMessage, sdk.NewAttribute(sdk.AttributeKeyAction, msg.Type())))
events = events.AppendEvents(msgResult.Events)
idxLog := sdk.ABCIMessageLog{MsgIndex: uint16(i), Log: msgResult.Log}
// stop execution and return on first failed message
if !msgResult.IsOK() {
idxLog.Success = false
idxLogs = append(idxLogs, idxLog)
code = msgResult.Code
codespace = msgResult.Codespace
break
}
idxLog.Success = true
idxLogs = append(idxLogs, idxLog)
}
}
// ----------------------------------------------------------------------------
// State
logJSON := codec.Cdc.MustMarshalJSON(idxLogs)
result = sdk.Result{
Code: code,
Codespace: codespace,
Data: data,
Log: strings.TrimSpace(string(logJSON)),
GasUsed: ctx.GasMeter().GasConsumed(),
Events: events,
}
type state struct {
ms sdk.CacheMultiStore
ctx sdk.Context
}
func (st *state) CacheMultiStore() sdk.CacheMultiStore {
return st.ms.CacheMultiStore()
}
func (st *state) Context() sdk.Context {
return st.ctx
return result
}

View File

@ -1,8 +0,0 @@
/*
Package baseapp contains data structures that provide basic data storage
functionality and act as a bridge between the ABCI interface and the SDK
abstractions.
BaseApp has no state except the CommitMultiStore you provide upon init.
*/
package baseapp

View File

@ -25,8 +25,7 @@ func (app *BaseApp) Deliver(tx sdk.Tx) (result sdk.Result) {
return app.runTx(runTxModeDeliver, nil, tx)
}
// Context with current {check, deliver}State of the app
// used by tests
// Context with current {check, deliver}State of the app used by tests.
func (app *BaseApp) NewContext(isCheckTx bool, header abci.Header) sdk.Context {
if isCheckTx {
return sdk.NewContext(app.checkState.ms, header, true, app.logger).

View File

@ -34,6 +34,12 @@ func SetHaltHeight(height uint64) func(*BaseApp) {
return func(bap *BaseApp) { bap.setHaltHeight(height) }
}
// SetInterBlockCache provides a BaseApp option function that sets the
// inter-block cache.
func SetInterBlockCache(cache sdk.MultiStorePersistentCache) func(*BaseApp) {
return func(app *BaseApp) { app.setInterBlockCache(cache) }
}
func (app *BaseApp) SetName(name string) {
if app.sealed {
panic("SetName() on sealed BaseApp")

21
baseapp/state.go Normal file
View File

@ -0,0 +1,21 @@
package baseapp
import (
sdk "github.com/cosmos/cosmos-sdk/types"
)
type state struct {
ms sdk.CacheMultiStore
ctx sdk.Context
}
// CacheMultiStore calls and returns a CacheMultiStore on the state's underling
// CacheMultiStore.
func (st *state) CacheMultiStore() sdk.CacheMultiStore {
return st.ms.CacheMultiStore()
}
// Context returns the Context of the state.
func (st *state) Context() sdk.Context {
return st.ctx
}

View File

@ -7,167 +7,207 @@
## Synopsis
This document describes `baseapp`, the abstraction that implements most of the common functionalities of an SDK application.
This document describes `BaseApp`, the abstraction that implements the core
functionalities of an SDK application.
- [Introduction](#introduction)
- [Type Definition](#type-definition)
- [Constructor](#constructor)
- [States](#states)
- [Routing](#routing)
- [Main ABCI Messages](#abci)
- [CheckTx](#checktx)
- [DeliverTx](#delivertx)
- [RunTx, AnteHandler and RunMsgs](#runtx-antehandler-and-runmsgs)
- [RunTx](#runtx)
- [AnteHandler](#antehandler)
- [RunMsgs](#runmsgs)
- [Other ABCI Message](#other-abci-message)
- [InitChain](#initchain)
- [BeginBlock](#beginblock)
- [EndBlock](#endblock)
- [Commit](#commit)
- [Info](#info)
- [Query](#query)
- [BaseApp](#baseapp)
- [Pre-requisite Reading](#pre-requisite-reading)
- [Synopsis](#synopsis)
- [Introduction](#introduction)
- [Type Definition](#type-definition)
- [Constructor](#constructor)
- [States](#states)
- [InitChain](#initchain)
- [CheckTx](#checktx)
- [BeginBlock](#beginblock)
- [DeliverTx](#delivertx)
- [Commit](#commit)
- [Routing](#routing)
- [Message Routing](#message-routing)
- [Query Routing](#query-routing)
- [Main ABCI Messages](#main-abci-messages)
- [CheckTx](#checktx-1)
- [DeliverTx](#delivertx-1)
- [RunTx, AnteHandler and RunMsgs](#runtx-antehandler-and-runmsgs)
- [RunTx](#runtx)
- [AnteHandler](#antehandler)
- [RunMsgs](#runmsgs)
- [Other ABCI Messages](#other-abci-messages)
- [InitChain](#initchain-1)
- [BeginBlock](#beginblock-1)
- [EndBlock](#endblock)
- [Commit](#commit-1)
- [Info](#info)
- [Query](#query)
## Introduction
`baseapp` is an abstraction that implements the core of an SDK application, namely:
`BaseApp` is a base type that implements the core of an SDK application, namely:
- The [Application-Blockchain Interface](#abci), for the state-machine to communicate with the underlying consensus engine (e.g. Tendermint).
- The [Application Blockchain Interface](#abci), for the state-machine to communicate with the
underlying consensus engine (e.g. Tendermint).
- A [Router](#routing), to route messages and queries to the appropriate module.
- Different [states](#states), as the state-machine can have different parallel states updated based on the ABCI message received.
- Different [states](#states), as the state-machine can have different volatile
states updated based on the ABCI message received.
The goal of `baseapp` is to provide a boilerplate SDK application that developers can easily extend to build their own custom application. Usually, developers will create a custom type for their application, like so:
The goal of `BaseApp` is to provide the fundamental layer of an SDK application
that developers can easily extend to build their own custom application. Usually,
developers will create a custom type for their application, like so:
```go
type app struct {
*bam.BaseApp // reference to baseapp
cdc *codec.Codec
type App struct {
// reference to a BaseApp
*bam.BaseApp
// list of application store keys
// list of application store keys
// list of application keepers
// list of application keepers
// module manager
// module manager
}
```
Extending the application with `baseapp` gives the former access to all of `baseapp`'s methods. This allows developers to compose their custom application with the modules they want, while not having to concern themselves with the hard work of implementing the ABCI, the routing and state management logic.
Extending the application with `BaseApp` gives the former access to all of `BaseApp`'s methods.
This allows developers to compose their custom application with the modules they want, while not
having to concern themselves with the hard work of implementing the ABCI, the routing and state
management logic.
## Type Definition
The [`baseapp` type](https://github.com/cosmos/cosmos-sdk/blob/master/baseapp/baseapp.go#L45-L91) holds many important parameters for any Cosmos SDK based application. Let us go through the most important components.
The [`BaseApp` type](https://github.com/cosmos/cosmos-sdk/blob/master/baseapp/baseapp.go#L53) holds
many important parameters for any Cosmos SDK based application. Let us go through the most
important components.
_Note: Not all parameters are described, only the most important ones. Refer to the [type definition](https://github.com/cosmos/cosmos-sdk/blob/master/baseapp/baseapp.go#L45-L91) for the full list_
> __Note__: Not all parameters are described, only the most important ones. Refer to the
[type definition](https://github.com/cosmos/cosmos-sdk/blob/master/baseapp/baseapp.go#L53) for the
full list.
First, the important parameters that are initialized during the initialization of the application:
First, the important parameters that are initialized during the bootstrapping of the application:
- [`CommitMultiStore`](./store.md#commit-multi-store): This is the main store of the application, which holds the canonical state that is committed at the [end of each block](#commit). This store is **not** cached, meaning it is not used to update the application's intermediate (un-committed) states. The `CommitMultiStore` is a multi-store, meaning a store of stores. Each module of the application uses one or multiple `KVStores` in the multi-store to persist their subset of the state.
- Database: The `db` is used by the `CommitMultiStore` to handle data storage.
- [Router](#message-routing): The `router` facilitates the routing of `messages` to the appropriate module for it to be processed. Here `message` refers to the transaction components that need to be processed by the application in order to update the state, and not to ABCI messages which implement the interface between the application and the underlying consensus engine.
- [Query Router](#query-routing): The `query router` facilitates the routing of queries to the appropriate module for it to be processed. These `queries` are not ABCI messages themselves, but they are relayed to the application from the underlying consensus engine via the ABCI message [`Query`](#query).
- [`TxDecoder`](https://godoc.org/github.com/cosmos/cosmos-sdk/types#TxDecoder): It is used to decode transaction `[]byte` relayed by the underlying Tendermint engine.
- [`BaseKey`]: This key is used to access the [main store](./store.md#main-store) in the `CommitMultiStore`. The main store is used to persist data related to the core of the application, like consensus parameters.
- [`AnteHandler`](#antehandler): This handler is used to handle signature verification and fee payment when a transaction is received.
- [`initChainer`](../basics/app-anatomy.md#initchainer), [`beginBlocker` and `endBlocker`](../basics/app-anatomy.md#beginblocker-and-endblocker): These are the functions executed when the application receives the [InitChain], [BeginBlock] and [EndBlock] ABCI messages from the underlying Tendermint engine.
- [`CommitMultiStore`](./store.md#commit-multi-store): This is the main store of the application,
which holds the canonical state that is committed at the [end of each block](#commit-1). This store
is **not** cached, meaning it is not used to update the application's volatile (un-committed) states.
The `CommitMultiStore` is a multi-store, meaning a store of stores. Each module of the application
uses one or multiple `KVStores` in the multi-store to persist their subset of the state.
- Database: The `db` is used by the `CommitMultiStore` to handle data persistence.
- [Router](#message-routing): The `router` facilitates the routing of `messages` to the appropriate
module for it to be processed. Here a `message` refers to the transaction components that need to be
processed by the application in order to update the state, and not to ABCI messages which implement
the interface between the application and the underlying consensus engine.
- [Query Router](#query-routing): The `query router` facilitates the routing of queries to the
appropriate module for it to be processed. These `queries` are not ABCI messages themselves, but they
are relayed to the application from the underlying consensus engine via the ABCI message [`Query`](#query).
- [`TxDecoder`](https://godoc.org/github.com/cosmos/cosmos-sdk/types#TxDecoder): It is used to decode
raw transaction bytes relayed by the underlying Tendermint engine.
- `BaseKey`: This key is used to access the main store in the `CommitMultiStore`. The main store is
used to persist data related to the core of the application, like consensus parameters.
- [`AnteHandler`](#antehandler): This handler is used to handle signature verification, fee payment,
and other pre-message execution checks when a transaction is received. It's executed during
[`CheckTx`](#checktx-1) and [`DeliverTx`](#delivertx-1).
- [`InitChainer`](../basics/app-anatomy.md#initchainer),
[`BeginBlocker` and `EndBlocker`](../basics/app-anatomy.md#beginblocker-and-endblocker): These are
the functions executed when the application receives the `InitChain`, `BeginBlock` and `EndBlock`
ABCI messages from the underlying Tendermint engine.
Then, parameters used to define [volatile states](#volatile-states) (i.e. cached states):
- `checkState`: This state is updated during [`CheckTx`](#checktx), and reset on [`Commit`](#commit).
- `deliverState`: This state is updated during [`DeliverTx`](#delivertx), and reset on [`Commit`](#commit).
- `checkState`: This state is updated during [`CheckTx`](#checktx-1), and reset on [`Commit`](#commit-1).
- `deliverState`: This state is updated during [`DeliverTx`](#delivertx-1), and set to `nil` on
[`Commit`](#commit-1) and gets re-initialized on BeginBlock.
Finally, a few more important parameters:
- `voteInfos`: This parameter carries the list of validators whose precommit is missing, either because they did not vote or because the proposer did not include their vote. This information is carried by the [context](#context) and can be used by the application for various things like punishing absent validators.
- `minGasPrices`: This parameter defines the minimum gas prices accepted by the node. This is a local parameter, meaning each full-node can set a different `minGasPrices`. It is run by the `anteHandler` during `CheckTx`, mainly as a spam protection mechanism. The transaction enters the [mempool](https://tendermint.com/docs/tendermint-core/mempool.html#transaction-ordering) only if the gas prices of the transaction is superior to one of the minimum gas price in `minGasPrices` (i.e. if `minGasPrices == 1uatom, 1upho`, the `gas-price` of the transaction must be superior to `1uatom` OR `1upho`).
- `appVersion`: Version of the application. It is set in the [application's constructor function](../basics/app-anatomy.md#constructor-function).
- `voteInfos`: This parameter carries the list of validators whose precommit is missing, either
because they did not vote or because the proposer did not include their vote. This information is
carried by the [Context](#context) and can be used by the application for various things like
punishing absent validators.
- `minGasPrices`: This parameter defines the minimum gas prices accepted by the node. This is a
**local** parameter, meaning each full-node can set a different `minGasPrices`. It is used in the
`AnteHandler` during [`CheckTx`](#checktx-1), mainly as a spam protection mechanism. The transaction
enters the [mempool](https://tendermint.com/docs/tendermint-core/mempool.html#transaction-ordering)
only if the gas prices of the transaction are greater than one of the minimum gas price in
`minGasPrices` (e.g. if `minGasPrices == 1uatom,1photon`, the `gas-price` of the transaction must be
greater than `1uatom` OR `1photon`).
- `appVersion`: Version of the application. It is set in the
[application's constructor function](../basics/app-anatomy.md#constructor-function).
## Constructor
`NewBaseApp(name string, logger log.Logger, db dbm.DB, txDecoder sdk.TxDecoder, options ...func(*BaseApp),)` is the constructor function for `baseapp`. It is called from the [application's constructor function](../basics/app-anatomy.md#constructor-function) each time the full-node is started.
```go
func NewBaseApp(
name string, logger log.Logger, db dbm.DB, txDecoder sdk.TxDecoder, options ...func(*BaseApp),
) *BaseApp {
`baseapp`'s constructor function is pretty straightforward. The only thing worth noting is the possibility to add additional [`options`](https://github.com/cosmos/cosmos-sdk/blob/master/baseapp/options.go) to `baseapp` by passing `options functions` to the constructor function, which will execute them in order. `options` are generally `setter` functions for important parameters, like `SetPruning()` to active pruning or `SetMinGasPrices()` to set the node's `min-gas-prices`.
// ...
}
```
A list of `options` examples can be found [here](https://github.com/cosmos/cosmos-sdk/blob/master/baseapp/options.go). Naturally, developers can add additional `options` based on their application's needs.
The `BaseApp` constructor function is pretty straightforward. The only thing worth noting is the
possibility to provide additional [`options`](https://github.com/cosmos/cosmos-sdk/blob/master/baseapp/options.go)
to the `BaseApp`, which will execute them in order. The `options` are generally `setter` functions
for important parameters, like `SetPruning()` to set pruning options or `SetMinGasPrices()` to set
the node's `min-gas-prices`.
A list of `options` examples can be found
[here](https://github.com/cosmos/cosmos-sdk/blob/master/baseapp/options.go). Naturally, developers
can add additional `options` based on their application's needs.
## States
`baseapp` handles various parallel states for different purposes. There is the [main state](#main-state), which is the canonical state of the application, and volatile states like [`checkState`](#checkState) and [`deliverState`](#deliverstate), which are used to handle temporary states in-between updates of the main state made during [`Commit`](#commit).
The `BaseApp` maintains two primary volatile states and a root or main state. The main state
is the canonical state of the application and the volatile states, `checkState` and `deliverState`,
are used to handle state transitions in-between the main state made during [`Commit`](#commit-1).
```
Updated whenever an unconfirmed Updated whenever a transaction To serve user queries relayed
transaction is received from the is received from the underlying from the underlying consensus
underlying consensus engine via consensus engine (as part of a block) engine via the Query ABCI message
CheckTx proposal via DeliverTx
+----------------------+ +----------------------+ +----------------------+
| CheckState(t)(0) | | DeliverState(t)(0) | | QueryState(t) |
+----------------------+ | | | |
CheckTx(tx1) | | | | |
v | | | |
+----------------------+ | | | |
| CheckState(t)(1) | | | | |
+----------------------+ | | | |
CheckTx(tx2) | | | | |
v | | | |
+----------------------+ | | | |
| CheckState(t)(2) | | | | |
+----------------------+ | | | |
CheckTx(tx3) | | | | |
v | | | |
+----------------------+ | | | |
| CheckState(t)(3) | | | | |
+----------------------+ +----------------------+ | |
DeliverTx(tx1) | | | |
v v | |
+----------------------+ +----------------------+ | |
| | | DeliverState(t)(1) | | |
| | +----------------------+ | |
DeliverTx(tx2) | | | | |
| | v | |
| | +----------------------+ | |
| | | DeliverState(t)(2) | | |
| | +----------------------+ | |
DeliverTx(tx3) | | | | |
| | v | |
| | +----------------------+ | |
| | | DeliverState(t)(3) | | |
+----------------------+ +----------------------+ +----------------------+
Commit() | | |
v v v
+----------------------+ +----------------------+ +----------------------+
| CheckState(t+1)(0) | | DeliverState(t+1)(0) | | QueryState(t+1) |
+----------------------+ | | | |
. . .
. . .
. . .
Internally, there is only a single `CommitMultiStore` which we refer to as the main or root state.
From this root state, we derive two volatile state through a mechanism called cache-wrapping. The
types can be illustrated as follows:
```
![Types](./baseapp_state_types.png)
### Main State
### InitChain
The main state is the canonical state of the application. It is initialized on [`InitChain`](#initchain) and updated on [`Commit`](#abci-commit) at the end of each block.
During `InitChain`, the two volatile states, `checkState` and `deliverState` are set by cache-wrapping
the root `CommitMultiStore`. Any subsequent reads and writes happen on cached versions of the `CommitMultiStore`.
```
+--------+ +--------+
| | | |
| S +----------------------------> | S' |
| | For each T in B: apply(T) | |
+--------+ +--------+
```
![InitChain](./baseapp_state-initchain.png)
The main state is held by `baseapp` in a structure called the `CommitMultiStore`. This multi-store is used by developers to instantiate all the stores they need for each of their application's modules.
### CheckTx
### Volatile States
During `CheckTx`, the `checkState`, which is based off of the last committed state from the root
store, is used for any reads and writes. Here we only execute the `AnteHandler` and verify a router
exists for every message in the transaction. Note, when we execute the `AnteHandler`, we cache-wrap
the already cache-wrapped `checkState`. This has the side effect that if the `AnteHandler` fails,
the state transitions won't be reflected in the `checkState` -- i.e. `checkState` is only updated on
success.
Volatile - or cached - states are used in between [`Commit`s](#commit) to manage temporary states. They are reset to the latest version of the main state after it is committed. There are two main volatile states:
![CheckTx](./baseapp_state-checktx.png)
- `checkState`: This cached state is initialized during [`InitChain`](#initchain), updated during [`CheckTx`](#abci-checktx) when an unconfirmed transaction is received, and reset to the [main state](#main-state) on [`Commit`](#abci-commit).
- `deliverState`: This cached state is initialized during [`BeginBlock`](#beginblock), updated during [`DeliverTx`](#abci-delivertx) when a transaction included in a block is processed, and reset to the [main state](#main-state) on [`Commit`](#abci-commit).
### BeginBlock
Both `checkState` and `deliverState` are of type [`state`](https://github.com/cosmos/cosmos-sdk/blob/master/baseapp/baseapp.go#L973-L976), which includes:
During `BeginBlock`, the `deliverState` is set for use in subsequent `DeliverTx` ABCI messages. The
`deliverState` is based off of the last committed state from the root store and is cache-wrapped.
Note, the `deliverState` is set to `nil` on [`Commit`](#commit-1).
- A [`CacheMultiStore`](https://github.com/cosmos/cosmos-sdk/blob/master/store/cachemulti/store.go), which is a cached version of the main `CommitMultiStore`. A new version of this store is committed at the end of each successful `CheckTx()`/`DeliverTx()` execution.
- A `Context`, which carries general information (like raw transaction size, block height, ...) that might be needed in order to process the transaction during `CheckTx()` and `DeliverTx()`. The `context` also holds a cache-wrapped version of the `CacheMultiStore`, so that the `CacheMultiStore` can maintain the correct version even if an internal step of `CheckTx()` or `DeliverTx()` fails.
![BeginBlock](./baseapp_state-begin_block.png)
### DeliverTx
The state flow for `DeliverTx` is nearly identical to `CheckTx` except state transitions occur on
the `deliverState` and messages in a transaction are executed. Similarly to `CheckTx`, state transitions
occur on a doubly cache-wrapped state -- `deliverState`. Successful message execution results in
writes being committed to `deliverState`. Note, if message execution fails, state transitions from
the AnteHandler are persisted.
![DeliverTx](./baseapp_state-deliver_tx.png)
### Commit
During `Commit` all the state transitions that occurred in the `deliverState` are finally written to
the root `CommitMultiStore` which in turn is committed to disk and results in a new application
root hash. These state transitions are now considered final. Finally, the `checkState` is set to the
newly committed state and `deliverState` is set to `nil` to be reset on `BeginBlock`.
![Commit](./baseapp_state-commit.png)
## Routing

Binary file not shown.

After

Width:  |  Height:  |  Size: 20 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 80 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 46 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 58 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 32 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 131 KiB

1
go.mod
View File

@ -11,6 +11,7 @@ require (
github.com/golang/mock v1.3.1-0.20190508161146-9fa652df1129
github.com/gorilla/mux v1.7.0
github.com/gorilla/websocket v1.4.1 // indirect
github.com/hashicorp/golang-lru v0.5.1
github.com/mattn/go-isatty v0.0.9
github.com/pelletier/go-toml v1.4.0
github.com/pkg/errors v0.8.1

14
go.sum
View File

@ -54,7 +54,6 @@ github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9
github.com/go-stack/stack v1.8.0 h1:5SgMzNM5HxrEjV0ww2lTmX6E2Izsfxas4+YHWRs3Lsk=
github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY=
github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
github.com/gogo/protobuf v1.2.1 h1:/s5zKNz0uPFCZ5hddgPdo2TK2TVrUNMn0OOX8/aZMTE=
github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4=
github.com/gogo/protobuf v1.3.0 h1:G8O7TerXerS4F6sx9OV7/nRfJdnXgHZu/S/7F2SN+UE=
github.com/gogo/protobuf v1.3.0/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o=
@ -64,7 +63,6 @@ github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfb
github.com/golang/mock v1.3.1-0.20190508161146-9fa652df1129 h1:tT8iWCYw4uOem71yYA3htfH+LNopJvcqZQshm56G5L4=
github.com/golang/mock v1.3.1-0.20190508161146-9fa652df1129/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y=
github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
github.com/golang/protobuf v1.3.0 h1:kbxbvI4Un1LUWKxufD+BiE6AEExYYgkQLQmLFqA1LFk=
github.com/golang/protobuf v1.3.0/go.mod h1:Qd/q+1AKNOZr9uGQzbzCmRO6sUih6GTPZv6a1/R87v0=
github.com/golang/protobuf v1.3.2 h1:6nsPYzhq5kReh6QImI3k5qWzO4PEbvbIW2cwSfR/6xs=
github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
@ -79,6 +77,8 @@ github.com/gorilla/mux v1.7.0/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2z
github.com/gorilla/websocket v1.2.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ=
github.com/gorilla/websocket v1.4.1 h1:q7AeDBpnBk8AogcD4DSag/Ukw/KV+YhzLj2bP5HvKCM=
github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE=
github.com/hashicorp/golang-lru v0.5.1 h1:0hERBMJE1eitiLkihrMvRVBYAkpHzc/J3QdDN+dAcgU=
github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
github.com/hashicorp/hcl v1.0.0 h1:0Anlzjpi4vEasTeNFn2mLJgTSwt0+6sfsiTG8qcWGx4=
github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ=
github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI=
@ -101,8 +101,6 @@ github.com/libp2p/go-buffer-pool v0.0.1 h1:9Rrn/H46cXjaA2HQ5Y8lyhOS1NhTkZ4yuEs2r
github.com/libp2p/go-buffer-pool v0.0.1/go.mod h1:xtyIz9PMobb13WaxR6Zo1Pd1zXJKYg0a8KiIvDp3TzQ=
github.com/magiconair/properties v1.8.0 h1:LLgXmsheXeRoUOBOjtwPQCWIYqM/LU1ayDtDePerRcY=
github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ=
github.com/mattn/go-isatty v0.0.6 h1:SrwhHcpV4nWrMGdNcC2kXpMfcBVYGDuTArqyhocJgvA=
github.com/mattn/go-isatty v0.0.6/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s=
github.com/mattn/go-isatty v0.0.9 h1:d5US/mDsogSGW37IV293h//ZFaeajb69h+EHFsv2xGg=
github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ=
github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU=
@ -116,7 +114,6 @@ github.com/onsi/ginkgo v1.7.0 h1:WSHQ+IS43OoUrWtD1/bbclrwK8TTH5hzp+umCiuxHgs=
github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
github.com/onsi/gomega v1.4.3 h1:RE1xgDvH7imwFD45h+u2SgIfERHlS2yNG4DObb5BSKU=
github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
github.com/pelletier/go-toml v1.2.0 h1:T5zMGML61Wp+FlcbWjRDT7yAxhJNAiPPLOFECq181zc=
github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic=
github.com/pelletier/go-toml v1.4.0 h1:u3Z1r+oOXJIkxqw34zVhyPgjBsm6X2wn21NWs/HfSeg=
github.com/pelletier/go-toml v1.4.0/go.mod h1:PN7xzY2wHTK0K9p34ErDQMlFxa51Fk0OUruD3k1mMwo=
@ -164,10 +161,8 @@ github.com/spf13/viper v1.0.0/go.mod h1:A8kyI5cUJhb8N+3pkfONlcEcZbueH6nhAm0Fq7Sr
github.com/spf13/viper v1.3.2 h1:VUFqw5KcqRf7i70GOzW7N+Q7+gxVBkSSqiXB12+JQ4M=
github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s=
github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A=
github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
github.com/stretchr/testify v1.3.0 h1:TivCn/peBQ7UY8ooIcPgZFpTNSz0Q2U6UrFlUfqbe0Q=
github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
github.com/stretchr/testify v1.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJyk=
github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
@ -197,7 +192,6 @@ golang.org/x/crypto v0.0.0-20170930174604-9419663f5a44/go.mod h1:6SG95UA2DQfeDnf
golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20190228161510-8dd112bcdc25/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2 h1:VklqNMn3ovrHsnt90PveolxSbWFaJdECFbxSq0Mqo2M=
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
golang.org/x/crypto v0.0.0-20190313024323-a1f597ede03a h1:YX8ljsm6wXlHZO+aRz9Exqr0evNhKRNe5K/gi+zKh4U=
golang.org/x/crypto v0.0.0-20190313024323-a1f597ede03a/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
@ -205,7 +199,6 @@ golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHl
golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20190311183353-d8887717615a h1:oWX7TPOiFAMXLq8o0ikBYfCJVlRHBcsciT5bXOrH628=
golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
golang.org/x/net v0.0.0-20190628185345-da137c7871d7 h1:rTIdg5QFRR7XCaK4LCjBiPbx8j4DQRpdYMnGn/bJUEU=
golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
@ -214,15 +207,12 @@ golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJ
golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20190423024810-112230192c58 h1:8gQV6CLnAEikrhgkHFbMAEhagSSnXWGV915qUMm9mrU=
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223 h1:DH4skfRX4EBpamg7iV4ZlCpblAHI6s6TDM39bFZumv8=
golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a h1:aYOabOQFp6Vj6W1F80affTUvO9UxmJRx8K0gsfABByQ=
golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg=

View File

@ -95,6 +95,10 @@ func (ms multiStore) GetStoreType() sdk.StoreType {
panic("not implemented")
}
func (ms multiStore) SetInterBlockCache(_ sdk.MultiStorePersistentCache) {
panic("not implemented")
}
var _ sdk.KVStore = kvStore{}
type kvStore struct {

View File

@ -20,13 +20,14 @@ import (
// Tendermint full-node start flags
const (
flagWithTendermint = "with-tendermint"
flagAddress = "address"
flagTraceStore = "trace-store"
flagPruning = "pruning"
flagCPUProfile = "cpu-profile"
FlagMinGasPrices = "minimum-gas-prices"
FlagHaltHeight = "halt-height"
flagWithTendermint = "with-tendermint"
flagAddress = "address"
flagTraceStore = "trace-store"
flagPruning = "pruning"
flagCPUProfile = "cpu-profile"
FlagMinGasPrices = "minimum-gas-prices"
FlagHaltHeight = "halt-height"
FlagInterBlockCache = "inter-block-cache"
)
// StartCmd runs the service passed in, either stand-alone or in-process with
@ -58,6 +59,7 @@ func StartCmd(ctx *Context, appCreator AppCreator) *cobra.Command {
"Minimum gas prices to accept for transactions; Any fee in a tx must meet this minimum (e.g. 0.01photino;0.0001stake)",
)
cmd.Flags().Uint64(FlagHaltHeight, 0, "Height at which to gracefully halt the chain and shutdown the node")
cmd.Flags().Bool(FlagInterBlockCache, true, "Enable inter-block caching")
cmd.Flags().String(flagCPUProfile, "", "Enable CPU profiling and write to the provided file")
// add support for all Tendermint-specific command line options

View File

@ -15,6 +15,7 @@ import (
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/baseapp"
"github.com/cosmos/cosmos-sdk/store"
sdk "github.com/cosmos/cosmos-sdk/types"
"github.com/cosmos/cosmos-sdk/x/auth"
authsimops "github.com/cosmos/cosmos-sdk/x/auth/simulation/operations"
@ -288,11 +289,18 @@ func invariants(app *SimApp) []sdk.Invariant {
return simulation.PeriodicInvariants(app.CrisisKeeper.Invariants(), flagPeriodValue, 0)
}
// Pass this in as an option to use a dbStoreAdapter instead of an IAVLStore for simulation speed.
// fauxMerkleModeOpt returns a BaseApp option to use a dbStoreAdapter instead of
// an IAVLStore for faster simulation speed.
func fauxMerkleModeOpt(bapp *baseapp.BaseApp) {
bapp.SetFauxMerkleMode()
}
// interBlockCacheOpt returns a BaseApp option function that sets the persistent
// inter-block write-through cache.
func interBlockCacheOpt() func(*baseapp.BaseApp) {
return baseapp.SetInterBlockCache(store.NewCommitKVStoreCacheManager())
}
// Profile with:
// /usr/local/go/bin/go test -benchmem -run=^$ github.com/cosmos/cosmos-sdk/simapp -bench ^BenchmarkFullAppSimulation$ -Commit=true -cpuprofile cpu.out
func BenchmarkFullAppSimulation(b *testing.B) {
@ -306,7 +314,8 @@ func BenchmarkFullAppSimulation(b *testing.B) {
db.Close()
os.RemoveAll(dir)
}()
app := NewSimApp(logger, db, nil, true, 0)
app := NewSimApp(logger, db, nil, true, 0, interBlockCacheOpt())
// Run randomized simulation
// TODO: parameterize numbers, save for a later PR
@ -593,6 +602,7 @@ func TestAppSimulationAfterImport(t *testing.T) {
newApp := NewSimApp(log.NewNopLogger(), newDB, nil, true, 0, fauxMerkleModeOpt)
require.Equal(t, "SimApp", newApp.Name())
newApp.InitChain(abci.RequestInitChain{
AppStateBytes: appState,
})
@ -630,7 +640,8 @@ func TestAppStateDeterminism(t *testing.T) {
for j := 0; j < numTimesToRunPerSeed; j++ {
logger := log.NewNopLogger()
db := dbm.NewMemDB()
app := NewSimApp(logger, db, nil, true, 0)
app := NewSimApp(logger, db, nil, true, 0, interBlockCacheOpt())
fmt.Printf(
"running non-determinism simulation; seed %d: %d/%d, attempt: %d/%d\n",
@ -671,7 +682,7 @@ func BenchmarkInvariants(b *testing.B) {
os.RemoveAll(dir)
}()
app := NewSimApp(logger, db, nil, true, 0)
app := NewSimApp(logger, db, nil, true, 0, interBlockCacheOpt())
// 2. Run parameterized simulation (w/o invariants)
_, simParams, simErr := simulation.SimulateFromSeed(

118
store/cache/cache.go vendored Normal file
View File

@ -0,0 +1,118 @@
package cache
import (
"fmt"
"github.com/cosmos/cosmos-sdk/store/cachekv"
"github.com/cosmos/cosmos-sdk/store/types"
lru "github.com/hashicorp/golang-lru"
)
var (
_ types.CommitKVStore = (*CommitKVStoreCache)(nil)
_ types.MultiStorePersistentCache = (*CommitKVStoreCacheManager)(nil)
// DefaultCommitKVStoreCacheSize defines the persistent ARC cache size for a
// CommitKVStoreCache.
DefaultCommitKVStoreCacheSize uint = 1000
)
type (
// CommitKVStoreCache implements an inter-block (persistent) cache that wraps a
// CommitKVStore. Reads first hit the internal ARC (Adaptive Replacement Cache).
// During a cache miss, the read is delegated to the underlying CommitKVStore
// and cached. Deletes and writes always happen to both the cache and the
// CommitKVStore in a write-through manner. Caching performed in the
// CommitKVStore and below is completely irrelevant to this layer.
CommitKVStoreCache struct {
types.CommitKVStore
cache *lru.ARCCache
}
// CommitKVStoreCacheManager maintains a mapping from a StoreKey to a
// CommitKVStoreCache. Each CommitKVStore, per StoreKey, is meant to be used
// in an inter-block (persistent) manner and typically provided by a
// CommitMultiStore.
CommitKVStoreCacheManager struct {
cacheSize uint
caches map[string]types.CommitKVStore
}
)
func NewCommitKVStoreCache(store types.CommitKVStore, size uint) *CommitKVStoreCache {
cache, err := lru.NewARC(int(size))
if err != nil {
panic(fmt.Errorf("failed to create KVStore cache: %s", err))
}
return &CommitKVStoreCache{
CommitKVStore: store,
cache: cache,
}
}
func NewCommitKVStoreCacheManager(size uint) *CommitKVStoreCacheManager {
return &CommitKVStoreCacheManager{
cacheSize: size,
caches: make(map[string]types.CommitKVStore),
}
}
// GetStoreCache returns a Cache from the CommitStoreCacheManager for a given
// StoreKey. If no Cache exists for the StoreKey, then one is created and set.
// The returned Cache is meant to be used in a persistent manner.
func (cmgr *CommitKVStoreCacheManager) GetStoreCache(key types.StoreKey, store types.CommitKVStore) types.CommitKVStore {
if cmgr.caches[key.Name()] == nil {
cmgr.caches[key.Name()] = NewCommitKVStoreCache(store, cmgr.cacheSize)
}
return cmgr.caches[key.Name()]
}
// Reset resets in the internal caches.
func (cmgr *CommitKVStoreCacheManager) Reset() {
cmgr.caches = make(map[string]types.CommitKVStore)
}
// CacheWrap returns the inter-block cache as a cache-wrapped CommitKVStore.
func (ckv *CommitKVStoreCache) CacheWrap() types.CacheWrap {
return cachekv.NewStore(ckv)
}
// Get retrieves a value by key. It will first look in the write-through cache.
// If the value doesn't exist in the write-through cache, the query is delegated
// to the underlying CommitKVStore.
func (ckv *CommitKVStoreCache) Get(key []byte) []byte {
types.AssertValidKey(key)
keyStr := string(key)
valueI, ok := ckv.cache.Get(keyStr)
if ok {
// cache hit
return valueI.([]byte)
}
// cache miss; write to cache
value := ckv.CommitKVStore.Get(key)
ckv.cache.Add(keyStr, value)
return value
}
// Set inserts a key/value pair into both the write-through cache and the
// underlying CommitKVStore.
func (ckv *CommitKVStoreCache) Set(key, value []byte) {
types.AssertValidKey(key)
types.AssertValidValue(value)
ckv.cache.Add(string(key), value)
ckv.CommitKVStore.Set(key, value)
}
// Delete removes a key/value pair from both the write-through cache and the
// underlying CommitKVStore.
func (ckv *CommitKVStoreCache) Delete(key []byte) {
ckv.cache.Remove(string(key))
ckv.CommitKVStore.Delete(key)
}

51
store/cache/cache_test.go vendored Normal file
View File

@ -0,0 +1,51 @@
package cache_test
import (
"fmt"
"testing"
"github.com/cosmos/cosmos-sdk/store/cache"
iavlstore "github.com/cosmos/cosmos-sdk/store/iavl"
"github.com/cosmos/cosmos-sdk/store/types"
"github.com/stretchr/testify/require"
"github.com/tendermint/iavl"
dbm "github.com/tendermint/tm-db"
)
func TestGetOrSetStoreCache(t *testing.T) {
db := dbm.NewMemDB()
mngr := cache.NewCommitKVStoreCacheManager(cache.DefaultCommitKVStoreCacheSize)
sKey := types.NewKVStoreKey("test")
store := iavlstore.UnsafeNewStore(iavl.NewMutableTree(db, 100), 10, 10)
store2 := mngr.GetStoreCache(sKey, store)
require.NotNil(t, store2)
require.Equal(t, store2, mngr.GetStoreCache(sKey, store))
}
func TestStoreCache(t *testing.T) {
db := dbm.NewMemDB()
mngr := cache.NewCommitKVStoreCacheManager(cache.DefaultCommitKVStoreCacheSize)
sKey := types.NewKVStoreKey("test")
store := iavlstore.UnsafeNewStore(iavl.NewMutableTree(db, 100), 10, 10)
kvStore := mngr.GetStoreCache(sKey, store)
for i := uint(0); i < cache.DefaultCommitKVStoreCacheSize*2; i++ {
key := []byte(fmt.Sprintf("key_%d", i))
value := []byte(fmt.Sprintf("value_%d", i))
kvStore.Set(key, value)
res := kvStore.Get(key)
require.Equal(t, res, value)
require.Equal(t, res, store.Get(key))
kvStore.Delete(key)
require.Nil(t, kvStore.Get(key))
require.Nil(t, store.Get(key))
}
}

View File

@ -34,7 +34,6 @@ type Store struct {
var _ types.CacheKVStore = (*Store)(nil)
// nolint
func NewStore(parent types.KVStore) *Store {
return &Store{
cache: make(map[string]*cValue),
@ -53,6 +52,7 @@ func (store *Store) GetStoreType() types.StoreType {
func (store *Store) Get(key []byte) (value []byte) {
store.mtx.Lock()
defer store.mtx.Unlock()
types.AssertValidKey(key)
cacheValue, ok := store.cache[string(key)]
@ -70,6 +70,7 @@ func (store *Store) Get(key []byte) (value []byte) {
func (store *Store) Set(key []byte, value []byte) {
store.mtx.Lock()
defer store.mtx.Unlock()
types.AssertValidKey(key)
types.AssertValidValue(value)
@ -86,6 +87,7 @@ func (store *Store) Has(key []byte) bool {
func (store *Store) Delete(key []byte) {
store.mtx.Lock()
defer store.mtx.Unlock()
types.AssertValidKey(key)
store.setCacheValue(key, nil, true, true)

View File

@ -29,10 +29,12 @@ type Store struct {
var _ types.CacheMultiStore = Store{}
// NewFromKVStore creates a new Store object from a mapping of store keys to
// CacheWrapper objects and a KVStore as the database. Each CacheWrapper store
// is cache-wrapped.
func NewFromKVStore(
store types.KVStore,
stores map[types.StoreKey]types.CacheWrapper, keys map[string]types.StoreKey,
traceWriter io.Writer, traceContext types.TraceContext,
store types.KVStore, stores map[types.StoreKey]types.CacheWrapper,
keys map[string]types.StoreKey, traceWriter io.Writer, traceContext types.TraceContext,
) Store {
cms := Store{
db: cachekv.NewStore(store),
@ -53,11 +55,13 @@ func NewFromKVStore(
return cms
}
// NewStore creates a new Store object from a mapping of store keys to
// CacheWrapper objects. Each CacheWrapper store is cache-wrapped.
func NewStore(
db dbm.DB,
stores map[types.StoreKey]types.CacheWrapper, keys map[string]types.StoreKey,
db dbm.DB, stores map[types.StoreKey]types.CacheWrapper, keys map[string]types.StoreKey,
traceWriter io.Writer, traceContext types.TraceContext,
) Store {
return NewFromKVStore(dbadapter.Store{DB: db}, stores, keys, traceWriter, traceContext)
}
@ -66,6 +70,7 @@ func newCacheMultiStoreFromCMS(cms Store) Store {
for k, v := range cms.stores {
stores[k] = v
}
return NewFromKVStore(cms.db, stores, nil, cms.traceWriter, cms.traceContext)
}

View File

@ -22,8 +22,34 @@ const (
defaultIAVLCacheSize = 10000
)
// LoadStore loads the iavl store
func LoadStore(db dbm.DB, id types.CommitID, pruning types.PruningOptions, lazyLoading bool) (types.CommitStore, error) {
var (
_ types.KVStore = (*Store)(nil)
_ types.CommitStore = (*Store)(nil)
_ types.CommitKVStore = (*Store)(nil)
_ types.Queryable = (*Store)(nil)
)
// Store Implements types.KVStore and CommitKVStore.
type Store struct {
tree Tree
// How many old versions we hold onto.
// A value of 0 means keep no recent states.
numRecent int64
// This is the distance between state-sync waypoint states to be stored.
// See https://github.com/tendermint/tendermint/issues/828
// A value of 1 means store every state.
// A value of 0 means store no waypoints. (node cannot assist in state-sync)
// By default this value should be set the same across all nodes,
// so that nodes can know the waypoints their peers store.
storeEvery int64
}
// LoadStore returns an IAVL Store as a CommitKVStore. Internally it will load the
// store's version (id) from the provided DB. An error is returned if the version
// fails to load.
func LoadStore(db dbm.DB, id types.CommitID, pruning types.PruningOptions, lazyLoading bool) (types.CommitKVStore, error) {
tree := iavl.NewMutableTree(db, defaultIAVLCacheSize)
var err error
@ -43,38 +69,15 @@ func LoadStore(db dbm.DB, id types.CommitID, pruning types.PruningOptions, lazyL
return iavl, nil
}
//----------------------------------------
var _ types.KVStore = (*Store)(nil)
var _ types.CommitStore = (*Store)(nil)
var _ types.Queryable = (*Store)(nil)
// Store Implements types.KVStore and CommitStore.
type Store struct {
tree Tree
// How many old versions we hold onto.
// A value of 0 means keep no recent states.
numRecent int64
// This is the distance between state-sync waypoint states to be stored.
// See https://github.com/tendermint/tendermint/issues/828
// A value of 1 means store every state.
// A value of 0 means store no waypoints. (node cannot assist in state-sync)
// By default this value should be set the same across all nodes,
// so that nodes can know the waypoints their peers store.
storeEvery int64
}
// CONTRACT: tree should be fully loaded.
// nolint: unparam
// UnsafeNewStore returns a reference to a new IAVL Store.
//
// CONTRACT: The IAVL tree should be fully loaded.
func UnsafeNewStore(tree *iavl.MutableTree, numRecent int64, storeEvery int64) *Store {
st := &Store{
return &Store{
tree: tree,
numRecent: numRecent,
storeEvery: storeEvery,
}
return st
}
// GetImmutable returns a reference to a new store backed by an immutable IAVL
@ -167,9 +170,9 @@ func (st *Store) Set(key, value []byte) {
}
// Implements types.KVStore.
func (st *Store) Get(key []byte) (value []byte) {
_, v := st.tree.Get(key)
return v
func (st *Store) Get(key []byte) []byte {
_, value := st.tree.Get(key)
return value
}
// Implements types.KVStore.

View File

@ -32,12 +32,14 @@ type Store struct {
lastCommitID types.CommitID
pruningOpts types.PruningOptions
storesParams map[types.StoreKey]storeParams
stores map[types.StoreKey]types.CommitStore
stores map[types.StoreKey]types.CommitKVStore
keysByName map[string]types.StoreKey
lazyLoading bool
traceWriter io.Writer
traceContext types.TraceContext
interBlockCache types.MultiStorePersistentCache
}
var _ types.CommitMultiStore = (*Store)(nil)
@ -48,7 +50,7 @@ func NewStore(db dbm.DB) *Store {
return &Store{
db: db,
storesParams: make(map[types.StoreKey]storeParams),
stores: make(map[types.StoreKey]types.CommitStore),
stores: make(map[types.StoreKey]types.CommitKVStore),
keysByName: make(map[string]types.StoreKey),
}
}
@ -141,7 +143,7 @@ func (rs *Store) loadVersion(ver int64, upgrades *types.StoreUpgrades) error {
}
// load each Store (note this doesn't panic on unmounted keys now)
var newStores = make(map[types.StoreKey]types.CommitStore)
var newStores = make(map[types.StoreKey]types.CommitKVStore)
for key, storeParams := range rs.storesParams {
// Load it
@ -220,6 +222,13 @@ func moveKVStoreData(oldDB types.KVStore, newDB types.KVStore) error {
return deleteKVStore(oldDB)
}
// SetInterBlockCache sets the Store's internal inter-block (persistent) cache.
// When this is defined, all CommitKVStores will be wrapped with their respective
// inter-block cache.
func (rs *Store) SetInterBlockCache(c types.MultiStorePersistentCache) {
rs.interBlockCache = c
}
// SetTracer sets the tracer for the MultiStore that the underlying
// stores will utilize to trace operations. A MultiStore is returned.
func (rs *Store) SetTracer(w io.Writer) types.MultiStore {
@ -292,7 +301,8 @@ func (rs *Store) CacheWrapWithTrace(_ io.Writer, _ types.TraceContext) types.Cac
//----------------------------------------
// +MultiStore
// Implements MultiStore.
// CacheMultiStore cache-wraps the multi-store and returns a CacheMultiStore.
// It implements the MultiStore interface.
func (rs *Store) CacheMultiStore() types.CacheMultiStore {
stores := make(map[types.StoreKey]types.CacheWrapper)
for k, v := range rs.stores {
@ -442,7 +452,7 @@ func parsePath(path string) (storeName string, subpath string, err errors.Error)
//----------------------------------------
// Note: why do we use key and params.key in different places. Seems like there should be only one key used.
func (rs *Store) loadCommitStoreFromParams(key types.StoreKey, id types.CommitID, params storeParams) (store types.CommitStore, err error) {
func (rs *Store) loadCommitStoreFromParams(key types.StoreKey, id types.CommitID, params storeParams) (types.CommitKVStore, error) {
var db dbm.DB
if params.db != nil {
@ -457,7 +467,19 @@ func (rs *Store) loadCommitStoreFromParams(key types.StoreKey, id types.CommitID
panic("recursive MultiStores not yet supported")
case types.StoreTypeIAVL:
return iavl.LoadStore(db, id, rs.pruningOpts, rs.lazyLoading)
store, err := iavl.LoadStore(db, id, rs.pruningOpts, rs.lazyLoading)
if err != nil {
return nil, err
}
if rs.interBlockCache != nil {
// Wrap and get a CommitKVStore with inter-block caching. Note, this should
// only wrap the primary CommitKVStore, not any store that is already
// cache-wrapped as that will create unexpected behavior.
store = rs.interBlockCache.GetStoreCache(key, store)
}
return store, err
case types.StoreTypeDB:
return commitDBStoreAdapter{Store: dbadapter.Store{DB: db}}, nil
@ -465,7 +487,7 @@ func (rs *Store) loadCommitStoreFromParams(key types.StoreKey, id types.CommitID
case types.StoreTypeTransient:
_, ok := key.(*types.TransientStoreKey)
if !ok {
return store, fmt.Errorf("invalid StoreKey for StoreTypeTransient: %s", key.String())
return nil, fmt.Errorf("invalid StoreKey for StoreTypeTransient: %s", key.String())
}
return transient.NewStore(), nil
@ -573,7 +595,7 @@ func setLatestVersion(batch dbm.Batch, version int64) {
}
// Commits each store and returns a new commitInfo.
func commitStores(version int64, storeMap map[types.StoreKey]types.CommitStore) commitInfo {
func commitStores(version int64, storeMap map[types.StoreKey]types.CommitKVStore) commitInfo {
storeInfos := make([]storeInfo, 0, len(storeMap))
for key, store := range storeMap {

View File

@ -410,7 +410,7 @@ func getExpectedCommitID(store *Store, ver int64) types.CommitID {
}
}
func hashStores(stores map[types.StoreKey]types.CommitStore) []byte {
func hashStores(stores map[types.StoreKey]types.CommitKVStore) []byte {
m := make(map[string][]byte, len(stores))
for key, store := range stores {
name := key.Name()

View File

@ -3,6 +3,7 @@ package store
import (
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/store/cache"
"github.com/cosmos/cosmos-sdk/store/rootmulti"
"github.com/cosmos/cosmos-sdk/store/types"
)
@ -11,6 +12,10 @@ func NewCommitMultiStore(db dbm.DB) types.CommitMultiStore {
return rootmulti.NewStore(db)
}
func NewCommitKVStoreCacheManager() types.MultiStorePersistentCache {
return cache.NewCommitKVStoreCacheManager(cache.DefaultCommitKVStoreCacheSize)
}
func NewPruningOptionsFromString(strategy string) (opt PruningOptions) {
switch strategy {
case "nothing":

View File

@ -151,6 +151,10 @@ type CommitMultiStore interface {
// must be idempotent (return the same commit id). Otherwise the behavior is
// undefined.
LoadVersion(ver int64) error
// Set an inter-block (persistent) cache that maintains a mapping from
// StoreKeys to CommitKVStores.
SetInterBlockCache(MultiStorePersistentCache)
}
//---------subsp-------------------------------
@ -328,3 +332,14 @@ type KVPair cmn.KVPair
// TraceContext contains TraceKVStore context data. It will be written with
// every trace operation.
type TraceContext map[string]interface{}
// MultiStorePersistentCache defines an interface which provides inter-block
// (persistent) caching capabilities for multiple CommitKVStores based on StoreKeys.
type MultiStorePersistentCache interface {
// Wrap and return the provided CommitKVStore with an inter-block (persistent)
// cache.
GetStoreCache(key StoreKey, store CommitKVStore) CommitKVStore
// Reset the entire set of internal caches.
Reset()
}

View File

@ -14,15 +14,16 @@ type (
// nolint - reexport
type (
Store = types.Store
Committer = types.Committer
CommitStore = types.CommitStore
Queryable = types.Queryable
MultiStore = types.MultiStore
CacheMultiStore = types.CacheMultiStore
CommitMultiStore = types.CommitMultiStore
KVStore = types.KVStore
Iterator = types.Iterator
Store = types.Store
Committer = types.Committer
CommitStore = types.CommitStore
Queryable = types.Queryable
MultiStore = types.MultiStore
CacheMultiStore = types.CacheMultiStore
CommitMultiStore = types.CommitMultiStore
MultiStorePersistentCache = types.MultiStorePersistentCache
KVStore = types.KVStore
Iterator = types.Iterator
)
// StoreDecoderRegistry defines each of the modules store decoders. Used for ImportExport

View File

@ -195,6 +195,8 @@ func (s Subspace) Update(ctx sdk.Context, key []byte, param []byte) error {
}
s.Set(ctx, key, dest)
// TODO: Remove; seems redundant as Set already does this.
tStore := s.transientStore(ctx)
tStore.Set(key, []byte{})

View File

@ -4,7 +4,6 @@ import (
"math/rand"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/secp256k1"
sdk "github.com/cosmos/cosmos-sdk/types"
@ -38,13 +37,8 @@ func RandomAccounts(r *rand.Rand, n int) []Account {
// don't need that much entropy for simulation
privkeySeed := make([]byte, 15)
r.Read(privkeySeed)
useSecp := r.Int63()%2 == 0
if useSecp {
accs[i].PrivKey = secp256k1.GenPrivKeySecp256k1(privkeySeed)
} else {
accs[i].PrivKey = ed25519.GenPrivKeyFromSecret(privkeySeed)
}
accs[i].PrivKey = secp256k1.GenPrivKeySecp256k1(privkeySeed)
accs[i].PubKey = accs[i].PrivKey.PubKey()
accs[i].Address = sdk.AccAddress(accs[i].PubKey.Address())
}