From 9e830270087480bde2a2c2634003e7d64a33c233 Mon Sep 17 00:00:00 2001 From: Jernej Kos Date: Sun, 16 Feb 2020 13:49:19 +0100 Subject: [PATCH] go/consensus/tendermint: Use MKVS for storing application state --- .changelog/1898.breaking.md | 1 + go/common/version/version.go | 2 +- go/consensus/tendermint/abci/context.go | 81 +- go/consensus/tendermint/abci/errors.go | 32 + go/consensus/tendermint/abci/mux.go | 41 +- go/consensus/tendermint/abci/prune.go | 111 ++- go/consensus/tendermint/abci/prune_test.go | 77 +- go/consensus/tendermint/abci/state.go | 375 +++++----- go/consensus/tendermint/abci/timer.go | 58 +- go/consensus/tendermint/apps/beacon/beacon.go | 4 +- .../tendermint/apps/beacon/genesis.go | 7 +- go/consensus/tendermint/apps/beacon/query.go | 26 +- .../tendermint/apps/beacon/state/state.go | 58 +- .../apps/epochtime_mock/epochtime_mock.go | 13 +- .../tendermint/apps/epochtime_mock/query.go | 26 +- .../tendermint/apps/epochtime_mock/state.go | 69 +- .../tendermint/apps/keymanager/genesis.go | 7 +- .../tendermint/apps/keymanager/keymanager.go | 19 +- .../tendermint/apps/keymanager/query.go | 28 +- .../tendermint/apps/keymanager/state/state.go | 69 +- .../apps/keymanager/transactions.go | 12 +- .../tendermint/apps/registry/genesis.go | 51 +- .../tendermint/apps/registry/query.go | 38 +- .../tendermint/apps/registry/registry.go | 20 +- .../tendermint/apps/registry/state/state.go | 701 ++++++++++-------- .../tendermint/apps/registry/transactions.go | 82 +- .../tendermint/apps/roothash/genesis.go | 18 +- .../tendermint/apps/roothash/query.go | 28 +- .../tendermint/apps/roothash/roothash.go | 83 ++- .../tendermint/apps/roothash/state/state.go | 95 ++- .../tendermint/apps/roothash/transactions.go | 26 +- .../tendermint/apps/scheduler/genesis.go | 16 +- .../tendermint/apps/scheduler/query.go | 30 +- .../tendermint/apps/scheduler/scheduler.go | 60 +- .../tendermint/apps/scheduler/state/state.go | 200 ++--- go/consensus/tendermint/apps/staking/fees.go | 38 +- .../tendermint/apps/staking/genesis.go | 126 ++-- .../apps/staking/proposing_rewards.go | 21 +- go/consensus/tendermint/apps/staking/query.go | 38 +- .../apps/staking/signing_rewards.go | 26 +- .../tendermint/apps/staking/slashing.go | 8 +- .../tendermint/apps/staking/slashing_test.go | 18 +- .../tendermint/apps/staking/staking.go | 47 +- .../apps/staking/state/accumulator.go | 55 +- .../apps/staking/state/accumulator_test.go | 26 +- .../tendermint/apps/staking/state/gas.go | 13 +- .../tendermint/apps/staking/state/state.go | 587 +++++++++------ .../apps/staking/state/state_test.go | 97 ++- .../tendermint/apps/staking/transactions.go | 151 ++-- go/consensus/tendermint/apps/staking/votes.go | 2 +- .../apps/supplementarysanity/checks.go | 114 +-- .../supplementarysanity.go | 8 +- go/consensus/tendermint/db/init.go | 5 + .../epochtime_mock/epochtime_mock.go | 4 + .../tendermint/inspector/inspector.go | 42 -- go/consensus/tendermint/service/service.go | 3 - go/consensus/tendermint/tendermint.go | 12 +- go/go.mod | 2 - go/go.sum | 2 - go/oasis-node/cmd/debug/debug.go | 2 - .../cmd/debug/tendermint/tendermint.go | 81 -- go/registry/api/api.go | 31 +- go/registry/api/sanity_check.go | 20 +- go/roothash/api/commitment/pool.go | 5 +- go/roothash/api/commitment/pool_test.go | 3 +- go/storage/api/api.go | 35 +- go/storage/client/tests/tests.go | 5 +- go/storage/database/database.go | 17 +- go/storage/database/database_test.go | 1 + go/storage/metrics.go | 61 +- go/storage/mkvs/urkel/db/api/api.go | 38 +- go/storage/mkvs/urkel/db/badger/badger.go | 69 +- go/storage/mkvs/urkel/insert.go | 20 +- go/storage/mkvs/urkel/urkel.go | 9 + go/storage/mkvs/urkel/urkel_test.go | 16 +- go/upgrade/migrations/dummy.go | 13 +- go/worker/common/committee/group.go | 2 +- go/worker/storage/committee/node.go | 8 +- 78 files changed, 2413 insertions(+), 2032 deletions(-) create mode 100644 .changelog/1898.breaking.md create mode 100644 go/consensus/tendermint/abci/errors.go delete mode 100644 go/consensus/tendermint/inspector/inspector.go delete mode 100644 go/oasis-node/cmd/debug/tendermint/tendermint.go diff --git a/.changelog/1898.breaking.md b/.changelog/1898.breaking.md new file mode 100644 index 00000000000..d0ddd9f4cd1 --- /dev/null +++ b/.changelog/1898.breaking.md @@ -0,0 +1 @@ +go/consensus/tendermint: Use MKVS for storing application state diff --git a/go/common/version/version.go b/go/common/version/version.go index 79272111672..3f247524920 100644 --- a/go/common/version/version.go +++ b/go/common/version/version.go @@ -70,7 +70,7 @@ var ( // // NOTE: Any change in the major or minor versions are considered // breaking changes for the protocol. - ConsensusProtocol = Version{Major: 0, Minor: 23, Patch: 0} + ConsensusProtocol = Version{Major: 0, Minor: 24, Patch: 0} // Tendermint exposes the tendermint core version. Tendermint = parseSemVerStr(version.TMCoreSemVer) diff --git a/go/consensus/tendermint/abci/context.go b/go/consensus/tendermint/abci/context.go index 0bd4cd55de7..bd3b587b9f4 100644 --- a/go/consensus/tendermint/abci/context.go +++ b/go/consensus/tendermint/abci/context.go @@ -3,14 +3,15 @@ package abci import ( "bytes" "context" + "fmt" "time" - "github.com/tendermint/iavl" "github.com/tendermint/tendermint/abci/types" "github.com/oasislabs/oasis-core/go/common/crypto/signature" "github.com/oasislabs/oasis-core/go/common/logging" "github.com/oasislabs/oasis-core/go/consensus/tendermint/api" + mkvs "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel" ) type contextKey struct{} @@ -57,7 +58,7 @@ func (m ContextMode) String() string { // Context is the context of processing a transaction/block. type Context struct { - ctx context.Context + context.Context mode ContextMode currentTime time.Time @@ -69,22 +70,13 @@ type Context struct { txSigner signature.PublicKey appState ApplicationState - state *iavl.MutableTree + state mkvs.Tree blockHeight int64 blockCtx *BlockContext - logger *logging.Logger -} + stateCheckpoint *StateCheckpoint -// NewMockContext creates a new mock context for use in tests. -func NewMockContext(mode ContextMode, now time.Time) *Context { - return &Context{ - ctx: context.Background(), - mode: mode, - currentTime: now, - gasAccountant: NewNopGasAccountant(), - logger: logging.GetLogger("consensus/tendermint/abci").With("mode", mode), - } + logger *logging.Logger } // FromCtx extracts an ABCI context from a context.Context if one has been @@ -99,13 +91,19 @@ func FromCtx(ctx context.Context) *Context { // After calling this method, the context should no longer be used. func (c *Context) Close() { if c.IsSimulation() { - c.state.Rollback() + if tree, ok := c.state.(mkvs.ClosableTree); ok { + tree.Close() + } } c.events = nil c.appState = nil c.state = nil c.blockCtx = nil + + if c.stateCheckpoint != nil { + panic("context: open checkpoint was never committed or discarded") + } } // Logger returns the logger associated with this context. @@ -113,11 +111,6 @@ func (c *Context) Logger() *logging.Logger { return c.logger } -// Ctx returns a context.Context that is associated with this ABCI context. -func (c *Context) Ctx() context.Context { - return c.ctx -} - // Mode returns the context mode. func (c *Context) Mode() ContextMode { return c.mode @@ -225,8 +218,11 @@ func (c *Context) Now() time.Time { return c.currentTime } -// State returns the mutable state tree. -func (c *Context) State() *iavl.MutableTree { +// State returns the state tree associated with this context. +func (c *Context) State() mkvs.KeyValueTree { + if c.stateCheckpoint != nil { + return c.stateCheckpoint.overlay + } return c.state } @@ -253,34 +249,47 @@ func (c *Context) BlockContext() *BlockContext { return c.blockCtx } -// NewStateCheckpoint creates a new state checkpoint. -func (c *Context) NewStateCheckpoint() *StateCheckpoint { - return &StateCheckpoint{ - ImmutableTree: *c.State().ImmutableTree, - ctx: c, +// StartCheckpoint starts a new state checkpoint. Any further updates to the context's state will +// be performed against the checkpoint and will only be committed in case of an explicit Commit. +// +// The caller must make sure to call either Close or Commit on the checkpoint, otherwise this will +// leak resources. +func (c *Context) StartCheckpoint() *StateCheckpoint { + if c.stateCheckpoint != nil { + panic("context: nested checkpoints are not allowed") } + c.stateCheckpoint = &StateCheckpoint{ + ctx: c, + overlay: mkvs.NewOverlay(c.state), + } + return c.stateCheckpoint } // StateCheckpoint is a state checkpoint that can be used to rollback state. type StateCheckpoint struct { - iavl.ImmutableTree - - ctx *Context + ctx *Context + overlay mkvs.OverlayTree } -// Close releases resources associated with the checkpoint. +// Close releases resources associated with the checkpoint without committing it. func (sc *StateCheckpoint) Close() { + if sc.ctx == nil { + return + } + sc.overlay.Close() + sc.ctx.stateCheckpoint = nil sc.ctx = nil } -// Rollback rolls back the active state to the one from the checkpoint. -func (sc *StateCheckpoint) Rollback() { +// Commit commits any changes performed since the checkpoint was created. +func (sc *StateCheckpoint) Commit() { if sc.ctx == nil { return } - st := sc.ctx.State() - st.Rollback() - st.ImmutableTree = &sc.ImmutableTree + if err := sc.overlay.Commit(sc.ctx); err != nil { + panic(fmt.Errorf("context: failed to commit checkpoint: %w", err)) + } + sc.Close() } // BlockContextKey is an interface for a block context key. diff --git a/go/consensus/tendermint/abci/errors.go b/go/consensus/tendermint/abci/errors.go new file mode 100644 index 00000000000..464016d3014 --- /dev/null +++ b/go/consensus/tendermint/abci/errors.go @@ -0,0 +1,32 @@ +package abci + +import ( + "errors" + "fmt" +) + +type errorUnavailableState struct { + inner error +} + +func (e *errorUnavailableState) Error() string { + return fmt.Sprintf("unavailable/corrupted state: %s", e.inner.Error()) +} + +func (e *errorUnavailableState) Unwrap() error { + return e.inner +} + +// UnavailableStateError wraps an error in an unavailable state error. +func UnavailableStateError(err error) error { + if err == nil { + return nil + } + return &errorUnavailableState{err} +} + +// IsUnavailableStateError returns true if any error in err's chain is an unavailable state error. +func IsUnavailableStateError(err error) bool { + var e *errorUnavailableState + return errors.As(err, &e) +} diff --git a/go/consensus/tendermint/abci/mux.go b/go/consensus/tendermint/abci/mux.go index 963fa55de3f..acfea781d29 100644 --- a/go/consensus/tendermint/abci/mux.go +++ b/go/consensus/tendermint/abci/mux.go @@ -32,7 +32,6 @@ import ( const ( stateKeyGenesisDigest = "OasisGenesisDigest" - stateKeyGenesisRequest = "OasisGenesisRequest" stateKeyInitChainEvents = "OasisInitChainEvents" metricsUpdateInterval = 10 * time.Second @@ -57,6 +56,7 @@ var ( // ApplicationConfig is the configuration for the consensus application. type ApplicationConfig struct { DataDir string + StorageBackend string Pruning PruneConfig HaltEpochHeight epochtime.EpochTime MinGasPrice uint64 @@ -218,11 +218,6 @@ func (a *ApplicationServer) RegisterHaltHook(hook func(ctx context.Context, bloc a.mux.registerHaltHook(hook) } -// Pruner returns the ABCI state pruner. -func (a *ApplicationServer) Pruner() StatePruner { - return a.mux.state.statePruner -} - // SetEpochtime sets the mux epochtime. // // Epochtime must be set before the multiplexer can be used. @@ -396,7 +391,10 @@ func (mux *abciMux) InitChain(req types.RequestInitChain) types.ResponseInitChai tmp := bytes.NewBuffer(nil) _ = types.WriteMessage(&req, tmp) genesisDigest := sha512.Sum512_256(tmp.Bytes()) - mux.state.deliverTxTree.Set([]byte(stateKeyGenesisDigest), genesisDigest[:]) + err = mux.state.deliverTxTree.Insert(mux.state.ctx, []byte(stateKeyGenesisDigest), genesisDigest[:]) + if err != nil { + panic(err) + } resp := mux.BaseApplication.InitChain(req) @@ -422,11 +420,6 @@ func (mux *abciMux) InitChain(req types.RequestInitChain) types.ResponseInitChai mux.logger.Debug("Genesis hook dispatch complete") }() - // TODO: remove stateKeyGenesisRequest here, see oasis-core#2426 - b, _ = req.Marshal() - mux.state.deliverTxTree.Set([]byte(stateKeyGenesisRequest), b) - mux.state.checkTxTree.Set([]byte(stateKeyGenesisRequest), b) - // Call InitChain() on all applications. mux.logger.Debug("InitChain: initializing applications") @@ -452,7 +445,10 @@ func (mux *abciMux) InitChain(req types.RequestInitChain) types.ResponseInitChai // Since returning emitted events doesn't work for InitChain() response yet, // we store those and return them in BeginBlock(). evBinary := cbor.Marshal(ctx.GetEvents()) - mux.state.deliverTxTree.Set([]byte(stateKeyInitChainEvents), evBinary) + err = mux.state.deliverTxTree.Insert(mux.state.ctx, []byte(stateKeyInitChainEvents), evBinary) + if err != nil { + panic(err) + } return resp } @@ -483,7 +479,7 @@ func (mux *abciMux) BeginBlock(req types.RequestBeginBlock) types.ResponseBeginB ctx := mux.state.NewContext(ContextBeginBlock, mux.currentTime) defer ctx.Close() - currentEpoch, err := mux.state.GetCurrentEpoch(ctx.Ctx()) + currentEpoch, err := mux.state.GetCurrentEpoch(ctx) if err != nil { panic("mux: can't get current epoch in BeginBlock") } @@ -547,14 +543,19 @@ func (mux *abciMux) BeginBlock(req types.RequestBeginBlock) types.ResponseBeginB // During the first block, also collect and prepend application events // generated during InitChain to BeginBlock events. if mux.state.BlockHeight() == 0 { - _, evBinary := mux.state.deliverTxTree.Get([]byte(stateKeyInitChainEvents)) + evBinary, err := mux.state.deliverTxTree.Get(ctx, []byte(stateKeyInitChainEvents)) + if err != nil { + panic(fmt.Errorf("mux: BeginBlock: failed to query init chain events: %w", err)) + } if evBinary != nil { var events []types.Event _ = cbor.Unmarshal(evBinary, &events) response.Events = append(events, response.Events...) - mux.state.deliverTxTree.Remove([]byte(stateKeyInitChainEvents)) + if err := mux.state.deliverTxTree.Remove(ctx, []byte(stateKeyInitChainEvents)); err != nil { + panic(fmt.Errorf("mux: BeginBlock: failed to remove init chain events: %w", err)) + } } } @@ -729,6 +730,14 @@ func (mux *abciMux) DeliverTx(req types.RequestDeliverTx) types.ResponseDeliverT defer ctx.Close() if err := mux.executeTx(ctx, req.Tx); err != nil { + if IsUnavailableStateError(err) { + // Make sure to not commit any transactions which include results based on unavailable + // and/or corrupted state -- doing so can further corrupt state. + ctx.Logger().Error("unavailable and/or corrupted state detected during tx processing", + "err", err, + ) + panic(err) + } module, code := errors.Code(err) return types.ResponseDeliverTx{ diff --git a/go/consensus/tendermint/abci/prune.go b/go/consensus/tendermint/abci/prune.go index f80293f8c29..ad957b9a8d8 100644 --- a/go/consensus/tendermint/abci/prune.go +++ b/go/consensus/tendermint/abci/prune.go @@ -1,13 +1,13 @@ package abci import ( + "context" "fmt" "strings" - "github.com/tendermint/iavl" - + "github.com/oasislabs/oasis-core/go/common" "github.com/oasislabs/oasis-core/go/common/logging" - "github.com/oasislabs/oasis-core/go/common/pubsub" + nodedb "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel/db/api" ) const ( @@ -60,111 +60,91 @@ type PruneConfig struct { Strategy PruneStrategy // NumKept is the number of versions retained when applicable. - NumKept int64 + NumKept uint64 } -// StatePruner is a concrete ABCI mux iAVL state pruner implementation. +// StatePruner is a concrete ABCI mux state pruner implementation. type StatePruner interface { - // Prune purges unneeded versions from the ABCI mux iAVL tree, + // Prune purges unneeded versions from the ABCI mux node database, // given the latest version, based on the underlying strategy. - Prune(latestVersion int64) - - // Subscribe subscribes to prune events. - Subscribe() (<-chan int64, *pubsub.Subscription, error) + Prune(ctx context.Context, latestRound uint64) error } type statePrunerInitializer interface { - Initialize(latestVersion int64) error + Initialize(latestRound uint64) error } type nonePruner struct{} -func (p *nonePruner) Prune(latestVersion int64) { +func (p *nonePruner) Prune(ctx context.Context, latestRound uint64) error { // Nothing to prune. -} - -func (p *nonePruner) Subscribe() (<-chan int64, *pubsub.Subscription, error) { - return nil, nil, nil + return nil } type genericPruner struct { logger *logging.Logger - tree *iavl.MutableTree - - eldestRetained int64 - keepN int64 + ndb nodedb.NodeDB - notifier *pubsub.Broker + earliestRound uint64 + keepN uint64 } -func (p *genericPruner) Initialize(latestVersion int64) error { +func (p *genericPruner) Initialize(latestRound uint64) error { // Figure out the eldest version currently present in the tree. - if p.eldestRetained = p.tree.EldestVersion(); p.eldestRetained == -1 { - // The tree is empty, nothing to purge. - p.eldestRetained = 0 - return nil + var err error + if p.earliestRound, err = p.ndb.GetEarliestRound(context.Background()); err != nil { + return fmt.Errorf("failed to get earliest round: %w", err) } - return p.doPrune(latestVersion) + return p.doPrune(context.Background(), latestRound) } -func (p *genericPruner) Prune(latestVersion int64) { - if err := p.doPrune(latestVersion); err != nil { +func (p *genericPruner) Prune(ctx context.Context, latestRound uint64) error { + if err := p.doPrune(ctx, latestRound); err != nil { p.logger.Error("Prune", "err", err, ) - panic(err) + return err } + return nil } -func (p *genericPruner) doPrune(latestVersion int64) error { - if latestVersion < p.keepN { +func (p *genericPruner) doPrune(ctx context.Context, latestRound uint64) error { + if latestRound < p.keepN { return nil } p.logger.Debug("Prune: Start", - "latest_version", latestVersion, - "start_version", p.eldestRetained, + "latest_version", latestRound, + "start_version", p.earliestRound, ) - preserveFrom := latestVersion - p.keepN - for i := p.eldestRetained; i <= latestVersion; i++ { - if p.tree.VersionExists(i) { - if i >= preserveFrom { - p.eldestRetained = i - break - } - - p.logger.Debug("Prune: Delete", - "latest_version", latestVersion, - "pruned_version", i, - ) + preserveFrom := latestRound - p.keepN + for i := p.earliestRound; i <= latestRound; i++ { + if i >= preserveFrom { + p.earliestRound = i + break + } - if err := p.tree.DeleteVersion(i); err != nil { - return err - } + p.logger.Debug("Prune: Delete", + "latest_version", latestRound, + "pruned_version", i, + ) - p.notifier.Broadcast(i) + if _, err := p.ndb.Prune(ctx, common.Namespace{}, i); err != nil { + return err } } p.logger.Debug("Prune: Finish", - "latest_version", latestVersion, - "eldest_version", p.eldestRetained, + "latest_version", latestRound, + "eldest_version", p.earliestRound, ) return nil } -func (p *genericPruner) Subscribe() (<-chan int64, *pubsub.Subscription, error) { - sub := p.notifier.Subscribe() - ch := make(chan int64) - sub.Unwrap(ch) - - return ch, sub, nil -} - -func newStatePruner(cfg *PruneConfig, tree *iavl.MutableTree, latestVersion int64) (StatePruner, error) { +func newStatePruner(cfg *PruneConfig, ndb nodedb.NodeDB, latestRound uint64) (StatePruner, error) { // The roothash checkCommittees call requires at least 1 previous block // for timekeeping purposes. const minKept = 1 @@ -181,17 +161,16 @@ func newStatePruner(cfg *PruneConfig, tree *iavl.MutableTree, latestVersion int6 } statePruner = &genericPruner{ - logger: logger, - tree: tree, - keepN: cfg.NumKept, - notifier: pubsub.NewBroker(false), + logger: logger, + ndb: ndb, + keepN: cfg.NumKept, } default: return nil, fmt.Errorf("abci/pruner: unsupported pruning strategy: %v", cfg.Strategy) } if initializer, ok := statePruner.(statePrunerInitializer); ok { - if err := initializer.Initialize(latestVersion); err != nil { + if err := initializer.Initialize(latestRound); err != nil { return nil, err } } diff --git a/go/consensus/tendermint/abci/prune_test.go b/go/consensus/tendermint/abci/prune_test.go index 856b6f48e09..2498ac7a098 100644 --- a/go/consensus/tendermint/abci/prune_test.go +++ b/go/consensus/tendermint/abci/prune_test.go @@ -1,36 +1,77 @@ package abci import ( + "context" "fmt" + "io/ioutil" + "os" "testing" "github.com/stretchr/testify/require" - "github.com/tendermint/iavl" - dbm "github.com/tendermint/tm-db" + + "github.com/oasislabs/oasis-core/go/common" + "github.com/oasislabs/oasis-core/go/common/crypto/hash" + mkvs "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel" + mkvsDB "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel/db/api" + mkvsBadgerDB "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel/db/badger" ) func TestPruneKeepN(t *testing.T) { - db := dbm.NewMemDB() - tree := iavl.NewMutableTree(db, 128) - - for i := int64(1); i <= 11; i++ { - tree.Set([]byte(fmt.Sprintf("key:%d", i)), []byte(fmt.Sprintf("value:%d", i))) - _, ver, err := tree.SaveVersion() - require.NoError(t, err, "SaveVersion: %d", i) - require.Equal(t, i, ver, "incorrect version on save") + require := require.New(t) + + // Create a new random temporary directory under /tmp. + dir, err := ioutil.TempDir("", "abci-prune.test.badger") + require.NoError(err, "TempDir") + defer os.RemoveAll(dir) + + // Create a Badger-backed Node DB. + ndb, err := mkvsBadgerDB.New(&mkvsDB.Config{ + DB: dir, + NoFsync: true, + MaxCacheSize: 16 * 1024 * 1024, + }) + require.NoError(err, "New") + tree := mkvs.New(nil, ndb) + + ctx := context.Background() + for i := uint64(1); i <= 11; i++ { + err = tree.Insert(ctx, []byte(fmt.Sprintf("key:%d", i)), []byte(fmt.Sprintf("value:%d", i))) + require.NoError(err, "Insert") + + var rootHash hash.Hash + _, rootHash, err = tree.Commit(ctx, common.Namespace{}, i) + require.NoError(err, "Commit") + err = ndb.Finalize(ctx, common.Namespace{}, i, []hash.Hash{rootHash}) + require.NoError(err, "Finalize") } + earliestRound, err := ndb.GetEarliestRound(ctx) + require.NoError(err, "GetEarliestRound") + require.EqualValues(1, earliestRound, "earliest round should be correct") + latestRound, err := ndb.GetLatestRound(ctx) + require.NoError(err, "GetLatestRound") + require.EqualValues(11, latestRound, "latest round should be correct") + pruner, err := newStatePruner(&PruneConfig{ Strategy: PruneKeepN, NumKept: 2, - }, tree, 10) - require.NoError(t, err, "newStatePruner failed") + }, ndb, 10) + require.NoError(err, "newStatePruner failed") - for i := int64(1); i <= 10; i++ { - require.Equal(t, i >= 8, tree.VersionExists(i), "VersionExists(%d)", i) - } + earliestRound, err = ndb.GetEarliestRound(ctx) + require.NoError(err, "GetEarliestRound") + require.EqualValues(8, earliestRound, "earliest round should be correct") + latestRound, err = ndb.GetLatestRound(ctx) + require.NoError(err, "GetLatestRound") + require.EqualValues(11, latestRound, "latest round should be correct") + + err = pruner.Prune(ctx, 11) + require.NoError(err, "Prune") - pruner.Prune(11) - require.False(t, tree.VersionExists(8), "VersionExists(8), should be pruned") - require.True(t, tree.VersionExists(11), "VersionExists(11)") + earliestRound, err = ndb.GetEarliestRound(ctx) + require.NoError(err, "GetEarliestRound") + require.EqualValues(9, earliestRound, "earliest round should be correct") + latestRound, err = ndb.GetLatestRound(ctx) + require.NoError(err, "GetLatestRound") + require.EqualValues(11, latestRound, "latest round should be correct") } diff --git a/go/consensus/tendermint/abci/state.go b/go/consensus/tendermint/abci/state.go index 7ef89d2c67a..6634fbbe1b6 100644 --- a/go/consensus/tendermint/abci/state.go +++ b/go/consensus/tendermint/abci/state.go @@ -1,7 +1,6 @@ package abci import ( - "bytes" "context" "encoding/json" "errors" @@ -10,19 +9,21 @@ import ( "sync" "time" - "github.com/tendermint/iavl" + "github.com/eapache/channels" "github.com/tendermint/tendermint/abci/types" - dbm "github.com/tendermint/tm-db" + "github.com/oasislabs/oasis-core/go/common" + "github.com/oasislabs/oasis-core/go/common/crypto/hash" "github.com/oasislabs/oasis-core/go/common/crypto/signature" "github.com/oasislabs/oasis-core/go/common/logging" "github.com/oasislabs/oasis-core/go/common/quantity" consensus "github.com/oasislabs/oasis-core/go/consensus/api" "github.com/oasislabs/oasis-core/go/consensus/api/transaction" - "github.com/oasislabs/oasis-core/go/consensus/tendermint/api" - "github.com/oasislabs/oasis-core/go/consensus/tendermint/db" epochtime "github.com/oasislabs/oasis-core/go/epochtime/api" genesis "github.com/oasislabs/oasis-core/go/genesis/api" + storage "github.com/oasislabs/oasis-core/go/storage/api" + storageDB "github.com/oasislabs/oasis-core/go/storage/database" + mkvs "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel" ) var ( @@ -33,8 +34,14 @@ var ( _ ApplicationState = (*mockApplicationState)(nil) ) +// appStateDir is the subdirectory which contains ABCI state. +const appStateDir = "abci-state" + // ApplicationState is the overall past, present and future state of all multiplexed applications. type ApplicationState interface { + // Storage returns the storage backend. + Storage() storage.LocalBackend + // BlockHeight returns the last committed block height. BlockHeight() int64 @@ -48,16 +55,6 @@ type ApplicationState interface { // and calls from anywhere else will cause races. BlockContext() *BlockContext - // DeliverTxTree returns the versioned tree to be used by queries - // to view comitted data, and transactions to build the next version. - DeliverTxTree() *iavl.MutableTree - - // CheckTxTree returns the state tree to be used for modifications - // inside CheckTx (mempool connection) calls. - // - // This state is never persisted. - CheckTxTree() *iavl.MutableTree - // GetBaseEpoch returns the base epoch. GetBaseEpoch() (epochtime.EpochTime, error) @@ -71,9 +68,6 @@ type ApplicationState interface { // last block. As a matter of convenience, the current epoch is returned. EpochChanged(ctx *Context) (bool, epochtime.EpochTime) - // Genesis returns the ABCI genesis state. - Genesis() *genesis.Document - // MinGasPrice returns the configured minimum gas price. MinGasPrice() *quantity.Quantity @@ -87,17 +81,21 @@ type ApplicationState interface { type applicationState struct { logger *logging.Logger - ctx context.Context - db dbm.DB - deliverTxTree *iavl.MutableTree - checkTxTree *iavl.MutableTree - statePruner StatePruner + ctx context.Context + cancelCtx context.CancelFunc - blockLock sync.RWMutex - blockHash []byte - blockHeight int64 - blockTime time.Time - blockCtx *BlockContext + stateRoot storage.Root + storage storage.LocalBackend + deliverTxTree mkvs.Tree + checkTxTree mkvs.Tree + + statePruner StatePruner + prunerClosedCh chan struct{} + prunerNotifyCh *channels.RingChannel + + blockLock sync.RWMutex + blockTime time.Time + blockCtx *BlockContext txAuthHandler TransactionAuthHandler @@ -109,7 +107,6 @@ type applicationState struct { minGasPrice quantity.Quantity ownTxSigner signature.PublicKey - metricsCloseCh chan struct{} metricsClosedCh chan struct{} } @@ -122,10 +119,10 @@ func (s *applicationState) NewContext(mode ContextMode, now time.Time) *Context currentTime: now, gasAccountant: NewNopGasAccountant(), appState: s, - blockHeight: s.blockHeight, + blockHeight: int64(s.stateRoot.Round), logger: logging.GetLogger("consensus/tendermint/abci").With("mode", mode), } - c.ctx = context.WithValue(s.ctx, contextKey{}, c) + c.Context = context.WithValue(s.ctx, contextKey{}, c) switch mode { case ContextInitChain: @@ -138,12 +135,7 @@ func (s *applicationState) NewContext(mode ContextMode, now time.Time) *Context case ContextSimulateTx: // Since simulation is running in parallel to any changes to the database, we make sure // to create a separate in-memory tree at the given block height. - c.state = iavl.NewMutableTree(s.db, 128) - // NOTE: This requires a specific implementation of `LoadVersion` which doesn't rely - // on cached metadata. Such an implementation is provided in our fork of IAVL. - if _, err := c.state.LoadVersion(c.blockHeight); err != nil { - panic(fmt.Errorf("context: failed to load state at height %d: %w", c.blockHeight, err)) - } + c.state = mkvs.NewWithRoot(nil, s.storage.NodeDB(), s.stateRoot, mkvs.WithoutWriteLog()) c.currentTime = s.blockTime default: panic(fmt.Errorf("context: invalid mode: %s (%d)", mode, mode)) @@ -152,12 +144,16 @@ func (s *applicationState) NewContext(mode ContextMode, now time.Time) *Context return c } +func (s *applicationState) Storage() storage.LocalBackend { + return s.storage +} + // BlockHeight returns the last committed block height. func (s *applicationState) BlockHeight() int64 { s.blockLock.RLock() defer s.blockLock.RUnlock() - return s.blockHeight + return int64(s.stateRoot.Round) } // BlockHash returns the last committed block hash. @@ -165,7 +161,11 @@ func (s *applicationState) BlockHash() []byte { s.blockLock.RLock() defer s.blockLock.RUnlock() - return append([]byte{}, s.blockHash...) + if s.stateRoot.Round == 0 { + // Tendermint expects a nil hash when there is no state otherwise it will panic. + return nil + } + return s.stateRoot.Hash[:] } // BlockContext returns the current block context which can be used @@ -177,20 +177,6 @@ func (s *applicationState) BlockContext() *BlockContext { return s.blockCtx } -// DeliverTxTree returns the versioned tree to be used by queries -// to view comitted data, and transactions to build the next version. -func (s *applicationState) DeliverTxTree() *iavl.MutableTree { - return s.deliverTxTree -} - -// CheckTxTree returns the state tree to be used for modifications -// inside CheckTx (mempool connection) calls. -// -// This state is never persisted. -func (s *applicationState) CheckTxTree() *iavl.MutableTree { - return s.checkTxTree -} - // GetBaseEpoch returns the base epoch. func (s *applicationState) GetBaseEpoch() (epochtime.EpochTime, error) { return s.timeSource.GetBaseEpoch(s.ctx) @@ -222,7 +208,7 @@ func (s *applicationState) EpochChanged(ctx *Context) (bool, epochtime.EpochTime return false, epochtime.EpochInvalid } - currentEpoch, err := s.timeSource.GetEpoch(ctx.Ctx(), blockHeight+1) + currentEpoch, err := s.timeSource.GetEpoch(ctx, blockHeight+1) if err != nil { s.logger.Error("EpochChanged: failed to get current epoch", "err", err, @@ -236,7 +222,7 @@ func (s *applicationState) EpochChanged(ctx *Context) (bool, epochtime.EpochTime return false, currentEpoch } - previousEpoch, err := s.timeSource.GetEpoch(ctx.Ctx(), blockHeight) + previousEpoch, err := s.timeSource.GetEpoch(ctx, blockHeight) if err != nil { s.logger.Error("EpochChanged: failed to get previous epoch", "err", err, @@ -256,30 +242,6 @@ func (s *applicationState) EpochChanged(ctx *Context) (bool, epochtime.EpochTime return true, currentEpoch } -// Genesis returns the ABCI genesis state. -func (s *applicationState) Genesis() *genesis.Document { - _, b := s.checkTxTree.Get([]byte(stateKeyGenesisRequest)) - - var req types.RequestInitChain - if err := req.Unmarshal(b); err != nil { - s.logger.Error("Genesis: corrupted defered genesis state", - "err", err, - ) - panic("Genesis: invalid defered genesis application state") - } - - st, err := parseGenesisAppState(req) - if err != nil { - s.logger.Error("failed to unmarshal genesis application state", - "err", err, - "state", req.AppStateBytes, - ) - panic("Genesis: invalid genesis application state") - } - - return st -} - // MinGasPrice returns the configured minimum gas price. func (s *applicationState) MinGasPrice() *quantity.Quantity { return &s.minGasPrice @@ -293,7 +255,7 @@ func (s *applicationState) OwnTxSigner() signature.PublicKey { func (s *applicationState) inHaltEpoch(ctx *Context) bool { blockHeight := s.BlockHeight() - currentEpoch, err := s.GetEpoch(ctx.Ctx(), blockHeight+1) + currentEpoch, err := s.GetEpoch(ctx, blockHeight+1) if err != nil { s.logger.Error("inHaltEpoch: failed to get epoch", "err", err, @@ -308,7 +270,7 @@ func (s *applicationState) inHaltEpoch(ctx *Context) bool { func (s *applicationState) afterHaltEpoch(ctx *Context) bool { blockHeight := s.BlockHeight() - currentEpoch, err := s.GetEpoch(ctx.Ctx(), blockHeight+1) + currentEpoch, err := s.GetEpoch(ctx, blockHeight+1) if err != nil { s.logger.Error("afterHaltEpoch: failed to get epoch", "err", err, @@ -321,62 +283,54 @@ func (s *applicationState) afterHaltEpoch(ctx *Context) bool { } func (s *applicationState) doCommit(now time.Time) error { - // Save the new version of the persistent tree. - blockHash, blockHeight, err := s.deliverTxTree.SaveVersion() - if err == nil { - s.blockLock.Lock() - s.blockHash = blockHash - s.blockHeight = blockHeight - s.blockTime = now - s.blockLock.Unlock() - - // Reset CheckTx state to latest version. This is safe because - // Tendermint holds a lock on the mempool for commit. - // - // WARNING: deliverTxTree and checkTxTree do not share internal - // state beyond the backing database. The `LoadVersion` - // implementation MUST be written in a way to avoid relying on - // cached metadata. - // - // This makes the upstream `LazyLoadVersion` and `LoadVersion` - // unsuitable for our use case. - _, cerr := s.checkTxTree.LoadVersion(blockHeight) - if cerr != nil { - panic(cerr) - } + s.blockLock.Lock() + defer s.blockLock.Unlock() - // Prune the iavl state according to the specified strategy. - s.statePruner.Prune(s.blockHeight) + _, stateRootHash, err := s.deliverTxTree.Commit(s.ctx, s.stateRoot.Namespace, s.stateRoot.Round+1) + if err != nil { + return fmt.Errorf("failed to commit: %w", err) + } + // NOTE: Finalization could be done in parallel, together with pruning since replay into + // non-finalized rounds is possible. + if err = s.storage.NodeDB().Finalize(s.ctx, s.stateRoot.Namespace, s.stateRoot.Round+1, []hash.Hash{stateRootHash}); err != nil { + return fmt.Errorf("failed to finalize round %d: %w", s.stateRoot.Round+1, err) } - return err + s.stateRoot.Hash = stateRootHash + s.stateRoot.Round++ + s.blockTime = now + + // Switch the CheckTx tree to the newly committed version. Note that this is safe as Tendermint + // holds the mempool lock while commit is in progress so no CheckTx can take place. + s.checkTxTree.Close() + s.checkTxTree = mkvs.NewWithRoot(nil, s.storage.NodeDB(), s.stateRoot, mkvs.WithoutWriteLog()) + + // Notify pruner of a new block. + s.prunerNotifyCh.In() <- s.stateRoot.Round + + return nil } func (s *applicationState) doCleanup() { - if s.db != nil { - // Don't close the DB out from under the metrics worker. - close(s.metricsCloseCh) + if s.storage != nil { + // Don't close the DB out from under the metrics/pruner worker. + s.cancelCtx() + <-s.prunerClosedCh <-s.metricsClosedCh - s.db.Close() - s.db = nil + s.storage.Cleanup() + s.storage = nil } } func (s *applicationState) updateMetrics() error { var dbSize int64 - - switch m := s.db.(type) { - case api.SizeableDB: - var err error - if dbSize, err = m.Size(); err != nil { - s.logger.Error("Size", - "err", err, - ) - return err - } - default: - return fmt.Errorf("state: unsupported DB for metrics") + var err error + if dbSize, err = s.storage.NodeDB().Size(); err != nil { + s.logger.Error("Size", + "err", err, + ) + return err } abciSize.Set(float64(dbSize) / 1024768.0) @@ -402,7 +356,7 @@ func (s *applicationState) metricsWorker() { for { select { - case <-s.metricsCloseCh: + case <-s.ctx.Done(): return case <-t.C: _ = s.updateMetrics() @@ -410,38 +364,94 @@ func (s *applicationState) metricsWorker() { } } +func (s *applicationState) pruneWorker() { + defer close(s.prunerClosedCh) + + for { + select { + case <-s.ctx.Done(): + return + case r := <-s.prunerNotifyCh.Out(): + round := r.(uint64) + + if err := s.statePruner.Prune(s.ctx, round); err != nil { + s.logger.Warn("failed to prune state", + "err", err, + "block_height", round, + ) + } + } + } +} + func newApplicationState(ctx context.Context, cfg *ApplicationConfig) (*applicationState, error) { - db, err := db.New(filepath.Join(cfg.DataDir, "abci-mux-state"), false) + baseDir := filepath.Join(cfg.DataDir, appStateDir) + if err := common.Mkdir(baseDir); err != nil { + return nil, fmt.Errorf("failed to create application state directory: %w", err) + } + + switch cfg.StorageBackend { + case storageDB.BackendNameBadgerDB: + default: + return nil, fmt.Errorf("unsupported storage backend: %s", cfg.StorageBackend) + } + + db, err := storageDB.New(&storage.Config{ + Backend: cfg.StorageBackend, + DB: filepath.Join(baseDir, storageDB.DefaultFileName(cfg.StorageBackend)), + MaxCacheSize: 64 * 1024 * 1024, // TODO: Make this configurable. + DiscardWriteLogs: true, + NoFsync: true, // This is safe as Tendermint will replay on crash. + }) if err != nil { return nil, err } + ldb := db.(storage.LocalBackend) + ndb := ldb.NodeDB() + + // Make sure to close the database in case we fail. + var ok bool + defer func() { + if !ok { + db.Cleanup() + } + }() - // Figure out the latest version/hash if any, and use that - // as the block height/hash. - deliverTxTree := iavl.NewMutableTree(db, 128) - blockHeight, err := deliverTxTree.Load() + // Figure out the latest round/hash if any, and use that as the block height/hash. + latestRound, err := ndb.GetLatestRound(ctx) if err != nil { - db.Close() return nil, err } - blockHash := deliverTxTree.Hash() - - checkTxTree := iavl.NewMutableTree(db, 128) - checkTxBlockHeight, err := checkTxTree.Load() + roots, err := ndb.GetRootsForRound(ctx, latestRound) if err != nil { - db.Close() return nil, err } - - if blockHeight != checkTxBlockHeight || !bytes.Equal(blockHash, checkTxTree.Hash()) { - db.Close() - return nil, fmt.Errorf("state: inconsistent trees") + stateRoot := storage.Root{ + Round: latestRound, + } + switch len(roots) { + case 0: + // No roots -- empty database. + if latestRound != 0 { + return nil, fmt.Errorf("state: no roots at non-zero height, corrupted database?") + } + stateRoot.Hash.Empty() + case 1: + // Exactly one root -- the usual case. + stateRoot.Hash = roots[0] + default: + // More roots -- should not happen for our use case. + return nil, fmt.Errorf("state: more than one root, corrupted database?") } - statePruner, err := newStatePruner(&cfg.Pruning, deliverTxTree, blockHeight) + // Use the node database directly to avoid going through the syncer interface. + deliverTxTree := mkvs.NewWithRoot(nil, ndb, stateRoot, mkvs.WithoutWriteLog()) + checkTxTree := mkvs.NewWithRoot(nil, ndb, stateRoot, mkvs.WithoutWriteLog()) + + // Initialize the state pruner. + statePruner, err := newStatePruner(&cfg.Pruning, ndb, latestRound) if err != nil { - db.Close() - return nil, err + return nil, fmt.Errorf("state: failed to create pruner: %w", err) } var minGasPrice quantity.Quantity @@ -449,22 +459,28 @@ func newApplicationState(ctx context.Context, cfg *ApplicationConfig) (*applicat return nil, fmt.Errorf("state: invalid minimum gas price: %w", err) } + ctx, cancelCtx := context.WithCancel(ctx) + + ok = true + s := &applicationState{ logger: logging.GetLogger("abci-mux/state"), ctx: ctx, - db: db, + cancelCtx: cancelCtx, deliverTxTree: deliverTxTree, checkTxTree: checkTxTree, + stateRoot: stateRoot, + storage: ldb, statePruner: statePruner, - blockHash: blockHash, - blockHeight: blockHeight, + prunerClosedCh: make(chan struct{}), + prunerNotifyCh: channels.NewRingChannel(1), haltEpochHeight: cfg.HaltEpochHeight, minGasPrice: minGasPrice, ownTxSigner: cfg.OwnTxSigner, - metricsCloseCh: make(chan struct{}), metricsClosedCh: make(chan struct{}), } go s.metricsWorker() + go s.pruneWorker() return s, nil } @@ -491,15 +507,17 @@ type MockApplicationStateConfig struct { MinGasPrice *quantity.Quantity OwnTxSigner signature.PublicKey - - Genesis *genesis.Document } type mockApplicationState struct { cfg MockApplicationStateConfig blockCtx *BlockContext - tree *iavl.MutableTree + tree mkvs.Tree +} + +func (ms *mockApplicationState) Storage() storage.LocalBackend { + panic("not implemented") } func (ms *mockApplicationState) BlockHeight() int64 { @@ -514,14 +532,6 @@ func (ms *mockApplicationState) BlockContext() *BlockContext { return ms.blockCtx } -func (ms *mockApplicationState) DeliverTxTree() *iavl.MutableTree { - return ms.tree -} - -func (ms *mockApplicationState) CheckTxTree() *iavl.MutableTree { - return ms.tree -} - func (ms *mockApplicationState) GetBaseEpoch() (epochtime.EpochTime, error) { return ms.cfg.BaseEpoch, nil } @@ -538,10 +548,6 @@ func (ms *mockApplicationState) EpochChanged(ctx *Context) (bool, epochtime.Epoc return ms.cfg.EpochChanged, ms.cfg.CurrentEpoch } -func (ms *mockApplicationState) Genesis() *genesis.Document { - return ms.cfg.Genesis -} - func (ms *mockApplicationState) MinGasPrice() *quantity.Quantity { return ms.cfg.MinGasPrice } @@ -561,15 +567,14 @@ func (ms *mockApplicationState) NewContext(mode ContextMode, now time.Time) *Con blockCtx: ms.blockCtx, logger: logging.GetLogger("consensus/tendermint/abci").With("mode", mode), } - c.ctx = context.WithValue(context.Background(), contextKey{}, c) + c.Context = context.WithValue(context.Background(), contextKey{}, c) return c } // NewMockApplicationState creates a new mock application state for testing. func NewMockApplicationState(cfg MockApplicationStateConfig) ApplicationState { - db := dbm.NewMemDB() - tree := iavl.NewMutableTree(db, 128) + tree := mkvs.New(nil, nil) blockCtx := NewBlockContext() if cfg.MaxBlockGas > 0 { @@ -587,15 +592,37 @@ func NewMockApplicationState(cfg MockApplicationStateConfig) ApplicationState { // ImmutableState is an immutable state wrapper. type ImmutableState struct { - // Snapshot is the backing immutable iAVL tree snapshot. - Snapshot *iavl.ImmutableTree + Tree mkvs.KeyValueTree +} + +// Close releases the resources associated with the immutable state wrapper. +// +// After calling this method, the immutable state wrapper should not be used anymore. +func (s *ImmutableState) Close() { + if tree, ok := s.Tree.(mkvs.ClosableTree); ok { + tree.Close() + } } // NewImmutableState creates a new immutable state wrapper. -func NewImmutableState(state ApplicationState, version int64) (*ImmutableState, error) { +func NewImmutableState(ctx context.Context, state ApplicationState, version int64) (*ImmutableState, error) { if state == nil { return nil, ErrNoState } + + // Check if this request was made from an ABCI application context. + if abciCtx := FromCtx(ctx); abciCtx != nil { + // Override used state with the one from the current context in the following cases: + // + // - If this request was made from InitChain, no blocks and states have been submitted yet. + // - If this request was made from an ABCI app and is for the current (future) height. + // + if abciCtx.IsInitChain() || version == abciCtx.BlockHeight()+1 { + return &ImmutableState{Tree: abciCtx.State()}, nil + } + } + + // Handle a regular (external) query where we need to create a new tree. if state.BlockHeight() == 0 { return nil, consensus.ErrNoCommittedBlocks } @@ -603,10 +630,18 @@ func NewImmutableState(state ApplicationState, version int64) (*ImmutableState, version = state.BlockHeight() } - snapshot, err := state.DeliverTxTree().GetImmutable(version) + ndb := state.Storage().NodeDB() + roots, err := ndb.GetRootsForRound(ctx, uint64(version)) if err != nil { return nil, err } + if len(roots) != 1 { + panic(fmt.Sprintf("corrupted state (%d): %+v", version, roots)) + } + tree := mkvs.NewWithRoot(nil, ndb, storage.Root{ + Round: uint64(version), + Hash: roots[0], + }, mkvs.WithoutWriteLog()) - return &ImmutableState{Snapshot: snapshot}, nil + return &ImmutableState{Tree: tree}, nil } diff --git a/go/consensus/tendermint/abci/timer.go b/go/consensus/tendermint/abci/timer.go index e628aa76ba5..dfc2afe69f2 100644 --- a/go/consensus/tendermint/abci/timer.go +++ b/go/consensus/tendermint/abci/timer.go @@ -2,6 +2,7 @@ package abci import ( "encoding/hex" + "fmt" "time" "github.com/oasislabs/oasis-core/go/common/keyformat" @@ -109,7 +110,10 @@ func (t *Timer) Data(ctx *Context) []byte { t.refreshState() if t.state.data == nil { - _, value := ctx.State().Get(t.ID) + value, err := ctx.State().Get(ctx, t.ID) + if err != nil { + panic(fmt.Errorf("timer: failed to fetch timer: %w", err)) + } if value == nil { logger.Error("timer not found", "id", hex.EncodeToString(t.ID), @@ -144,7 +148,10 @@ func (t *Timer) Reset(ctx *Context, duration time.Duration, data []byte) { } // Create timer entry. - ctx.State().Set(t.state.getKey(), t.state.data) + err := ctx.State().Insert(ctx, t.state.getKey(), t.state.data) + if err != nil { + panic(fmt.Errorf("timer: failed to set timer: %w", err)) + } t.ID = t.state.getKey() } @@ -166,7 +173,7 @@ func (t *Timer) remove(ctx *Context) { return } - if _, removed := ctx.State().Remove(t.ID); !removed { + if existing, err := ctx.State().RemoveExisting(ctx, t.ID); existing == nil || err != nil { logger.Error("timer not removed", "id", hex.EncodeToString(t.ID), ) @@ -176,25 +183,30 @@ func (t *Timer) remove(ctx *Context) { func fireTimers(ctx *Context, app Application) (err error) { // Iterate through all timers which have already expired. - ctx.State().IterateRange( - timerKeyFmt.Encode(), - timerKeyFmt.Encode(uint64(ctx.Now().Unix())+1), - true, - func(key, value []byte) bool { - var ts timerState - ts.fromKeyValue(key, value) - - // Skip timers that are not for this application. - if app.ID() != ts.app { - return false - } - - if err = app.FireTimer(ctx, &Timer{ID: key, state: &ts}); err != nil { - return true - } - - return false - }, - ) + it := ctx.State().NewIterator(ctx) + defer it.Close() + + now := uint64(ctx.Now().Unix()) + for it.Seek(timerKeyFmt.Encode()); it.Valid(); it.Next() { + var decDeadline uint64 + if !timerKeyFmt.Decode(it.Key(), &decDeadline) || decDeadline > now { + break + } + + var ts timerState + ts.fromKeyValue(it.Key(), it.Value()) + + // Skip timers that are not for this application. + if app.ID() != ts.app { + continue + } + + if err = app.FireTimer(ctx, &Timer{ID: it.Key(), state: &ts}); err != nil { + return err + } + } + if it.Err() != nil { + return UnavailableStateError(it.Err()) + } return } diff --git a/go/consensus/tendermint/apps/beacon/beacon.go b/go/consensus/tendermint/apps/beacon/beacon.go index b28e8cec2d3..07683ae3d90 100644 --- a/go/consensus/tendermint/apps/beacon/beacon.go +++ b/go/consensus/tendermint/apps/beacon/beacon.go @@ -84,7 +84,7 @@ func (app *beaconApplication) onBeaconEpochChange(ctx *abci.Context, epoch epoch var entropyCtx, entropy []byte state := beaconState.NewMutableState(ctx.State()) - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { ctx.Logger().Error("failed to fetch consensus parameters", "err", err, @@ -143,7 +143,7 @@ func (app *beaconApplication) onBeaconEpochChange(ctx *abci.Context, epoch epoch func (app *beaconApplication) onNewBeacon(ctx *abci.Context, beacon []byte) error { state := beaconState.NewMutableState(ctx.State()) - if err := state.SetBeacon(beacon); err != nil { + if err := state.SetBeacon(ctx, beacon); err != nil { ctx.Logger().Error("onNewBeacon: failed to set beacon", "err", err, ) diff --git a/go/consensus/tendermint/apps/beacon/genesis.go b/go/consensus/tendermint/apps/beacon/genesis.go index f2bbb86b703..25aae74f19c 100644 --- a/go/consensus/tendermint/apps/beacon/genesis.go +++ b/go/consensus/tendermint/apps/beacon/genesis.go @@ -2,6 +2,7 @@ package beacon import ( "context" + "fmt" "github.com/tendermint/tendermint/abci/types" @@ -19,7 +20,9 @@ func (app *beaconApplication) InitChain(ctx *abci.Context, req types.RequestInit // It is not super important for now as the epoch will transition // immediately on the first block under normal circumstances. state := beaconState.NewMutableState(ctx.State()) - state.SetConsensusParameters(&doc.Beacon.Parameters) + if err := state.SetConsensusParameters(ctx, &doc.Beacon.Parameters); err != nil { + return fmt.Errorf("failed to set consensus parameters: %w", err) + } if doc.Beacon.Parameters.DebugDeterministic { ctx.Logger().Warn("Determistic beacon entropy is NOT FOR PRODUCTION USE") @@ -28,7 +31,7 @@ func (app *beaconApplication) InitChain(ctx *abci.Context, req types.RequestInit } func (bq *beaconQuerier) Genesis(ctx context.Context) (*beacon.Genesis, error) { - params, err := bq.state.ConsensusParameters() + params, err := bq.state.ConsensusParameters(ctx) if err != nil { return nil, err } diff --git a/go/consensus/tendermint/apps/beacon/query.go b/go/consensus/tendermint/apps/beacon/query.go index 3082a544bfa..773f141128a 100644 --- a/go/consensus/tendermint/apps/beacon/query.go +++ b/go/consensus/tendermint/apps/beacon/query.go @@ -4,7 +4,6 @@ import ( "context" beacon "github.com/oasislabs/oasis-core/go/beacon/api" - "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" beaconState "github.com/oasislabs/oasis-core/go/consensus/tendermint/apps/beacon/state" ) @@ -21,27 +20,10 @@ type QueryFactory struct { // QueryAt returns the beacon query interface for a specific height. func (sf *QueryFactory) QueryAt(ctx context.Context, height int64) (Query, error) { - var state *beaconState.ImmutableState - var err error - abciCtx := abci.FromCtx(ctx) - - // If this request was made from InitChain, no blocks and states have been - // submitted yet, so we use the existing state instead. - if abciCtx != nil && abciCtx.IsInitChain() { - state = beaconState.NewMutableState(abciCtx.State()).ImmutableState - } else { - state, err = beaconState.NewImmutableState(sf.app.state, height) - if err != nil { - return nil, err - } + state, err := beaconState.NewImmutableState(ctx, sf.app.state, height) + if err != nil { + return nil, err } - - // If this request was made from an ABCI app, make sure to use the associated - // context for querying state instead of the default one. - if abciCtx != nil && height == abciCtx.BlockHeight()+1 { - state.Snapshot = abciCtx.State().ImmutableTree - } - return &beaconQuerier{state}, nil } @@ -50,7 +32,7 @@ type beaconQuerier struct { } func (bq *beaconQuerier) Beacon(ctx context.Context) ([]byte, error) { - return bq.state.Beacon() + return bq.state.Beacon(ctx) } func (app *beaconApplication) QueryFactory() interface{} { diff --git a/go/consensus/tendermint/apps/beacon/state/state.go b/go/consensus/tendermint/apps/beacon/state/state.go index 8fcc63579c9..47137897a3a 100644 --- a/go/consensus/tendermint/apps/beacon/state/state.go +++ b/go/consensus/tendermint/apps/beacon/state/state.go @@ -1,15 +1,15 @@ package state import ( + "context" "errors" "fmt" - "github.com/tendermint/iavl" - beacon "github.com/oasislabs/oasis-core/go/beacon/api" "github.com/oasislabs/oasis-core/go/common/cbor" "github.com/oasislabs/oasis-core/go/common/keyformat" "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" + mkvs "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel" ) var ( @@ -27,8 +27,8 @@ type ImmutableState struct { *abci.ImmutableState } -func NewImmutableState(state abci.ApplicationState, version int64) (*ImmutableState, error) { - inner, err := abci.NewImmutableState(state, version) +func NewImmutableState(ctx context.Context, state abci.ApplicationState, version int64) (*ImmutableState, error) { + inner, err := abci.NewImmutableState(ctx, state, version) if err != nil { return nil, err } @@ -37,53 +37,57 @@ func NewImmutableState(state abci.ApplicationState, version int64) (*ImmutableSt } // Beacon gets the current random beacon value. -func (s *ImmutableState) Beacon() ([]byte, error) { - _, b := s.Snapshot.Get(beaconKeyFmt.Encode()) - if b == nil { +func (s *ImmutableState) Beacon(ctx context.Context) ([]byte, error) { + data, err := s.Tree.Get(ctx, beaconKeyFmt.Encode()) + if err != nil { + return nil, abci.UnavailableStateError(err) + } + if data == nil { return nil, beacon.ErrBeaconNotAvailable } - - return b, nil + return data, nil } -func (s *ImmutableState) ConsensusParameters() (*beacon.ConsensusParameters, error) { - _, raw := s.Snapshot.Get(parametersKeyFmt.Encode()) - if raw == nil { +func (s *ImmutableState) ConsensusParameters(ctx context.Context) (*beacon.ConsensusParameters, error) { + data, err := s.Tree.Get(ctx, parametersKeyFmt.Encode()) + if err != nil { + return nil, abci.UnavailableStateError(err) + } + if data == nil { return nil, errors.New("tendermint/beacon: expected consensus parameters to be present in app state") } var params beacon.ConsensusParameters - err := cbor.Unmarshal(raw, ¶ms) - return ¶ms, err + if err = cbor.Unmarshal(data, ¶ms); err != nil { + return nil, abci.UnavailableStateError(err) + } + return ¶ms, nil } // MutableState is a mutable beacon state wrapper. type MutableState struct { *ImmutableState - - tree *iavl.MutableTree } -func (s *MutableState) SetBeacon(newBeacon []byte) error { +func (s *MutableState) SetBeacon(ctx context.Context, newBeacon []byte) error { if l := len(newBeacon); l != beacon.BeaconSize { return fmt.Errorf("tendermint/beacon: unexpected beacon size: %d", l) } - s.tree.Set(beaconKeyFmt.Encode(), newBeacon) - - return nil + err := s.Tree.Insert(ctx, beaconKeyFmt.Encode(), newBeacon) + return abci.UnavailableStateError(err) } -func (s *MutableState) SetConsensusParameters(params *beacon.ConsensusParameters) { - s.tree.Set(parametersKeyFmt.Encode(), cbor.Marshal(params)) +func (s *MutableState) SetConsensusParameters(ctx context.Context, params *beacon.ConsensusParameters) error { + err := s.Tree.Insert(ctx, parametersKeyFmt.Encode(), cbor.Marshal(params)) + return abci.UnavailableStateError(err) } // NewMutableState creates a new mutable beacon state wrapper. -func NewMutableState(tree *iavl.MutableTree) *MutableState { - inner := &abci.ImmutableState{Snapshot: tree.ImmutableTree} - +func NewMutableState(tree mkvs.KeyValueTree) *MutableState { return &MutableState{ - ImmutableState: &ImmutableState{inner}, - tree: tree, + ImmutableState: &ImmutableState{ + &abci.ImmutableState{Tree: tree}, + }, } } diff --git a/go/consensus/tendermint/apps/epochtime_mock/epochtime_mock.go b/go/consensus/tendermint/apps/epochtime_mock/epochtime_mock.go index c8a593be013..9e40108e42c 100644 --- a/go/consensus/tendermint/apps/epochtime_mock/epochtime_mock.go +++ b/go/consensus/tendermint/apps/epochtime_mock/epochtime_mock.go @@ -54,14 +54,16 @@ func (app *epochTimeMockApplication) InitChain(ctx *abci.Context, request types. func (app *epochTimeMockApplication) BeginBlock(ctx *abci.Context, request types.RequestBeginBlock) error { state := newMutableState(ctx.State()) - future, err := state.getFutureEpoch() + future, err := state.getFutureEpoch(ctx) if err != nil { return fmt.Errorf("BeginBlock: failed to get future epoch: %w", err) } if future == nil { return nil } - defer state.clearFutureEpoch() + defer func() { + _ = state.clearFutureEpoch(ctx) + }() height := ctx.BlockHeight() if future.Height != height { @@ -77,7 +79,10 @@ func (app *epochTimeMockApplication) BeginBlock(ctx *abci.Context, request types "current_height", height, ) - state.setEpoch(future.Epoch, height) + if err = state.setEpoch(ctx, future.Epoch, height); err != nil { + return fmt.Errorf("epochtime_mock: failed to set epoch: %w", err) + } + ctx.EmitEvent(api.NewEventBuilder(app.Name()).Attribute(KeyEpoch, cbor.Marshal(future.Epoch))) return nil @@ -125,7 +130,7 @@ func (app *epochTimeMockApplication) setEpoch( "is_check_only", ctx.IsCheckOnly(), ) - return state.setFutureEpoch(epoch, height+1) + return state.setFutureEpoch(ctx, epoch, height+1) } // New constructs a new mock epochtime application instance. diff --git a/go/consensus/tendermint/apps/epochtime_mock/query.go b/go/consensus/tendermint/apps/epochtime_mock/query.go index 571a584f48c..5e27da7d0c2 100644 --- a/go/consensus/tendermint/apps/epochtime_mock/query.go +++ b/go/consensus/tendermint/apps/epochtime_mock/query.go @@ -3,7 +3,6 @@ package epochtimemock import ( "context" - "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" epochtime "github.com/oasislabs/oasis-core/go/epochtime/api" ) @@ -19,27 +18,10 @@ type QueryFactory struct { // QueryAt returns the mock epochtime query interface for a specific height. func (sf *QueryFactory) QueryAt(ctx context.Context, height int64) (Query, error) { - var state *immutableState - var err error - abciCtx := abci.FromCtx(ctx) - - // If this request was made from InitChain, no blocks and states have been - // submitted yet, so we use the existing state instead. - if abciCtx != nil && abciCtx.IsInitChain() { - state = newMutableState(abciCtx.State()).immutableState - } else { - state, err = newImmutableState(sf.app.state, height) - if err != nil { - return nil, err - } + state, err := newImmutableState(ctx, sf.app.state, height) + if err != nil { + return nil, err } - - // If this request was made from an ABCI app, make sure to use the associated - // context for querying state instead of the default one. - if abciCtx != nil && height == abciCtx.BlockHeight()+1 { - state.Snapshot = abciCtx.State().ImmutableTree - } - return &epochtimeMockQuerier{state}, nil } @@ -48,7 +30,7 @@ type epochtimeMockQuerier struct { } func (eq *epochtimeMockQuerier) Epoch(ctx context.Context) (epochtime.EpochTime, int64, error) { - return eq.state.getEpoch() + return eq.state.getEpoch(ctx) } func (app *epochTimeMockApplication) QueryFactory() interface{} { diff --git a/go/consensus/tendermint/apps/epochtime_mock/state.go b/go/consensus/tendermint/apps/epochtime_mock/state.go index 0f1906d989b..6c86643fa45 100644 --- a/go/consensus/tendermint/apps/epochtime_mock/state.go +++ b/go/consensus/tendermint/apps/epochtime_mock/state.go @@ -1,13 +1,14 @@ package epochtimemock import ( - "github.com/pkg/errors" - "github.com/tendermint/iavl" + "context" + "fmt" "github.com/oasislabs/oasis-core/go/common/cbor" "github.com/oasislabs/oasis-core/go/common/keyformat" "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" "github.com/oasislabs/oasis-core/go/epochtime/api" + mkvs "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel" ) var ( @@ -30,32 +31,40 @@ type immutableState struct { *abci.ImmutableState } -func (s *immutableState) getEpoch() (api.EpochTime, int64, error) { - _, raw := s.Snapshot.Get(epochCurrentKeyFmt.Encode()) - if raw == nil { +func (s *immutableState) getEpoch(ctx context.Context) (api.EpochTime, int64, error) { + data, err := s.Tree.Get(ctx, epochCurrentKeyFmt.Encode()) + if err != nil { + return api.EpochInvalid, 0, abci.UnavailableStateError(err) + } + if data == nil { return api.EpochTime(0), 0, nil } var state mockEpochTimeState - err := cbor.Unmarshal(raw, &state) - return state.Epoch, state.Height, err + if err = cbor.Unmarshal(data, &state); err != nil { + return api.EpochInvalid, 0, abci.UnavailableStateError(err) + } + return state.Epoch, state.Height, nil } -func (s *immutableState) getFutureEpoch() (*mockEpochTimeState, error) { - _, raw := s.Snapshot.Get(epochFutureKeyFmt.Encode()) - if raw == nil { +func (s *immutableState) getFutureEpoch(ctx context.Context) (*mockEpochTimeState, error) { + data, err := s.Tree.Get(ctx, epochFutureKeyFmt.Encode()) + if err != nil { + return nil, abci.UnavailableStateError(err) + } + if data == nil { return nil, nil } var state mockEpochTimeState - if err := cbor.Unmarshal(raw, &state); err != nil { - return nil, errors.Wrap(err, "epochtime_mock: failed to unmarshal future epoch") + if err := cbor.Unmarshal(data, &state); err != nil { + return nil, abci.UnavailableStateError(err) } return &state, nil } -func newImmutableState(state abci.ApplicationState, version int64) (*immutableState, error) { - inner, err := abci.NewImmutableState(state, version) +func newImmutableState(ctx context.Context, state abci.ApplicationState, version int64) (*immutableState, error) { + inner, err := abci.NewImmutableState(ctx, state, version) if err != nil { return nil, err } @@ -65,39 +74,37 @@ func newImmutableState(state abci.ApplicationState, version int64) (*immutableSt type mutableState struct { *immutableState - - tree *iavl.MutableTree } -func (s *mutableState) setEpoch(epoch api.EpochTime, height int64) { +func (s *mutableState) setEpoch(ctx context.Context, epoch api.EpochTime, height int64) error { state := mockEpochTimeState{Epoch: epoch, Height: height} - s.tree.Set(epochCurrentKeyFmt.Encode(), cbor.Marshal(state)) + err := s.Tree.Insert(ctx, epochCurrentKeyFmt.Encode(), cbor.Marshal(state)) + return abci.UnavailableStateError(err) } -func (s *mutableState) setFutureEpoch(epoch api.EpochTime, height int64) error { - future, err := s.getFutureEpoch() +func (s *mutableState) setFutureEpoch(ctx context.Context, epoch api.EpochTime, height int64) error { + future, err := s.getFutureEpoch(ctx) if err != nil { return err } if future != nil { - return errors.New("epochtime_mock: future epoch already pending") + return fmt.Errorf("epochtime_mock: future epoch already pending") } state := mockEpochTimeState{Epoch: epoch, Height: height} - s.tree.Set(epochFutureKeyFmt.Encode(), cbor.Marshal(state)) - - return nil + err = s.Tree.Insert(ctx, epochFutureKeyFmt.Encode(), cbor.Marshal(state)) + return abci.UnavailableStateError(err) } -func (s *mutableState) clearFutureEpoch() { - s.tree.Remove(epochFutureKeyFmt.Encode()) +func (s *mutableState) clearFutureEpoch(ctx context.Context) error { + err := s.Tree.Remove(ctx, epochFutureKeyFmt.Encode()) + return abci.UnavailableStateError(err) } -func newMutableState(tree *iavl.MutableTree) *mutableState { - inner := &abci.ImmutableState{Snapshot: tree.ImmutableTree} - +func newMutableState(tree mkvs.KeyValueTree) *mutableState { return &mutableState{ - immutableState: &immutableState{inner}, - tree: tree, + immutableState: &immutableState{ + &abci.ImmutableState{Tree: tree}, + }, } } diff --git a/go/consensus/tendermint/apps/keymanager/genesis.go b/go/consensus/tendermint/apps/keymanager/genesis.go index d000fa3cb92..83429bf6d58 100644 --- a/go/consensus/tendermint/apps/keymanager/genesis.go +++ b/go/consensus/tendermint/apps/keymanager/genesis.go @@ -4,6 +4,7 @@ import ( "context" "encoding/json" "errors" + "fmt" "github.com/tendermint/tendermint/abci/types" @@ -68,7 +69,9 @@ func (app *keymanagerApplication) InitChain(ctx *abci.Context, request types.Req } // Set, enqueue for emit. - state.SetStatus(v) + if err := state.SetStatus(ctx, v); err != nil { + return fmt.Errorf("tendermint/keymanager: failed to set status: %w", err) + } toEmit = append(toEmit, v) } @@ -80,7 +83,7 @@ func (app *keymanagerApplication) InitChain(ctx *abci.Context, request types.Req } func (kq *keymanagerQuerier) Genesis(ctx context.Context) (*keymanager.Genesis, error) { - statuses, err := kq.state.Statuses() + statuses, err := kq.state.Statuses(ctx) if err != nil { return nil, err } diff --git a/go/consensus/tendermint/apps/keymanager/keymanager.go b/go/consensus/tendermint/apps/keymanager/keymanager.go index 05a0a474c84..53e7e2b325d 100644 --- a/go/consensus/tendermint/apps/keymanager/keymanager.go +++ b/go/consensus/tendermint/apps/keymanager/keymanager.go @@ -5,7 +5,6 @@ import ( "encoding/hex" "fmt" - "github.com/pkg/errors" "github.com/tendermint/tendermint/abci/types" "golang.org/x/crypto/sha3" @@ -86,17 +85,17 @@ func (app *keymanagerApplication) EndBlock(ctx *abci.Context, request types.Requ } func (app *keymanagerApplication) FireTimer(ctx *abci.Context, timer *abci.Timer) error { - return errors.New("tendermint/keymanager: unexpected timer") + return fmt.Errorf("tendermint/keymanager: unexpected timer") } func (app *keymanagerApplication) onEpochChange(ctx *abci.Context, epoch epochtime.EpochTime) error { // Query the runtime and node lists. regState := registryState.NewMutableState(ctx.State()) - runtimes, _ := regState.Runtimes() - nodes, _ := regState.Nodes() + runtimes, _ := regState.Runtimes(ctx) + nodes, _ := regState.Nodes(ctx) registry.SortNodeList(nodes) - params, err := regState.ConsensusParameters() + params, err := regState.ConsensusParameters(ctx) if err != nil { return fmt.Errorf("failed to get consensus parameters: %w", err) } @@ -130,7 +129,7 @@ func (app *keymanagerApplication) onEpochChange(ctx *abci.Context, epoch epochti ) // Suspend runtime. - if err := regState.SuspendRuntime(rt.ID); err != nil { + if err := regState.SuspendRuntime(ctx, rt.ID); err != nil { return err } @@ -139,7 +138,7 @@ func (app *keymanagerApplication) onEpochChange(ctx *abci.Context, epoch epochti } var forceEmit bool - oldStatus, err := state.Status(rt.ID) + oldStatus, err := state.Status(ctx, rt.ID) switch err { case nil: case api.ErrNoSuchStatus: @@ -154,7 +153,7 @@ func (app *keymanagerApplication) onEpochChange(ctx *abci.Context, epoch epochti "id", rt.ID, "err", err, ) - return errors.Wrap(err, "failed to query key manager status") + return fmt.Errorf("failed to query key manager status: %w", err) } newStatus := app.generateStatus(ctx, rt, oldStatus, nodes) @@ -168,7 +167,9 @@ func (app *keymanagerApplication) onEpochChange(ctx *abci.Context, epoch epochti ) // Set, enqueue for emit. - state.SetStatus(newStatus) + if err := state.SetStatus(ctx, newStatus); err != nil { + return fmt.Errorf("failed to set key manager status: %w", err) + } toEmit = append(toEmit, newStatus) } } diff --git a/go/consensus/tendermint/apps/keymanager/query.go b/go/consensus/tendermint/apps/keymanager/query.go index d76024399b2..8b6a965b341 100644 --- a/go/consensus/tendermint/apps/keymanager/query.go +++ b/go/consensus/tendermint/apps/keymanager/query.go @@ -4,7 +4,6 @@ import ( "context" "github.com/oasislabs/oasis-core/go/common" - "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" keymanagerState "github.com/oasislabs/oasis-core/go/consensus/tendermint/apps/keymanager/state" keymanager "github.com/oasislabs/oasis-core/go/keymanager/api" ) @@ -23,27 +22,10 @@ type QueryFactory struct { // QueryAt returns the key manager query interface for a specific height. func (sf *QueryFactory) QueryAt(ctx context.Context, height int64) (Query, error) { - var state *keymanagerState.ImmutableState - var err error - abciCtx := abci.FromCtx(ctx) - - // If this request was made from InitChain, no blocks and states have been - // submitted yet, so we use the existing state instead. - if abciCtx != nil && abciCtx.IsInitChain() { - state = keymanagerState.NewMutableState(abciCtx.State()).ImmutableState - } else { - state, err = keymanagerState.NewImmutableState(sf.app.state, height) - if err != nil { - return nil, err - } - } - - // If this request was made from an ABCI app, make sure to use the associated - // context for querying state instead of the default one. - if abciCtx != nil && height == abciCtx.BlockHeight()+1 { - state.Snapshot = abciCtx.State().ImmutableTree + state, err := keymanagerState.NewImmutableState(ctx, sf.app.state, height) + if err != nil { + return nil, err } - return &keymanagerQuerier{state}, nil } @@ -52,11 +34,11 @@ type keymanagerQuerier struct { } func (kq *keymanagerQuerier) Status(ctx context.Context, id common.Namespace) (*keymanager.Status, error) { - return kq.state.Status(id) + return kq.state.Status(ctx, id) } func (kq *keymanagerQuerier) Statuses(ctx context.Context) ([]*keymanager.Status, error) { - return kq.state.Statuses() + return kq.state.Statuses(ctx) } func (app *keymanagerApplication) QueryFactory() interface{} { diff --git a/go/consensus/tendermint/apps/keymanager/state/state.go b/go/consensus/tendermint/apps/keymanager/state/state.go index ef2f7c1eb55..8373f206a87 100644 --- a/go/consensus/tendermint/apps/keymanager/state/state.go +++ b/go/consensus/tendermint/apps/keymanager/state/state.go @@ -1,13 +1,14 @@ package state import ( - "github.com/tendermint/iavl" + "context" "github.com/oasislabs/oasis-core/go/common" "github.com/oasislabs/oasis-core/go/common/cbor" "github.com/oasislabs/oasis-core/go/common/keyformat" "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" "github.com/oasislabs/oasis-core/go/keymanager/api" + mkvs "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel" ) var ( @@ -21,8 +22,8 @@ type ImmutableState struct { *abci.ImmutableState } -func (st *ImmutableState) Statuses() ([]*api.Status, error) { - rawStatuses, err := st.getStatusesRaw() +func (st *ImmutableState) Statuses(ctx context.Context) ([]*api.Status, error) { + rawStatuses, err := st.getStatusesRaw(ctx) if err != nil { return nil, err } @@ -31,7 +32,7 @@ func (st *ImmutableState) Statuses() ([]*api.Status, error) { for _, raw := range rawStatuses { var status api.Status if err = cbor.Unmarshal(raw, &status); err != nil { - return nil, err + return nil, abci.UnavailableStateError(err) } statuses = append(statuses, &status) } @@ -39,39 +40,41 @@ func (st *ImmutableState) Statuses() ([]*api.Status, error) { return statuses, nil } -func (st *ImmutableState) getStatusesRaw() ([][]byte, error) { - var rawVec [][]byte - st.Snapshot.IterateRange( - statusKeyFmt.Encode(), - nil, - true, - func(key, value []byte) bool { - if !statusKeyFmt.Decode(key) { - return true - } - rawVec = append(rawVec, value) - return false - }, - ) +func (st *ImmutableState) getStatusesRaw(ctx context.Context) ([][]byte, error) { + it := st.Tree.NewIterator(ctx) + defer it.Close() + var rawVec [][]byte + for it.Seek(statusKeyFmt.Encode()); it.Valid(); it.Next() { + if !statusKeyFmt.Decode(it.Key()) { + break + } + rawVec = append(rawVec, it.Value()) + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } return rawVec, nil } -func (st *ImmutableState) Status(id common.Namespace) (*api.Status, error) { - _, raw := st.Snapshot.Get(statusKeyFmt.Encode(&id)) - if raw == nil { +func (st *ImmutableState) Status(ctx context.Context, id common.Namespace) (*api.Status, error) { + data, err := st.Tree.Get(ctx, statusKeyFmt.Encode(&id)) + if err != nil { + return nil, abci.UnavailableStateError(err) + } + if data == nil { return nil, api.ErrNoSuchStatus } var status api.Status - if err := cbor.Unmarshal(raw, &status); err != nil { - return nil, err + if err := cbor.Unmarshal(data, &status); err != nil { + return nil, abci.UnavailableStateError(err) } return &status, nil } -func NewImmutableState(state abci.ApplicationState, version int64) (*ImmutableState, error) { - inner, err := abci.NewImmutableState(state, version) +func NewImmutableState(ctx context.Context, state abci.ApplicationState, version int64) (*ImmutableState, error) { + inner, err := abci.NewImmutableState(ctx, state, version) if err != nil { return nil, err } @@ -81,20 +84,18 @@ func NewImmutableState(state abci.ApplicationState, version int64) (*ImmutableSt // MutableState is a mutable key manager state wrapper. type MutableState struct { *ImmutableState - - tree *iavl.MutableTree } -func (st *MutableState) SetStatus(status *api.Status) { - st.tree.Set(statusKeyFmt.Encode(&status.ID), cbor.Marshal(status)) +func (st *MutableState) SetStatus(ctx context.Context, status *api.Status) error { + err := st.Tree.Insert(ctx, statusKeyFmt.Encode(&status.ID), cbor.Marshal(status)) + return abci.UnavailableStateError(err) } // NewMutableState creates a new mutable key manager state wrapper. -func NewMutableState(tree *iavl.MutableTree) *MutableState { - inner := &abci.ImmutableState{Snapshot: tree.ImmutableTree} - +func NewMutableState(tree mkvs.KeyValueTree) *MutableState { return &MutableState{ - ImmutableState: &ImmutableState{inner}, - tree: tree, + ImmutableState: &ImmutableState{ + &abci.ImmutableState{Tree: tree}, + }, } } diff --git a/go/consensus/tendermint/apps/keymanager/transactions.go b/go/consensus/tendermint/apps/keymanager/transactions.go index 548174bc0dc..8922e42378a 100644 --- a/go/consensus/tendermint/apps/keymanager/transactions.go +++ b/go/consensus/tendermint/apps/keymanager/transactions.go @@ -19,7 +19,7 @@ func (app *keymanagerApplication) updatePolicy( ) error { // Ensure that the runtime exists and is a key manager. regState := registryState.NewMutableState(ctx.State()) - rt, err := regState.Runtime(sigPol.Policy.ID) + rt, err := regState.Runtime(ctx, sigPol.Policy.ID) if err != nil { return err } @@ -33,7 +33,7 @@ func (app *keymanagerApplication) updatePolicy( } // Get the existing policy document, if one exists. - oldStatus, err := state.Status(rt.ID) + oldStatus, err := state.Status(ctx, rt.ID) if err != nil { return err } @@ -48,7 +48,7 @@ func (app *keymanagerApplication) updatePolicy( } // Charge gas for this operation. - regParams, err := regState.ConsensusParameters() + regParams, err := regState.ConsensusParameters(ctx) if err != nil { return err } @@ -67,10 +67,12 @@ func (app *keymanagerApplication) updatePolicy( // TODO: It would be possible to update the cohort on each // node-reregistration, but I'm not sure how often the policy // will get updated. - nodes, _ := regState.Nodes() + nodes, _ := regState.Nodes(ctx) registry.SortNodeList(nodes) newStatus := app.generateStatus(ctx, rt, oldStatus, nodes) - state.SetStatus(newStatus) + if err := state.SetStatus(ctx, newStatus); err != nil { + panic(fmt.Errorf("failed to set keymanager status: %w", err)) + } ctx.EmitEvent(tmapi.NewEventBuilder(app.Name()).Attribute(KeyStatusUpdate, cbor.Marshal([]*api.Status{newStatus}))) diff --git a/go/consensus/tendermint/apps/registry/genesis.go b/go/consensus/tendermint/apps/registry/genesis.go index e86fab1c2e4..01c70ca5e0e 100644 --- a/go/consensus/tendermint/apps/registry/genesis.go +++ b/go/consensus/tendermint/apps/registry/genesis.go @@ -1,16 +1,13 @@ package registry import ( - "bytes" "context" "encoding/json" - "sort" + "fmt" - "github.com/pkg/errors" "github.com/tendermint/tendermint/abci/types" "github.com/oasislabs/oasis-core/go/common/cbor" - "github.com/oasislabs/oasis-core/go/common/crypto/signature" "github.com/oasislabs/oasis-core/go/common/node" "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" registryState "github.com/oasislabs/oasis-core/go/consensus/tendermint/apps/registry/state" @@ -27,7 +24,9 @@ func (app *registryApplication) InitChain(ctx *abci.Context, request types.Reque ) state := registryState.NewMutableState(ctx.State()) - state.SetConsensusParameters(&st.Parameters) + if err := state.SetConsensusParameters(ctx, &st.Parameters); err != nil { + return fmt.Errorf("failed to set consensus parameters: %w", err) + } for _, v := range st.Entities { ctx.Logger().Debug("InitChain: Registering genesis entity", @@ -38,7 +37,7 @@ func (app *registryApplication) InitChain(ctx *abci.Context, request types.Reque "err", err, "entity", v, ) - return errors.Wrap(err, "registry: genesis entity registration failure") + return fmt.Errorf("registry: genesis entity registration failure: %w", err) } } // Register runtimes. First key manager and then compute runtime(s). @@ -59,7 +58,7 @@ func (app *registryApplication) InitChain(ctx *abci.Context, request types.Reque "err", err, "runtime", v, ) - return errors.Wrap(err, "registry: genesis runtime registration failure") + return fmt.Errorf("registry: genesis runtime registration failure: %w", err) } } } @@ -72,14 +71,14 @@ func (app *registryApplication) InitChain(ctx *abci.Context, request types.Reque "err", err, "runtime", v, ) - return errors.Wrap(err, "registry: genesis suspended runtime registration failure") + return fmt.Errorf("registry: genesis suspended runtime registration failure: %w", err) } var rt registry.Runtime if err := cbor.Unmarshal(v.Blob, &rt); err != nil { - return errors.Wrap(err, "registry: malformed genesis suspended runtime") + return fmt.Errorf("registry: malformed genesis suspended runtime: %w", err) } - if err := state.SuspendRuntime(rt.ID); err != nil { - return errors.Wrap(err, "registry: failed to suspend runtime at genesis") + if err := state.SuspendRuntime(ctx, rt.ID); err != nil { + return fmt.Errorf("registry: failed to suspend runtime at genesis: %w", err) } } for _, v := range st.Nodes { @@ -93,26 +92,16 @@ func (app *registryApplication) InitChain(ctx *abci.Context, request types.Reque "err", err, "node", v, ) - return errors.Wrap(err, "registry: genesis node registration failure") + return fmt.Errorf("registry: genesis node registration failure: %w", err) } } - type nodeStatus struct { - id signature.PublicKey - status *registry.NodeStatus - } - var ns []*nodeStatus - for k, v := range st.NodeStatuses { - ns = append(ns, &nodeStatus{k, v}) - } - // Make sure that we apply node status updates in a canonical order. - sort.SliceStable(ns, func(i, j int) bool { return bytes.Compare(ns[i].id[:], ns[j].id[:]) < 0 }) - for _, s := range ns { - if err := state.SetNodeStatus(s.id, s.status); err != nil { + for id, status := range st.NodeStatuses { + if err := state.SetNodeStatus(ctx, id, status); err != nil { ctx.Logger().Error("InitChain: failed to set node status", "err", err, ) - return errors.Wrap(err, "registry: genesis node status set failure") + return fmt.Errorf("registry: genesis node status set failure: %w", err) } } @@ -121,19 +110,19 @@ func (app *registryApplication) InitChain(ctx *abci.Context, request types.Reque func (rq *registryQuerier) Genesis(ctx context.Context) (*registry.Genesis, error) { // Fetch entities, runtimes, and nodes from state. - signedEntities, err := rq.state.SignedEntities() + signedEntities, err := rq.state.SignedEntities(ctx) if err != nil { return nil, err } - signedRuntimes, err := rq.state.SignedRuntimes() + signedRuntimes, err := rq.state.SignedRuntimes(ctx) if err != nil { return nil, err } - suspendedRuntimes, err := rq.state.SuspendedRuntimes() + suspendedRuntimes, err := rq.state.SuspendedRuntimes(ctx) if err != nil { return nil, err } - signedNodes, err := rq.state.SignedNodes() + signedNodes, err := rq.state.SignedNodes(ctx) if err != nil { return nil, err } @@ -154,12 +143,12 @@ func (rq *registryQuerier) Genesis(ctx context.Context) (*registry.Genesis, erro } } - nodeStatuses, err := rq.state.NodeStatuses() + nodeStatuses, err := rq.state.NodeStatuses(ctx) if err != nil { return nil, err } - params, err := rq.state.ConsensusParameters() + params, err := rq.state.ConsensusParameters(ctx) if err != nil { return nil, err } diff --git a/go/consensus/tendermint/apps/registry/query.go b/go/consensus/tendermint/apps/registry/query.go index c763c984cec..5e838533ede 100644 --- a/go/consensus/tendermint/apps/registry/query.go +++ b/go/consensus/tendermint/apps/registry/query.go @@ -8,7 +8,6 @@ import ( "github.com/oasislabs/oasis-core/go/common/crypto/signature" "github.com/oasislabs/oasis-core/go/common/entity" "github.com/oasislabs/oasis-core/go/common/node" - "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" registryState "github.com/oasislabs/oasis-core/go/consensus/tendermint/apps/registry/state" registry "github.com/oasislabs/oasis-core/go/registry/api" ) @@ -32,27 +31,10 @@ type QueryFactory struct { // QueryAt returns the registry query interface for a specific height. func (sf *QueryFactory) QueryAt(ctx context.Context, height int64) (Query, error) { - var state *registryState.ImmutableState - var err error - abciCtx := abci.FromCtx(ctx) - - // If this request was made from InitChain, no blocks and states have been - // submitted yet, so we use the existing state instead. - if abciCtx != nil && abciCtx.IsInitChain() { - state = registryState.NewMutableState(abciCtx.State()).ImmutableState - } else { - state, err = registryState.NewImmutableState(sf.app.state, height) - if err != nil { - return nil, err - } - } - - // If this request was made from an ABCI app, make sure to use the associated - // context for querying state instead of the default one. - if abciCtx != nil && height == abciCtx.BlockHeight()+1 { - state.Snapshot = abciCtx.State().ImmutableTree + state, err := registryState.NewImmutableState(ctx, sf.app.state, height) + if err != nil { + return nil, err } - return ®istryQuerier{sf.app, state, height}, nil } @@ -63,11 +45,11 @@ type registryQuerier struct { } func (rq *registryQuerier) Entity(ctx context.Context, id signature.PublicKey) (*entity.Entity, error) { - return rq.state.Entity(id) + return rq.state.Entity(ctx, id) } func (rq *registryQuerier) Entities(ctx context.Context) ([]*entity.Entity, error) { - return rq.state.Entities() + return rq.state.Entities(ctx) } func (rq *registryQuerier) Node(ctx context.Context, id signature.PublicKey) (*node.Node, error) { @@ -76,7 +58,7 @@ func (rq *registryQuerier) Node(ctx context.Context, id signature.PublicKey) (*n return nil, fmt.Errorf("failed to get epoch: %w", err) } - node, err := rq.state.Node(id) + node, err := rq.state.Node(ctx, id) if err != nil { return nil, err } @@ -89,7 +71,7 @@ func (rq *registryQuerier) Node(ctx context.Context, id signature.PublicKey) (*n } func (rq *registryQuerier) NodeStatus(ctx context.Context, id signature.PublicKey) (*registry.NodeStatus, error) { - return rq.state.NodeStatus(id) + return rq.state.NodeStatus(ctx, id) } func (rq *registryQuerier) Nodes(ctx context.Context) ([]*node.Node, error) { @@ -98,7 +80,7 @@ func (rq *registryQuerier) Nodes(ctx context.Context) ([]*node.Node, error) { return nil, fmt.Errorf("failed to get epoch: %w", err) } - nodes, err := rq.state.Nodes() + nodes, err := rq.state.Nodes(ctx) if err != nil { return nil, err } @@ -115,11 +97,11 @@ func (rq *registryQuerier) Nodes(ctx context.Context) ([]*node.Node, error) { } func (rq *registryQuerier) Runtime(ctx context.Context, id common.Namespace) (*registry.Runtime, error) { - return rq.state.Runtime(id) + return rq.state.Runtime(ctx, id) } func (rq *registryQuerier) Runtimes(ctx context.Context) ([]*registry.Runtime, error) { - return rq.state.Runtimes() + return rq.state.Runtimes(ctx) } func (app *registryApplication) QueryFactory() interface{} { diff --git a/go/consensus/tendermint/apps/registry/registry.go b/go/consensus/tendermint/apps/registry/registry.go index 7635694eaa7..2b84f220c08 100644 --- a/go/consensus/tendermint/apps/registry/registry.go +++ b/go/consensus/tendermint/apps/registry/registry.go @@ -112,11 +112,11 @@ func (app *registryApplication) FireTimer(*abci.Context, *abci.Timer) error { return fmt.Errorf("tendermint/registry: unexpected timer") } -func (app *registryApplication) onRegistryEpochChanged(ctx *abci.Context, registryEpoch epochtime.EpochTime) error { +func (app *registryApplication) onRegistryEpochChanged(ctx *abci.Context, registryEpoch epochtime.EpochTime) (err error) { state := registryState.NewMutableState(ctx.State()) stakeState := stakingState.NewMutableState(ctx.State()) - nodes, err := state.Nodes() + nodes, err := state.Nodes(ctx) if err != nil { ctx.Logger().Error("onRegistryEpochChanged: failed to get nodes", "err", err, @@ -124,7 +124,7 @@ func (app *registryApplication) onRegistryEpochChanged(ctx *abci.Context, regist return fmt.Errorf("registry: onRegistryEpochChanged: failed to get nodes: %w", err) } - debondingInterval, err := stakeState.DebondingInterval() + debondingInterval, err := stakeState.DebondingInterval(ctx) if err != nil { ctx.Logger().Error("onRegistryEpochChanged: failed to get debonding interval", "err", err, @@ -132,7 +132,7 @@ func (app *registryApplication) onRegistryEpochChanged(ctx *abci.Context, regist return fmt.Errorf("registry: onRegistryEpochChanged: failed to get debonding interval: %w", err) } - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { ctx.Logger().Error("onRegistryEpochChanged: failed to fetch consensus parameters", "err", err, @@ -146,7 +146,9 @@ func (app *registryApplication) onRegistryEpochChanged(ctx *abci.Context, regist if err != nil { return fmt.Errorf("failed to create stake accumulator cache: %w", err) } - defer stakeAcc.Commit() + defer func() { + err = stakeAcc.Commit() + }() } // When a node expires, it is kept around for up to the debonding @@ -163,7 +165,7 @@ func (app *registryApplication) onRegistryEpochChanged(ctx *abci.Context, regist // node expiration (this is required so that we don't emit expiration // events every epoch). var status *registry.NodeStatus - status, err = state.NodeStatus(node.ID) + status, err = state.NodeStatus(ctx, node.ID) if err != nil { return fmt.Errorf("registry: onRegistryEpochChanged: couldn't get node status: %w", err) } @@ -171,7 +173,7 @@ func (app *registryApplication) onRegistryEpochChanged(ctx *abci.Context, regist if !status.ExpirationProcessed { expiredNodes = append(expiredNodes, node) status.ExpirationProcessed = true - if err = state.SetNodeStatus(node.ID, status); err != nil { + if err = state.SetNodeStatus(ctx, node.ID, status); err != nil { return fmt.Errorf("registry: onRegistryEpochChanged: couldn't set node status: %w", err) } } @@ -185,7 +187,9 @@ func (app *registryApplication) onRegistryEpochChanged(ctx *abci.Context, regist ctx.Logger().Debug("removing expired node", "node_id", node.ID, ) - state.RemoveNode(node) + if err = state.RemoveNode(ctx, node); err != nil { + return fmt.Errorf("registry: onRegistryEpochChanged: couldn't remove node: %w", err) + } // Remove the stake claim for the given node. if !params.DebugBypassStake { diff --git a/go/consensus/tendermint/apps/registry/state/state.go b/go/consensus/tendermint/apps/registry/state/state.go index 98e0bd3141f..c563a51e099 100644 --- a/go/consensus/tendermint/apps/registry/state/state.go +++ b/go/consensus/tendermint/apps/registry/state/state.go @@ -1,10 +1,9 @@ package state import ( + "context" "errors" - "github.com/tendermint/iavl" - "github.com/oasislabs/oasis-core/go/common" "github.com/oasislabs/oasis-core/go/common/cbor" "github.com/oasislabs/oasis-core/go/common/crypto/hash" @@ -15,10 +14,12 @@ import ( "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" tmcrypto "github.com/oasislabs/oasis-core/go/consensus/tendermint/crypto" registry "github.com/oasislabs/oasis-core/go/registry/api" + mkvs "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel" ) var ( - _ registry.NodeLookup = (*ImmutableState)(nil) + _ registry.NodeLookup = (*ImmutableState)(nil) + _ registry.RuntimeLookup = (*ImmutableState)(nil) // signedEntityKeyFmt is the key format used for signed entities. // @@ -79,89 +80,92 @@ type ImmutableState struct { *abci.ImmutableState } -func (s *ImmutableState) getSignedEntityRaw(id signature.PublicKey) ([]byte, error) { - _, value := s.Snapshot.Get(signedEntityKeyFmt.Encode(&id)) - return value, nil +func (s *ImmutableState) getSignedEntityRaw(ctx context.Context, id signature.PublicKey) ([]byte, error) { + data, err := s.Tree.Get(ctx, signedEntityKeyFmt.Encode(&id)) + return data, abci.UnavailableStateError(err) } -func (s *ImmutableState) Entity(id signature.PublicKey) (*entity.Entity, error) { - signedEntityRaw, err := s.getSignedEntityRaw(id) - if err != nil || signedEntityRaw == nil { +// Entity looks up a registered entity by its identifier. +func (s *ImmutableState) Entity(ctx context.Context, id signature.PublicKey) (*entity.Entity, error) { + signedEntityRaw, err := s.getSignedEntityRaw(ctx, id) + if err != nil { + return nil, err + } + if signedEntityRaw == nil { return nil, registry.ErrNoSuchEntity } var signedEntity entity.SignedEntity if err = cbor.Unmarshal(signedEntityRaw, &signedEntity); err != nil { - return nil, err + return nil, abci.UnavailableStateError(err) } var entity entity.Entity if err = cbor.Unmarshal(signedEntity.Blob, &entity); err != nil { - return nil, err + return nil, abci.UnavailableStateError(err) } return &entity, nil } -func (s *ImmutableState) Entities() ([]*entity.Entity, error) { +// Entities returns a list of all registered entities. +func (s *ImmutableState) Entities(ctx context.Context) ([]*entity.Entity, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() + var entities []*entity.Entity - s.Snapshot.IterateRange( - signedEntityKeyFmt.Encode(), - nil, - true, - func(key, value []byte) bool { - if !signedEntityKeyFmt.Decode(key) { - return true - } - - var signedEntity entity.SignedEntity - if err := cbor.Unmarshal(value, &signedEntity); err != nil { - panic("tendermint/registry: corrupted state: " + err.Error()) - } - var entity entity.Entity - if err := cbor.Unmarshal(signedEntity.Blob, &entity); err != nil { - panic("tendermint/registry: corrupted state: " + err.Error()) - } - - entities = append(entities, &entity) - - return false - }, - ) + for it.Seek(signedEntityKeyFmt.Encode()); it.Valid(); it.Next() { + if !signedEntityKeyFmt.Decode(it.Key()) { + break + } + var signedEntity entity.SignedEntity + if err := cbor.Unmarshal(it.Value(), &signedEntity); err != nil { + return nil, abci.UnavailableStateError(err) + } + var entity entity.Entity + if err := cbor.Unmarshal(signedEntity.Blob, &entity); err != nil { + return nil, abci.UnavailableStateError(err) + } + + entities = append(entities, &entity) + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } return entities, nil } -func (s *ImmutableState) SignedEntities() ([]*entity.SignedEntity, error) { +// SignedEntities returns a list of all registered entities (signed). +func (s *ImmutableState) SignedEntities(ctx context.Context) ([]*entity.SignedEntity, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() + var entities []*entity.SignedEntity - s.Snapshot.IterateRange( - signedEntityKeyFmt.Encode(), - nil, - true, - func(key, value []byte) bool { - if !signedEntityKeyFmt.Decode(key) { - return true - } - - var signedEntity entity.SignedEntity - if err := cbor.Unmarshal(value, &signedEntity); err != nil { - panic("tendermint/registry: corrupted state: " + err.Error()) - } - - entities = append(entities, &signedEntity) - - return false - }, - ) + for it.Seek(signedEntityKeyFmt.Encode()); it.Valid(); it.Next() { + if !signedEntityKeyFmt.Decode(it.Key()) { + break + } + + var signedEntity entity.SignedEntity + if err := cbor.Unmarshal(it.Value(), &signedEntity); err != nil { + return nil, abci.UnavailableStateError(err) + } + entities = append(entities, &signedEntity) + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } return entities, nil } -func (s *ImmutableState) getSignedNodeRaw(id signature.PublicKey) ([]byte, error) { - _, value := s.Snapshot.Get(signedNodeKeyFmt.Encode(&id)) - return value, nil +func (s *ImmutableState) getSignedNodeRaw(ctx context.Context, id signature.PublicKey) ([]byte, error) { + data, err := s.Tree.Get(ctx, signedNodeKeyFmt.Encode(&id)) + return data, abci.UnavailableStateError(err) } -func (s *ImmutableState) Node(id signature.PublicKey) (*node.Node, error) { - signedNodeRaw, err := s.getSignedNodeRaw(id) +// Node looks up a specific node by its identifier. +func (s *ImmutableState) Node(ctx context.Context, id signature.PublicKey) (*node.Node, error) { + signedNodeRaw, err := s.getSignedNodeRaw(ctx, id) if err != nil { return nil, err } @@ -171,103 +175,108 @@ func (s *ImmutableState) Node(id signature.PublicKey) (*node.Node, error) { var signedNode node.MultiSignedNode if err = cbor.Unmarshal(signedNodeRaw, &signedNode); err != nil { - return nil, err + return nil, abci.UnavailableStateError(err) } var node node.Node if err = cbor.Unmarshal(signedNode.Blob, &node); err != nil { - return nil, err + return nil, abci.UnavailableStateError(err) } return &node, nil } -func (s *ImmutableState) NodeByConsensusAddress(address []byte) (*node.Node, error) { - _, rawID := s.Snapshot.Get(nodeByConsAddressKeyFmt.Encode(address)) +// NodeByConsensusAddress looks up a specific node by its consensus address. +func (s *ImmutableState) NodeByConsensusAddress(ctx context.Context, address []byte) (*node.Node, error) { + rawID, err := s.Tree.Get(ctx, nodeByConsAddressKeyFmt.Encode(address)) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if rawID == nil { return nil, registry.ErrNoSuchNode } var id signature.PublicKey if err := id.UnmarshalBinary(rawID); err != nil { - return nil, err + return nil, abci.UnavailableStateError(err) } - return s.Node(id) + return s.Node(ctx, id) } -func (s *ImmutableState) Nodes() ([]*node.Node, error) { +// Nodes returns a list of all registered nodes. +func (s *ImmutableState) Nodes(ctx context.Context) ([]*node.Node, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() + var nodes []*node.Node - s.Snapshot.IterateRange( - signedNodeKeyFmt.Encode(), - nil, - true, - func(key, value []byte) bool { - if !signedNodeKeyFmt.Decode(key) { - return true - } - - var signedNode node.MultiSignedNode - if err := cbor.Unmarshal(value, &signedNode); err != nil { - panic("tendermint/registry: corrupted state: " + err.Error()) - } - var node node.Node - if err := cbor.Unmarshal(signedNode.Blob, &node); err != nil { - panic("tendermint/registry: corrupted state: " + err.Error()) - } - - nodes = append(nodes, &node) - - return false - }, - ) + for it.Seek(signedNodeKeyFmt.Encode()); it.Valid(); it.Next() { + if !signedNodeKeyFmt.Decode(it.Key()) { + break + } + var signedNode node.MultiSignedNode + if err := cbor.Unmarshal(it.Value(), &signedNode); err != nil { + return nil, abci.UnavailableStateError(err) + } + var node node.Node + if err := cbor.Unmarshal(signedNode.Blob, &node); err != nil { + return nil, abci.UnavailableStateError(err) + } + + nodes = append(nodes, &node) + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } return nodes, nil } -func (s *ImmutableState) SignedNodes() ([]*node.MultiSignedNode, error) { +// SignedNodes returns a list of all registered nodes (in signed form). +func (s *ImmutableState) SignedNodes(ctx context.Context) ([]*node.MultiSignedNode, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() + var nodes []*node.MultiSignedNode - s.Snapshot.IterateRange( - signedNodeKeyFmt.Encode(), - nil, - true, - func(key, value []byte) bool { - if !signedNodeKeyFmt.Decode(key) { - return true - } - - var signedNode node.MultiSignedNode - if err := cbor.Unmarshal(value, &signedNode); err != nil { - panic("tendermint/registry: corrupted state: " + err.Error()) - } - - nodes = append(nodes, &signedNode) - - return false - }, - ) + for it.Seek(signedNodeKeyFmt.Encode()); it.Valid(); it.Next() { + if !signedNodeKeyFmt.Decode(it.Key()) { + break + } + var signedNode node.MultiSignedNode + if err := cbor.Unmarshal(it.Value(), &signedNode); err != nil { + return nil, abci.UnavailableStateError(err) + } + + nodes = append(nodes, &signedNode) + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } return nodes, nil } -func (s *ImmutableState) getSignedRuntime(keyFmt *keyformat.KeyFormat, id common.Namespace) (*registry.SignedRuntime, error) { - _, raw := s.Snapshot.Get(keyFmt.Encode(&id)) +func (s *ImmutableState) getSignedRuntime(ctx context.Context, keyFmt *keyformat.KeyFormat, id common.Namespace) (*registry.SignedRuntime, error) { + raw, err := s.Tree.Get(ctx, keyFmt.Encode(&id)) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if raw == nil { return nil, registry.ErrNoSuchRuntime } var signedRuntime registry.SignedRuntime if err := cbor.Unmarshal(raw, &signedRuntime); err != nil { - return nil, err + return nil, abci.UnavailableStateError(err) } return &signedRuntime, nil } -func (s *ImmutableState) getRuntime(keyFmt *keyformat.KeyFormat, id common.Namespace) (*registry.Runtime, error) { - signedRuntime, err := s.getSignedRuntime(keyFmt, id) +func (s *ImmutableState) getRuntime(ctx context.Context, keyFmt *keyformat.KeyFormat, id common.Namespace) (*registry.Runtime, error) { + signedRuntime, err := s.getSignedRuntime(ctx, keyFmt, id) if err != nil { return nil, err } var runtime registry.Runtime if err = cbor.Unmarshal(signedRuntime.Blob, &runtime); err != nil { - return nil, err + return nil, abci.UnavailableStateError(err) } return &runtime, nil } @@ -276,21 +285,21 @@ func (s *ImmutableState) getRuntime(keyFmt *keyformat.KeyFormat, id common.Names // // This excludes any suspended runtimes, use SuspendedRuntime to query // suspended runtimes only. -func (s *ImmutableState) Runtime(id common.Namespace) (*registry.Runtime, error) { - return s.getRuntime(signedRuntimeKeyFmt, id) +func (s *ImmutableState) Runtime(ctx context.Context, id common.Namespace) (*registry.Runtime, error) { + return s.getRuntime(ctx, signedRuntimeKeyFmt, id) } // SuspendedRuntime looks up a suspended runtime by its identifier and // returns it. -func (s *ImmutableState) SuspendedRuntime(id common.Namespace) (*registry.Runtime, error) { - return s.getRuntime(suspendedRuntimeKeyFmt, id) +func (s *ImmutableState) SuspendedRuntime(ctx context.Context, id common.Namespace) (*registry.Runtime, error) { + return s.getRuntime(ctx, suspendedRuntimeKeyFmt, id) } // AnyRuntime looks up either an active or suspended runtime by its identifier and returns it. -func (s *ImmutableState) AnyRuntime(id common.Namespace) (rt *registry.Runtime, err error) { - rt, err = s.Runtime(id) +func (s *ImmutableState) AnyRuntime(ctx context.Context, id common.Namespace) (rt *registry.Runtime, err error) { + rt, err = s.Runtime(ctx, id) if err == registry.ErrNoSuchRuntime { - rt, err = s.SuspendedRuntime(id) + rt, err = s.SuspendedRuntime(ctx, id) } return } @@ -299,224 +308,231 @@ func (s *ImmutableState) AnyRuntime(id common.Namespace) (rt *registry.Runtime, // // This excludes any suspended runtimes, use SuspendedSignedRuntime to query // suspended runtimes only. -func (s *ImmutableState) SignedRuntime(id common.Namespace) (*registry.SignedRuntime, error) { - return s.getSignedRuntime(signedRuntimeKeyFmt, id) +func (s *ImmutableState) SignedRuntime(ctx context.Context, id common.Namespace) (*registry.SignedRuntime, error) { + return s.getSignedRuntime(ctx, signedRuntimeKeyFmt, id) } // SignedSuspendedRuntime looks up a (signed) suspended runtime by its identifier and returns it. -func (s *ImmutableState) SignedSuspendedRuntime(id common.Namespace) (*registry.SignedRuntime, error) { - return s.getSignedRuntime(suspendedRuntimeKeyFmt, id) +func (s *ImmutableState) SignedSuspendedRuntime(ctx context.Context, id common.Namespace) (*registry.SignedRuntime, error) { + return s.getSignedRuntime(ctx, suspendedRuntimeKeyFmt, id) } func (s *ImmutableState) iterateRuntimes( + ctx context.Context, keyFmt *keyformat.KeyFormat, - cb func(*registry.SignedRuntime), -) { - s.Snapshot.IterateRange( - keyFmt.Encode(), - nil, - true, - func(key, value []byte) bool { - if !keyFmt.Decode(key) { - return true - } - - var signedRt registry.SignedRuntime - if err := cbor.Unmarshal(value, &signedRt); err != nil { - panic("tendermint/registry: corrupted state: " + err.Error()) - } - - cb(&signedRt) - - return false - }, - ) + cb func(*registry.SignedRuntime) error, +) error { + it := s.Tree.NewIterator(ctx) + defer it.Close() + + for it.Seek(keyFmt.Encode()); it.Valid(); it.Next() { + if !keyFmt.Decode(it.Key()) { + break + } + + var signedRt registry.SignedRuntime + if err := cbor.Unmarshal(it.Value(), &signedRt); err != nil { + return abci.UnavailableStateError(err) + } + + if err := cb(&signedRt); err != nil { + return err + } + } + return abci.UnavailableStateError(it.Err()) } // SignedRuntimes returns a list of all registered runtimes (signed). // // This excludes any suspended runtimes. -func (s *ImmutableState) SignedRuntimes() ([]*registry.SignedRuntime, error) { +func (s *ImmutableState) SignedRuntimes(ctx context.Context) ([]*registry.SignedRuntime, error) { var runtimes []*registry.SignedRuntime - s.iterateRuntimes(signedRuntimeKeyFmt, func(rt *registry.SignedRuntime) { + err := s.iterateRuntimes(ctx, signedRuntimeKeyFmt, func(rt *registry.SignedRuntime) error { runtimes = append(runtimes, rt) + return nil }) - + if err != nil { + return nil, err + } return runtimes, nil } // SuspendedRuntimes returns a list of all suspended runtimes (signed). -func (s *ImmutableState) SuspendedRuntimes() ([]*registry.SignedRuntime, error) { +func (s *ImmutableState) SuspendedRuntimes(ctx context.Context) ([]*registry.SignedRuntime, error) { var runtimes []*registry.SignedRuntime - s.iterateRuntimes(suspendedRuntimeKeyFmt, func(rt *registry.SignedRuntime) { + err := s.iterateRuntimes(ctx, suspendedRuntimeKeyFmt, func(rt *registry.SignedRuntime) error { runtimes = append(runtimes, rt) + return nil }) - + if err != nil { + return nil, err + } return runtimes, nil } // AllSignedRuntimes returns a list of all runtimes (suspended included). -func (s *ImmutableState) AllSignedRuntimes() ([]*registry.SignedRuntime, error) { +func (s *ImmutableState) AllSignedRuntimes(ctx context.Context) ([]*registry.SignedRuntime, error) { var runtimes []*registry.SignedRuntime - s.iterateRuntimes(signedRuntimeKeyFmt, func(rt *registry.SignedRuntime) { + err := s.iterateRuntimes(ctx, signedRuntimeKeyFmt, func(rt *registry.SignedRuntime) error { runtimes = append(runtimes, rt) + return nil }) - s.iterateRuntimes(suspendedRuntimeKeyFmt, func(rt *registry.SignedRuntime) { + if err != nil { + return nil, err + } + err = s.iterateRuntimes(ctx, suspendedRuntimeKeyFmt, func(rt *registry.SignedRuntime) error { runtimes = append(runtimes, rt) + return nil }) - + if err != nil { + return nil, err + } return runtimes, nil } // Runtimes returns a list of all registered runtimes. // // This excludes any suspended runtimes. -func (s *ImmutableState) Runtimes() ([]*registry.Runtime, error) { +func (s *ImmutableState) Runtimes(ctx context.Context) ([]*registry.Runtime, error) { var runtimes []*registry.Runtime - s.iterateRuntimes(signedRuntimeKeyFmt, func(sigRt *registry.SignedRuntime) { + err := s.iterateRuntimes(ctx, signedRuntimeKeyFmt, func(sigRt *registry.SignedRuntime) error { var rt registry.Runtime if err := cbor.Unmarshal(sigRt.Blob, &rt); err != nil { - panic("tendermint/registry: corrupted state: " + err.Error()) + return abci.UnavailableStateError(err) } runtimes = append(runtimes, &rt) + return nil }) - + if err != nil { + return nil, err + } return runtimes, nil } // AllRuntimes returns a list of all registered runtimes (suspended included). -func (s *ImmutableState) AllRuntimes() ([]*registry.Runtime, error) { +func (s *ImmutableState) AllRuntimes(ctx context.Context) ([]*registry.Runtime, error) { var runtimes []*registry.Runtime - unpackFn := func(sigRt *registry.SignedRuntime) { + unpackFn := func(sigRt *registry.SignedRuntime) error { var rt registry.Runtime if err := cbor.Unmarshal(sigRt.Blob, &rt); err != nil { - panic("tendermint/registry: corrupted state: " + err.Error()) + return abci.UnavailableStateError(err) } runtimes = append(runtimes, &rt) + return nil + } + if err := s.iterateRuntimes(ctx, signedRuntimeKeyFmt, unpackFn); err != nil { + return nil, err + } + if err := s.iterateRuntimes(ctx, suspendedRuntimeKeyFmt, unpackFn); err != nil { + return nil, err } - s.iterateRuntimes(signedRuntimeKeyFmt, unpackFn) - s.iterateRuntimes(suspendedRuntimeKeyFmt, unpackFn) - return runtimes, nil } -func (s *ImmutableState) NodeStatus(id signature.PublicKey) (*registry.NodeStatus, error) { - _, value := s.Snapshot.Get(nodeStatusKeyFmt.Encode(&id)) +// NodeStatus returns a specific node status. +func (s *ImmutableState) NodeStatus(ctx context.Context, id signature.PublicKey) (*registry.NodeStatus, error) { + value, err := s.Tree.Get(ctx, nodeStatusKeyFmt.Encode(&id)) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if value == nil { return nil, registry.ErrNoSuchNode } var status registry.NodeStatus if err := cbor.Unmarshal(value, &status); err != nil { - return nil, err + return nil, abci.UnavailableStateError(err) } return &status, nil } -func (s *ImmutableState) NodeStatuses() (map[signature.PublicKey]*registry.NodeStatus, error) { +// NodeStatuses returns all of the node statuses. +func (s *ImmutableState) NodeStatuses(ctx context.Context) (map[signature.PublicKey]*registry.NodeStatus, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() + statuses := make(map[signature.PublicKey]*registry.NodeStatus) - s.Snapshot.IterateRange( - nodeStatusKeyFmt.Encode(), - nil, - true, - func(key, value []byte) bool { - var nodeID signature.PublicKey - if !nodeStatusKeyFmt.Decode(key, &nodeID) { - return true - } - - var status registry.NodeStatus - if err := cbor.Unmarshal(value, &status); err != nil { - panic("tendermint/registry: corrupted state: " + err.Error()) - } - - statuses[nodeID] = &status - - return false - }, - ) + for it.Seek(nodeStatusKeyFmt.Encode()); it.Valid(); it.Next() { + var nodeID signature.PublicKey + if !nodeStatusKeyFmt.Decode(it.Key(), &nodeID) { + break + } + + var status registry.NodeStatus + if err := cbor.Unmarshal(it.Value(), &status); err != nil { + return nil, abci.UnavailableStateError(err) + } + statuses[nodeID] = &status + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } return statuses, nil } -func (s *ImmutableState) HasEntityNodes(id signature.PublicKey) (bool, error) { - result := true - s.Snapshot.IterateRange( - signedNodeByEntityKeyFmt.Encode(&id), - nil, - true, - func(key, value []byte) bool { - var entityID signature.PublicKey - if !signedNodeByEntityKeyFmt.Decode(key, &entityID) || !entityID.Equal(id) { - result = false - } - // Stop immediately as we are only interested in one result. - return true - }, - ) - return result, nil -} +// HasEntityNodes checks whether an entity has any registered nodes. +func (s *ImmutableState) HasEntityNodes(ctx context.Context, id signature.PublicKey) (bool, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() -func (s *ImmutableState) NumEntityNodes(id signature.PublicKey) (int, error) { - var n int - s.Snapshot.IterateRange( - signedNodeByEntityKeyFmt.Encode(&id), - nil, - true, - func(key, value []byte) bool { - var entityID signature.PublicKey - if !signedNodeByEntityKeyFmt.Decode(key, &entityID) || !entityID.Equal(id) { - return true - } + if it.Seek(signedNodeByEntityKeyFmt.Encode(&id)); it.Valid() { + var entityID signature.PublicKey + if !signedNodeByEntityKeyFmt.Decode(it.Key(), &entityID) || !entityID.Equal(id) { + return false, nil + } + return true, nil + } + return false, abci.UnavailableStateError(it.Err()) +} - n++ +// HasEntityRuntimes checks whether an entity has any registered runtimes. +func (s *ImmutableState) HasEntityRuntimes(ctx context.Context, id signature.PublicKey) (bool, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() - return false - }, - ) - return n, nil -} - -func (s *ImmutableState) HasEntityRuntimes(id signature.PublicKey) (bool, error) { - result := true - s.Snapshot.IterateRange( - signedRuntimeByEntityKeyFmt.Encode(&id), - nil, - true, - func(key, value []byte) bool { - var entityID signature.PublicKey - if !signedRuntimeByEntityKeyFmt.Decode(key, &entityID) || !entityID.Equal(id) { - result = false - } - // Stop immediately as we are only interested in one result. - return true - }, - ) - return result, nil + if it.Seek(signedRuntimeByEntityKeyFmt.Encode(&id)); it.Valid() { + var entityID signature.PublicKey + if !signedRuntimeByEntityKeyFmt.Decode(it.Key(), &entityID) || !entityID.Equal(id) { + return false, nil + } + return true, nil + } + return false, abci.UnavailableStateError(it.Err()) } -func (s *ImmutableState) ConsensusParameters() (*registry.ConsensusParameters, error) { - _, raw := s.Snapshot.Get(parametersKeyFmt.Encode()) +// ConsensusParameters returns the registry consensus parameters. +func (s *ImmutableState) ConsensusParameters(ctx context.Context) (*registry.ConsensusParameters, error) { + raw, err := s.Tree.Get(ctx, parametersKeyFmt.Encode()) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if raw == nil { return nil, errors.New("tendermint/registry: expected consensus parameters to be present in app state") } var params registry.ConsensusParameters - err := cbor.Unmarshal(raw, ¶ms) - return ¶ms, err + if err := cbor.Unmarshal(raw, ¶ms); err != nil { + return nil, abci.UnavailableStateError(err) + } + return ¶ms, nil } -func (s *ImmutableState) NodeByConsensusOrP2PKey(key signature.PublicKey) (*node.Node, error) { - _, rawID := s.Snapshot.Get(keyMapKeyFmt.Encode(&key)) +// NodeByConsensusOrP2PKey looks up a specific node by its consensus or P2P key. +func (s *ImmutableState) NodeByConsensusOrP2PKey(ctx context.Context, key signature.PublicKey) (*node.Node, error) { + rawID, err := s.Tree.Get(ctx, keyMapKeyFmt.Encode(&key)) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if rawID == nil { return nil, registry.ErrNoSuchNode } var id signature.PublicKey if err := id.UnmarshalBinary(rawID); err != nil { - return nil, err + return nil, abci.UnavailableStateError(err) } - return s.Node(id) + return s.Node(ctx, id) } // Hashes a node's committee certificate into a key for the certificate to node ID map. @@ -526,22 +542,26 @@ func nodeCertificateToMapKey(cert []byte) hash.Hash { return h } -func (s *ImmutableState) NodeByCertificate(cert []byte) (*node.Node, error) { +// NodeByCertificate looks up a specific node by its certificate. +func (s *ImmutableState) NodeByCertificate(ctx context.Context, cert []byte) (*node.Node, error) { certHash := nodeCertificateToMapKey(cert) - _, rawID := s.Snapshot.Get(certificateMapKeyFmt.Encode(&certHash)) + rawID, err := s.Tree.Get(ctx, certificateMapKeyFmt.Encode(&certHash)) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if rawID == nil { return nil, registry.ErrNoSuchNode } var id signature.PublicKey if err := id.UnmarshalBinary(rawID); err != nil { - return nil, err + return nil, abci.UnavailableStateError(err) } - return s.Node(id) + return s.Node(ctx, id) } -func NewImmutableState(state abci.ApplicationState, version int64) (*ImmutableState, error) { - inner, err := abci.NewImmutableState(state, version) +func NewImmutableState(ctx context.Context, state abci.ApplicationState, version int64) (*ImmutableState, error) { + inner, err := abci.NewImmutableState(ctx, state, version) if err != nil { return nil, err } @@ -552,123 +572,156 @@ func NewImmutableState(state abci.ApplicationState, version int64) (*ImmutableSt // MutableState is a mutable registry state wrapper. type MutableState struct { *ImmutableState - - tree *iavl.MutableTree } -func (s *MutableState) SetEntity(ent *entity.Entity, sigEnt *entity.SignedEntity) { - s.tree.Set(signedEntityKeyFmt.Encode(&ent.ID), cbor.Marshal(sigEnt)) +// SetEntity sets a signed entity descriptor for a registered entity. +func (s *MutableState) SetEntity(ctx context.Context, ent *entity.Entity, sigEnt *entity.SignedEntity) error { + err := s.Tree.Insert(ctx, signedEntityKeyFmt.Encode(&ent.ID), cbor.Marshal(sigEnt)) + return abci.UnavailableStateError(err) } -func (s *MutableState) RemoveEntity(id signature.PublicKey) (*entity.Entity, error) { - data, removed := s.tree.Remove(signedEntityKeyFmt.Encode(&id)) - if removed { +// RemoveEntity removes a previously registered entity. +func (s *MutableState) RemoveEntity(ctx context.Context, id signature.PublicKey) (*entity.Entity, error) { + data, err := s.Tree.RemoveExisting(ctx, signedEntityKeyFmt.Encode(&id)) + if err != nil { + return nil, abci.UnavailableStateError(err) + } + if data != nil { var removedSignedEntity entity.SignedEntity + if err = cbor.Unmarshal(data, &removedSignedEntity); err != nil { + return nil, abci.UnavailableStateError(err) + } var removedEntity entity.Entity - - cbor.MustUnmarshal(data, &removedSignedEntity) - cbor.MustUnmarshal(removedSignedEntity.Blob, &removedEntity) + if err = cbor.Unmarshal(removedSignedEntity.Blob, &removedEntity); err != nil { + return nil, abci.UnavailableStateError(err) + } return &removedEntity, nil } - return nil, registry.ErrNoSuchEntity } -func (s *MutableState) SetNode(node *node.Node, signedNode *node.MultiSignedNode) error { - // Ensure that the entity exists. - ent, err := s.getSignedEntityRaw(node.EntityID) - if ent == nil || err != nil { - return registry.ErrNoSuchEntity - } - - s.tree.Set(signedNodeKeyFmt.Encode(&node.ID), cbor.Marshal(signedNode)) - s.tree.Set(signedNodeByEntityKeyFmt.Encode(&node.EntityID, &node.ID), []byte("")) - +// SetNode sets a signed node descriptor for a registered node. +func (s *MutableState) SetNode(ctx context.Context, node *node.Node, signedNode *node.MultiSignedNode) error { address := []byte(tmcrypto.PublicKeyToTendermint(&node.Consensus.ID).Address()) rawNodeID, err := node.ID.MarshalBinary() if err != nil { return err } - s.tree.Set(nodeByConsAddressKeyFmt.Encode(address), rawNodeID) - s.tree.Set(keyMapKeyFmt.Encode(&node.Consensus.ID), rawNodeID) - s.tree.Set(keyMapKeyFmt.Encode(&node.P2P.ID), rawNodeID) + if err = s.Tree.Insert(ctx, signedNodeKeyFmt.Encode(&node.ID), cbor.Marshal(signedNode)); err != nil { + return abci.UnavailableStateError(err) + } + if err = s.Tree.Insert(ctx, signedNodeByEntityKeyFmt.Encode(&node.EntityID, &node.ID), []byte("")); err != nil { + return abci.UnavailableStateError(err) + } + + if err = s.Tree.Insert(ctx, nodeByConsAddressKeyFmt.Encode(address), rawNodeID); err != nil { + return abci.UnavailableStateError(err) + } + + if err = s.Tree.Insert(ctx, keyMapKeyFmt.Encode(&node.Consensus.ID), rawNodeID); err != nil { + return abci.UnavailableStateError(err) + } + if err = s.Tree.Insert(ctx, keyMapKeyFmt.Encode(&node.P2P.ID), rawNodeID); err != nil { + return abci.UnavailableStateError(err) + } certHash := nodeCertificateToMapKey(node.Committee.Certificate) - s.tree.Set(certificateMapKeyFmt.Encode(&certHash), rawNodeID) + if err = s.Tree.Insert(ctx, certificateMapKeyFmt.Encode(&certHash), rawNodeID); err != nil { + return abci.UnavailableStateError(err) + } return nil } -func (s *MutableState) RemoveNode(node *node.Node) { - s.tree.Remove(signedNodeKeyFmt.Encode(&node.ID)) - s.tree.Remove(signedNodeByEntityKeyFmt.Encode(&node.EntityID, &node.ID)) - s.tree.Remove(nodeStatusKeyFmt.Encode(&node.ID)) +// RemoveNode removes a registered node. +func (s *MutableState) RemoveNode(ctx context.Context, node *node.Node) error { + if err := s.Tree.Remove(ctx, signedNodeKeyFmt.Encode(&node.ID)); err != nil { + return abci.UnavailableStateError(err) + } + if err := s.Tree.Remove(ctx, signedNodeByEntityKeyFmt.Encode(&node.EntityID, &node.ID)); err != nil { + return abci.UnavailableStateError(err) + } + if err := s.Tree.Remove(ctx, nodeStatusKeyFmt.Encode(&node.ID)); err != nil { + return abci.UnavailableStateError(err) + } address := []byte(tmcrypto.PublicKeyToTendermint(&node.Consensus.ID).Address()) - s.tree.Remove(nodeByConsAddressKeyFmt.Encode(address)) + if err := s.Tree.Remove(ctx, nodeByConsAddressKeyFmt.Encode(address)); err != nil { + return abci.UnavailableStateError(err) + } - s.tree.Remove(keyMapKeyFmt.Encode(&node.Consensus.ID)) - s.tree.Remove(keyMapKeyFmt.Encode(&node.P2P.ID)) + if err := s.Tree.Remove(ctx, keyMapKeyFmt.Encode(&node.Consensus.ID)); err != nil { + return abci.UnavailableStateError(err) + } + if err := s.Tree.Remove(ctx, keyMapKeyFmt.Encode(&node.P2P.ID)); err != nil { + return abci.UnavailableStateError(err) + } certHash := nodeCertificateToMapKey(node.Committee.Certificate) - s.tree.Remove(certificateMapKeyFmt.Encode(&certHash)) + if err := s.Tree.Remove(ctx, certificateMapKeyFmt.Encode(&certHash)); err != nil { + return abci.UnavailableStateError(err) + } + + return nil } -func (s *MutableState) SetRuntime(rt *registry.Runtime, sigRt *registry.SignedRuntime, suspended bool) error { - ent, err := s.getSignedEntityRaw(rt.EntityID) - if ent == nil || err != nil { - return registry.ErrNoSuchEntity +// SetRuntime sets a signed runtime descriptor for a registered runtime. +func (s *MutableState) SetRuntime(ctx context.Context, rt *registry.Runtime, sigRt *registry.SignedRuntime, suspended bool) error { + if err := s.Tree.Insert(ctx, signedRuntimeByEntityKeyFmt.Encode(&rt.EntityID, &rt.ID), []byte("")); err != nil { + return abci.UnavailableStateError(err) } - s.tree.Set(signedRuntimeByEntityKeyFmt.Encode(&rt.EntityID, &rt.ID), []byte("")) - + var err error if suspended { - s.tree.Set(suspendedRuntimeKeyFmt.Encode(&rt.ID), cbor.Marshal(sigRt)) + err = s.Tree.Insert(ctx, suspendedRuntimeKeyFmt.Encode(&rt.ID), cbor.Marshal(sigRt)) } else { - s.tree.Set(signedRuntimeKeyFmt.Encode(&rt.ID), cbor.Marshal(sigRt)) + err = s.Tree.Insert(ctx, signedRuntimeKeyFmt.Encode(&rt.ID), cbor.Marshal(sigRt)) } - - return nil + return abci.UnavailableStateError(err) } -func (s *MutableState) SuspendRuntime(id common.Namespace) error { - _, raw := s.Snapshot.Get(signedRuntimeKeyFmt.Encode(&id)) - if raw == nil { +// SuspendRuntime marks a runtime as suspended. +func (s *MutableState) SuspendRuntime(ctx context.Context, id common.Namespace) error { + data, err := s.Tree.RemoveExisting(ctx, signedRuntimeKeyFmt.Encode(&id)) + if err != nil { + return abci.UnavailableStateError(err) + } + if data == nil { return registry.ErrNoSuchRuntime } - - s.tree.Remove(signedRuntimeKeyFmt.Encode(&id)) - s.tree.Set(suspendedRuntimeKeyFmt.Encode(&id), raw) - return nil + return s.Tree.Insert(ctx, suspendedRuntimeKeyFmt.Encode(&id), data) } -func (s *MutableState) ResumeRuntime(id common.Namespace) error { - _, raw := s.Snapshot.Get(suspendedRuntimeKeyFmt.Encode(&id)) - if raw == nil { +// ResumeRuntime resumes a previously suspended runtime. +func (s *MutableState) ResumeRuntime(ctx context.Context, id common.Namespace) error { + data, err := s.Tree.RemoveExisting(ctx, suspendedRuntimeKeyFmt.Encode(&id)) + if err != nil { + return abci.UnavailableStateError(err) + } + if data == nil { return registry.ErrNoSuchRuntime } - - s.tree.Remove(suspendedRuntimeKeyFmt.Encode(&id)) - s.tree.Set(signedRuntimeKeyFmt.Encode(&id), raw) - return nil + return s.Tree.Insert(ctx, signedRuntimeKeyFmt.Encode(&id), data) } -func (s *MutableState) SetNodeStatus(id signature.PublicKey, status *registry.NodeStatus) error { - s.tree.Set(nodeStatusKeyFmt.Encode(&id), cbor.Marshal(status)) - return nil +// SetNodeStatus sets a status for a registered node. +func (s *MutableState) SetNodeStatus(ctx context.Context, id signature.PublicKey, status *registry.NodeStatus) error { + err := s.Tree.Insert(ctx, nodeStatusKeyFmt.Encode(&id), cbor.Marshal(status)) + return abci.UnavailableStateError(err) } -func (s *MutableState) SetConsensusParameters(params *registry.ConsensusParameters) { - s.tree.Set(parametersKeyFmt.Encode(), cbor.Marshal(params)) +// SetConsensusParameters sets registry consensus parameters. +func (s *MutableState) SetConsensusParameters(ctx context.Context, params *registry.ConsensusParameters) error { + err := s.Tree.Insert(ctx, parametersKeyFmt.Encode(), cbor.Marshal(params)) + return abci.UnavailableStateError(err) } // NewMutableState creates a new mutable registry state wrapper. -func NewMutableState(tree *iavl.MutableTree) *MutableState { - inner := &abci.ImmutableState{Snapshot: tree.ImmutableTree} - +func NewMutableState(tree mkvs.KeyValueTree) *MutableState { return &MutableState{ - ImmutableState: &ImmutableState{inner}, - tree: tree, + ImmutableState: &ImmutableState{ + &abci.ImmutableState{Tree: tree}, + }, } } diff --git a/go/consensus/tendermint/apps/registry/transactions.go b/go/consensus/tendermint/apps/registry/transactions.go index c03bf813845..b43454e9304 100644 --- a/go/consensus/tendermint/apps/registry/transactions.go +++ b/go/consensus/tendermint/apps/registry/transactions.go @@ -29,7 +29,7 @@ func (app *registryApplication) registerEntity( } // Charge gas for this transaction. - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { ctx.Logger().Error("RegisterEntity: failed to fetch consensus parameters", "err", err, @@ -60,7 +60,9 @@ func (app *registryApplication) registerEntity( } } - state.SetEntity(ent, sigEnt) + if err = state.SetEntity(ctx, ent, sigEnt); err != nil { + return fmt.Errorf("failed to set entity: %w", err) + } ctx.Logger().Debug("RegisterEntity: registered", "entity", ent, @@ -77,7 +79,7 @@ func (app *registryApplication) deregisterEntity(ctx *abci.Context, state *regis } // Charge gas for this transaction. - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { ctx.Logger().Error("DeregisterEntity: failed to fetch consensus parameters", "err", err, @@ -91,7 +93,7 @@ func (app *registryApplication) deregisterEntity(ctx *abci.Context, state *regis id := ctx.TxSigner() // Prevent entity deregistration if there are any registered nodes. - hasNodes, err := state.HasEntityNodes(id) + hasNodes, err := state.HasEntityNodes(ctx, id) if err != nil { ctx.Logger().Error("DeregisterEntity: failed to check for nodes", "err", err, @@ -105,7 +107,7 @@ func (app *registryApplication) deregisterEntity(ctx *abci.Context, state *regis return registry.ErrEntityHasNodes } // Prevent entity deregistration if there are any registered runtimes. - hasRuntimes, err := state.HasEntityRuntimes(id) + hasRuntimes, err := state.HasEntityRuntimes(ctx, id) if err != nil { ctx.Logger().Error("DeregisterEntity: failed to check for runtimes", "err", err, @@ -119,9 +121,13 @@ func (app *registryApplication) deregisterEntity(ctx *abci.Context, state *regis return registry.ErrEntityHasRuntimes } - removedEntity, err := state.RemoveEntity(id) - if err != nil { + removedEntity, err := state.RemoveEntity(ctx, id) + switch err { + case nil: + case registry.ErrNoSuchEntity: return err + default: + return fmt.Errorf("DeregisterEntity: failed to remove entity: %w", err) } if !params.DebugBypassStake { @@ -160,7 +166,7 @@ func (app *registryApplication) registerNode( // nolint: gocyclo ) return err } - untrustedEntity, err := state.Entity(untrustedNode.EntityID) + untrustedEntity, err := state.Entity(ctx, untrustedNode.EntityID) if err != nil { ctx.Logger().Error("RegisterNode: failed to query owning entity", "err", err, @@ -169,7 +175,7 @@ func (app *registryApplication) registerNode( // nolint: gocyclo return err } - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { ctx.Logger().Error("RegisterNode: failed to fetch consensus parameters", "err", err, @@ -177,7 +183,7 @@ func (app *registryApplication) registerNode( // nolint: gocyclo return err } - epoch, err := app.state.GetEpoch(ctx.Ctx(), ctx.BlockHeight()+1) + epoch, err := app.state.GetEpoch(ctx, ctx.BlockHeight()+1) if err != nil { ctx.Logger().Error("RegisterNode: failed to get epoch", "err", err, @@ -186,6 +192,7 @@ func (app *registryApplication) registerNode( // nolint: gocyclo } newNode, paidRuntimes, err := registry.VerifyRegisterNodeArgs( + ctx, params, ctx.Logger(), sigNode, @@ -254,7 +261,7 @@ func (app *registryApplication) registerNode( // nolint: gocyclo } // Check if node exists. - existingNode, err := state.Node(newNode.ID) + existingNode, err := state.Node(ctx, newNode.ID) isNewNode := err == registry.ErrNoSuchNode isExpiredNode := err == nil && existingNode.IsExpired(uint64(epoch)) if !isNewNode && err != nil { @@ -288,12 +295,13 @@ func (app *registryApplication) registerNode( // nolint: gocyclo // Create a new state checkpoint and rollback in case we fail. var ok bool - sc := ctx.NewStateCheckpoint() + sc := ctx.StartCheckpoint() defer func() { - if !ok { - sc.Rollback() + if ok { + sc.Commit() + } else { + sc.Close() } - sc.Close() }() // Check that the entity has enough stake for this node registration. @@ -314,24 +322,26 @@ func (app *registryApplication) registerNode( // nolint: gocyclo ) return err } - stakeAcc.Commit() + if err = stakeAcc.Commit(); err != nil { + return fmt.Errorf("failed to commit stake accumulator updates: %w", err) + } } if isNewNode || isExpiredNode { // Node doesn't exist (or is expired). Create node. - if err = state.SetNode(newNode, sigNode); err != nil { + if err = state.SetNode(ctx, newNode, sigNode); err != nil { ctx.Logger().Error("RegisterNode: failed to create node", "err", err, "node", newNode, "entity", newNode.EntityID, ) - return registry.ErrBadEntityForNode + return fmt.Errorf("failed to set node: %w", err) } var status *registry.NodeStatus if existingNode != nil { // Node exists but is expired, fetch existing status. - if status, err = state.NodeStatus(newNode.ID); err != nil { + if status, err = state.NodeStatus(ctx, newNode.ID); err != nil { ctx.Logger().Error("RegisterNode: failed to get node status", "err", err, ) @@ -345,11 +355,11 @@ func (app *registryApplication) registerNode( // nolint: gocyclo status = ®istry.NodeStatus{} } - if err = state.SetNodeStatus(newNode.ID, status); err != nil { + if err = state.SetNodeStatus(ctx, newNode.ID, status); err != nil { ctx.Logger().Error("RegisterNode: failed to set node status", "err", err, ) - return registry.ErrInvalidArgument + return fmt.Errorf("failed to set node status: %w", err) } } else { // The node already exists, validate and update the node's entry. @@ -362,13 +372,13 @@ func (app *registryApplication) registerNode( // nolint: gocyclo ) return err } - if err = state.SetNode(newNode, sigNode); err != nil { + if err = state.SetNode(ctx, newNode, sigNode); err != nil { ctx.Logger().Error("RegisterNode: failed to update node", "err", err, "node", newNode, "entity", newNode.EntityID, ) - return registry.ErrBadEntityForNode + return fmt.Errorf("failed to set node: %w", err) } } @@ -383,7 +393,7 @@ func (app *registryApplication) registerNode( // nolint: gocyclo } } - err := state.ResumeRuntime(rt.ID) + err := state.ResumeRuntime(ctx, rt.ID) switch err { case nil: ctx.Logger().Debug("RegisterNode: resumed runtime", @@ -424,7 +434,7 @@ func (app *registryApplication) unfreezeNode( } // Charge gas for this transaction. - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { ctx.Logger().Error("UnfreezeNode: failed to fetch consensus parameters", "err", err, @@ -436,7 +446,7 @@ func (app *registryApplication) unfreezeNode( } // Fetch node descriptor. - node, err := state.Node(unfreeze.NodeID) + node, err := state.Node(ctx, unfreeze.NodeID) if err != nil { ctx.Logger().Error("UnfreezeNode: failed to fetch node", "err", err, @@ -450,7 +460,7 @@ func (app *registryApplication) unfreezeNode( } // Fetch node status. - status, err := state.NodeStatus(unfreeze.NodeID) + status, err := state.NodeStatus(ctx, unfreeze.NodeID) if err != nil { ctx.Logger().Error("UnfreezeNode: failed to fetch node status", "err", err, @@ -461,7 +471,7 @@ func (app *registryApplication) unfreezeNode( } // Ensure if we can actually unfreeze. - epoch, err := app.state.GetEpoch(ctx.Ctx(), ctx.BlockHeight()+1) + epoch, err := app.state.GetEpoch(ctx, ctx.BlockHeight()+1) if err != nil { return err } @@ -471,8 +481,8 @@ func (app *registryApplication) unfreezeNode( // Reset frozen status. status.Unfreeze() - if err = state.SetNodeStatus(node.ID, status); err != nil { - return err + if err = state.SetNodeStatus(ctx, node.ID, status); err != nil { + return fmt.Errorf("failed to set node status: %w", err) } ctx.Logger().Debug("UnfreezeNode: unfrozen", @@ -489,7 +499,7 @@ func (app *registryApplication) registerRuntime( // nolint: gocyclo state *registryState.MutableState, sigRt *registry.SignedRuntime, ) error { - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { ctx.Logger().Error("RegisterRuntime: failed to fetch consensus parameters", "err", err, @@ -518,7 +528,7 @@ func (app *registryApplication) registerRuntime( // nolint: gocyclo } if rt.Kind == registry.KindCompute { - if err = registry.VerifyRegisterComputeRuntimeArgs(ctx.Logger(), rt, state); err != nil { + if err = registry.VerifyRegisterComputeRuntimeArgs(ctx, ctx.Logger(), rt, state); err != nil { return err } } @@ -555,12 +565,12 @@ func (app *registryApplication) registerRuntime( // nolint: gocyclo // Make sure the runtime doesn't exist yet. var suspended bool - existingRt, err := state.Runtime(rt.ID) + existingRt, err := state.Runtime(ctx, rt.ID) switch err { case nil: case registry.ErrNoSuchRuntime: // Make sure the runtime isn't suspended. - existingRt, err = state.SuspendedRuntime(rt.ID) + existingRt, err = state.SuspendedRuntime(ctx, rt.ID) switch err { case nil: suspended = true @@ -593,13 +603,13 @@ func (app *registryApplication) registerRuntime( // nolint: gocyclo } } - if err = state.SetRuntime(rt, sigRt, suspended); err != nil { + if err = state.SetRuntime(ctx, rt, sigRt, suspended); err != nil { ctx.Logger().Error("RegisterRuntime: failed to create runtime", "err", err, "runtime", rt, "entity", rt.EntityID, ) - return registry.ErrBadEntityForRuntime + return fmt.Errorf("failed to set runtime: %w", err) } if !suspended { diff --git a/go/consensus/tendermint/apps/roothash/genesis.go b/go/consensus/tendermint/apps/roothash/genesis.go index 41bd1fe5106..e125920b1b1 100644 --- a/go/consensus/tendermint/apps/roothash/genesis.go +++ b/go/consensus/tendermint/apps/roothash/genesis.go @@ -2,6 +2,7 @@ package roothash import ( "context" + "fmt" "github.com/tendermint/tendermint/abci/types" @@ -20,7 +21,9 @@ func (app *rootHashApplication) InitChain(ctx *abci.Context, request types.Reque st := doc.RootHash state := roothashState.NewMutableState(ctx.State()) - state.SetConsensusParameters(&st.Parameters) + if err := state.SetConsensusParameters(ctx, &st.Parameters); err != nil { + return fmt.Errorf("failed to set consensus parameters: %w", err) + } // The per-runtime roothash state is done primarily via DeliverTx, but // also needs to be done here since the genesis state can have runtime @@ -30,19 +33,24 @@ func (app *rootHashApplication) InitChain(ctx *abci.Context, request types.Reque // carved out it's entries by this point. regState := registryState.NewMutableState(ctx.State()) - runtimes, _ := regState.Runtimes() + runtimes, _ := regState.Runtimes(ctx) for _, v := range runtimes { ctx.Logger().Info("InitChain: allocating per-runtime state", "runtime", v.ID, ) - app.onNewRuntime(ctx, v, &st) + if err := app.onNewRuntime(ctx, v, &st); err != nil { + return fmt.Errorf("failed to initialize runtime state: %w", err) + } } return nil } func (rq *rootHashQuerier) Genesis(ctx context.Context) (*roothashAPI.Genesis, error) { - runtimes := rq.state.Runtimes() + runtimes, err := rq.state.Runtimes(ctx) + if err != nil { + return nil, err + } // Get per-runtime states. rtStates := make(map[common.Namespace]*api.RuntimeGenesis) @@ -58,7 +66,7 @@ func (rq *rootHashQuerier) Genesis(ctx context.Context) (*roothashAPI.Genesis, e rtStates[rt.Runtime.ID] = &rtState } - params, err := rq.state.ConsensusParameters() + params, err := rq.state.ConsensusParameters(ctx) if err != nil { return nil, err } diff --git a/go/consensus/tendermint/apps/roothash/query.go b/go/consensus/tendermint/apps/roothash/query.go index bedd55faa9a..4f313b5fb6d 100644 --- a/go/consensus/tendermint/apps/roothash/query.go +++ b/go/consensus/tendermint/apps/roothash/query.go @@ -4,7 +4,6 @@ import ( "context" "github.com/oasislabs/oasis-core/go/common" - "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" roothashState "github.com/oasislabs/oasis-core/go/consensus/tendermint/apps/roothash/state" roothash "github.com/oasislabs/oasis-core/go/roothash/api" "github.com/oasislabs/oasis-core/go/roothash/api/block" @@ -24,27 +23,10 @@ type QueryFactory struct { // QueryAt returns the roothash query interface for a specific height. func (sf *QueryFactory) QueryAt(ctx context.Context, height int64) (Query, error) { - var state *roothashState.ImmutableState - var err error - abciCtx := abci.FromCtx(ctx) - - // If this request was made from InitChain, no blocks and states have been - // submitted yet, so we use the existing state instead. - if abciCtx != nil && abciCtx.IsInitChain() { - state = roothashState.NewMutableState(abciCtx.State()).ImmutableState - } else { - state, err = roothashState.NewImmutableState(sf.app.state, height) - if err != nil { - return nil, err - } - } - - // If this request was made from an ABCI app, make sure to use the associated - // context for querying state instead of the default one. - if abciCtx != nil && height == abciCtx.BlockHeight()+1 { - state.Snapshot = abciCtx.State().ImmutableTree + state, err := roothashState.NewImmutableState(ctx, sf.app.state, height) + if err != nil { + return nil, err } - return &rootHashQuerier{state}, nil } @@ -53,7 +35,7 @@ type rootHashQuerier struct { } func (rq *rootHashQuerier) LatestBlock(ctx context.Context, id common.Namespace) (*block.Block, error) { - runtime, err := rq.state.RuntimeState(id) + runtime, err := rq.state.RuntimeState(ctx, id) if err != nil { return nil, err } @@ -61,7 +43,7 @@ func (rq *rootHashQuerier) LatestBlock(ctx context.Context, id common.Namespace) } func (rq *rootHashQuerier) GenesisBlock(ctx context.Context, id common.Namespace) (*block.Block, error) { - runtime, err := rq.state.RuntimeState(id) + runtime, err := rq.state.RuntimeState(ctx, id) if err != nil { return nil, err } diff --git a/go/consensus/tendermint/apps/roothash/roothash.go b/go/consensus/tendermint/apps/roothash/roothash.go index 1f30baa6a0c..73688b4f84a 100644 --- a/go/consensus/tendermint/apps/roothash/roothash.go +++ b/go/consensus/tendermint/apps/roothash/roothash.go @@ -4,9 +4,9 @@ package roothash import ( "bytes" "encoding/binary" + "errors" "fmt" - "github.com/pkg/errors" "github.com/tendermint/tendermint/abci/types" beacon "github.com/oasislabs/oasis-core/go/beacon/api" @@ -91,9 +91,9 @@ func (app *rootHashApplication) onCommitteeChanged(ctx *abci.Context, epoch epoc state := roothashState.NewMutableState(ctx.State()) schedState := schedulerState.NewMutableState(ctx.State()) regState := registryState.NewMutableState(ctx.State()) - runtimes, _ := regState.Runtimes() + runtimes, _ := regState.Runtimes(ctx) - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { return fmt.Errorf("failed to get consensus parameters: %w", err) } @@ -115,7 +115,7 @@ func (app *rootHashApplication) onCommitteeChanged(ctx *abci.Context, epoch epoc continue } - rtState, err := state.RuntimeState(rt.ID) + rtState, err := state.RuntimeState(ctx, rt.ID) if err != nil { return fmt.Errorf("failed to fetch runtime state: %w", err) } @@ -147,7 +147,7 @@ func (app *rootHashApplication) onCommitteeChanged(ctx *abci.Context, epoch epoc } } if (empty || !sufficientStake) && !params.DebugDoNotSuspendRuntimes { - if err := app.suspendUnpaidRuntime(ctx, rtState, regState); err != nil { + if err = app.suspendUnpaidRuntime(ctx, rtState, regState); err != nil { return err } } @@ -179,7 +179,9 @@ func (app *rootHashApplication) onCommitteeChanged(ctx *abci.Context, epoch epoc // Update the runtime descriptor to the latest per-epoch value. rtState.Runtime = rt - state.SetRuntimeState(rtState) + if err = state.SetRuntimeState(ctx, rtState); err != nil { + return fmt.Errorf("failed to set runtime state: %w", err) + } } return nil @@ -194,7 +196,7 @@ func (app *rootHashApplication) suspendUnpaidRuntime( "runtime_id", rtState.Runtime.ID, ) - if err := regState.SuspendRuntime(rtState.Runtime.ID); err != nil { + if err := regState.SuspendRuntime(ctx, rtState.Runtime.ID); err != nil { return err } @@ -246,7 +248,7 @@ func (app *rootHashApplication) prepareNewCommittees( // NOTE: There will later be multiple executor committees. var executorCommittees []*scheduler.Committee - xc1, err := schedState.Committee(scheduler.KindComputeExecutor, rtID) + xc1, err := schedState.Committee(ctx, scheduler.KindComputeExecutor, rtID) if err != nil { ctx.Logger().Error("checkCommittees: failed to get executor committee from scheduler", "err", err, @@ -279,7 +281,7 @@ func (app *rootHashApplication) prepareNewCommittees( mergePool = new(commitment.Pool) committeeIDParts = append(committeeIDParts, []byte("merge committee follows")) - mergeCommittee, err := schedState.Committee(scheduler.KindComputeMerge, rtID) + mergeCommittee, err := schedState.Committee(ctx, scheduler.KindComputeMerge, rtID) if err != nil { ctx.Logger().Error("checkCommittees: failed to get merge committee from scheduler", "err", err, @@ -342,11 +344,6 @@ func (app *rootHashApplication) ExecuteTx(ctx *abci.Context, tx *transaction.Tra } func (app *rootHashApplication) ForeignExecuteTx(ctx *abci.Context, other abci.Application, tx *transaction.Transaction) error { - var st *roothash.Genesis - ensureGenesis := func() { - st = &app.state.Genesis().RootHash - } - switch other.Name() { case registryapp.AppName: for _, ev := range ctx.GetEvents() { @@ -358,15 +355,16 @@ func (app *rootHashApplication) ForeignExecuteTx(ctx *abci.Context, other abci.A if bytes.Equal(pair.GetKey(), registryapp.KeyRuntimeRegistered) { var rt registry.Runtime if err := cbor.Unmarshal(pair.GetValue(), &rt); err != nil { - return errors.Wrap(err, "roothash: failed to deserialize new runtime") + return fmt.Errorf("roothash: failed to deserialize new runtime: %w", err) } ctx.Logger().Debug("ForeignDeliverTx: new runtime", "runtime", rt.ID, ) - ensureGenesis() - app.onNewRuntime(ctx, &rt, st) + if err := app.onNewRuntime(ctx, &rt, nil); err != nil { + return err + } } } } @@ -375,22 +373,27 @@ func (app *rootHashApplication) ForeignExecuteTx(ctx *abci.Context, other abci.A return nil } -func (app *rootHashApplication) onNewRuntime(ctx *abci.Context, runtime *registry.Runtime, genesis *roothash.Genesis) { - state := roothashState.NewMutableState(ctx.State()) - +func (app *rootHashApplication) onNewRuntime(ctx *abci.Context, runtime *registry.Runtime, genesis *roothash.Genesis) error { if !runtime.IsCompute() { ctx.Logger().Warn("onNewRuntime: ignoring non-compute runtime", "runtime", runtime, ) - return + return nil } // Check if state already exists for the given runtime. - if _, err := state.RuntimeState(runtime.ID); err != roothash.ErrInvalidRuntime { + state := roothashState.NewMutableState(ctx.State()) + _, err := state.RuntimeState(ctx, runtime.ID) + switch err { + case nil: ctx.Logger().Warn("onNewRuntime: state for runtime already exists", "runtime", runtime, ) - return + return nil + case roothash.ErrInvalidRuntime: + // Runtime does not yet exist. + default: + return fmt.Errorf("failed to fetch runtime state: %w", err) } // Create genesis block. @@ -401,6 +404,7 @@ func (app *rootHashApplication) onNewRuntime(ctx *abci.Context, runtime *registr genesisBlock.Header.StateRoot = runtime.Genesis.StateRoot genesisBlock.Header.StorageSignatures = runtime.Genesis.StorageReceipts if ctx.IsInitChain() { + // NOTE: Outside InitChain the genesis argument will be nil. genesisRts := genesis.RuntimeStates[runtime.ID] if genesisRts != nil { genesisBlock.Header.Round = genesisRts.Round @@ -411,12 +415,15 @@ func (app *rootHashApplication) onNewRuntime(ctx *abci.Context, runtime *registr // Create new state containing the genesis block. timerCtx := &timerContext{ID: runtime.ID} - state.SetRuntimeState(&roothashState.RuntimeState{ + err = state.SetRuntimeState(ctx, &roothashState.RuntimeState{ Runtime: runtime, CurrentBlock: genesisBlock, GenesisBlock: genesisBlock, Timer: *abci.NewTimer(ctx, app, timerKindRound, runtime.ID[:], cbor.Marshal(timerCtx)), }) + if err != nil { + return fmt.Errorf("failed to set runtime state: %w", err) + } ctx.Logger().Debug("onNewRuntime: created genesis state for runtime", "runtime", runtime, @@ -428,20 +435,21 @@ func (app *rootHashApplication) onNewRuntime(ctx *abci.Context, runtime *registr Round: genesisBlock.Header.Round, } ctx.EmitEvent(tmapi.NewEventBuilder(app.Name()).Attribute(KeyFinalized, cbor.Marshal(tagV))) + return nil } func (app *rootHashApplication) EndBlock(ctx *abci.Context, request types.RequestEndBlock) (types.ResponseEndBlock, error) { return types.ResponseEndBlock{}, nil } -func (app *rootHashApplication) FireTimer(ctx *abci.Context, timer *abci.Timer) error { +func (app *rootHashApplication) FireTimer(ctx *abci.Context, timer *abci.Timer) (err error) { if timer.Kind() != timerKindRound { return errors.New("tendermint/roothash: unexpected timer") } var tCtx timerContext - if err := cbor.Unmarshal(timer.Data(ctx), &tCtx); err != nil { - return err + if err = cbor.Unmarshal(timer.Data(ctx), &tCtx); err != nil { + return fmt.Errorf("failed to unmarshal timer data: %w", err) } ctx.Logger().Warn("FireTimer: timer fired", @@ -449,12 +457,12 @@ func (app *rootHashApplication) FireTimer(ctx *abci.Context, timer *abci.Timer) ) state := roothashState.NewMutableState(ctx.State()) - rtState, err := state.RuntimeState(tCtx.ID) + rtState, err := state.RuntimeState(ctx, tCtx.ID) if err != nil { ctx.Logger().Error("FireTimer: failed to get state associated with timer", "err", err, ) - return err + return fmt.Errorf("failed to get runtime state: %w", err) } latestBlock := rtState.CurrentBlock @@ -476,14 +484,18 @@ func (app *rootHashApplication) FireTimer(ctx *abci.Context, timer *abci.Timer) "timer_round", tCtx.Round, ) - defer state.SetRuntimeState(rtState) + defer func() { + if err2 := state.SetRuntimeState(ctx, rtState); err2 != nil { + err = fmt.Errorf("failed to set runtime state: %w", err2) + } + }() if rtState.Round.MergePool.IsTimeout(ctx.Now()) { - if err := app.tryFinalizeBlock(ctx, rtState, true); err != nil { + if err = app.tryFinalizeBlock(ctx, rtState, true); err != nil { ctx.Logger().Error("failed to finalize block", "err", err, ) - panic(err) + return fmt.Errorf("failed to finalize block: %w", err) } } for _, pool := range rtState.Round.ExecutorPool.GetTimeoutCommittees(ctx.Now()) { @@ -666,7 +678,7 @@ func (app *rootHashApplication) tryFinalizeMerge( } func (app *rootHashApplication) postProcessFinalizedBlock(ctx *abci.Context, rtState *roothashState.RuntimeState, blk *block.Block) error { - checkpoint := ctx.NewStateCheckpoint() + checkpoint := ctx.StartCheckpoint() defer checkpoint.Close() for _, message := range blk.Header.Messages { @@ -682,9 +694,6 @@ func (app *rootHashApplication) postProcessFinalizedBlock(ctx *abci.Context, rtS logging.LogEvent, roothash.LogEventMessageUnsat, ) - // Roll back changes from message handling. - checkpoint.Rollback() - // Substitute empty block. app.emitEmptyBlock(ctx, rtState, block.RoundFailed) @@ -692,6 +701,8 @@ func (app *rootHashApplication) postProcessFinalizedBlock(ctx *abci.Context, rtS } } + checkpoint.Commit() + // All good. Hook up the new block. rtState.Timer.Stop(ctx) rtState.CurrentBlock = blk diff --git a/go/consensus/tendermint/apps/roothash/state/state.go b/go/consensus/tendermint/apps/roothash/state/state.go index 15797aa85a2..38847cfd6f8 100644 --- a/go/consensus/tendermint/apps/roothash/state/state.go +++ b/go/consensus/tendermint/apps/roothash/state/state.go @@ -1,10 +1,9 @@ package state import ( + "context" "fmt" - "github.com/tendermint/iavl" - "github.com/oasislabs/oasis-core/go/common" "github.com/oasislabs/oasis-core/go/common/cbor" "github.com/oasislabs/oasis-core/go/common/keyformat" @@ -12,6 +11,7 @@ import ( registry "github.com/oasislabs/oasis-core/go/registry/api" roothash "github.com/oasislabs/oasis-core/go/roothash/api" "github.com/oasislabs/oasis-core/go/roothash/api/block" + mkvs "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel" ) var ( @@ -41,8 +41,8 @@ type ImmutableState struct { *abci.ImmutableState } -func NewImmutableState(state abci.ApplicationState, version int64) (*ImmutableState, error) { - inner, err := abci.NewImmutableState(state, version) +func NewImmutableState(ctx context.Context, state abci.ApplicationState, version int64) (*ImmutableState, error) { + inner, err := abci.NewImmutableState(ctx, state, version) if err != nil { return nil, err } @@ -50,69 +50,84 @@ func NewImmutableState(state abci.ApplicationState, version int64) (*ImmutableSt return &ImmutableState{inner}, nil } -func (s *ImmutableState) RuntimeState(id common.Namespace) (*RuntimeState, error) { - _, raw := s.Snapshot.Get(runtimeKeyFmt.Encode(&id)) +// RuntimeState returns the roothash runtime state for a specific runtime. +func (s *ImmutableState) RuntimeState(ctx context.Context, id common.Namespace) (*RuntimeState, error) { + raw, err := s.Tree.Get(ctx, runtimeKeyFmt.Encode(&id)) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if raw == nil { return nil, roothash.ErrInvalidRuntime } var state RuntimeState - err := cbor.Unmarshal(raw, &state) + if err = cbor.Unmarshal(raw, &state); err != nil { + return nil, abci.UnavailableStateError(err) + } return &state, err } -func (s *ImmutableState) Runtimes() []*RuntimeState { +// Runtimes returns the list of all roothash runtime states. +func (s *ImmutableState) Runtimes(ctx context.Context) ([]*RuntimeState, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() + var runtimes []*RuntimeState - s.Snapshot.IterateRange( - runtimeKeyFmt.Encode(), - nil, - true, - func(key, value []byte) bool { - if !runtimeKeyFmt.Decode(key) { - return true - } - - var state RuntimeState - cbor.MustUnmarshal(value, &state) - - runtimes = append(runtimes, &state) - return false - }, - ) + for it.Seek(runtimeKeyFmt.Encode()); it.Valid(); it.Next() { + if !runtimeKeyFmt.Decode(it.Key()) { + break + } - return runtimes + var state RuntimeState + if err := cbor.Unmarshal(it.Value(), &state); err != nil { + return nil, abci.UnavailableStateError(err) + } + + runtimes = append(runtimes, &state) + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } + return runtimes, nil } -func (s *ImmutableState) ConsensusParameters() (*roothash.ConsensusParameters, error) { - _, raw := s.Snapshot.Get(parametersKeyFmt.Encode()) +// ConsensusParameters returns the roothash consensus parameters. +func (s *ImmutableState) ConsensusParameters(ctx context.Context) (*roothash.ConsensusParameters, error) { + raw, err := s.Tree.Get(ctx, parametersKeyFmt.Encode()) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if raw == nil { return nil, fmt.Errorf("tendermint/roothash: expected consensus parameters to be present in app state") } var params roothash.ConsensusParameters - err := cbor.Unmarshal(raw, ¶ms) - return ¶ms, err + if err = cbor.Unmarshal(raw, ¶ms); err != nil { + return nil, abci.UnavailableStateError(err) + } + return ¶ms, nil } type MutableState struct { *ImmutableState - - tree *iavl.MutableTree } -func NewMutableState(tree *iavl.MutableTree) *MutableState { - inner := &abci.ImmutableState{Snapshot: tree.ImmutableTree} - +func NewMutableState(tree mkvs.KeyValueTree) *MutableState { return &MutableState{ - ImmutableState: &ImmutableState{inner}, - tree: tree, + ImmutableState: &ImmutableState{ + &abci.ImmutableState{Tree: tree}, + }, } } -func (s *MutableState) SetRuntimeState(state *RuntimeState) { - s.tree.Set(runtimeKeyFmt.Encode(&state.Runtime.ID), cbor.Marshal(state)) +// SetRuntimeState sets a runtime's roothash state. +func (s *MutableState) SetRuntimeState(ctx context.Context, state *RuntimeState) error { + err := s.Tree.Insert(ctx, runtimeKeyFmt.Encode(&state.Runtime.ID), cbor.Marshal(state)) + return abci.UnavailableStateError(err) } -func (s *MutableState) SetConsensusParameters(params *roothash.ConsensusParameters) { - s.tree.Set(parametersKeyFmt.Encode(), cbor.Marshal(params)) +// SetConsensusParameters sets roothash consensus parameters. +func (s *MutableState) SetConsensusParameters(ctx context.Context, params *roothash.ConsensusParameters) error { + err := s.Tree.Insert(ctx, parametersKeyFmt.Encode(), cbor.Marshal(params)) + return abci.UnavailableStateError(err) } diff --git a/go/consensus/tendermint/apps/roothash/transactions.go b/go/consensus/tendermint/apps/roothash/transactions.go index 655a4eadf7b..5dc939a69e1 100644 --- a/go/consensus/tendermint/apps/roothash/transactions.go +++ b/go/consensus/tendermint/apps/roothash/transactions.go @@ -19,6 +19,7 @@ import ( var _ commitment.SignatureVerifier = (*roothashSignatureVerifier)(nil) type roothashSignatureVerifier struct { + ctx *abci.Context runtimeID common.Namespace scheduler *schedulerState.MutableState } @@ -32,7 +33,7 @@ func (sv *roothashSignatureVerifier) VerifyCommitteeSignatures(kind scheduler.Co return nil } - committee, err := sv.scheduler.Committee(kind, sv.runtimeID) + committee, err := sv.scheduler.Committee(sv.ctx, kind, sv.runtimeID) if err != nil { return err } @@ -62,7 +63,7 @@ func (app *rootHashApplication) getRuntimeState( id common.Namespace, ) (*roothashState.RuntimeState, commitment.SignatureVerifier, commitment.NodeLookup, error) { // Fetch current runtime state. - rtState, err := state.RuntimeState(id) + rtState, err := state.RuntimeState(ctx, id) if err != nil { return nil, nil, nil, fmt.Errorf("roothash: failed to fetch runtime state: %w", err) } @@ -75,6 +76,7 @@ func (app *rootHashApplication) getRuntimeState( // Create signature verifier. sv := &roothashSignatureVerifier{ + ctx: ctx, runtimeID: id, scheduler: schedulerState.NewMutableState(ctx.State()), } @@ -98,13 +100,13 @@ func (app *rootHashApplication) executorCommit( ctx *abci.Context, state *roothashState.MutableState, cc *roothash.ExecutorCommit, -) error { +) (err error) { if ctx.IsCheckOnly() { return nil } // Charge gas for this transaction. - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { ctx.Logger().Error("ComputeCommit: failed to fetch consensus parameters", "err", err, @@ -119,7 +121,11 @@ func (app *rootHashApplication) executorCommit( if err != nil { return err } - defer state.SetRuntimeState(rtState) + defer func() { + if err2 := state.SetRuntimeState(ctx, rtState); err2 != nil { + err = fmt.Errorf("failed to set runtime state: %w", err2) + } + }() pools := make(map[*commitment.Pool]bool) for _, commit := range cc.Commits { @@ -147,13 +153,13 @@ func (app *rootHashApplication) mergeCommit( ctx *abci.Context, state *roothashState.MutableState, mc *roothash.MergeCommit, -) error { +) (err error) { if ctx.IsCheckOnly() { return nil } // Charge gas for this transaction. - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { ctx.Logger().Error("MergeCommit: failed to fetch consensus parameters", "err", err, @@ -168,7 +174,11 @@ func (app *rootHashApplication) mergeCommit( if err != nil { return err } - defer state.SetRuntimeState(rtState) + defer func() { + if err2 := state.SetRuntimeState(ctx, rtState); err2 != nil { + err = fmt.Errorf("failed to set runtime state: %w", err2) + } + }() // Add commitments. for _, commit := range mc.Commits { diff --git a/go/consensus/tendermint/apps/scheduler/genesis.go b/go/consensus/tendermint/apps/scheduler/genesis.go index 13a32e48b4c..7747e0cf3f1 100644 --- a/go/consensus/tendermint/apps/scheduler/genesis.go +++ b/go/consensus/tendermint/apps/scheduler/genesis.go @@ -25,7 +25,9 @@ func (app *schedulerApplication) InitChain(ctx *abci.Context, req types.RequestI app.baseEpoch = baseEpoch state := schedulerState.NewMutableState(ctx.State()) - state.SetConsensusParameters(&doc.Scheduler.Parameters) + if err = state.SetConsensusParameters(ctx, &doc.Scheduler.Parameters); err != nil { + return fmt.Errorf("failed to set consensus parameters: %w", err) + } if doc.Scheduler.Parameters.DebugStaticValidators { ctx.Logger().Warn("static validators are configured") @@ -55,7 +57,9 @@ func (app *schedulerApplication) InitChain(ctx *abci.Context, req types.RequestI } // Add the current validator set to ABCI, so that we can query it later. - state.PutCurrentValidators(staticValidators) + if err = state.PutCurrentValidators(ctx, staticValidators); err != nil { + return fmt.Errorf("failed to set validator set: %w", err) + } return nil } @@ -77,7 +81,7 @@ func (app *schedulerApplication) InitChain(ctx *abci.Context, req types.RequestI } regState := registryState.NewMutableState(ctx.State()) - nodes, err := regState.Nodes() + nodes, err := regState.Nodes(ctx) if err != nil { return fmt.Errorf("tendermint/scheduler: couldn't get nodes: %w", err) } @@ -139,13 +143,15 @@ func (app *schedulerApplication) InitChain(ctx *abci.Context, req types.RequestI // // Sort of stupid it needs to be done this way, but tendermint doesn't // appear to pass ABCI the validator set anywhere other than InitChain. - state.PutCurrentValidators(currentValidators) + if err := state.PutCurrentValidators(ctx, currentValidators); err != nil { + return fmt.Errorf("failed to set validator set: %w", err) + } return nil } func (sq *schedulerQuerier) Genesis(ctx context.Context) (*scheduler.Genesis, error) { - params, err := sq.state.ConsensusParameters() + params, err := sq.state.ConsensusParameters(ctx) if err != nil { return nil, err } diff --git a/go/consensus/tendermint/apps/scheduler/query.go b/go/consensus/tendermint/apps/scheduler/query.go index e59bb195571..c7e4ddb9c4e 100644 --- a/go/consensus/tendermint/apps/scheduler/query.go +++ b/go/consensus/tendermint/apps/scheduler/query.go @@ -4,7 +4,6 @@ import ( "context" consensus "github.com/oasislabs/oasis-core/go/consensus/api" - "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" schedulerState "github.com/oasislabs/oasis-core/go/consensus/tendermint/apps/scheduler/state" scheduler "github.com/oasislabs/oasis-core/go/scheduler/api" ) @@ -24,27 +23,10 @@ type QueryFactory struct { // QueryAt returns the scheduler query interface for a specific height. func (sf *QueryFactory) QueryAt(ctx context.Context, height int64) (Query, error) { - var state *schedulerState.ImmutableState - var err error - abciCtx := abci.FromCtx(ctx) - - // If this request was made from InitChain, no blocks and states have been - // submitted yet, so we use the existing state instead. - if abciCtx != nil && abciCtx.IsInitChain() { - state = schedulerState.NewMutableState(abciCtx.State()).ImmutableState - } else { - state, err = schedulerState.NewImmutableState(sf.app.state, height) - if err != nil { - return nil, err - } - } - - // If this request was made from an ABCI app, make sure to use the associated - // context for querying state instead of the default one. - if abciCtx != nil && height == abciCtx.BlockHeight()+1 { - state.Snapshot = abciCtx.State().ImmutableTree + state, err := schedulerState.NewImmutableState(ctx, sf.app.state, height) + if err != nil { + return nil, err } - return &schedulerQuerier{state}, nil } @@ -53,7 +35,7 @@ type schedulerQuerier struct { } func (sq *schedulerQuerier) Validators(ctx context.Context) ([]*scheduler.Validator, error) { - valPks, err := sq.state.CurrentValidators() + valPks, err := sq.state.CurrentValidators(ctx) if err != nil { return nil, err } @@ -73,11 +55,11 @@ func (sq *schedulerQuerier) Validators(ctx context.Context) ([]*scheduler.Valida } func (sq *schedulerQuerier) AllCommittees(ctx context.Context) ([]*scheduler.Committee, error) { - return sq.state.AllCommittees() + return sq.state.AllCommittees(ctx) } func (sq *schedulerQuerier) KindsCommittees(ctx context.Context, kinds []scheduler.CommitteeKind) ([]*scheduler.Committee, error) { - return sq.state.KindsCommittees(kinds) + return sq.state.KindsCommittees(ctx, kinds) } func (app *schedulerApplication) QueryFactory() interface{} { diff --git a/go/consensus/tendermint/apps/scheduler/scheduler.go b/go/consensus/tendermint/apps/scheduler/scheduler.go index 87da4bfc197..f680015b592 100644 --- a/go/consensus/tendermint/apps/scheduler/scheduler.go +++ b/go/consensus/tendermint/apps/scheduler/scheduler.go @@ -97,17 +97,17 @@ func (app *schedulerApplication) BeginBlock(ctx *abci.Context, request types.Req } beacState := beaconState.NewMutableState(ctx.State()) - beacon, err := beacState.Beacon() + beacon, err := beacState.Beacon(ctx) if err != nil { return errors.Wrap(err, "tendermint/scheduler: couldn't get beacon") } regState := registryState.NewMutableState(ctx.State()) - runtimes, err := regState.Runtimes() + runtimes, err := regState.Runtimes(ctx) if err != nil { return errors.Wrap(err, "tendermint/scheduler: couldn't get runtimes") } - allNodes, err := regState.Nodes() + allNodes, err := regState.Nodes(ctx) if err != nil { return errors.Wrap(err, "tendermint/scheduler: couldn't get nodes") } @@ -116,7 +116,7 @@ func (app *schedulerApplication) BeginBlock(ctx *abci.Context, request types.Req var nodes []*node.Node for _, node := range allNodes { var status *registry.NodeStatus - status, err = regState.NodeStatus(node.ID) + status, err = regState.NodeStatus(ctx, node.ID) if err != nil { return errors.Wrap(err, "tendermint/scheduler: couldn't get node status") } @@ -134,7 +134,7 @@ func (app *schedulerApplication) BeginBlock(ctx *abci.Context, request types.Req } state := schedulerState.NewMutableState(ctx.State()) - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { ctx.Logger().Error("failed to fetch consensus parameters", "err", err, @@ -198,7 +198,7 @@ func (app *schedulerApplication) BeginBlock(ctx *abci.Context, request types.Req if entitiesEligibleForReward != nil { accounts := publicKeyMapToSortedSlice(entitiesEligibleForReward) stakingSt := stakingState.NewMutableState(ctx.State()) - if err = stakingSt.AddRewards(epoch, ¶ms.RewardFactorEpochElectionAny, accounts); err != nil { + if err = stakingSt.AddRewards(ctx, epoch, ¶ms.RewardFactorEpochElectionAny, accounts); err != nil { return errors.Wrap(err, "adding rewards") } } @@ -218,7 +218,7 @@ func (app *schedulerApplication) EndBlock(ctx *abci.Context, req types.RequestEn var resp types.ResponseEndBlock state := schedulerState.NewMutableState(ctx.State()) - pendingValidators, err := state.PendingValidators() + pendingValidators, err := state.PendingValidators(ctx) if err != nil { return resp, errors.Wrap(err, "scheduler/tendermint: failed to query pending validators") } @@ -227,13 +227,15 @@ func (app *schedulerApplication) EndBlock(ctx *abci.Context, req types.RequestEn return resp, nil } - currentValidators, err := state.CurrentValidators() + currentValidators, err := state.CurrentValidators(ctx) if err != nil { return resp, errors.Wrap(err, "scheduler/tendermint: failed to query current validators") } // Clear out the pending validator update. - state.PutPendingValidators(nil) + if err = state.PutPendingValidators(ctx, nil); err != nil { + return resp, fmt.Errorf("failed to clear validators: %w", err) + } // Tendermint expects a vector of ValidatorUpdate that expresses // the difference between the current validator set (tracked manually @@ -284,7 +286,9 @@ func (app *schedulerApplication) EndBlock(ctx *abci.Context, req types.RequestEn resp.ValidatorUpdates = updates // Stash the updated validator set. - state.PutCurrentValidators(pendingValidators) + if err = state.PutCurrentValidators(ctx, pendingValidators); err != nil { + return resp, fmt.Errorf("failed to set validators: %w", err) + } return resp, nil } @@ -456,8 +460,7 @@ func (app *schedulerApplication) electCommittee( "kind", kind, "runtime_id", rt.ID, ) - schedulerState.NewMutableState(ctx.State()).DropCommittee(kind, rt.ID) - return nil + return schedulerState.NewMutableState(ctx.State()).DropCommittee(ctx, kind, rt.ID) } nrNodes, wantedNodes := len(nodeList), workerSize+backupSize @@ -469,8 +472,7 @@ func (app *schedulerApplication) electCommittee( "backup_size", backupSize, "nr_nodes", nrNodes, ) - schedulerState.NewMutableState(ctx.State()).DropCommittee(kind, rt.ID) - return nil + return schedulerState.NewMutableState(ctx.State()).DropCommittee(ctx, kind, rt.ID) } // Do the actual election. @@ -504,17 +506,15 @@ func (app *schedulerApplication) electCommittee( "backup_size", backupSize, "available", len(members), ) - schedulerState.NewMutableState(ctx.State()).DropCommittee(kind, rt.ID) - return nil + return schedulerState.NewMutableState(ctx.State()).DropCommittee(ctx, kind, rt.ID) } - schedulerState.NewMutableState(ctx.State()).PutCommittee(&scheduler.Committee{ + return schedulerState.NewMutableState(ctx.State()).PutCommittee(ctx, &scheduler.Committee{ Kind: kind, RuntimeID: rt.ID, Members: members, ValidFor: epoch, }) - return nil } // Operates on consensus connection. @@ -632,7 +632,9 @@ electLoop: // Set the new pending validator set in the ABCI state. It needs to be // applied in EndBlock. state := schedulerState.NewMutableState(ctx.State()) - state.PutPendingValidators(newValidators) + if err = state.PutPendingValidators(ctx, newValidators); err != nil { + return fmt.Errorf("failed to set pending validators: %w", err) + } return nil } @@ -662,12 +664,24 @@ func publicKeyMapToSliceByStake( return entities, nil } - // Stable-sort the shuffled slice by decending escrow balance. + // Stable-sort the shuffled slice by descending escrow balance. + var balanceErr error sort.SliceStable(entities, func(i, j int) bool { - iBal := stakeAcc.GetEscrowBalance(entities[i]) - jBal := stakeAcc.GetEscrowBalance(entities[j]) - return iBal.Cmp(&jBal) == 1 // Note: Not -1 to get a reversed sort. + iBal, err := stakeAcc.GetEscrowBalance(entities[i]) + if err != nil { + balanceErr = err + return false + } + jBal, err := stakeAcc.GetEscrowBalance(entities[j]) + if err != nil { + balanceErr = err + return false + } + return iBal.Cmp(jBal) == 1 // Note: Not -1 to get a reversed sort. }) + if balanceErr != nil { + return nil, fmt.Errorf("failed to fetch escrow balance: %w", balanceErr) + } return entities, nil } diff --git a/go/consensus/tendermint/apps/scheduler/state/state.go b/go/consensus/tendermint/apps/scheduler/state/state.go index 349193e8184..e5bf65e246e 100644 --- a/go/consensus/tendermint/apps/scheduler/state/state.go +++ b/go/consensus/tendermint/apps/scheduler/state/state.go @@ -1,17 +1,16 @@ package state import ( + "context" "errors" - "github.com/tendermint/iavl" - "github.com/oasislabs/oasis-core/go/common" "github.com/oasislabs/oasis-core/go/common/cbor" "github.com/oasislabs/oasis-core/go/common/crypto/signature" "github.com/oasislabs/oasis-core/go/common/keyformat" - "github.com/oasislabs/oasis-core/go/common/logging" "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" "github.com/oasislabs/oasis-core/go/scheduler/api" + mkvs "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel" ) var ( @@ -33,119 +32,133 @@ var ( // // Value is CBOR-serialized api.ConsensusParameters. parametersKeyFmt = keyformat.New(0x63) - - logger = logging.GetLogger("tendermint/scheduler") ) type ImmutableState struct { *abci.ImmutableState } -func (s *ImmutableState) Committee(kind api.CommitteeKind, runtimeID common.Namespace) (*api.Committee, error) { - _, raw := s.Snapshot.Get(committeeKeyFmt.Encode(uint8(kind), &runtimeID)) +// Committee returns a specific elected committee. +func (s *ImmutableState) Committee(ctx context.Context, kind api.CommitteeKind, runtimeID common.Namespace) (*api.Committee, error) { + raw, err := s.Tree.Get(ctx, committeeKeyFmt.Encode(uint8(kind), &runtimeID)) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if raw == nil { return nil, nil } var committee *api.Committee - err := cbor.Unmarshal(raw, &committee) - return committee, err + if err = cbor.Unmarshal(raw, &committee); err != nil { + return nil, abci.UnavailableStateError(err) + } + return committee, nil } -func (s *ImmutableState) AllCommittees() ([]*api.Committee, error) { +// AllCommittees returns a list of all elected committees. +func (s *ImmutableState) AllCommittees(ctx context.Context) ([]*api.Committee, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() + var committees []*api.Committee - s.Snapshot.IterateRange( - committeeKeyFmt.Encode(), - nil, - true, - func(key, value []byte) bool { - if !committeeKeyFmt.Decode(key) { - return true - } + for it.Seek(committeeKeyFmt.Encode()); it.Valid(); it.Next() { + if !committeeKeyFmt.Decode(it.Key()) { + break + } - var c *api.Committee - err := cbor.Unmarshal(value, &c) - if err != nil { - logger.Error("couldn't get committee from state entry", - "key", key, - "value", value, - "err", err, - ) - return false - } - committees = append(committees, c) - return false - }, - ) + var c api.Committee + if err := cbor.Unmarshal(it.Value(), &c); err != nil { + return nil, abci.UnavailableStateError(err) + } + + committees = append(committees, &c) + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } return committees, nil } -func (s *ImmutableState) KindsCommittees(kinds []api.CommitteeKind) ([]*api.Committee, error) { +// KindsCommittees returns a list of all committees of specific kinds. +func (s *ImmutableState) KindsCommittees(ctx context.Context, kinds []api.CommitteeKind) ([]*api.Committee, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() + var committees []*api.Committee for _, kind := range kinds { - s.Snapshot.IterateRange( - committeeKeyFmt.Encode(uint8(kind)), - nil, - true, - func(key, value []byte) bool { - var k uint8 - if !committeeKeyFmt.Decode(key, &k) || k != uint8(kind) { - return true - } - - var c *api.Committee - err := cbor.Unmarshal(value, &c) - if err != nil { - logger.Error("couldn't get committee from state entry", - "key", key, - "value", value, - "err", err, - ) - return false - } - committees = append(committees, c) - return false - }, - ) + for it.Seek(committeeKeyFmt.Encode(uint8(kind))); it.Valid(); it.Next() { + var k uint8 + if !committeeKeyFmt.Decode(it.Key(), &k) || k != uint8(kind) { + break + } + + var c api.Committee + if err := cbor.Unmarshal(it.Value(), &c); err != nil { + return nil, abci.UnavailableStateError(err) + } + + committees = append(committees, &c) + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } } return committees, nil } -func (s *ImmutableState) CurrentValidators() ([]signature.PublicKey, error) { - _, raw := s.Snapshot.Get(validatorsCurrentKeyFmt.Encode()) +// CurrentValidators returns a list of current validators. +func (s *ImmutableState) CurrentValidators(ctx context.Context) ([]signature.PublicKey, error) { + raw, err := s.Tree.Get(ctx, validatorsCurrentKeyFmt.Encode()) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if raw == nil { return nil, nil } var validators []signature.PublicKey - err := cbor.Unmarshal(raw, &validators) - return validators, err + if err = cbor.Unmarshal(raw, &validators); err != nil { + return nil, abci.UnavailableStateError(err) + } + return validators, nil } -func (s *ImmutableState) PendingValidators() ([]signature.PublicKey, error) { - _, raw := s.Snapshot.Get(validatorsPendingKeyFmt.Encode()) +// PendingValidators returns a list of pending validators. +func (s *ImmutableState) PendingValidators(ctx context.Context) ([]signature.PublicKey, error) { + raw, err := s.Tree.Get(ctx, validatorsPendingKeyFmt.Encode()) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if raw == nil { return nil, nil } var validators []signature.PublicKey - err := cbor.Unmarshal(raw, &validators) - return validators, err + if err = cbor.Unmarshal(raw, &validators); err != nil { + return nil, abci.UnavailableStateError(err) + } + return validators, nil } -func (s *ImmutableState) ConsensusParameters() (*api.ConsensusParameters, error) { - _, raw := s.Snapshot.Get(parametersKeyFmt.Encode()) +// ConsensusParameters returns scheduler consensus parameters. +func (s *ImmutableState) ConsensusParameters(ctx context.Context) (*api.ConsensusParameters, error) { + raw, err := s.Tree.Get(ctx, parametersKeyFmt.Encode()) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if raw == nil { return nil, errors.New("tendermint/scheduler: expected consensus parameters to be present in app state") } var params api.ConsensusParameters - err := cbor.Unmarshal(raw, ¶ms) - return ¶ms, err + if err = cbor.Unmarshal(raw, ¶ms); err != nil { + return nil, abci.UnavailableStateError(err) + } + return ¶ms, nil } -func NewImmutableState(state abci.ApplicationState, version int64) (*ImmutableState, error) { - inner, err := abci.NewImmutableState(state, version) +func NewImmutableState(ctx context.Context, state abci.ApplicationState, version int64) (*ImmutableState, error) { + inner, err := abci.NewImmutableState(ctx, state, version) if err != nil { return nil, err } @@ -156,40 +169,47 @@ func NewImmutableState(state abci.ApplicationState, version int64) (*ImmutableSt // MutableState is a mutable scheduler state wrapper. type MutableState struct { *ImmutableState - - tree *iavl.MutableTree } -func (s *MutableState) PutCommittee(c *api.Committee) { - s.tree.Set(committeeKeyFmt.Encode(uint8(c.Kind), &c.RuntimeID), cbor.Marshal(c)) +// PutCommittee sets an elected committee for a specific runtime. +func (s *MutableState) PutCommittee(ctx context.Context, c *api.Committee) error { + err := s.Tree.Insert(ctx, committeeKeyFmt.Encode(uint8(c.Kind), &c.RuntimeID), cbor.Marshal(c)) + return abci.UnavailableStateError(err) } -func (s *MutableState) DropCommittee(kind api.CommitteeKind, runtimeID common.Namespace) { - s.tree.Remove(committeeKeyFmt.Encode(uint8(kind), &runtimeID)) +// DropCommittee removes an elected committee of a specific kind for a specific runtime. +func (s *MutableState) DropCommittee(ctx context.Context, kind api.CommitteeKind, runtimeID common.Namespace) error { + err := s.Tree.Remove(ctx, committeeKeyFmt.Encode(uint8(kind), &runtimeID)) + return abci.UnavailableStateError(err) } -func (s *MutableState) PutCurrentValidators(validators []signature.PublicKey) { - s.tree.Set(validatorsCurrentKeyFmt.Encode(), cbor.Marshal(validators)) +// PutCurrentValidators stores the current set of validators. +func (s *MutableState) PutCurrentValidators(ctx context.Context, validators []signature.PublicKey) error { + err := s.Tree.Insert(ctx, validatorsCurrentKeyFmt.Encode(), cbor.Marshal(validators)) + return abci.UnavailableStateError(err) } -func (s *MutableState) PutPendingValidators(validators []signature.PublicKey) { +// PutPendingValidators stores the pending set of validators. +func (s *MutableState) PutPendingValidators(ctx context.Context, validators []signature.PublicKey) error { if validators == nil { - s.tree.Remove(validatorsPendingKeyFmt.Encode()) - return + err := s.Tree.Remove(ctx, validatorsPendingKeyFmt.Encode()) + return abci.UnavailableStateError(err) } - s.tree.Set(validatorsPendingKeyFmt.Encode(), cbor.Marshal(validators)) + err := s.Tree.Insert(ctx, validatorsPendingKeyFmt.Encode(), cbor.Marshal(validators)) + return abci.UnavailableStateError(err) } -func (s *MutableState) SetConsensusParameters(params *api.ConsensusParameters) { - s.tree.Set(parametersKeyFmt.Encode(), cbor.Marshal(params)) +// SetConsensusParameters sets the scheduler consensus parameters. +func (s *MutableState) SetConsensusParameters(ctx context.Context, params *api.ConsensusParameters) error { + err := s.Tree.Insert(ctx, parametersKeyFmt.Encode(), cbor.Marshal(params)) + return abci.UnavailableStateError(err) } // NewMutableState creates a new mutable scheduler state wrapper. -func NewMutableState(tree *iavl.MutableTree) *MutableState { - inner := &abci.ImmutableState{Snapshot: tree.ImmutableTree} - +func NewMutableState(tree mkvs.KeyValueTree) *MutableState { return &MutableState{ - ImmutableState: &ImmutableState{inner}, - tree: tree, + ImmutableState: &ImmutableState{ + &abci.ImmutableState{Tree: tree}, + }, } } diff --git a/go/consensus/tendermint/apps/staking/fees.go b/go/consensus/tendermint/apps/staking/fees.go index 422552bcce6..676b8bf20e9 100644 --- a/go/consensus/tendermint/apps/staking/fees.go +++ b/go/consensus/tendermint/apps/staking/fees.go @@ -12,8 +12,14 @@ import ( // disburseFees disburses fees. // // In case of errors the state may be inconsistent. -func (app *stakingApplication) disburseFees(ctx *abci.Context, stakeState *stakingState.MutableState, proposerEntity *signature.PublicKey, numEligibleValidators int, signingEntities []signature.PublicKey) error { - totalFees, err := stakeState.LastBlockFees() +func (app *stakingApplication) disburseFees( + ctx *abci.Context, + stakeState *stakingState.MutableState, + proposerEntity *signature.PublicKey, + numEligibleValidators int, + signingEntities []signature.PublicKey, +) error { + totalFees, err := stakeState.LastBlockFees(ctx) if err != nil { return fmt.Errorf("staking: failed to query last block fees: %w", err) } @@ -34,7 +40,7 @@ func (app *stakingApplication) disburseFees(ctx *abci.Context, stakeState *staki // proposer $P$. The ratio of this split are controlled by `FeeSplitVote` and `FeeSplitPropose`. // Portions corresponding to validators that don't sign the block go to the common pool. - consensusParameters, err := stakeState.ConsensusParameters() + consensusParameters, err := stakeState.ConsensusParameters(ctx) if err != nil { return fmt.Errorf("staking: failed to load consensus parameters: %w", err) } @@ -87,7 +93,10 @@ func (app *stakingApplication) disburseFees(ctx *abci.Context, stakeState *staki if !proposeTotal.IsZero() { if proposerEntity != nil { // Perform the transfer. - acct := stakeState.Account(*proposerEntity) + acct, err := stakeState.Account(ctx, *proposerEntity) + if err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } if err = quantity.Move(&acct.General.Balance, totalFees, proposeTotal); err != nil { ctx.Logger().Error("failed to disburse fees (propose)", "err", err, @@ -96,14 +105,19 @@ func (app *stakingApplication) disburseFees(ctx *abci.Context, stakeState *staki ) return fmt.Errorf("staking: failed to disburse fees (propose): %w", err) } - stakeState.SetAccount(*proposerEntity, acct) + if err = stakeState.SetAccount(ctx, *proposerEntity, acct); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } } } // Pay voters. if !perVIVote.IsZero() { for _, voterEntity := range signingEntities { // Perform the transfer. - acct := stakeState.Account(voterEntity) + acct, err := stakeState.Account(ctx, voterEntity) + if err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } if err = quantity.Move(&acct.General.Balance, totalFees, perVIVote); err != nil { ctx.Logger().Error("failed to disburse fees (vote)", "err", err, @@ -112,23 +126,27 @@ func (app *stakingApplication) disburseFees(ctx *abci.Context, stakeState *staki ) return fmt.Errorf("staking: failed to disburse fees (vote): %w", err) } - stakeState.SetAccount(voterEntity, acct) + if err = stakeState.SetAccount(ctx, voterEntity, acct); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } } } // Any remainder goes to the common pool. if !totalFees.IsZero() { - commonPool, err := stakeState.CommonPool() + commonPool, err := stakeState.CommonPool(ctx) if err != nil { return fmt.Errorf("staking: failed to query common pool: %w", err) } - if err := quantity.Move(commonPool, totalFees, totalFees); err != nil { + if err = quantity.Move(commonPool, totalFees, totalFees); err != nil { ctx.Logger().Error("failed to move remainder to common pool", "err", err, "amount", totalFees, ) return fmt.Errorf("staking: failed to move to common pool: %w", err) } - stakeState.SetCommonPool(commonPool) + if err = stakeState.SetCommonPool(ctx, commonPool); err != nil { + return fmt.Errorf("failed to set common pool: %w", err) + } } return nil diff --git a/go/consensus/tendermint/apps/staking/genesis.go b/go/consensus/tendermint/apps/staking/genesis.go index d3a087872b9..abb633c211d 100644 --- a/go/consensus/tendermint/apps/staking/genesis.go +++ b/go/consensus/tendermint/apps/staking/genesis.go @@ -1,10 +1,8 @@ package staking import ( - "bytes" "context" "fmt" - "sort" "github.com/pkg/errors" "github.com/tendermint/tendermint/abci/types" @@ -22,7 +20,9 @@ func (app *stakingApplication) initParameters(ctx *abci.Context, state *stakingS return fmt.Errorf("staking/tendermint: sanity check failed: %w", err) } - state.SetConsensusParameters(&st.Parameters) + if err := state.SetConsensusParameters(ctx, &st.Parameters); err != nil { + return fmt.Errorf("staking/tendermint: failed to set consensus parameters: %w", err) + } return nil } @@ -40,16 +40,13 @@ func (app *stakingApplication) initCommonPool(ctx *abci.Context, st *staking.Gen return nil } -func (app *stakingApplication) initLedger(ctx *abci.Context, state *stakingState.MutableState, st *staking.Genesis, totalSupply *quantity.Quantity) error { - type ledgerUpdate struct { - id signature.PublicKey - account *staking.Account - } - - var ups []ledgerUpdate - for k, v := range st.Ledger { - id := k - +func (app *stakingApplication) initLedger( + ctx *abci.Context, + state *stakingState.MutableState, + st *staking.Genesis, + totalSupply *quantity.Quantity, +) error { + for id, v := range st.Ledger { if !v.General.Balance.IsValid() { ctx.Logger().Error("InitChain: invalid genesis general balance", "id", id, @@ -81,7 +78,6 @@ func (app *stakingApplication) initLedger(ctx *abci.Context, state *stakingState return errors.New("staking/tendermint: non-empty stake accumulator in genesis") } - ups = append(ups, ledgerUpdate{id, v}) if err := totalSupply.Add(&v.General.Balance); err != nil { ctx.Logger().Error("InitChain: failed to add general balance", "err", err, @@ -100,34 +96,38 @@ func (app *stakingApplication) initLedger(ctx *abci.Context, state *stakingState ) return errors.Wrap(err, "staking/tendermint: failed to add debonding escrow balance") } - } - // Make sure that we apply ledger updates in a canonical order. - sort.SliceStable(ups, func(i, j int) bool { return bytes.Compare(ups[i].id[:], ups[j].id[:]) < 0 }) - for _, u := range ups { - state.SetAccount(u.id, u.account) + + if err := state.SetAccount(ctx, id, v); err != nil { + return fmt.Errorf("staking/tendermint: failed to set account: %w", err) + } } return nil } -func (app *stakingApplication) initTotalSupply(ctx *abci.Context, state *stakingState.MutableState, st *staking.Genesis, totalSupply *quantity.Quantity) { +func (app *stakingApplication) initTotalSupply( + ctx *abci.Context, + state *stakingState.MutableState, + st *staking.Genesis, + totalSupply *quantity.Quantity, +) error { if totalSupply.Cmp(&st.TotalSupply) != 0 { ctx.Logger().Error("InitChain: total supply mismatch", "expected", st.TotalSupply, "actual", totalSupply, ) + return fmt.Errorf("total supply mismatch") } - state.SetCommonPool(&st.CommonPool) - state.SetTotalSupply(totalSupply) + if err := state.SetCommonPool(ctx, &st.CommonPool); err != nil { + return fmt.Errorf("failed to set common pool: %w", err) + } + if err := state.SetTotalSupply(ctx, totalSupply); err != nil { + return fmt.Errorf("failed to set total supply: %w", err) + } + return nil } func (app *stakingApplication) initDelegations(ctx *abci.Context, state *stakingState.MutableState, st *staking.Genesis) error { - type delegationUpdate struct { - escrowID signature.PublicKey - delegatorID signature.PublicKey - delegation *staking.Delegation - } - var dups []delegationUpdate for escrowID, delegations := range st.Delegations { delegationShares := quantity.NewQuantity() for delegatorID, delegation := range delegations { @@ -137,10 +137,15 @@ func (app *stakingApplication) initDelegations(ctx *abci.Context, state *staking ) return errors.Wrap(err, "staking/tendermint: failed to add delegation shares") } - dups = append(dups, delegationUpdate{escrowID, delegatorID, delegation}) + if err := state.SetDelegation(ctx, delegatorID, escrowID, delegation); err != nil { + return fmt.Errorf("failed to set delegation: %w", err) + } } - acc := state.Account(escrowID) + acc, err := state.Account(ctx, escrowID) + if err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } if acc.Escrow.Active.TotalShares.Cmp(delegationShares) != 0 { ctx.Logger().Error("InitChain: total shares mismatch", "escrow_id", escrowID, @@ -150,27 +155,10 @@ func (app *stakingApplication) initDelegations(ctx *abci.Context, state *staking return errors.New("staking/tendermint: total shares mismatch") } } - // Make sure that we apply delegation updates in a canonical order. - sort.SliceStable(dups, func(i, j int) bool { - if c := bytes.Compare(dups[i].escrowID[:], dups[j].escrowID[:]); c != 0 { - return c < 0 - } - return bytes.Compare(dups[i].delegatorID[:], dups[j].delegatorID[:]) < 0 - }) - for _, u := range dups { - state.SetDelegation(u.delegatorID, u.escrowID, u.delegation) - } return nil } func (app *stakingApplication) initDebondingDelegations(ctx *abci.Context, state *stakingState.MutableState, st *staking.Genesis) error { - type debondingDelegationUpdate struct { - escrowID signature.PublicKey - delegatorID signature.PublicKey - seq uint64 - delegation *staking.DebondingDelegation - } - var deups []debondingDelegationUpdate for escrowID, delegators := range st.DebondingDelegations { debondingShares := quantity.NewQuantity() for delegatorID, delegations := range delegators { @@ -182,11 +170,16 @@ func (app *stakingApplication) initDebondingDelegations(ctx *abci.Context, state return errors.Wrap(err, "staking/tendermint: failed to add debonding delegation shares") } - deups = append(deups, debondingDelegationUpdate{escrowID, delegatorID, uint64(idx), delegation}) + if err := state.SetDebondingDelegation(ctx, delegatorID, escrowID, uint64(idx), delegation); err != nil { + return fmt.Errorf("failed to set debonding delegation: %w", err) + } } } - acc := state.Account(escrowID) + acc, err := state.Account(ctx, escrowID) + if err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } if acc.Escrow.Debonding.TotalShares.Cmp(debondingShares) != 0 { ctx.Logger().Error("InitChain: debonding shares mismatch", "escrow_id", escrowID, @@ -196,19 +189,6 @@ func (app *stakingApplication) initDebondingDelegations(ctx *abci.Context, state return errors.New("staking/tendermint: debonding shares mismatch") } } - // Make sure that we apply delegation updates in a canonical order. - sort.SliceStable(deups, func(i, j int) bool { - if c := bytes.Compare(deups[i].escrowID[:], deups[j].escrowID[:]); c != 0 { - return c < 0 - } - if c := bytes.Compare(deups[i].delegatorID[:], deups[j].delegatorID[:]); c != 0 { - return c < 0 - } - return deups[i].seq < deups[j].seq - }) - for _, u := range deups { - state.SetDebondingDelegation(u.delegatorID, u.escrowID, u.seq, u.delegation) - } return nil } @@ -233,7 +213,9 @@ func (app *stakingApplication) InitChain(ctx *abci.Context, request types.Reques return err } - app.initTotalSupply(ctx, state, st, &totalSupply) + if err := app.initTotalSupply(ctx, state, st, &totalSupply); err != nil { + return err + } if err := app.initDelegations(ctx, state, st); err != nil { return err @@ -253,39 +235,43 @@ func (app *stakingApplication) InitChain(ctx *abci.Context, request types.Reques // Genesis exports current state in genesis format. func (sq *stakingQuerier) Genesis(ctx context.Context) (*staking.Genesis, error) { - totalSupply, err := sq.state.TotalSupply() + totalSupply, err := sq.state.TotalSupply(ctx) if err != nil { return nil, err } - commonPool, err := sq.state.CommonPool() + commonPool, err := sq.state.CommonPool(ctx) if err != nil { return nil, err } - accounts, err := sq.state.Accounts() + accounts, err := sq.state.Accounts(ctx) if err != nil { return nil, err } ledger := make(map[signature.PublicKey]*staking.Account) for _, acctID := range accounts { - acct := sq.state.Account(acctID) + var acct *staking.Account + acct, err = sq.state.Account(ctx, acctID) + if err != nil { + return nil, fmt.Errorf("failed to fetch account: %w", err) + } // Make sure that export resets the stake accumulator state as that should be re-initialized // during genesis (a genesis document with non-empty stake accumulator is invalid). acct.Escrow.StakeAccumulator = staking.StakeAccumulator{} ledger[acctID] = acct } - delegations, err := sq.state.Delegations() + delegations, err := sq.state.Delegations(ctx) if err != nil { return nil, err } - debondingDelegations, err := sq.state.DebondingDelegations() + debondingDelegations, err := sq.state.DebondingDelegations(ctx) if err != nil { return nil, err } - params, err := sq.state.ConsensusParameters() + params, err := sq.state.ConsensusParameters(ctx) if err != nil { return nil, err } diff --git a/go/consensus/tendermint/apps/staking/proposing_rewards.go b/go/consensus/tendermint/apps/staking/proposing_rewards.go index 4f3f20a46f9..5a78182261f 100644 --- a/go/consensus/tendermint/apps/staking/proposing_rewards.go +++ b/go/consensus/tendermint/apps/staking/proposing_rewards.go @@ -13,9 +13,13 @@ import ( epochtime "github.com/oasislabs/oasis-core/go/epochtime/api" ) -func (app *stakingApplication) resolveEntityIDFromProposer(regState *registryState.MutableState, request types.RequestBeginBlock, ctx *abci.Context) *signature.PublicKey { +func (app *stakingApplication) resolveEntityIDFromProposer( + ctx *abci.Context, + regState *registryState.MutableState, + request types.RequestBeginBlock, +) *signature.PublicKey { var proposingEntity *signature.PublicKey - proposerNode, err := regState.NodeByConsensusAddress(request.Header.ProposerAddress) + proposerNode, err := regState.NodeByConsensusAddress(ctx, request.Header.ProposerAddress) if err != nil { ctx.Logger().Warn("failed to get proposer node", "err", err, @@ -27,17 +31,22 @@ func (app *stakingApplication) resolveEntityIDFromProposer(regState *registrySta return proposingEntity } -func (app *stakingApplication) rewardBlockProposing(ctx *abci.Context, stakeState *stakingState.MutableState, proposingEntity *signature.PublicKey, numEligibleValidators int, numSigningEntities int) error { +func (app *stakingApplication) rewardBlockProposing( + ctx *abci.Context, + stakeState *stakingState.MutableState, + proposingEntity *signature.PublicKey, + numEligibleValidators, numSigningEntities int, +) error { if proposingEntity == nil { return nil } - params, err := stakeState.ConsensusParameters() + params, err := stakeState.ConsensusParameters(ctx) if err != nil { return fmt.Errorf("staking mutable state getting consensus parameters: %w", err) } - epoch, err := app.state.GetCurrentEpoch(ctx.Ctx()) + epoch, err := app.state.GetCurrentEpoch(ctx) if err != nil { return fmt.Errorf("app state getting current epoch: %w", err) } @@ -46,7 +55,7 @@ func (app *stakingApplication) rewardBlockProposing(ctx *abci.Context, stakeStat return nil } // Reward the proposer based on the `(number of included votes) / (size of the validator set)` ratio. - if err = stakeState.AddRewardSingleAttenuated(epoch, ¶ms.RewardFactorBlockProposed, numSigningEntities, numEligibleValidators, *proposingEntity); err != nil { + if err = stakeState.AddRewardSingleAttenuated(ctx, epoch, ¶ms.RewardFactorBlockProposed, numSigningEntities, numEligibleValidators, *proposingEntity); err != nil { return fmt.Errorf("adding rewards: %w", err) } return nil diff --git a/go/consensus/tendermint/apps/staking/query.go b/go/consensus/tendermint/apps/staking/query.go index b4bb6379bdc..3670dad7bd3 100644 --- a/go/consensus/tendermint/apps/staking/query.go +++ b/go/consensus/tendermint/apps/staking/query.go @@ -5,7 +5,6 @@ import ( "github.com/oasislabs/oasis-core/go/common/crypto/signature" "github.com/oasislabs/oasis-core/go/common/quantity" - "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" stakingState "github.com/oasislabs/oasis-core/go/consensus/tendermint/apps/staking/state" epochtime "github.com/oasislabs/oasis-core/go/epochtime/api" staking "github.com/oasislabs/oasis-core/go/staking/api" @@ -30,27 +29,10 @@ type QueryFactory struct { // QueryAt returns the staking query interface for a specific height. func (sf *QueryFactory) QueryAt(ctx context.Context, height int64) (Query, error) { - var state *stakingState.ImmutableState - var err error - abciCtx := abci.FromCtx(ctx) - - // If this request was made from InitChain, no blocks and states have been - // submitted yet, so we use the existing state instead. - if abciCtx != nil && abciCtx.IsInitChain() { - state = stakingState.NewMutableState(abciCtx.State()).ImmutableState - } else { - state, err = stakingState.NewImmutableState(sf.app.state, height) - if err != nil { - return nil, err - } - } - - // If this request was made from an ABCI app, make sure to use the associated - // context for querying state instead of the default one. - if abciCtx != nil && height == abciCtx.BlockHeight()+1 { - state.Snapshot = abciCtx.State().ImmutableTree + state, err := stakingState.NewImmutableState(ctx, sf.app.state, height) + if err != nil { + return nil, err } - return &stakingQuerier{state}, nil } @@ -59,15 +41,15 @@ type stakingQuerier struct { } func (sq *stakingQuerier) TotalSupply(ctx context.Context) (*quantity.Quantity, error) { - return sq.state.TotalSupply() + return sq.state.TotalSupply(ctx) } func (sq *stakingQuerier) CommonPool(ctx context.Context) (*quantity.Quantity, error) { - return sq.state.CommonPool() + return sq.state.CommonPool(ctx) } func (sq *stakingQuerier) Threshold(ctx context.Context, kind staking.ThresholdKind) (*quantity.Quantity, error) { - thresholds, err := sq.state.Thresholds() + thresholds, err := sq.state.Thresholds(ctx) if err != nil { return nil, err } @@ -80,19 +62,19 @@ func (sq *stakingQuerier) Threshold(ctx context.Context, kind staking.ThresholdK } func (sq *stakingQuerier) DebondingInterval(ctx context.Context) (epochtime.EpochTime, error) { - return sq.state.DebondingInterval() + return sq.state.DebondingInterval(ctx) } func (sq *stakingQuerier) Accounts(ctx context.Context) ([]signature.PublicKey, error) { - return sq.state.Accounts() + return sq.state.Accounts(ctx) } func (sq *stakingQuerier) AccountInfo(ctx context.Context, id signature.PublicKey) (*staking.Account, error) { - return sq.state.Account(id), nil + return sq.state.Account(ctx, id) } func (sq *stakingQuerier) DebondingDelegations(ctx context.Context, id signature.PublicKey) (map[signature.PublicKey][]*staking.DebondingDelegation, error) { - return sq.state.DebondingDelegationsFor(id) + return sq.state.DebondingDelegationsFor(ctx, id) } func (app *stakingApplication) QueryFactory() interface{} { diff --git a/go/consensus/tendermint/apps/staking/signing_rewards.go b/go/consensus/tendermint/apps/staking/signing_rewards.go index 058ead92241..fba27e7bc87 100644 --- a/go/consensus/tendermint/apps/staking/signing_rewards.go +++ b/go/consensus/tendermint/apps/staking/signing_rewards.go @@ -9,8 +9,12 @@ import ( epochtime "github.com/oasislabs/oasis-core/go/epochtime/api" ) -func (app *stakingApplication) updateEpochSigning(ctx *abci.Context, stakeState *stakingState.MutableState, signingEntities []signature.PublicKey) error { - epochSigning, err := stakeState.EpochSigning() +func (app *stakingApplication) updateEpochSigning( + ctx *abci.Context, + stakeState *stakingState.MutableState, + signingEntities []signature.PublicKey, +) error { + epochSigning, err := stakeState.EpochSigning(ctx) if err != nil { return fmt.Errorf("loading epoch signing info: %w", err) } @@ -19,7 +23,9 @@ func (app *stakingApplication) updateEpochSigning(ctx *abci.Context, stakeState return err } - stakeState.SetEpochSigning(epochSigning) + if err := stakeState.SetEpochSigning(ctx, epochSigning); err != nil { + return fmt.Errorf("failed to set epoch signing info: %w", err) + } return nil } @@ -27,21 +33,25 @@ func (app *stakingApplication) updateEpochSigning(ctx *abci.Context, stakeState func (app *stakingApplication) rewardEpochSigning(ctx *abci.Context, time epochtime.EpochTime) error { stakeState := stakingState.NewMutableState(ctx.State()) - params, err := stakeState.ConsensusParameters() + params, err := stakeState.ConsensusParameters(ctx) if err != nil { return fmt.Errorf("loading consensus parameters: %w", err) } if params.SigningRewardThresholdDenominator == 0 { - stakeState.ClearEpochSigning() + if err = stakeState.ClearEpochSigning(ctx); err != nil { + return fmt.Errorf("failed to clear epoch signing: %w", err) + } return nil } - epochSigning, err := stakeState.EpochSigning() + epochSigning, err := stakeState.EpochSigning(ctx) if err != nil { return fmt.Errorf("loading epoch signing info: %w", err) } - stakeState.ClearEpochSigning() + if err = stakeState.ClearEpochSigning(ctx); err != nil { + return fmt.Errorf("failed to clear epoch signing: %w", err) + } if epochSigning.Total == 0 { return nil @@ -52,7 +62,7 @@ func (app *stakingApplication) rewardEpochSigning(ctx *abci.Context, time epocht return fmt.Errorf("determining eligibility: %w", err) } - if err := stakeState.AddRewards(time, ¶ms.RewardFactorEpochSigned, eligibleEntities); err != nil { + if err := stakeState.AddRewards(ctx, time, ¶ms.RewardFactorEpochSigned, eligibleEntities); err != nil { return fmt.Errorf("adding rewards: %w", err) } diff --git a/go/consensus/tendermint/apps/staking/slashing.go b/go/consensus/tendermint/apps/staking/slashing.go index fe09d726609..d619fe9136e 100644 --- a/go/consensus/tendermint/apps/staking/slashing.go +++ b/go/consensus/tendermint/apps/staking/slashing.go @@ -29,7 +29,7 @@ func onEvidenceDoubleSign( // Resolve consensus node. Note that in order for this to work even in light // of node expirations, the node descriptor must be available for at least // the debonding period after expiration. - node, err := regState.NodeByConsensusAddress(addr) + node, err := regState.NodeByConsensusAddress(ctx, addr) if err != nil { ctx.Logger().Warn("failed to get validator node", "err", err, @@ -38,7 +38,7 @@ func onEvidenceDoubleSign( return nil } - nodeStatus, err := regState.NodeStatus(node.ID) + nodeStatus, err := regState.NodeStatus(ctx, node.ID) if err != nil { ctx.Logger().Warn("failed to get validator node status", "err", err, @@ -58,7 +58,7 @@ func onEvidenceDoubleSign( } // Retrieve the slash procedure for double signing. - st, err := stakeState.Slashing() + st, err := stakeState.Slashing(ctx) if err != nil { ctx.Logger().Error("failed to get slashing table entry for double signing", "err", err, @@ -96,7 +96,7 @@ func onEvidenceDoubleSign( return err } - if err = regState.SetNodeStatus(node.ID, nodeStatus); err != nil { + if err = regState.SetNodeStatus(ctx, node.ID, nodeStatus); err != nil { ctx.Logger().Error("failed to set validator node status", "err", err, "node_id", node.ID, diff --git a/go/consensus/tendermint/apps/staking/slashing_test.go b/go/consensus/tendermint/apps/staking/slashing_test.go index 97c44d08349..38c36af3e50 100644 --- a/go/consensus/tendermint/apps/staking/slashing_test.go +++ b/go/consensus/tendermint/apps/staking/slashing_test.go @@ -45,7 +45,8 @@ func TestOnEvidenceDoubleSign(t *testing.T) { ent, entitySigner, _ := entity.TestEntity() sigEntity, err := entity.SignEntity(entitySigner, registry.RegisterEntitySignatureContext, ent) require.NoError(err, "SignEntity") - regState.SetEntity(ent, sigEntity) + err = regState.SetEntity(ctx, ent, sigEntity) + require.NoError(err, "SetEntity") // Add node. nodeSigner := memorySigner.NewTestSigner("node test signer") nod := &node.Node{ @@ -57,7 +58,7 @@ func TestOnEvidenceDoubleSign(t *testing.T) { } sigNode, err := node.MultiSignNode([]signature.Signer{nodeSigner}, registry.RegisterNodeSignatureContext, nod) require.NoError(err, "MultiSignNode") - err = regState.SetNode(nod, sigNode) + err = regState.SetNode(ctx, nod, sigNode) require.NoError(err, "SetNode") // Should not fail if node status is not available. @@ -65,7 +66,7 @@ func TestOnEvidenceDoubleSign(t *testing.T) { require.NoError(err, "should not fail when node status is not available") // Add node status. - err = regState.SetNodeStatus(nod.ID, ®istry.NodeStatus{}) + err = regState.SetNodeStatus(ctx, nod.ID, ®istry.NodeStatus{}) require.NoError(err, "SetNodeStatus") // Should fail if unable to get the slashing procedure. @@ -75,7 +76,7 @@ func TestOnEvidenceDoubleSign(t *testing.T) { // Add slashing procedure. var slashAmount quantity.Quantity _ = slashAmount.FromUint64(100) - stakeState.SetConsensusParameters(&staking.ConsensusParameters{ + err = stakeState.SetConsensusParameters(ctx, &staking.ConsensusParameters{ Slashing: map[staking.SlashReason]staking.Slash{ staking.SlashDoubleSigning: staking.Slash{ Amount: slashAmount, @@ -83,6 +84,7 @@ func TestOnEvidenceDoubleSign(t *testing.T) { }, }, }) + require.NoError(err, "SetConsensusParameters") // Should fail as the validator has no stake (which is an invariant violation as a validator // needs to have some stake). @@ -94,7 +96,7 @@ func TestOnEvidenceDoubleSign(t *testing.T) { _ = balance.FromUint64(200) var totalShares quantity.Quantity _ = totalShares.FromUint64(200) - stakeState.SetAccount(ent.ID, &staking.Account{ + err = stakeState.SetAccount(ctx, ent.ID, &staking.Account{ Escrow: staking.EscrowAccount{ Active: staking.SharePool{ Balance: balance, @@ -102,18 +104,20 @@ func TestOnEvidenceDoubleSign(t *testing.T) { }, }, }) + require.NoError(err, "SetAccount") // Should slash. err = onEvidenceDoubleSign(ctx, validatorAddress, 1, now, 1) require.NoError(err, "slashing should succeed") // Entity stake should be slashed. - acct := stakeState.Account(ent.ID) + acct, err := stakeState.Account(ctx, ent.ID) + require.NoError(err, "Account") _ = balance.Sub(&slashAmount) require.EqualValues(balance, acct.Escrow.Active.Balance, "entity stake should be slashed") // Node should be frozen. - status, err := regState.NodeStatus(nod.ID) + status, err := regState.NodeStatus(ctx, nod.ID) require.NoError(err, "NodeStatus") require.True(status.IsFrozen(), "node should be frozen after slashing") require.EqualValues(registry.FreezeForever, status.FreezeEndTime, "node should be frozen forever") diff --git a/go/consensus/tendermint/apps/staking/staking.go b/go/consensus/tendermint/apps/staking/staking.go index 70d0924a921..6273c4fab29 100644 --- a/go/consensus/tendermint/apps/staking/staking.go +++ b/go/consensus/tendermint/apps/staking/staking.go @@ -4,7 +4,6 @@ package staking import ( "fmt" - "github.com/pkg/errors" "github.com/tendermint/tendermint/abci/types" tmtypes "github.com/tendermint/tendermint/types" @@ -57,7 +56,7 @@ func (app *stakingApplication) BeginBlock(ctx *abci.Context, request types.Reque stakeState := stakingState.NewMutableState(ctx.State()) // Look up the proposer's entity. - proposingEntity := app.resolveEntityIDFromProposer(regState, request, ctx) + proposingEntity := app.resolveEntityIDFromProposer(ctx, regState, request) // Go through all signers of the previous block and resolve entities. // numEligibleValidators is how many total validators are in the validator set, while @@ -160,47 +159,65 @@ func (app *stakingApplication) onEpochChange(ctx *abci.Context, epoch epochtime. state := stakingState.NewMutableState(ctx.State()) // Delegation unbonding after debonding period elapses. - for _, e := range state.ExpiredDebondingQueue(epoch) { + expiredDebondingQueue, err := state.ExpiredDebondingQueue(ctx, epoch) + if err != nil { + return fmt.Errorf("failed to query expired debonding queue: %w", err) + } + for _, e := range expiredDebondingQueue { deb := e.Delegation shareAmount := deb.Shares.Clone() - delegator := state.Account(e.DelegatorID) + delegator, err := state.Account(ctx, e.DelegatorID) + if err != nil { + return fmt.Errorf("failed to query delegator account: %w", err) + } // NOTE: Could be the same account, so make sure to not have two duplicate // copies of it and overwrite it later. var escrow *staking.Account if e.DelegatorID.Equal(e.EscrowID) { escrow = delegator } else { - escrow = state.Account(e.EscrowID) + escrow, err = state.Account(ctx, e.EscrowID) + if err != nil { + return fmt.Errorf("failed to query escrow account: %w", err) + } } var tokens quantity.Quantity - if err := escrow.Escrow.Debonding.Withdraw(&tokens, &deb.Shares, shareAmount); err != nil { + if err = escrow.Escrow.Debonding.Withdraw(&tokens, &deb.Shares, shareAmount); err != nil { ctx.Logger().Error("failed to redeem debonding shares", "err", err, "escrow_id", e.EscrowID, "delegator_id", e.DelegatorID, "shares", deb.Shares, ) - return errors.Wrap(err, "staking/tendermint: failed to redeem debonding shares") + return fmt.Errorf("staking/tendermint: failed to redeem debonding shares: %w", err) } tokenAmount := tokens.Clone() - if err := quantity.Move(&delegator.General.Balance, &tokens, tokenAmount); err != nil { + if err = quantity.Move(&delegator.General.Balance, &tokens, tokenAmount); err != nil { ctx.Logger().Error("failed to move debonded tokens", "err", err, "escrow_id", e.EscrowID, "delegator_id", e.DelegatorID, "shares", deb.Shares, ) - return errors.Wrap(err, "staking/tendermint: failed to redeem debonding shares") + return fmt.Errorf("staking/tendermint: failed to redeem debonding shares: %w", err) } // Update state. - state.RemoveFromDebondingQueue(e.Epoch, e.DelegatorID, e.EscrowID, e.Seq) - state.SetDebondingDelegation(e.DelegatorID, e.EscrowID, e.Seq, nil) - state.SetAccount(e.DelegatorID, delegator) + if err = state.RemoveFromDebondingQueue(ctx, e.Epoch, e.DelegatorID, e.EscrowID, e.Seq); err != nil { + return fmt.Errorf("failed to remove from debonding queue: %w", err) + } + if err = state.SetDebondingDelegation(ctx, e.DelegatorID, e.EscrowID, e.Seq, nil); err != nil { + return fmt.Errorf("failed to set debonding delegation: %w", err) + } + if err = state.SetAccount(ctx, e.DelegatorID, delegator); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } if !e.DelegatorID.Equal(e.EscrowID) { - state.SetAccount(e.EscrowID, escrow) + if err = state.SetAccount(ctx, e.EscrowID, escrow); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } } ctx.Logger().Debug("released tokens", @@ -222,14 +239,14 @@ func (app *stakingApplication) onEpochChange(ctx *abci.Context, epoch epochtime. ctx.Logger().Error("failed to add signing rewards", "err", err, ) - return errors.Wrap(err, "staking/tendermint: failed to add signing rewards") + return fmt.Errorf("staking/tendermint: failed to add signing rewards: %w", err) } return nil } func (app *stakingApplication) FireTimer(ctx *abci.Context, timer *abci.Timer) error { - return errors.New("tendermint/staking: unexpected timer") + return fmt.Errorf("tendermint/staking: unexpected timer") } // New constructs a new staking application instance. diff --git a/go/consensus/tendermint/apps/staking/state/accumulator.go b/go/consensus/tendermint/apps/staking/state/accumulator.go index b208c5cf9a1..8f966cf2069 100644 --- a/go/consensus/tendermint/apps/staking/state/accumulator.go +++ b/go/consensus/tendermint/apps/staking/state/accumulator.go @@ -1,6 +1,7 @@ package state import ( + "context" "fmt" "github.com/oasislabs/oasis-core/go/common/crypto/signature" @@ -12,6 +13,7 @@ import ( // StakeAccumulatorCache is a thin wrapper around an underlying escrow accounts which caches some // values (like the threshold map) between operations. type StakeAccumulatorCache struct { + ctx context.Context // state is the mutable state. state *MutableState @@ -23,25 +25,32 @@ type StakeAccumulatorCache struct { thresholds map[staking.ThresholdKind]quantity.Quantity } -func (c *StakeAccumulatorCache) getAccount(id signature.PublicKey) *staking.Account { +func (c *StakeAccumulatorCache) getAccount(id signature.PublicKey) (*staking.Account, error) { if c.accounts == nil { c.accounts = make(map[signature.PublicKey]*staking.Account) c.accountOrder = nil } if a := c.accounts[id]; a != nil { - return a + return a, nil } - a := c.state.Account(id) + a, err := c.state.Account(c.ctx, id) + if err != nil { + return nil, err + } c.accounts[id] = a c.accountOrder = append(c.accountOrder, id) - return a + return a, nil } // CheckStakeClaims checks whether the escrow account balance satisfies all the stake claims. func (c *StakeAccumulatorCache) CheckStakeClaims(id signature.PublicKey) error { - return c.getAccount(id).Escrow.CheckStakeClaims(c.thresholds) + acct, err := c.getAccount(id) + if err != nil { + return err + } + return acct.Escrow.CheckStakeClaims(c.thresholds) } // AddStakeClaim attempts to add a stake claim to the given escrow account. @@ -49,27 +58,42 @@ func (c *StakeAccumulatorCache) CheckStakeClaims(id signature.PublicKey) error { // In case there is insufficient stake to cover the claim or an error occurrs, no modifications are // made to the stake accumulator. func (c *StakeAccumulatorCache) AddStakeClaim(id signature.PublicKey, claim staking.StakeClaim, thresholds []staking.ThresholdKind) error { - return c.getAccount(id).Escrow.AddStakeClaim(c.thresholds, claim, thresholds) + acct, err := c.getAccount(id) + if err != nil { + return err + } + return acct.Escrow.AddStakeClaim(c.thresholds, claim, thresholds) } // RemoveStakeClaim removes a given stake claim. // // It is an error if the stake claim does not exist. func (c *StakeAccumulatorCache) RemoveStakeClaim(id signature.PublicKey, claim staking.StakeClaim) error { - return c.getAccount(id).Escrow.RemoveStakeClaim(claim) + acct, err := c.getAccount(id) + if err != nil { + return err + } + return acct.Escrow.RemoveStakeClaim(claim) } // GetEscrowBalance returns a given account's escrow balance. -func (c *StakeAccumulatorCache) GetEscrowBalance(id signature.PublicKey) quantity.Quantity { - return *c.getAccount(id).Escrow.Active.Balance.Clone() +func (c *StakeAccumulatorCache) GetEscrowBalance(id signature.PublicKey) (*quantity.Quantity, error) { + acct, err := c.getAccount(id) + if err != nil { + return nil, err + } + return acct.Escrow.Active.Balance.Clone(), nil } // Commit commits the stake accumulator changes. The caller must ensure that this does not overwrite // any outstanding account updates. -func (c *StakeAccumulatorCache) Commit() { +func (c *StakeAccumulatorCache) Commit() error { for _, id := range c.accountOrder { - c.state.SetAccount(id, c.accounts[id]) + if err := c.state.SetAccount(c.ctx, id, c.accounts[id]); err != nil { + return fmt.Errorf("failed to set account %s: %w", id, err) + } } + return nil } // Discard discards any stake accumulator changes. @@ -82,12 +106,13 @@ func (c *StakeAccumulatorCache) Discard() { func NewStakeAccumulatorCache(ctx *abci.Context) (*StakeAccumulatorCache, error) { state := NewMutableState(ctx.State()) - thresholds, err := state.Thresholds() + thresholds, err := state.Thresholds(ctx) if err != nil { return nil, fmt.Errorf("staking/tendermint: failed to query thresholds: %w", err) } return &StakeAccumulatorCache{ + ctx: ctx, state: state, thresholds: thresholds, }, nil @@ -105,8 +130,7 @@ func AddStakeClaim(ctx *abci.Context, id signature.PublicKey, claim staking.Stak if err = sa.AddStakeClaim(id, claim, thresholds); err != nil { return err } - sa.Commit() - return nil + return sa.Commit() } // RemoveStakeClaim is a convenience function for removing a single stake claim from an entity. @@ -121,8 +145,7 @@ func RemoveStakeClaim(ctx *abci.Context, id signature.PublicKey, claim staking.S if err = sa.RemoveStakeClaim(id, claim); err != nil { return err } - sa.Commit() - return nil + return sa.Commit() } // CheckStakeClaims is a convenience function for checking a single entity's stake claims. diff --git a/go/consensus/tendermint/apps/staking/state/accumulator_test.go b/go/consensus/tendermint/apps/staking/state/accumulator_test.go index fb2166dc089..c49e54f4758 100644 --- a/go/consensus/tendermint/apps/staking/state/accumulator_test.go +++ b/go/consensus/tendermint/apps/staking/state/accumulator_test.go @@ -26,11 +26,12 @@ func TestStakeAccumulatorCache(t *testing.T) { stakeState := NewMutableState(ctx.State()) var q quantity.Quantity _ = q.FromInt64(1_000) - stakeState.SetConsensusParameters(&staking.ConsensusParameters{ + err = stakeState.SetConsensusParameters(ctx, &staking.ConsensusParameters{ Thresholds: map[staking.ThresholdKind]quantity.Quantity{ staking.KindEntity: *q.Clone(), }, }) + require.NoError(err, "SetConsensusParameters") acc, err := NewStakeAccumulatorCache(ctx) require.NoError(err, "NewStakeAccumulatorCache") @@ -42,7 +43,8 @@ func TestStakeAccumulatorCache(t *testing.T) { ent, _, _ := entity.TestEntity() var acct staking.Account acct.Escrow.Active.Balance = *q.Clone() - stakeState.SetAccount(ent.ID, &acct) + err = stakeState.SetAccount(ctx, ent.ID, &acct) + require.NoError(err, "SetAccount") err = acc.AddStakeClaim(ent.ID, staking.StakeClaim("claim"), []staking.ThresholdKind{staking.KindEntity}) require.NoError(err, "AddStakeClaim") @@ -50,27 +52,33 @@ func TestStakeAccumulatorCache(t *testing.T) { err = acc.CheckStakeClaims(ent.ID) require.NoError(err, "CheckStakeClaims") - balance := acc.GetEscrowBalance(ent.ID) - require.Equal(acct.Escrow.Active.Balance, balance, "GetEscrowBalance should return the correct balance") + balance, err := acc.GetEscrowBalance(ent.ID) + require.NoError(err, "GetEscrowBalance") + require.Equal(&acct.Escrow.Active.Balance, balance, "GetEscrowBalance should return the correct balance") // Check that nothing has been committed yet. - acct2 := stakeState.Account(ent.ID) + acct2, err := stakeState.Account(ctx, ent.ID) + require.NoError(err, "Account") require.Len(acct2.Escrow.StakeAccumulator.Claims, 0, "claims should not be updated yet") // Now commit and re-check. - acc.Commit() - acct2 = stakeState.Account(ent.ID) + err = acc.Commit() + require.NoError(err, "Commit") + acct2, err = stakeState.Account(ctx, ent.ID) + require.NoError(err, "Account") require.Len(acct2.Escrow.StakeAccumulator.Claims, 1, "claims should be correct") err = acc.RemoveStakeClaim(ent.ID, staking.StakeClaim("claim")) require.NoError(err, "RemoveStakeClaim") // Check that nothing has been committed. - acct2 = stakeState.Account(ent.ID) + acct2, err = stakeState.Account(ctx, ent.ID) + require.NoError(err, "Account") require.Len(acct2.Escrow.StakeAccumulator.Claims, 1, "claims should not be updated") acc.Discard() - acct2 = stakeState.Account(ent.ID) + acct2, err = stakeState.Account(ctx, ent.ID) + require.NoError(err, "Account") require.Len(acct2.Escrow.StakeAccumulator.Claims, 1, "claims should not be updated") // Test convenience functions. diff --git a/go/consensus/tendermint/apps/staking/state/gas.go b/go/consensus/tendermint/apps/staking/state/gas.go index 7d36ba09a18..91e6a384da3 100644 --- a/go/consensus/tendermint/apps/staking/state/gas.go +++ b/go/consensus/tendermint/apps/staking/state/gas.go @@ -45,7 +45,10 @@ func AuthenticateAndPayFees( } // Fetch account and make sure the nonce is correct. - account := state.Account(id) + account, err := state.Account(ctx, id) + if err != nil { + return fmt.Errorf("failed to fetch account state: %w", err) + } if account.General.Nonce != nonce { logger.Error("invalid account nonce", "account_id", id, @@ -89,7 +92,9 @@ func AuthenticateAndPayFees( } account.General.Nonce++ - state.SetAccount(id, account) + if err := state.SetAccount(ctx, id, account); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } // Configure gas accountant on the context. ctx.SetGasAccountant(abci.NewCompositeGasAccountant( @@ -106,5 +111,7 @@ func PersistBlockFees(ctx *abci.Context) { fees := ctx.BlockContext().Get(feeAccumulatorKey{}).(*feeAccumulator).balance state := NewMutableState(ctx.State()) - state.SetLastBlockFees(&fees) + if err := state.SetLastBlockFees(ctx, &fees); err != nil { + panic(err) + } } diff --git a/go/consensus/tendermint/apps/staking/state/state.go b/go/consensus/tendermint/apps/staking/state/state.go index 6cda4ae1b5f..f5f65dfa1cb 100644 --- a/go/consensus/tendermint/apps/staking/state/state.go +++ b/go/consensus/tendermint/apps/staking/state/state.go @@ -2,12 +2,12 @@ package state import ( "bytes" + "context" "fmt" "math" "sort" "github.com/pkg/errors" - "github.com/tendermint/iavl" "github.com/oasislabs/oasis-core/go/common/cbor" "github.com/oasislabs/oasis-core/go/common/crypto/signature" @@ -18,6 +18,7 @@ import ( "github.com/oasislabs/oasis-core/go/consensus/tendermint/api" epochtime "github.com/oasislabs/oasis-core/go/epochtime/api" staking "github.com/oasislabs/oasis-core/go/staking/api" + mkvs "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel" ) var ( @@ -76,48 +77,57 @@ type ImmutableState struct { *abci.ImmutableState } -func (s *ImmutableState) TotalSupply() (*quantity.Quantity, error) { - _, value := s.Snapshot.Get(totalSupplyKeyFmt.Encode()) +func (s *ImmutableState) TotalSupply(ctx context.Context) (*quantity.Quantity, error) { + value, err := s.Tree.Get(ctx, totalSupplyKeyFmt.Encode()) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if value == nil { return &quantity.Quantity{}, nil } var q quantity.Quantity if err := cbor.Unmarshal(value, &q); err != nil { - return nil, err + return nil, abci.UnavailableStateError(err) } - return &q, nil } // CommonPool returns the balance of the global common pool. -func (s *ImmutableState) CommonPool() (*quantity.Quantity, error) { - _, value := s.Snapshot.Get(commonPoolKeyFmt.Encode()) +func (s *ImmutableState) CommonPool(ctx context.Context) (*quantity.Quantity, error) { + value, err := s.Tree.Get(ctx, commonPoolKeyFmt.Encode()) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if value == nil { return &quantity.Quantity{}, nil } var q quantity.Quantity if err := cbor.Unmarshal(value, &q); err != nil { - return nil, err + return nil, abci.UnavailableStateError(err) } - return &q, nil } -func (s *ImmutableState) ConsensusParameters() (*staking.ConsensusParameters, error) { - _, raw := s.Snapshot.Get(parametersKeyFmt.Encode()) +func (s *ImmutableState) ConsensusParameters(ctx context.Context) (*staking.ConsensusParameters, error) { + raw, err := s.Tree.Get(ctx, parametersKeyFmt.Encode()) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if raw == nil { return nil, errors.New("tendermint/staking: expected consensus parameters to be present in app state") } var params staking.ConsensusParameters - err := cbor.Unmarshal(raw, ¶ms) + if err = cbor.Unmarshal(raw, ¶ms); err != nil { + return nil, abci.UnavailableStateError(err) + } return ¶ms, err } -func (s *ImmutableState) DebondingInterval() (epochtime.EpochTime, error) { - params, err := s.ConsensusParameters() +func (s *ImmutableState) DebondingInterval(ctx context.Context) (epochtime.EpochTime, error) { + params, err := s.ConsensusParameters(ctx) if err != nil { return epochtime.EpochInvalid, err } @@ -125,8 +135,8 @@ func (s *ImmutableState) DebondingInterval() (epochtime.EpochTime, error) { return params.DebondingInterval, nil } -func (s *ImmutableState) RewardSchedule() ([]staking.RewardStep, error) { - params, err := s.ConsensusParameters() +func (s *ImmutableState) RewardSchedule(ctx context.Context) ([]staking.RewardStep, error) { + params, err := s.ConsensusParameters(ctx) if err != nil { return nil, err } @@ -134,8 +144,8 @@ func (s *ImmutableState) RewardSchedule() ([]staking.RewardStep, error) { return params.RewardSchedule, nil } -func (s *ImmutableState) CommissionScheduleRules() (*staking.CommissionScheduleRules, error) { - params, err := s.ConsensusParameters() +func (s *ImmutableState) CommissionScheduleRules(ctx context.Context) (*staking.CommissionScheduleRules, error) { + params, err := s.ConsensusParameters(ctx) if err != nil { return nil, err } @@ -144,8 +154,8 @@ func (s *ImmutableState) CommissionScheduleRules() (*staking.CommissionScheduleR } // Thresholds returns the currently configured thresholds if any. -func (s *ImmutableState) Thresholds() (map[staking.ThresholdKind]quantity.Quantity, error) { - params, err := s.ConsensusParameters() +func (s *ImmutableState) Thresholds(ctx context.Context) (map[staking.ThresholdKind]quantity.Quantity, error) { + params, err := s.ConsensusParameters(ctx) if err != nil { return nil, err } @@ -153,158 +163,170 @@ func (s *ImmutableState) Thresholds() (map[staking.ThresholdKind]quantity.Quanti return params.Thresholds, nil } -func (s *ImmutableState) Accounts() ([]signature.PublicKey, error) { - var accounts []signature.PublicKey - s.Snapshot.IterateRange( - accountKeyFmt.Encode(), - nil, - true, - func(key, value []byte) bool { - var id signature.PublicKey - if !accountKeyFmt.Decode(key, &id) { - return true - } - - accounts = append(accounts, id) +func (s *ImmutableState) Accounts(ctx context.Context) ([]signature.PublicKey, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() - return false - }, - ) + var accounts []signature.PublicKey + for it.Seek(accountKeyFmt.Encode()); it.Valid(); it.Next() { + var id signature.PublicKey + if !accountKeyFmt.Decode(it.Key(), &id) { + break + } + accounts = append(accounts, id) + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } return accounts, nil } -func (s *ImmutableState) Account(id signature.PublicKey) *staking.Account { - _, value := s.Snapshot.Get(accountKeyFmt.Encode(&id)) +func (s *ImmutableState) Account(ctx context.Context, id signature.PublicKey) (*staking.Account, error) { + value, err := s.Tree.Get(ctx, accountKeyFmt.Encode(&id)) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if value == nil { - return &staking.Account{} + return &staking.Account{}, nil } var ent staking.Account - if err := cbor.Unmarshal(value, &ent); err != nil { - panic("staking: corrupt account state: " + err.Error()) + if err = cbor.Unmarshal(value, &ent); err != nil { + return nil, abci.UnavailableStateError(err) } - return &ent + return &ent, nil } // EscrowBalance returns the escrow balance for the ID. -func (s *ImmutableState) EscrowBalance(id signature.PublicKey) *quantity.Quantity { - account := s.Account(id) - - return account.Escrow.Active.Balance.Clone() +func (s *ImmutableState) EscrowBalance(ctx context.Context, id signature.PublicKey) (*quantity.Quantity, error) { + account, err := s.Account(ctx, id) + if err != nil { + return nil, err + } + return &account.Escrow.Active.Balance, nil } -func (s *ImmutableState) Delegations() (map[signature.PublicKey]map[signature.PublicKey]*staking.Delegation, error) { - delegations := make(map[signature.PublicKey]map[signature.PublicKey]*staking.Delegation) - s.Snapshot.IterateRange( - delegationKeyFmt.Encode(), - nil, - true, - func(key, value []byte) bool { - var escrowID signature.PublicKey - var delegatorID signature.PublicKey - if !delegationKeyFmt.Decode(key, &escrowID, &delegatorID) { - return true - } +func (s *ImmutableState) Delegations(ctx context.Context) (map[signature.PublicKey]map[signature.PublicKey]*staking.Delegation, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() - var del staking.Delegation - if err := cbor.Unmarshal(value, &del); err != nil { - panic("staking: corrupt delegation state: " + err.Error()) - } - - if delegations[escrowID] == nil { - delegations[escrowID] = make(map[signature.PublicKey]*staking.Delegation) - } - delegations[escrowID][delegatorID] = &del + delegations := make(map[signature.PublicKey]map[signature.PublicKey]*staking.Delegation) + for it.Seek(delegationKeyFmt.Encode()); it.Valid(); it.Next() { + var escrowID signature.PublicKey + var delegatorID signature.PublicKey + if !delegationKeyFmt.Decode(it.Key(), &escrowID, &delegatorID) { + break + } - return false - }, - ) + var del staking.Delegation + if err := cbor.Unmarshal(it.Value(), &del); err != nil { + return nil, abci.UnavailableStateError(err) + } + if delegations[escrowID] == nil { + delegations[escrowID] = make(map[signature.PublicKey]*staking.Delegation) + } + delegations[escrowID][delegatorID] = &del + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } return delegations, nil } -func (s *ImmutableState) Delegation(delegatorID, escrowID signature.PublicKey) *staking.Delegation { - _, value := s.Snapshot.Get(delegationKeyFmt.Encode(&escrowID, &delegatorID)) +func (s *ImmutableState) Delegation(ctx context.Context, delegatorID, escrowID signature.PublicKey) (*staking.Delegation, error) { + value, err := s.Tree.Get(ctx, delegationKeyFmt.Encode(&escrowID, &delegatorID)) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if value == nil { - return &staking.Delegation{} + return &staking.Delegation{}, nil } var del staking.Delegation - if err := cbor.Unmarshal(value, &del); err != nil { - panic("staking: corrupt delegation state: " + err.Error()) + if err = cbor.Unmarshal(value, &del); err != nil { + return nil, abci.UnavailableStateError(err) } - return &del + return &del, nil } -func (s *ImmutableState) DebondingDelegations() (map[signature.PublicKey]map[signature.PublicKey][]*staking.DebondingDelegation, error) { - delegations := make(map[signature.PublicKey]map[signature.PublicKey][]*staking.DebondingDelegation) - s.Snapshot.IterateRange( - debondingDelegationKeyFmt.Encode(), - nil, - true, - func(key, value []byte) bool { - var escrowID signature.PublicKey - var delegatorID signature.PublicKey - if !debondingDelegationKeyFmt.Decode(key, &delegatorID, &escrowID) { - return true - } - - var deb staking.DebondingDelegation - if err := cbor.Unmarshal(value, &deb); err != nil { - panic("staking: corrupt debonding delegation state: " + err.Error()) - } +func (s *ImmutableState) DebondingDelegations( + ctx context.Context, +) (map[signature.PublicKey]map[signature.PublicKey][]*staking.DebondingDelegation, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() - if delegations[escrowID] == nil { - delegations[escrowID] = make(map[signature.PublicKey][]*staking.DebondingDelegation) - } - delegations[escrowID][delegatorID] = append(delegations[escrowID][delegatorID], &deb) + delegations := make(map[signature.PublicKey]map[signature.PublicKey][]*staking.DebondingDelegation) + for it.Seek(debondingDelegationKeyFmt.Encode()); it.Valid(); it.Next() { + var escrowID signature.PublicKey + var delegatorID signature.PublicKey + if !debondingDelegationKeyFmt.Decode(it.Key(), &delegatorID, &escrowID) { + break + } - return false - }, - ) + var deb staking.DebondingDelegation + if err := cbor.Unmarshal(it.Value(), &deb); err != nil { + return nil, abci.UnavailableStateError(err) + } + if delegations[escrowID] == nil { + delegations[escrowID] = make(map[signature.PublicKey][]*staking.DebondingDelegation) + } + delegations[escrowID][delegatorID] = append(delegations[escrowID][delegatorID], &deb) + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } return delegations, nil } -func (s *ImmutableState) DebondingDelegationsFor(delegatorID signature.PublicKey) (map[signature.PublicKey][]*staking.DebondingDelegation, error) { - delegations := make(map[signature.PublicKey][]*staking.DebondingDelegation) - s.Snapshot.IterateRange( - debondingDelegationKeyFmt.Encode(&delegatorID), - nil, - true, - func(key, value []byte) bool { - var escrowID signature.PublicKey - var decDelegatorID signature.PublicKey - if !debondingDelegationKeyFmt.Decode(key, &decDelegatorID, &escrowID) || !decDelegatorID.Equal(delegatorID) { - return true - } +func (s *ImmutableState) DebondingDelegationsFor( + ctx context.Context, + delegatorID signature.PublicKey, +) (map[signature.PublicKey][]*staking.DebondingDelegation, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() - var deb staking.DebondingDelegation - if err := cbor.Unmarshal(value, &deb); err != nil { - panic("staking: corrupt debonding delegation state: " + err.Error()) - } - - delegations[escrowID] = append(delegations[escrowID], &deb) + delegations := make(map[signature.PublicKey][]*staking.DebondingDelegation) + for it.Seek(debondingDelegationKeyFmt.Encode(&delegatorID)); it.Valid(); it.Next() { + var escrowID signature.PublicKey + var decDelegatorID signature.PublicKey + if !debondingDelegationKeyFmt.Decode(it.Key(), &decDelegatorID, &escrowID) || !decDelegatorID.Equal(delegatorID) { + break + } - return false - }, - ) + var deb staking.DebondingDelegation + if err := cbor.Unmarshal(it.Value(), &deb); err != nil { + return nil, abci.UnavailableStateError(err) + } + delegations[escrowID] = append(delegations[escrowID], &deb) + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } return delegations, nil } -func (s *ImmutableState) DebondingDelegation(delegatorID, escrowID signature.PublicKey, seq uint64) *staking.DebondingDelegation { - _, value := s.Snapshot.Get(debondingDelegationKeyFmt.Encode(&delegatorID, &escrowID, seq)) +func (s *ImmutableState) DebondingDelegation( + ctx context.Context, + delegatorID, escrowID signature.PublicKey, + seq uint64, +) (*staking.DebondingDelegation, error) { + value, err := s.Tree.Get(ctx, debondingDelegationKeyFmt.Encode(&delegatorID, &escrowID, seq)) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if value == nil { - return &staking.DebondingDelegation{} + return &staking.DebondingDelegation{}, nil } var deb staking.DebondingDelegation - if err := cbor.Unmarshal(value, &deb); err != nil { - panic("staking: corrupt debonding delegation state: " + err.Error()) + if err = cbor.Unmarshal(value, &deb); err != nil { + return nil, abci.UnavailableStateError(err) } - return &deb + return &deb, nil } type DebondingQueueEntry struct { @@ -315,37 +337,39 @@ type DebondingQueueEntry struct { Delegation *staking.DebondingDelegation } -func (s *ImmutableState) ExpiredDebondingQueue(epoch epochtime.EpochTime) []*DebondingQueueEntry { - var entries []*DebondingQueueEntry - s.Snapshot.IterateRange( - debondingQueueKeyFmt.Encode(), - debondingQueueKeyFmt.Encode(uint64(epoch)+1), - true, - func(key, value []byte) bool { - var decEpoch, seq uint64 - var escrowID signature.PublicKey - var delegatorID signature.PublicKey - if !debondingQueueKeyFmt.Decode(key, &decEpoch, &delegatorID, &escrowID, &seq) { - return true - } +func (s *ImmutableState) ExpiredDebondingQueue(ctx context.Context, epoch epochtime.EpochTime) ([]*DebondingQueueEntry, error) { + it := s.Tree.NewIterator(ctx) + defer it.Close() - deb := s.DebondingDelegation(delegatorID, escrowID, seq) - entries = append(entries, &DebondingQueueEntry{ - Epoch: epochtime.EpochTime(decEpoch), - DelegatorID: delegatorID, - EscrowID: escrowID, - Seq: seq, - Delegation: deb, - }) + var entries []*DebondingQueueEntry + for it.Seek(debondingQueueKeyFmt.Encode()); it.Valid(); it.Next() { + var decEpoch, seq uint64 + var escrowID signature.PublicKey + var delegatorID signature.PublicKey + if !debondingQueueKeyFmt.Decode(it.Key(), &decEpoch, &delegatorID, &escrowID, &seq) || decEpoch > uint64(epoch) { + break + } - return false - }, - ) - return entries + deb, err := s.DebondingDelegation(ctx, delegatorID, escrowID, seq) + if err != nil { + return nil, err + } + entries = append(entries, &DebondingQueueEntry{ + Epoch: epochtime.EpochTime(decEpoch), + DelegatorID: delegatorID, + EscrowID: escrowID, + Seq: seq, + Delegation: deb, + }) + } + if it.Err() != nil { + return nil, abci.UnavailableStateError(it.Err()) + } + return entries, nil } -func (s *ImmutableState) Slashing() (map[staking.SlashReason]staking.Slash, error) { - params, err := s.ConsensusParameters() +func (s *ImmutableState) Slashing(ctx context.Context) (map[staking.SlashReason]staking.Slash, error) { + params, err := s.ConsensusParameters(ctx) if err != nil { return nil, err } @@ -353,17 +377,19 @@ func (s *ImmutableState) Slashing() (map[staking.SlashReason]staking.Slash, erro return params.Slashing, nil } -func (s *ImmutableState) LastBlockFees() (*quantity.Quantity, error) { - _, value := s.Snapshot.Get(lastBlockFeesKeyFmt.Encode()) +func (s *ImmutableState) LastBlockFees(ctx context.Context) (*quantity.Quantity, error) { + value, err := s.Tree.Get(ctx, lastBlockFeesKeyFmt.Encode()) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if value == nil { return &quantity.Quantity{}, nil } var q quantity.Quantity - if err := cbor.Unmarshal(value, &q); err != nil { - return nil, err + if err = cbor.Unmarshal(value, &q); err != nil { + return nil, abci.UnavailableStateError(err) } - return &q, nil } @@ -411,8 +437,11 @@ func (es *EpochSigning) EligibleEntities(thresholdNumerator, thresholdDenominato return eligibleEntities, nil } -func (s *ImmutableState) EpochSigning() (*EpochSigning, error) { - _, value := s.Snapshot.Get(epochSigningKeyFmt.Encode()) +func (s *ImmutableState) EpochSigning(ctx context.Context) (*EpochSigning, error) { + value, err := s.Tree.Get(ctx, epochSigningKeyFmt.Encode()) + if err != nil { + return nil, abci.UnavailableStateError(err) + } if value == nil { // Not present means zero everything. return &EpochSigning{ @@ -421,15 +450,14 @@ func (s *ImmutableState) EpochSigning() (*EpochSigning, error) { } var es EpochSigning - if err := cbor.Unmarshal(value, &es); err != nil { - return nil, err + if err = cbor.Unmarshal(value, &es); err != nil { + return nil, abci.UnavailableStateError(err) } - return &es, nil } -func NewImmutableState(state abci.ApplicationState, version int64) (*ImmutableState, error) { - inner, err := abci.NewImmutableState(state, version) +func NewImmutableState(ctx context.Context, state abci.ApplicationState, version int64) (*ImmutableState, error) { + inner, err := abci.NewImmutableState(ctx, state, version) if err != nil { return nil, err } @@ -440,65 +468,87 @@ func NewImmutableState(state abci.ApplicationState, version int64) (*ImmutableSt // MutableState is a mutable staking state wrapper. type MutableState struct { *ImmutableState - - tree *iavl.MutableTree } -func (s *MutableState) SetAccount(id signature.PublicKey, account *staking.Account) { - s.tree.Set(accountKeyFmt.Encode(&id), cbor.Marshal(account)) +func (s *MutableState) SetAccount(ctx context.Context, id signature.PublicKey, account *staking.Account) error { + err := s.Tree.Insert(ctx, accountKeyFmt.Encode(&id), cbor.Marshal(account)) + return abci.UnavailableStateError(err) } -func (s *MutableState) SetTotalSupply(q *quantity.Quantity) { - s.tree.Set(totalSupplyKeyFmt.Encode(), cbor.Marshal(q)) +func (s *MutableState) SetTotalSupply(ctx context.Context, q *quantity.Quantity) error { + err := s.Tree.Insert(ctx, totalSupplyKeyFmt.Encode(), cbor.Marshal(q)) + return abci.UnavailableStateError(err) } -func (s *MutableState) SetCommonPool(q *quantity.Quantity) { - s.tree.Set(commonPoolKeyFmt.Encode(), cbor.Marshal(q)) +func (s *MutableState) SetCommonPool(ctx context.Context, q *quantity.Quantity) error { + err := s.Tree.Insert(ctx, commonPoolKeyFmt.Encode(), cbor.Marshal(q)) + return abci.UnavailableStateError(err) } -func (s *MutableState) SetConsensusParameters(params *staking.ConsensusParameters) { - s.tree.Set(parametersKeyFmt.Encode(), cbor.Marshal(params)) +func (s *MutableState) SetConsensusParameters(ctx context.Context, params *staking.ConsensusParameters) error { + err := s.Tree.Insert(ctx, parametersKeyFmt.Encode(), cbor.Marshal(params)) + return abci.UnavailableStateError(err) } -func (s *MutableState) SetDelegation(delegatorID, escrowID signature.PublicKey, d *staking.Delegation) { +func (s *MutableState) SetDelegation(ctx context.Context, delegatorID, escrowID signature.PublicKey, d *staking.Delegation) error { // Remove delegation if there are no more shares in it. if d.Shares.IsZero() { - s.tree.Remove(delegationKeyFmt.Encode(&escrowID, &delegatorID)) - return + err := s.Tree.Remove(ctx, delegationKeyFmt.Encode(&escrowID, &delegatorID)) + return abci.UnavailableStateError(err) } - s.tree.Set(delegationKeyFmt.Encode(&escrowID, &delegatorID), cbor.Marshal(d)) + err := s.Tree.Insert(ctx, delegationKeyFmt.Encode(&escrowID, &delegatorID), cbor.Marshal(d)) + return abci.UnavailableStateError(err) } -func (s *MutableState) SetDebondingDelegation(delegatorID, escrowID signature.PublicKey, seq uint64, d *staking.DebondingDelegation) { +func (s *MutableState) SetDebondingDelegation( + ctx context.Context, + delegatorID, escrowID signature.PublicKey, + seq uint64, + d *staking.DebondingDelegation, +) error { key := debondingDelegationKeyFmt.Encode(&delegatorID, &escrowID, seq) if d == nil { // Remove descriptor. - s.tree.Remove(key) - return + err := s.Tree.Remove(ctx, key) + return abci.UnavailableStateError(err) } // Add to debonding queue. - s.tree.Set(debondingQueueKeyFmt.Encode(uint64(d.DebondEndTime), &delegatorID, &escrowID, seq), []byte{}) + if err := s.Tree.Insert(ctx, debondingQueueKeyFmt.Encode(uint64(d.DebondEndTime), &delegatorID, &escrowID, seq), []byte{}); err != nil { + return abci.UnavailableStateError(err) + } // Add descriptor. - s.tree.Set(key, cbor.Marshal(d)) + if err := s.Tree.Insert(ctx, key, cbor.Marshal(d)); err != nil { + return abci.UnavailableStateError(err) + } + return nil } -func (s *MutableState) RemoveFromDebondingQueue(epoch epochtime.EpochTime, delegatorID, escrowID signature.PublicKey, seq uint64) { - s.tree.Remove(debondingQueueKeyFmt.Encode(uint64(epoch), &delegatorID, &escrowID, seq)) +func (s *MutableState) RemoveFromDebondingQueue( + ctx context.Context, + epoch epochtime.EpochTime, + delegatorID, escrowID signature.PublicKey, + seq uint64, +) error { + err := s.Tree.Remove(ctx, debondingQueueKeyFmt.Encode(uint64(epoch), &delegatorID, &escrowID, seq)) + return abci.UnavailableStateError(err) } -func (s *MutableState) SetLastBlockFees(q *quantity.Quantity) { - s.tree.Set(lastBlockFeesKeyFmt.Encode(), cbor.Marshal(q)) +func (s *MutableState) SetLastBlockFees(ctx context.Context, q *quantity.Quantity) error { + err := s.Tree.Insert(ctx, lastBlockFeesKeyFmt.Encode(), cbor.Marshal(q)) + return abci.UnavailableStateError(err) } -func (s *MutableState) SetEpochSigning(es *EpochSigning) { - s.tree.Set(epochSigningKeyFmt.Encode(), cbor.Marshal(es)) +func (s *MutableState) SetEpochSigning(ctx context.Context, es *EpochSigning) error { + err := s.Tree.Insert(ctx, epochSigningKeyFmt.Encode(), cbor.Marshal(es)) + return abci.UnavailableStateError(err) } -func (s *MutableState) ClearEpochSigning() { - s.tree.Remove(epochSigningKeyFmt.Encode()) +func (s *MutableState) ClearEpochSigning(ctx context.Context) error { + err := s.Tree.Remove(ctx, epochSigningKeyFmt.Encode()) + return abci.UnavailableStateError(err) } func slashPool(dst *quantity.Quantity, p *staking.SharePool, amount, total *quantity.Quantity) error { @@ -525,12 +575,15 @@ func slashPool(dst *quantity.Quantity, p *staking.SharePool, amount, total *quan // WARNING: This is an internal routine to be used to implement staking policy, // and MUST NOT be exposed outside of backend implementations. func (s *MutableState) SlashEscrow(ctx *abci.Context, fromID signature.PublicKey, amount *quantity.Quantity) (bool, error) { - commonPool, err := s.CommonPool() + commonPool, err := s.CommonPool(ctx) if err != nil { return false, fmt.Errorf("staking: failed to query common pool for slash: %w", err) } - from := s.Account(fromID) + from, err := s.Account(ctx, fromID) + if err != nil { + return false, fmt.Errorf("staking: failed to query account %s: %w", fromID, err) + } // Compute the amount we need to slash each pool. The amount is split // between the pools based on relative total balance. @@ -557,8 +610,12 @@ func (s *MutableState) SlashEscrow(ctx *abci.Context, fromID signature.PublicKey return false, errors.Wrap(err, "moving tokens to common pool") } - s.SetCommonPool(commonPool) - s.SetAccount(fromID, from) + if err = s.SetCommonPool(ctx, commonPool); err != nil { + return false, fmt.Errorf("failed to set common pool: %w", err) + } + if err = s.SetAccount(ctx, fromID, from); err != nil { + return false, fmt.Errorf("failed to set account. %w", err) + } if !ctx.IsCheckOnly() { ev := cbor.Marshal(&staking.TakeEscrowEvent{ @@ -578,12 +635,15 @@ func (s *MutableState) SlashEscrow(ctx *abci.Context, fromID signature.PublicKey // WARNING: This is an internal routine to be used to implement incentivization // policy, and MUST NOT be exposed outside of backend implementations. func (s *MutableState) TransferFromCommon(ctx *abci.Context, toID signature.PublicKey, amount *quantity.Quantity) (bool, error) { - commonPool, err := s.CommonPool() + commonPool, err := s.CommonPool(ctx) if err != nil { return false, errors.Wrap(err, "staking: failed to query common pool for transfer") } - to := s.Account(toID) + to, err := s.Account(ctx, toID) + if err != nil { + return false, fmt.Errorf("failed to query account %s: %w", toID, err) + } transfered, err := quantity.MoveUpTo(&to.General.Balance, commonPool, amount) if err != nil { return false, errors.Wrap(err, "staking: failed to transfer from common pool") @@ -591,8 +651,12 @@ func (s *MutableState) TransferFromCommon(ctx *abci.Context, toID signature.Publ ret := !transfered.IsZero() if ret { - s.SetCommonPool(commonPool) - s.SetAccount(toID, to) + if err = s.SetCommonPool(ctx, commonPool); err != nil { + return false, fmt.Errorf("failed to set common pool: %w", err) + } + if err = s.SetAccount(ctx, toID, to); err != nil { + return false, fmt.Errorf("failed to set account %s: %w", toID, err) + } if !ctx.IsCheckOnly() { ev := cbor.Marshal(&staking.TransferEvent{ @@ -613,8 +677,13 @@ func (s *MutableState) TransferFromCommon(ctx *abci.Context, toID signature.Publ // returned error's cause will be `staking.ErrInsufficientBalance`, and it should // be safe for the caller to roll back to an earlier state tree and continue from // there. -func (s *MutableState) AddRewards(time epochtime.EpochTime, factor *quantity.Quantity, accounts []signature.PublicKey) error { - steps, err := s.RewardSchedule() +func (s *MutableState) AddRewards( + ctx context.Context, + time epochtime.EpochTime, + factor *quantity.Quantity, + accounts []signature.PublicKey, +) error { + steps, err := s.RewardSchedule(ctx) if err != nil { return err } @@ -630,23 +699,27 @@ func (s *MutableState) AddRewards(time epochtime.EpochTime, factor *quantity.Qua return nil } - commonPool, err := s.CommonPool() + commonPool, err := s.CommonPool(ctx) if err != nil { - return errors.Wrap(err, "loading common pool") + return fmt.Errorf("loading common pool: %w", err) } for _, id := range accounts { - ent := s.Account(id) + var ent *staking.Account + ent, err = s.Account(ctx, id) + if err != nil { + return fmt.Errorf("failed to fetch account %s: %w", id, err) + } q := ent.Escrow.Active.Balance.Clone() // Multiply first. - if err := q.Mul(factor); err != nil { + if err = q.Mul(factor); err != nil { return errors.Wrap(err, "multiplying by reward factor") } - if err := q.Mul(&activeStep.Scale); err != nil { + if err = q.Mul(&activeStep.Scale); err != nil { return errors.Wrap(err, "multiplying by reward step scale") } - if err := q.Quo(staking.RewardAmountDenominator); err != nil { + if err = q.Quo(staking.RewardAmountDenominator); err != nil { return errors.Wrap(err, "dividing by reward amount denominator") } @@ -659,47 +732,63 @@ func (s *MutableState) AddRewards(time epochtime.EpochTime, factor *quantity.Qua if rate != nil { com = q.Clone() // Multiply first. - if err := com.Mul(rate); err != nil { + if err = com.Mul(rate); err != nil { return errors.Wrap(err, "multiplying by commission rate") } - if err := com.Quo(staking.CommissionRateDenominator); err != nil { + if err = com.Quo(staking.CommissionRateDenominator); err != nil { return errors.Wrap(err, "dividing by commission rate denominator") } - if err := q.Sub(com); err != nil { + if err = q.Sub(com); err != nil { return errors.Wrap(err, "subtracting commission") } } if !q.IsZero() { - if err := quantity.Move(&ent.Escrow.Active.Balance, commonPool, q); err != nil { + if err = quantity.Move(&ent.Escrow.Active.Balance, commonPool, q); err != nil { return errors.Wrap(err, "transferring to active escrow balance from common pool") } } if com != nil && !com.IsZero() { - delegation := s.Delegation(id, id) + var delegation *staking.Delegation + delegation, err = s.Delegation(ctx, id, id) + if err != nil { + return fmt.Errorf("failed to query delegation: %w", err) + } - if err := ent.Escrow.Active.Deposit(&delegation.Shares, commonPool, com); err != nil { + if err = ent.Escrow.Active.Deposit(&delegation.Shares, commonPool, com); err != nil { return errors.Wrap(err, "depositing commission") } - s.SetDelegation(id, id, delegation) + if err = s.SetDelegation(ctx, id, id, delegation); err != nil { + return fmt.Errorf("failed to set delegation: %w", err) + } } - s.SetAccount(id, ent) + if err = s.SetAccount(ctx, id, ent); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } } - s.SetCommonPool(commonPool) + if err = s.SetCommonPool(ctx, commonPool); err != nil { + return fmt.Errorf("failed to set common pool") + } return nil } // AddRewardSingleAttenuated computes, scales, and transfers a staking reward to an active escrow account. -func (s *MutableState) AddRewardSingleAttenuated(time epochtime.EpochTime, factor *quantity.Quantity, attenuationNumerator, attenuationDenominator int, account signature.PublicKey) error { - steps, err := s.RewardSchedule() +func (s *MutableState) AddRewardSingleAttenuated( + ctx context.Context, + time epochtime.EpochTime, + factor *quantity.Quantity, + attenuationNumerator, attenuationDenominator int, + account signature.PublicKey, +) error { + steps, err := s.RewardSchedule(ctx) if err != nil { - return err + return fmt.Errorf("failed to query reward schedule: %w", err) } var activeStep *staking.RewardStep for _, step := range steps { @@ -721,28 +810,31 @@ func (s *MutableState) AddRewardSingleAttenuated(time epochtime.EpochTime, facto return errors.Wrapf(err, "importing attenuation denominator %d", attenuationDenominator) } - commonPool, err := s.CommonPool() + commonPool, err := s.CommonPool(ctx) if err != nil { - return errors.Wrap(err, "loading common pool") + return fmt.Errorf("loading common pool: %w", err) } - ent := s.Account(account) + ent, err := s.Account(ctx, account) + if err != nil { + return fmt.Errorf("failed to query account %s: %w", account, err) + } q := ent.Escrow.Active.Balance.Clone() // Multiply first. - if err := q.Mul(factor); err != nil { + if err = q.Mul(factor); err != nil { return errors.Wrap(err, "multiplying by reward factor") } - if err := q.Mul(&activeStep.Scale); err != nil { + if err = q.Mul(&activeStep.Scale); err != nil { return errors.Wrap(err, "multiplying by reward step scale") } - if err := q.Mul(&numQ); err != nil { + if err = q.Mul(&numQ); err != nil { return errors.Wrap(err, "multiplying by attenuation numerator") } - if err := q.Quo(staking.RewardAmountDenominator); err != nil { + if err = q.Quo(staking.RewardAmountDenominator); err != nil { return errors.Wrap(err, "dividing by reward amount denominator") } - if err := q.Quo(&denQ); err != nil { + if err = q.Quo(&denQ); err != nil { return errors.Wrap(err, "dividing by attenuation denominator") } @@ -755,47 +847,56 @@ func (s *MutableState) AddRewardSingleAttenuated(time epochtime.EpochTime, facto if rate != nil { com = q.Clone() // Multiply first. - if err := com.Mul(rate); err != nil { + if err = com.Mul(rate); err != nil { return errors.Wrap(err, "multiplying by commission rate") } - if err := com.Quo(staking.CommissionRateDenominator); err != nil { + if err = com.Quo(staking.CommissionRateDenominator); err != nil { return errors.Wrap(err, "dividing by commission rate denominator") } - if err := q.Sub(com); err != nil { + if err = q.Sub(com); err != nil { return errors.Wrap(err, "subtracting commission") } } if !q.IsZero() { - if err := quantity.Move(&ent.Escrow.Active.Balance, commonPool, q); err != nil { + if err = quantity.Move(&ent.Escrow.Active.Balance, commonPool, q); err != nil { return errors.Wrap(err, "transferring to active escrow balance from common pool") } } if com != nil && !com.IsZero() { - delegation := s.Delegation(account, account) + var delegation *staking.Delegation + delegation, err = s.Delegation(ctx, account, account) + if err != nil { + return fmt.Errorf("failed to query delegation: %w", err) + } - if err := ent.Escrow.Active.Deposit(&delegation.Shares, commonPool, com); err != nil { + if err = ent.Escrow.Active.Deposit(&delegation.Shares, commonPool, com); err != nil { return errors.Wrap(err, "depositing commission") } - s.SetDelegation(account, account, delegation) + if err = s.SetDelegation(ctx, account, account, delegation); err != nil { + return fmt.Errorf("failed to set delegation: %w", err) + } } - s.SetAccount(account, ent) + if err = s.SetAccount(ctx, account, ent); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } - s.SetCommonPool(commonPool) + if err = s.SetCommonPool(ctx, commonPool); err != nil { + return fmt.Errorf("failed to set common pool: %w", err) + } return nil } // NewMutableState creates a new mutable staking state wrapper. -func NewMutableState(tree *iavl.MutableTree) *MutableState { - inner := &abci.ImmutableState{Snapshot: tree.ImmutableTree} - +func NewMutableState(tree mkvs.KeyValueTree) *MutableState { return &MutableState{ - ImmutableState: &ImmutableState{inner}, - tree: tree, + ImmutableState: &ImmutableState{ + &abci.ImmutableState{Tree: tree}, + }, } } diff --git a/go/consensus/tendermint/apps/staking/state/state_test.go b/go/consensus/tendermint/apps/staking/state/state_test.go index 996aef7adb4..737b4fb56c8 100644 --- a/go/consensus/tendermint/apps/staking/state/state_test.go +++ b/go/consensus/tendermint/apps/staking/state/state_test.go @@ -7,8 +7,6 @@ import ( "time" "github.com/stretchr/testify/require" - "github.com/tendermint/iavl" - dbm "github.com/tendermint/tm-db" "github.com/oasislabs/oasis-core/go/common/crypto/signature" memorySigner "github.com/oasislabs/oasis-core/go/common/crypto/signature/signers/memory" @@ -69,11 +67,14 @@ func TestRewardAndSlash(t *testing.T) { deb.DebondEndTime = 21 require.NoError(t, escrowAccount.Escrow.Debonding.Deposit(&deb.Shares, &delegatorAccount.General.Balance, mustInitQuantityP(t, 100)), "debonding escrow deposit") - db := dbm.NewMemDB() - tree := iavl.NewMutableTree(db, 128) - s := NewMutableState(tree) + now := time.Unix(1580461674, 0) + appState := abci.NewMockApplicationState(abci.MockApplicationStateConfig{}) + ctx := appState.NewContext(abci.ContextBeginBlock, now) + defer ctx.Close() - s.SetConsensusParameters(&staking.ConsensusParameters{ + s := NewMutableState(ctx.State()) + + err = s.SetConsensusParameters(ctx, &staking.ConsensusParameters{ DebondingInterval: 21, RewardSchedule: []staking.RewardStep{ { @@ -92,81 +93,99 @@ func TestRewardAndSlash(t *testing.T) { MaxBoundSteps: 12, }, }) - s.SetCommonPool(mustInitQuantityP(t, 10000)) - - s.SetAccount(delegatorID, delegatorAccount) - s.SetAccount(escrowID, escrowAccount) - s.SetDelegation(delegatorID, escrowID, del) - s.SetDebondingDelegation(delegatorID, escrowID, 1, deb) + require.NoError(t, err, "SetConsensusParameters") + err = s.SetCommonPool(ctx, mustInitQuantityP(t, 10000)) + require.NoError(t, err, "SetCommonPool") + + err = s.SetAccount(ctx, delegatorID, delegatorAccount) + require.NoError(t, err, "SetAccount") + err = s.SetAccount(ctx, escrowID, escrowAccount) + require.NoError(t, err, "SetAccount") + err = s.SetDelegation(ctx, delegatorID, escrowID, del) + require.NoError(t, err, "SetDelegation") + err = s.SetDebondingDelegation(ctx, delegatorID, escrowID, 1, deb) + require.NoError(t, err, "SetDebondingDelegation") // Epoch 10 is during the first step. - require.NoError(t, s.AddRewards(10, mustInitQuantityP(t, 100), escrowAccountOnly), "add rewards epoch 10") + require.NoError(t, s.AddRewards(ctx, 10, mustInitQuantityP(t, 100), escrowAccountOnly), "add rewards epoch 10") // 100% gain. - delegatorAccount = s.Account(delegatorID) + delegatorAccount, err = s.Account(ctx, delegatorID) + require.NoError(t, err, "Account") require.Equal(t, mustInitQuantity(t, 100), delegatorAccount.General.Balance, "reward first step - delegator general") - escrowAccount = s.Account(escrowID) + escrowAccount, err = s.Account(ctx, escrowID) + require.NoError(t, err, "Account") require.Equal(t, mustInitQuantity(t, 200), escrowAccount.Escrow.Active.Balance, "reward first step - escrow active escrow") require.Equal(t, mustInitQuantity(t, 100), escrowAccount.Escrow.Debonding.Balance, "reward first step - escrow debonding escrow") // Reward is 100 tokens, with 80 added to the pool and 20 deposited as commission. // We add to the pool first, so the delegation becomes 100 shares : 180 tokens. // Then we deposit the 20 for commission, which comes out to 11 shares. - del = s.Delegation(delegatorID, escrowID) + del, err = s.Delegation(ctx, delegatorID, escrowID) + require.NoError(t, err, "Delegation") require.Equal(t, mustInitQuantity(t, 100), del.Shares, "reward first step - delegation shares") - escrowSelfDel := s.Delegation(escrowID, escrowID) + escrowSelfDel, err := s.Delegation(ctx, escrowID, escrowID) + require.NoError(t, err, "Delegation") require.Equal(t, mustInitQuantity(t, 11), escrowSelfDel.Shares, "reward first step - escrow self delegation shares") - commonPool, err := s.CommonPool() + commonPool, err := s.CommonPool(ctx) require.NoError(t, err, "load common pool") require.Equal(t, mustInitQuantityP(t, 9900), commonPool, "reward first step - common pool") // Epoch 30 is in the second step. - require.NoError(t, s.AddRewards(30, mustInitQuantityP(t, 100), escrowAccountOnly), "add rewards epoch 30") + require.NoError(t, s.AddRewards(ctx, 30, mustInitQuantityP(t, 100), escrowAccountOnly), "add rewards epoch 30") // 50% gain. - escrowAccount = s.Account(escrowID) + escrowAccount, err = s.Account(ctx, escrowID) + require.NoError(t, err, "Account") require.Equal(t, mustInitQuantity(t, 300), escrowAccount.Escrow.Active.Balance, "reward boundary epoch - escrow active escrow") - commonPool, err = s.CommonPool() + commonPool, err = s.CommonPool(ctx) require.NoError(t, err, "load common pool") require.Equal(t, mustInitQuantityP(t, 9800), commonPool, "reward first step - common pool") // Epoch 99 is after the end of the schedule - require.NoError(t, s.AddRewards(99, mustInitQuantityP(t, 100), escrowAccountOnly), "add rewards epoch 99") + require.NoError(t, s.AddRewards(ctx, 99, mustInitQuantityP(t, 100), escrowAccountOnly), "add rewards epoch 99") // No change. - escrowAccount = s.Account(escrowID) + escrowAccount, err = s.Account(ctx, escrowID) + require.NoError(t, err, "Account") require.Equal(t, mustInitQuantity(t, 300), escrowAccount.Escrow.Active.Balance, "reward late epoch - escrow active escrow") - slashedNonzero, err := s.SlashEscrow(abci.NewMockContext(abci.ContextDeliverTx, time.Now()), escrowID, mustInitQuantityP(t, 40)) + slashedNonzero, err := s.SlashEscrow(ctx, escrowID, mustInitQuantityP(t, 40)) require.NoError(t, err, "slash escrow") require.True(t, slashedNonzero, "slashed nonzero") // 40 token loss. - delegatorAccount = s.Account(delegatorID) + delegatorAccount, err = s.Account(ctx, delegatorID) + require.NoError(t, err, "Account") require.Equal(t, mustInitQuantity(t, 100), delegatorAccount.General.Balance, "slash - delegator general") - escrowAccount = s.Account(escrowID) + escrowAccount, err = s.Account(ctx, escrowID) + require.NoError(t, err, "Account") require.Equal(t, mustInitQuantity(t, 270), escrowAccount.Escrow.Active.Balance, "slash - escrow active escrow") require.Equal(t, mustInitQuantity(t, 90), escrowAccount.Escrow.Debonding.Balance, "slash - escrow debonding escrow") - commonPool, err = s.CommonPool() + commonPool, err = s.CommonPool(ctx) require.NoError(t, err, "load common pool") require.Equal(t, mustInitQuantityP(t, 9840), commonPool, "slash - common pool") // Epoch 10 is during the first step. - require.NoError(t, s.AddRewardSingleAttenuated(10, mustInitQuantityP(t, 10), 5, 10, escrowID), "add attenuated rewards epoch 30") + require.NoError(t, s.AddRewardSingleAttenuated(ctx, 10, mustInitQuantityP(t, 10), 5, 10, escrowID), "add attenuated rewards epoch 30") // 5% gain. - escrowAccount = s.Account(escrowID) + escrowAccount, err = s.Account(ctx, escrowID) + require.NoError(t, err, "Account") require.Equal(t, mustInitQuantity(t, 283), escrowAccount.Escrow.Active.Balance, "attenuated reward - escrow active escrow") - commonPool, err = s.CommonPool() + commonPool, err = s.CommonPool(ctx) require.NoError(t, err, "load common pool") require.Equal(t, mustInitQuantityP(t, 9827), commonPool, "reward attenuated - common pool") } func TestEpochSigning(t *testing.T) { - db := dbm.NewMemDB() - tree := iavl.NewMutableTree(db, 128) - s := NewMutableState(tree) + now := time.Unix(1580461674, 0) + appState := abci.NewMockApplicationState(abci.MockApplicationStateConfig{}) + ctx := appState.NewContext(abci.ContextBeginBlock, now) + defer ctx.Close() + + s := NewMutableState(ctx.State()) - es, err := s.EpochSigning() + es, err := s.EpochSigning(ctx) require.NoError(t, err, "load epoch signing info") require.Zero(t, es.Total, "empty epoch signing info total") require.Empty(t, es.ByEntity, "empty epoch signing info by entity") @@ -183,8 +202,9 @@ func TestEpochSigning(t *testing.T) { require.EqualValues(t, 4, es.Total, "populated epoch signing info total") require.Len(t, es.ByEntity, 3, "populated epoch signing info by entity") - s.SetEpochSigning(es) - esRoundTrip, err := s.EpochSigning() + err = s.SetEpochSigning(ctx, es) + require.NoError(t, err, "SetEpochSigning") + esRoundTrip, err := s.EpochSigning(ctx) require.NoError(t, err, "load epoch signing info 2") require.Equal(t, es, esRoundTrip, "epoch signing info round trip") @@ -195,8 +215,9 @@ func TestEpochSigning(t *testing.T) { require.Contains(t, eligibleEntities, exact, "'exact' eligible") require.Contains(t, eligibleEntities, perfect, "'perfect' eligible") - s.ClearEpochSigning() - esClear, err := s.EpochSigning() + err = s.ClearEpochSigning(ctx) + require.NoError(t, err, "ClearEpochSigning") + esClear, err := s.EpochSigning(ctx) require.NoError(t, err, "load cleared epoch signing info") require.Zero(t, esClear.Total, "cleared epoch signing info total") require.Empty(t, esClear.ByEntity, "cleared epoch signing info by entity") diff --git a/go/consensus/tendermint/apps/staking/transactions.go b/go/consensus/tendermint/apps/staking/transactions.go index 7538c395557..b3309325452 100644 --- a/go/consensus/tendermint/apps/staking/transactions.go +++ b/go/consensus/tendermint/apps/staking/transactions.go @@ -33,20 +33,23 @@ func (app *stakingApplication) transfer(ctx *abci.Context, state *stakingState.M } // Charge gas for this transaction. - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { - return err + return fmt.Errorf("failed to fetch consensus parameters: %w", err) } if err = ctx.Gas().UseGas(1, staking.GasOpTransfer, params.GasCosts); err != nil { return err } fromID := ctx.TxSigner() - epoch, err := app.state.GetCurrentEpoch(ctx.Ctx()) + epoch, err := app.state.GetCurrentEpoch(ctx) if err != nil { return fmt.Errorf("getting current epoch: %w", err) } - from := state.Account(fromID) + from, err := state.Account(ctx, fromID) + if err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } if !isTransferPermitted(params, fromID, from, epoch) { return staking.ErrForbidden } @@ -54,7 +57,7 @@ func (app *stakingApplication) transfer(ctx *abci.Context, state *stakingState.M if fromID.Equal(xfer.To) { // Handle transfer to self as just a balance check. if from.General.Balance.Cmp(&xfer.Tokens) < 0 { - err := staking.ErrInsufficientBalance + err = staking.ErrInsufficientBalance ctx.Logger().Error("Transfer: self-transfer greater than balance", "err", err, "from", fromID, @@ -66,8 +69,12 @@ func (app *stakingApplication) transfer(ctx *abci.Context, state *stakingState.M } else { // Source and destination MUST be separate accounts with how // quantity.Move is implemented. - to := state.Account(xfer.To) - if err := quantity.Move(&to.General.Balance, &from.General.Balance, &xfer.Tokens); err != nil { + var to *staking.Account + to, err = state.Account(ctx, xfer.To) + if err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } + if err = quantity.Move(&to.General.Balance, &from.General.Balance, &xfer.Tokens); err != nil { ctx.Logger().Error("Transfer: failed to move balance", "err", err, "from", fromID, @@ -77,10 +84,14 @@ func (app *stakingApplication) transfer(ctx *abci.Context, state *stakingState.M return err } - state.SetAccount(xfer.To, to) + if err = state.SetAccount(ctx, xfer.To, to); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } } - state.SetAccount(fromID, from) + if err = state.SetAccount(ctx, fromID, from); err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } ctx.Logger().Debug("Transfer: executed transfer", "from", fromID, @@ -104,18 +115,21 @@ func (app *stakingApplication) burn(ctx *abci.Context, state *stakingState.Mutab } // Charge gas for this transaction. - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { - return err + return fmt.Errorf("failed to fetch consensus parameters: %w", err) } - if err := ctx.Gas().UseGas(1, staking.GasOpBurn, params.GasCosts); err != nil { + if err = ctx.Gas().UseGas(1, staking.GasOpBurn, params.GasCosts); err != nil { return err } id := ctx.TxSigner() - from := state.Account(id) + from, err := state.Account(ctx, id) + if err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } - if err := from.General.Balance.Sub(&burn.Tokens); err != nil { + if err = from.General.Balance.Sub(&burn.Tokens); err != nil { ctx.Logger().Error("Burn: failed to burn tokens", "err", err, "from", id, "amount", burn.Tokens, @@ -123,12 +137,19 @@ func (app *stakingApplication) burn(ctx *abci.Context, state *stakingState.Mutab return err } - totalSupply, _ := state.TotalSupply() + totalSupply, err := state.TotalSupply(ctx) + if err != nil { + return fmt.Errorf("failed to fetch total supply: %w", err) + } _ = totalSupply.Sub(&burn.Tokens) - state.SetAccount(id, from) - state.SetTotalSupply(totalSupply) + if err = state.SetAccount(ctx, id, from); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } + if err = state.SetTotalSupply(ctx, totalSupply); err != nil { + return fmt.Errorf("failed to set total supply: %w", err) + } ctx.Logger().Debug("Burn: burnt tokens", "from", id, @@ -150,11 +171,11 @@ func (app *stakingApplication) addEscrow(ctx *abci.Context, state *stakingState. } // Charge gas for this transaction. - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { - return err + return fmt.Errorf("failed to fetch consensus parameters: %w", err) } - if err := ctx.Gas().UseGas(1, staking.GasOpAddEscrow, params.GasCosts); err != nil { + if err = ctx.Gas().UseGas(1, staking.GasOpAddEscrow, params.GasCosts); err != nil { return err } @@ -164,7 +185,10 @@ func (app *stakingApplication) addEscrow(ctx *abci.Context, state *stakingState. } id := ctx.TxSigner() - from := state.Account(id) + from, err := state.Account(ctx, id) + if err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } // Fetch escrow account. // @@ -177,13 +201,19 @@ func (app *stakingApplication) addEscrow(ctx *abci.Context, state *stakingState. if params.DisableDelegation { return staking.ErrForbidden } - to = state.Account(escrow.Account) + to, err = state.Account(ctx, escrow.Account) + if err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } } // Fetch delegation. - delegation := state.Delegation(id, escrow.Account) + delegation, err := state.Delegation(ctx, id, escrow.Account) + if err != nil { + return fmt.Errorf("failed to fetch delegation: %w", err) + } - if err := to.Escrow.Active.Deposit(&delegation.Shares, &from.General.Balance, &escrow.Tokens); err != nil { + if err = to.Escrow.Active.Deposit(&delegation.Shares, &from.General.Balance, &escrow.Tokens); err != nil { ctx.Logger().Error("AddEscrow: failed to escrow tokens", "err", err, "from", id, @@ -194,12 +224,18 @@ func (app *stakingApplication) addEscrow(ctx *abci.Context, state *stakingState. } // Commit accounts. - state.SetAccount(id, from) + if err = state.SetAccount(ctx, id, from); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } if !id.Equal(escrow.Account) { - state.SetAccount(escrow.Account, to) + if err = state.SetAccount(ctx, escrow.Account, to); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } } // Commit delegation descriptor. - state.SetDelegation(id, escrow.Account, delegation) + if err = state.SetDelegation(ctx, id, escrow.Account, delegation); err != nil { + return fmt.Errorf("failed to set delegation: %w", err) + } ctx.Logger().Debug("AddEscrow: escrowed tokens", "from", id, @@ -228,16 +264,19 @@ func (app *stakingApplication) reclaimEscrow(ctx *abci.Context, state *stakingSt } // Charge gas for this transaction. - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { - return err + return fmt.Errorf("failed to fetch consensus parameters: %w", err) } if err = ctx.Gas().UseGas(1, staking.GasOpReclaimEscrow, params.GasCosts); err != nil { return err } id := ctx.TxSigner() - to := state.Account(id) + to, err := state.Account(ctx, id) + if err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } // Fetch escrow account. // @@ -250,21 +289,27 @@ func (app *stakingApplication) reclaimEscrow(ctx *abci.Context, state *stakingSt if params.DisableDelegation { return staking.ErrForbidden } - from = state.Account(reclaim.Account) + from, err = state.Account(ctx, reclaim.Account) + if err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } } // Fetch delegation. - delegation := state.Delegation(id, reclaim.Account) + delegation, err := state.Delegation(ctx, id, reclaim.Account) + if err != nil { + return fmt.Errorf("failed to fetch delegation: %w", err) + } // Fetch debonding interval and current epoch. - debondingInterval, err := state.DebondingInterval() + debondingInterval, err := state.DebondingInterval(ctx) if err != nil { ctx.Logger().Error("ReclaimEscrow: failed to query debonding interval", "err", err, ) return err } - epoch, err := app.state.GetEpoch(ctx.Ctx(), ctx.BlockHeight()+1) + epoch, err := app.state.GetEpoch(ctx, ctx.BlockHeight()+1) if err != nil { return err } @@ -275,7 +320,7 @@ func (app *stakingApplication) reclaimEscrow(ctx *abci.Context, state *stakingSt var tokens quantity.Quantity - if err := from.Escrow.Active.Withdraw(&tokens, &delegation.Shares, &reclaim.Shares); err != nil { + if err = from.Escrow.Active.Withdraw(&tokens, &delegation.Shares, &reclaim.Shares); err != nil { ctx.Logger().Error("ReclaimEscrow: failed to redeem escrow shares", "err", err, "to", id, @@ -286,7 +331,7 @@ func (app *stakingApplication) reclaimEscrow(ctx *abci.Context, state *stakingSt } tokenAmount := tokens.Clone() - if err := from.Escrow.Debonding.Deposit(&deb.Shares, &tokens, tokenAmount); err != nil { + if err = from.Escrow.Debonding.Deposit(&deb.Shares, &tokens, tokenAmount); err != nil { ctx.Logger().Error("ReclaimEscrow: failed to debond shares", "err", err, "to", id, @@ -303,14 +348,21 @@ func (app *stakingApplication) reclaimEscrow(ctx *abci.Context, state *stakingSt return staking.ErrInvalidArgument } - // Include the nonce as the final disambiguator to prevent overwriting debonding - // delegations. - state.SetDebondingDelegation(id, reclaim.Account, to.General.Nonce, &deb) + // Include the nonce as the final disambiguator to prevent overwriting debonding delegations. + if err = state.SetDebondingDelegation(ctx, id, reclaim.Account, to.General.Nonce, &deb); err != nil { + return fmt.Errorf("failed to set debonding delegation: %w", err) + } - state.SetDelegation(id, reclaim.Account, delegation) - state.SetAccount(id, to) + if err = state.SetDelegation(ctx, id, reclaim.Account, delegation); err != nil { + return fmt.Errorf("failed to set delegation: %w", err) + } + if err = state.SetAccount(ctx, id, to); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } if !id.Equal(reclaim.Account) { - state.SetAccount(reclaim.Account, from) + if err = state.SetAccount(ctx, reclaim.Account, from); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } } return nil @@ -326,21 +378,24 @@ func (app *stakingApplication) amendCommissionSchedule( } // Charge gas for this transaction. - params, err := state.ConsensusParameters() + params, err := state.ConsensusParameters(ctx) if err != nil { - return err + return fmt.Errorf("failed to fetch consensus parameters: %w", err) } if err = ctx.Gas().UseGas(1, staking.GasOpAmendCommissionSchedule, params.GasCosts); err != nil { return err } - epoch, err := app.state.GetEpoch(ctx.Ctx(), ctx.BlockHeight()+1) + epoch, err := app.state.GetEpoch(ctx, ctx.BlockHeight()+1) if err != nil { return err } id := ctx.TxSigner() - from := state.Account(id) + from, err := state.Account(ctx, id) + if err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } if err = from.Escrow.CommissionSchedule.AmendAndPruneAndValidate(&amendCommissionSchedule.Amendment, ¶ms.CommissionScheduleRules, epoch); err != nil { ctx.Logger().Error("AmendCommissionSchedule: amendment not acceptable", @@ -350,7 +405,9 @@ func (app *stakingApplication) amendCommissionSchedule( return err } - state.SetAccount(id, from) + if err = state.SetAccount(ctx, id, from); err != nil { + return fmt.Errorf("failed to set account: %w", err) + } return nil } diff --git a/go/consensus/tendermint/apps/staking/votes.go b/go/consensus/tendermint/apps/staking/votes.go index f93a7c4366a..b8e978ce17b 100644 --- a/go/consensus/tendermint/apps/staking/votes.go +++ b/go/consensus/tendermint/apps/staking/votes.go @@ -19,7 +19,7 @@ func (app *stakingApplication) resolveEntityIDsFromVotes(ctx *abci.Context, regS valAddr := a.Validator.Address // Map address to node/entity. - node, err := regState.NodeByConsensusAddress(valAddr) + node, err := regState.NodeByConsensusAddress(ctx, valAddr) if err != nil { ctx.Logger().Warn("failed to get validator node", "err", err, diff --git a/go/consensus/tendermint/apps/supplementarysanity/checks.go b/go/consensus/tendermint/apps/supplementarysanity/checks.go index cfb26e5c962..390777c6cc8 100644 --- a/go/consensus/tendermint/apps/supplementarysanity/checks.go +++ b/go/consensus/tendermint/apps/supplementarysanity/checks.go @@ -3,11 +3,10 @@ package supplementarysanity import ( "fmt" - "github.com/tendermint/iavl" - "github.com/oasislabs/oasis-core/go/common" "github.com/oasislabs/oasis-core/go/common/crypto/signature" "github.com/oasislabs/oasis-core/go/common/quantity" + "github.com/oasislabs/oasis-core/go/consensus/tendermint/abci" keymanagerState "github.com/oasislabs/oasis-core/go/consensus/tendermint/apps/keymanager/state" registryState "github.com/oasislabs/oasis-core/go/consensus/tendermint/apps/registry/state" roothashState "github.com/oasislabs/oasis-core/go/consensus/tendermint/apps/roothash/state" @@ -20,7 +19,7 @@ import ( staking "github.com/oasislabs/oasis-core/go/staking/api" ) -func checkEpochTime(state *iavl.MutableTree, now epochtime.EpochTime) error { +func checkEpochTime(ctx *abci.Context, now epochtime.EpochTime) error { if now == epochtime.EpochInvalid { return fmt.Errorf("current epoch is invalid") } @@ -29,11 +28,11 @@ func checkEpochTime(state *iavl.MutableTree, now epochtime.EpochTime) error { return nil } -func checkRegistry(state *iavl.MutableTree, now epochtime.EpochTime) error { - st := registryState.NewMutableState(state) +func checkRegistry(ctx *abci.Context, now epochtime.EpochTime) error { + st := registryState.NewMutableState(ctx.State()) // Check entities. - entities, err := st.SignedEntities() + entities, err := st.SignedEntities(ctx) if err != nil { return fmt.Errorf("SignedEntities: %w", err) } @@ -43,15 +42,15 @@ func checkRegistry(state *iavl.MutableTree, now epochtime.EpochTime) error { } // Check runtimes. - runtimes, err := st.SignedRuntimes() + runtimes, err := st.SignedRuntimes(ctx) if err != nil { return fmt.Errorf("AllSignedRuntimes: %w", err) } - suspendedRuntimes, err := st.SuspendedRuntimes() + suspendedRuntimes, err := st.SuspendedRuntimes(ctx) if err != nil { return fmt.Errorf("SuspendedRuntimes: %w", err) } - params, err := st.ConsensusParameters() + params, err := st.ConsensusParameters(ctx) if err != nil { return fmt.Errorf("ConsensusParameters: %w", err) } @@ -61,7 +60,7 @@ func checkRegistry(state *iavl.MutableTree, now epochtime.EpochTime) error { } // Check nodes. - nodes, err := st.SignedNodes() + nodes, err := st.SignedNodes(ctx) if err != nil { return fmt.Errorf("SignedNodes: %w", err) } @@ -73,17 +72,20 @@ func checkRegistry(state *iavl.MutableTree, now epochtime.EpochTime) error { return nil } -func checkRootHash(state *iavl.MutableTree, now epochtime.EpochTime) error { - st := roothashState.NewMutableState(state) +func checkRootHash(ctx *abci.Context, now epochtime.EpochTime) error { + st := roothashState.NewMutableState(ctx.State()) // Check blocks. - runtimes := st.Runtimes() + runtimes, err := st.Runtimes(ctx) + if err != nil { + return fmt.Errorf("Runtimes: %w", err) + } blocks := make(map[common.Namespace]*block.Block) for _, rt := range runtimes { blocks[rt.Runtime.ID] = rt.CurrentBlock } - err := roothash.SanityCheckBlocks(blocks) + err = roothash.SanityCheckBlocks(blocks) if err != nil { return fmt.Errorf("SanityCheckBlocks: %w", err) } @@ -92,15 +94,15 @@ func checkRootHash(state *iavl.MutableTree, now epochtime.EpochTime) error { return nil } -func checkStaking(state *iavl.MutableTree, now epochtime.EpochTime) error { - st := stakingState.NewMutableState(state) +func checkStaking(ctx *abci.Context, now epochtime.EpochTime) error { + st := stakingState.NewMutableState(ctx.State()) - parameters, err := st.ConsensusParameters() + parameters, err := st.ConsensusParameters(ctx) if err != nil { return fmt.Errorf("ConsensusParameters: %w", err) } - totalSupply, err := st.TotalSupply() + totalSupply, err := st.TotalSupply(ctx) if err != nil { return fmt.Errorf("TotalSupply: %w", err) } @@ -108,7 +110,7 @@ func checkStaking(state *iavl.MutableTree, now epochtime.EpochTime) error { return fmt.Errorf("total supply %v is invalid", totalSupply) } - commonPool, err := st.CommonPool() + commonPool, err := st.CommonPool(ctx) if err != nil { return fmt.Errorf("CommonPool: %w", err) } @@ -119,18 +121,23 @@ func checkStaking(state *iavl.MutableTree, now epochtime.EpochTime) error { // Check if the total supply adds up (common pool + all balances in the ledger). // Check all commission schedules. var total quantity.Quantity - accounts, err := st.Accounts() + accounts, err := st.Accounts(ctx) if err != nil { return fmt.Errorf("Accounts: %w", err) } + var acct *staking.Account for _, id := range accounts { - err = staking.SanityCheckAccount(&total, parameters, now, id, st.Account(id)) + acct, err = st.Account(ctx, id) + if err != nil { + return fmt.Errorf("Account: %w", err) + } + err = staking.SanityCheckAccount(&total, parameters, now, id, acct) if err != nil { return fmt.Errorf("SanityCheckAccount %s: %w", id, err) } } - totalFees, err := st.LastBlockFees() + totalFees, err := st.LastBlockFees(ctx) if err != nil { return fmt.Errorf("LastBlockFees: %w", err) } @@ -145,32 +152,44 @@ func checkStaking(state *iavl.MutableTree, now epochtime.EpochTime) error { } // All shares of all delegations for a given account must add up to account's Escrow.Active.TotalShares. - delegationses, err := st.Delegations() + delegationses, err := st.Delegations(ctx) if err != nil { return fmt.Errorf("Delegations: %w", err) } - for acct, delegations := range delegationses { - err = staking.SanityCheckDelegations(st.Account(acct), delegations) + for id, delegations := range delegationses { + acct, err = st.Account(ctx, id) + if err != nil { + return fmt.Errorf("Account: %w", err) + } + err = staking.SanityCheckDelegations(acct, delegations) if err != nil { - return fmt.Errorf("SanityCheckDelegations %s: %w", acct, err) + return fmt.Errorf("SanityCheckDelegations %s: %w", id, err) } } // All shares of all debonding delegations for a given account must add up to account's Escrow.Debonding.TotalShares. - debondingDelegationses, err := st.DebondingDelegations() + debondingDelegationses, err := st.DebondingDelegations(ctx) if err != nil { return fmt.Errorf("DebondingDelegations: %w", err) } - for acct, debondingDelegations := range debondingDelegationses { - err := staking.SanityCheckDebondingDelegations(st.Account(acct), debondingDelegations) + for id, debondingDelegations := range debondingDelegationses { + acct, err = st.Account(ctx, id) if err != nil { - return fmt.Errorf("SanityCheckDebondingDelegations %s: %w", acct, err) + return fmt.Errorf("Account: %w", err) + } + err = staking.SanityCheckDebondingDelegations(acct, debondingDelegations) + if err != nil { + return fmt.Errorf("SanityCheckDebondingDelegations %s: %w", id, err) } } // Check the above two invariants for each account as well. for _, id := range accounts { - err := staking.SanityCheckAccountShares(st.Account(id), delegationses[id], debondingDelegationses[id]) + acct, err = st.Account(ctx, id) + if err != nil { + return fmt.Errorf("Account: %w", err) + } + err = staking.SanityCheckAccountShares(acct, delegationses[id], debondingDelegationses[id]) if err != nil { return fmt.Errorf("SanityCheckAccountShares %s: %w", id, err) } @@ -179,10 +198,10 @@ func checkStaking(state *iavl.MutableTree, now epochtime.EpochTime) error { return nil } -func checkKeyManager(state *iavl.MutableTree, now epochtime.EpochTime) error { - st := keymanagerState.NewMutableState(state) +func checkKeyManager(ctx *abci.Context, now epochtime.EpochTime) error { + st := keymanagerState.NewMutableState(ctx.State()) - statuses, err := st.Statuses() + statuses, err := st.Statuses(ctx) if err != nil { return fmt.Errorf("Statuses: %w", err) } @@ -194,31 +213,31 @@ func checkKeyManager(state *iavl.MutableTree, now epochtime.EpochTime) error { return nil } -func checkScheduler(*iavl.MutableTree, epochtime.EpochTime) error { +func checkScheduler(*abci.Context, epochtime.EpochTime) error { // nothing to check yet return nil } -func checkBeacon(*iavl.MutableTree, epochtime.EpochTime) error { +func checkBeacon(*abci.Context, epochtime.EpochTime) error { // nothing to check yet return nil } -func checkConsensus(*iavl.MutableTree, epochtime.EpochTime) error { +func checkConsensus(*abci.Context, epochtime.EpochTime) error { // nothing to check yet return nil } -func checkHalt(*iavl.MutableTree, epochtime.EpochTime) error { +func checkHalt(*abci.Context, epochtime.EpochTime) error { // nothing to check yet return nil } -func checkStakeClaims(state *iavl.MutableTree, now epochtime.EpochTime) error { - regSt := registryState.NewMutableState(state) - stakeSt := stakingState.NewMutableState(state) +func checkStakeClaims(ctx *abci.Context, now epochtime.EpochTime) error { + regSt := registryState.NewMutableState(ctx.State()) + stakeSt := stakingState.NewMutableState(ctx.State()) - params, err := regSt.ConsensusParameters() + params, err := regSt.ConsensusParameters(ctx) if err != nil { return fmt.Errorf("failed to get consensus parameters: %w", err) } @@ -231,7 +250,7 @@ func checkStakeClaims(state *iavl.MutableTree, now epochtime.EpochTime) error { // Claims in the stake accumulators should be consistent with general state. claims := make(map[signature.PublicKey]map[staking.StakeClaim][]staking.ThresholdKind) // Entity registrations. - entities, err := regSt.Entities() + entities, err := regSt.Entities(ctx) if err != nil { return fmt.Errorf("failed to get entities: %w", err) } @@ -241,7 +260,7 @@ func checkStakeClaims(state *iavl.MutableTree, now epochtime.EpochTime) error { } } // Node registrations. - nodes, err := regSt.Nodes() + nodes, err := regSt.Nodes(ctx) if err != nil { return fmt.Errorf("failed to get node registrations: %w", err) } @@ -249,7 +268,7 @@ func checkStakeClaims(state *iavl.MutableTree, now epochtime.EpochTime) error { claims[node.EntityID][registry.StakeClaimForNode(node.ID)] = registry.StakeThresholdsForNode(node) } // Runtime registrations. - runtimes, err := regSt.AllRuntimes() + runtimes, err := regSt.AllRuntimes(ctx) if err != nil { return fmt.Errorf("failed to get runtime registrations: %w", err) } @@ -259,7 +278,10 @@ func checkStakeClaims(state *iavl.MutableTree, now epochtime.EpochTime) error { // Compare with actual accumulator state. for _, entity := range entities { - acct := stakeSt.Account(entity.ID) + acct, err := stakeSt.Account(ctx, entity.ID) + if err != nil { + return fmt.Errorf("failed to fetch account: %w", err) + } expectedClaims := claims[entity.ID] actualClaims := acct.Escrow.StakeAccumulator.Claims if len(expectedClaims) != len(actualClaims) { diff --git a/go/consensus/tendermint/apps/supplementarysanity/supplementarysanity.go b/go/consensus/tendermint/apps/supplementarysanity/supplementarysanity.go index 131a5718eff..0aeb9e1f757 100644 --- a/go/consensus/tendermint/apps/supplementarysanity/supplementarysanity.go +++ b/go/consensus/tendermint/apps/supplementarysanity/supplementarysanity.go @@ -4,7 +4,6 @@ import ( "math/rand" "github.com/pkg/errors" - "github.com/tendermint/iavl" "github.com/tendermint/tendermint/abci/types" "github.com/oasislabs/oasis-core/go/common/logging" @@ -107,14 +106,13 @@ func (app *supplementarySanityApplication) endBlockImpl(ctx *abci.Context, reque logger.Debug("checking this block", "height", request.Height) - now, err := app.state.GetEpoch(ctx.Ctx(), ctx.BlockHeight()+1) + now, err := app.state.GetEpoch(ctx, ctx.BlockHeight()+1) if err != nil { return errors.Wrap(err, "GetEpoch") } - state := ctx.State() for _, tt := range []struct { name string - checker func(state *iavl.MutableTree, now epochtime.EpochTime) error + checker func(ctx *abci.Context, now epochtime.EpochTime) error }{ {"checkEpochTime", checkEpochTime}, {"checkRegistry", checkRegistry}, @@ -127,7 +125,7 @@ func (app *supplementarySanityApplication) endBlockImpl(ctx *abci.Context, reque {"checkHalt", checkHalt}, {"checkStakeClaims", checkStakeClaims}, } { - if err := tt.checker(state, now); err != nil { + if err := tt.checker(ctx, now); err != nil { return errors.Wrap(err, tt.name) } } diff --git a/go/consensus/tendermint/db/init.go b/go/consensus/tendermint/db/init.go index 259c977688a..f486dd5c43d 100644 --- a/go/consensus/tendermint/db/init.go +++ b/go/consensus/tendermint/db/init.go @@ -19,6 +19,11 @@ const cfgBackend = "tendermint.db.backend" // Flags has the configuration flags. var Flags = flag.NewFlagSet("", flag.ContinueOnError) +// GetBackendName returns the currently configured Tendermint database backend. +func GetBackendName() string { + return viper.GetString(cfgBackend) +} + // GetProvider returns the currently configured Tendermint DBProvider. func GetProvider() (node.DBProvider, error) { backend := viper.GetString(cfgBackend) diff --git a/go/consensus/tendermint/epochtime_mock/epochtime_mock.go b/go/consensus/tendermint/epochtime_mock/epochtime_mock.go index 2f3e536644c..0a8cd7ac381 100644 --- a/go/consensus/tendermint/epochtime_mock/epochtime_mock.go +++ b/go/consensus/tendermint/epochtime_mock/epochtime_mock.go @@ -147,6 +147,10 @@ func (t *tendermintMockBackend) worker(ctx context.Context) { t.currentBlock = height t.notifier.Broadcast(t.epoch) t.Unlock() + } else { + t.logger.Warn("unable to query initial epoch", + "err", err, + ) } for { diff --git a/go/consensus/tendermint/inspector/inspector.go b/go/consensus/tendermint/inspector/inspector.go deleted file mode 100644 index 4d1bc4eb164..00000000000 --- a/go/consensus/tendermint/inspector/inspector.go +++ /dev/null @@ -1,42 +0,0 @@ -// Package inspector contains utility functions for inspecting Tendermint state. -package inspector - -import ( - "github.com/tendermint/iavl" - dbm "github.com/tendermint/tm-db" - - "github.com/oasislabs/oasis-core/go/consensus/tendermint/db" -) - -// MuxState is an open ABCI mux state database. -type MuxState struct { - db dbm.DB - tree *iavl.ImmutableTree -} - -// Close closes the underlying database -func (s *MuxState) Close() { - s.db.Close() -} - -// Tree returns the immutable tree representing ABCI mux state. -func (s *MuxState) Tree() *iavl.ImmutableTree { - return s.tree -} - -// OpenMuxState opens the ABCI mux state for inspection. -func OpenMuxState(filename string) (*MuxState, error) { - db, err := db.New(filename, true) - if err != nil { - return nil, err - } - - tree := iavl.NewMutableTree(db, 128) - _, err = tree.Load() - if err != nil { - db.Close() - return nil, err - } - - return &MuxState{db: db, tree: tree.ImmutableTree}, nil -} diff --git a/go/consensus/tendermint/service/service.go b/go/consensus/tendermint/service/service.go index 06488d7bdfb..cf07963756b 100644 --- a/go/consensus/tendermint/service/service.go +++ b/go/consensus/tendermint/service/service.go @@ -55,9 +55,6 @@ type TendermintService interface { // Unsubscribe unsubscribes from tendermint events. Unsubscribe(subscriber string, query tmpubsub.Query) error - - // Pruner returns the ABCI state pruner. - Pruner() abci.StatePruner } // GenesisProvider is a tendermint specific genesis document provider. diff --git a/go/consensus/tendermint/tendermint.go b/go/consensus/tendermint/tendermint.go index 4c30065de9a..5862ce618ac 100644 --- a/go/consensus/tendermint/tendermint.go +++ b/go/consensus/tendermint/tendermint.go @@ -597,10 +597,6 @@ func (t *tendermintService) Unsubscribe(subscriber string, query tmpubsub.Query) return fmt.Errorf("tendermint: unsubscribe called with no backing service") } -func (t *tendermintService) Pruner() abci.StatePruner { - return t.mux.Pruner() -} - func (t *tendermintService) RegisterApplication(app abci.Application) error { return t.mux.Register(app) } @@ -880,11 +876,11 @@ func (t *tendermintService) lazyInit() error { if err = pruneCfg.Strategy.FromString(pruneStrat); err != nil { return err } - pruneNumKept := int64(viper.GetInt(cfgABCIPruneNumKept)) - pruneCfg.NumKept = pruneNumKept + pruneCfg.NumKept = viper.GetUint64(cfgABCIPruneNumKept) appConfig := &abci.ApplicationConfig{ - DataDir: t.dataDir, + DataDir: filepath.Join(t.dataDir, StateDir), + StorageBackend: db.GetBackendName(), Pruning: pruneCfg, HaltEpochHeight: t.genesis.HaltEpoch, MinGasPrice: viper.GetUint64(CfgConsensusMinGasPrice), @@ -1341,7 +1337,7 @@ func init() { Flags.String(CfgCoreListenAddress, "tcp://0.0.0.0:26656", "tendermint core listen address") Flags.String(cfgCoreExternalAddress, "", "tendermint address advertised to other nodes") Flags.String(cfgABCIPruneStrategy, abci.PruneDefault, "ABCI state pruning strategy") - Flags.Int64(cfgABCIPruneNumKept, 3600, "ABCI state versions kept (when applicable)") + Flags.Uint64(cfgABCIPruneNumKept, 3600, "ABCI state versions kept (when applicable)") Flags.StringSlice(CfgSentryUpstreamAddress, []string{}, "Tendermint nodes for which we act as sentry of the form ID@ip:port") Flags.StringSlice(CfgP2PPersistentPeer, []string{}, "Tendermint persistent peer(s) of the form ID@ip:port") Flags.Bool(CfgP2PDisablePeerExchange, false, "Disable Tendermint's peer-exchange reactor") diff --git a/go/go.mod b/go/go.mod index fe414cc239d..c00c4d0a3f9 100644 --- a/go/go.mod +++ b/go/go.mod @@ -1,7 +1,6 @@ module github.com/oasislabs/oasis-core/go replace ( - github.com/tendermint/iavl => github.com/oasislabs/iavl v0.12.0-ekiden3 github.com/tendermint/tendermint => github.com/oasislabs/tendermint v0.32.8-oasis2 golang.org/x/crypto/curve25519 => github.com/oasislabs/ed25519/extra/x25519 v0.0.0-20191022155220-a426dcc8ad5f golang.org/x/crypto/ed25519 => github.com/oasislabs/ed25519 v0.0.0-20191109133925-b197a691e30d @@ -61,7 +60,6 @@ require ( github.com/stretchr/testify v1.4.0 github.com/tecbot/gorocksdb v0.0.0-20190519120508-025c3cf4ffb4 // indirect github.com/tendermint/go-amino v0.15.0 // indirect - github.com/tendermint/iavl v0.12.2 github.com/tendermint/tendermint v0.32.8 github.com/tendermint/tm-db v0.2.0 github.com/uber-go/atomic v1.4.0 // indirect diff --git a/go/go.sum b/go/go.sum index c495df216f3..f74eb4db13d 100644 --- a/go/go.sum +++ b/go/go.sum @@ -350,8 +350,6 @@ github.com/oasislabs/deoxysii v0.0.0-20190807103041-6159f99c2236 h1:eTbRemVO4uAX github.com/oasislabs/deoxysii v0.0.0-20190807103041-6159f99c2236/go.mod h1:gFIu170Sklo1wPRTYMTDxA664TYdgrl9NENFXfC+u3g= github.com/oasislabs/ed25519 v0.0.0-20191122104632-9d9ffc15f526 h1:xKlK+m6tNFucKVOP4V0GDgU4IgaLbS+HRoiVbN3W8Y4= github.com/oasislabs/ed25519 v0.0.0-20191122104632-9d9ffc15f526/go.mod h1:xIpCyrK2ouGA4QBGbiNbkoONrvJ00u9P3QOkXSOAC0c= -github.com/oasislabs/iavl v0.12.0-ekiden3 h1:8544fXJb57urhAEpTlIwDBdTJukgpPS/FCS/yj14I8E= -github.com/oasislabs/iavl v0.12.0-ekiden3/go.mod h1:B/tMpl5cg7n42n3xYQTCckJzQezoI75jedkc8FOiOF0= github.com/oasislabs/safeopen v0.0.0-20200117113835-6aa648f43ff8 h1:KC7dcrx0WEeyAWGAG+vdJjmIW36PUfw1x/LUnHjLm2E= github.com/oasislabs/safeopen v0.0.0-20200117113835-6aa648f43ff8/go.mod h1:ABsG2IHM7bpTRIH3EvQ8CZQEBkzuhLxXFxaYApYMB9Y= github.com/oasislabs/tendermint v0.32.8-oasis1 h1:y+RZsI7D6jFqV/OufXA0meDOgn6eJLsXa5a0iyRLIRU= diff --git a/go/oasis-node/cmd/debug/debug.go b/go/oasis-node/cmd/debug/debug.go index d69dc86de48..81044fed010 100644 --- a/go/oasis-node/cmd/debug/debug.go +++ b/go/oasis-node/cmd/debug/debug.go @@ -8,7 +8,6 @@ import ( "github.com/oasislabs/oasis-core/go/oasis-node/cmd/debug/control" "github.com/oasislabs/oasis-core/go/oasis-node/cmd/debug/fixgenesis" "github.com/oasislabs/oasis-core/go/oasis-node/cmd/debug/storage" - "github.com/oasislabs/oasis-core/go/oasis-node/cmd/debug/tendermint" "github.com/oasislabs/oasis-core/go/oasis-node/cmd/debug/txsource" ) @@ -20,7 +19,6 @@ var debugCmd = &cobra.Command{ // Register registers the debug sub-command and all of it's children. func Register(parentCmd *cobra.Command) { storage.Register(debugCmd) - tendermint.Register(debugCmd) byzantine.Register(debugCmd) txsource.Register(debugCmd) fixgenesis.Register(debugCmd) diff --git a/go/oasis-node/cmd/debug/tendermint/tendermint.go b/go/oasis-node/cmd/debug/tendermint/tendermint.go deleted file mode 100644 index 017cf1c4652..00000000000 --- a/go/oasis-node/cmd/debug/tendermint/tendermint.go +++ /dev/null @@ -1,81 +0,0 @@ -// Package tendermint implements the tendermint debug sub-commands. -package tendermint - -import ( - "bytes" - "encoding/hex" - "encoding/json" - "fmt" - - "github.com/spf13/cobra" - - "github.com/oasislabs/oasis-core/go/common/cbor" - "github.com/oasislabs/oasis-core/go/common/logging" - "github.com/oasislabs/oasis-core/go/consensus/tendermint/inspector" - cmdCommon "github.com/oasislabs/oasis-core/go/oasis-node/cmd/common" -) - -var ( - stateFilename string - - tmCmd = &cobra.Command{ - Use: "tendermint", - Short: "tendermint backend utilities", - } - - tmDumpMuxStateCmd = &cobra.Command{ - Use: "dump-abci-mux-state", - Short: "dump ABCI mux state as JSON", - Run: doDumpMuxState, - } -) - -func doDumpMuxState(cmd *cobra.Command, args []string) { - if err := cmdCommon.Init(); err != nil { - cmdCommon.EarlyLogAndExit(err) - } - - logger := logging.GetLogger("cmd/debug/tendermint/dump-abci-mux-state") - - state, err := inspector.OpenMuxState(stateFilename) - if err != nil { - logger.Error("failed to open ABCI mux state", - "err", err, - ) - return - } - defer state.Close() - - output := make(map[string]interface{}) - state.Tree().Iterate(func(key, value []byte) bool { - // Try to decode as CBOR and if that fails, output as hex. - var decoded interface{} - if err = cbor.Unmarshal(value, &decoded); err != nil { - decoded = hex.EncodeToString(value) - } - - output[string(key)] = decoded - return false - }) - - buf := bytes.NewBuffer(nil) - enc := json.NewEncoder(buf) - enc.SetEscapeHTML(false) - enc.SetIndent("", " ") - - if err = enc.Encode(output); err != nil { - logger.Error("failed to encode ABCI mux state", - "err", err, - ) - return - } - - fmt.Printf("%s\n", buf.Bytes()) -} - -// Register registers the tendermint sub-command and all of it's children. -func Register(parentCmd *cobra.Command) { - tmDumpMuxStateCmd.Flags().StringVarP(&stateFilename, "state", "s", "abci-mux-state.bolt.db", "ABCI mux state file to dump") - tmCmd.AddCommand(tmDumpMuxStateCmd) - parentCmd.AddCommand(tmCmd) -} diff --git a/go/registry/api/api.go b/go/registry/api/api.go index 3aa6c88b4c7..b515b5604d7 100644 --- a/go/registry/api/api.go +++ b/go/registry/api/api.go @@ -289,10 +289,10 @@ type NodeList struct { // functions to look-up nodes in the registry's state. type NodeLookup interface { // Returns the node that corresponds to the given consensus or P2P ID. - NodeByConsensusOrP2PKey(key signature.PublicKey) (*node.Node, error) + NodeByConsensusOrP2PKey(ctx context.Context, key signature.PublicKey) (*node.Node, error) // Returns the node that corresponds to the given committee certificate. - NodeByCertificate(cert []byte) (*node.Node, error) + NodeByCertificate(ctx context.Context, cert []byte) (*node.Node, error) } // RuntimeLookup interface implements various ways for the verification @@ -301,14 +301,14 @@ type RuntimeLookup interface { // Runtime looks up a runtime by its identifier and returns it. // // This excludes any suspended runtimes, use SuspendedRuntime to query suspended runtimes only. - Runtime(id common.Namespace) (*Runtime, error) + Runtime(ctx context.Context, id common.Namespace) (*Runtime, error) // SuspendedRuntime looks up a suspended runtime by its identifier and // returns it. - SuspendedRuntime(id common.Namespace) (*Runtime, error) + SuspendedRuntime(ctx context.Context, id common.Namespace) (*Runtime, error) // AnyRuntime looks up either an active or suspended runtime by its identifier and returns it. - AnyRuntime(id common.Namespace) (*Runtime, error) + AnyRuntime(ctx context.Context, id common.Namespace) (*Runtime, error) } // VerifyRegisterEntityArgs verifies arguments for RegisterEntity. @@ -367,6 +367,7 @@ func VerifyRegisterEntityArgs(logger *logging.Logger, sigEnt *entity.SignedEntit // // Returns the node descriptor and a list of runtime descriptors the node is registering for. func VerifyRegisterNodeArgs( // nolint: gocyclo + ctx context.Context, params *ConsensusParameters, logger *logging.Logger, sigNode *node.MultiSignedNode, @@ -382,15 +383,15 @@ func VerifyRegisterNodeArgs( // nolint: gocyclo return nil, nil, ErrInvalidArgument } - var ctx signature.Context + var sigCtx signature.Context switch isGenesis { case true: - ctx = RegisterGenesisNodeSignatureContext + sigCtx = RegisterGenesisNodeSignatureContext case false: - ctx = RegisterNodeSignatureContext + sigCtx = RegisterNodeSignatureContext } - if err := sigNode.Open(ctx, &n); err != nil { + if err := sigNode.Open(sigCtx, &n); err != nil { logger.Error("RegisterNode: invalid signature", "signed_node", sigNode, ) @@ -502,7 +503,7 @@ func VerifyRegisterNodeArgs( // nolint: gocyclo rtMap[rt.ID] = true // Make sure that the claimed runtime actually exists. - regRt, err := runtimeLookup.AnyRuntime(rt.ID) + regRt, err := runtimeLookup.AnyRuntime(ctx, rt.ID) if err != nil { logger.Error("RegisterNode: failed to fetch supported runtime", "err", err, @@ -623,7 +624,7 @@ func VerifyRegisterNodeArgs( // nolint: gocyclo return nil, nil, fmt.Errorf("%w: P2P and Consensus IDs not unique", ErrInvalidArgument) } - existingNode, err := nodeLookup.NodeByConsensusOrP2PKey(n.Consensus.ID) + existingNode, err := nodeLookup.NodeByConsensusOrP2PKey(ctx, n.Consensus.ID) if err != nil && err != ErrNoSuchNode { logger.Error("RegisterNode: failed to get node by consensus ID", "err", err, @@ -639,7 +640,7 @@ func VerifyRegisterNodeArgs( // nolint: gocyclo return nil, nil, fmt.Errorf("%w: duplicate node consensus ID", ErrInvalidArgument) } - existingNode, err = nodeLookup.NodeByConsensusOrP2PKey(n.P2P.ID) + existingNode, err = nodeLookup.NodeByConsensusOrP2PKey(ctx, n.P2P.ID) if err != nil && err != ErrNoSuchNode { logger.Error("RegisterNode: failed to get node by P2P ID", "err", err, @@ -655,7 +656,7 @@ func VerifyRegisterNodeArgs( // nolint: gocyclo return nil, nil, fmt.Errorf("%w: duplicate node P2P ID", ErrInvalidArgument) } - existingNode, err = nodeLookup.NodeByCertificate(n.Committee.Certificate) + existingNode, err = nodeLookup.NodeByCertificate(ctx, n.Committee.Certificate) if err != nil && err != ErrNoSuchNode { logger.Error("RegisterNode: failed to get node by committee certificate", "err", err, @@ -1157,10 +1158,10 @@ func VerifyRegisterRuntimeStorageArgs(rt *Runtime, logger *logging.Logger) error } // VerifyRegisterComputeRuntimeArgs verifies compute runtime-specific arguments for RegisterRuntime. -func VerifyRegisterComputeRuntimeArgs(logger *logging.Logger, rt *Runtime, runtimeLookup RuntimeLookup) error { +func VerifyRegisterComputeRuntimeArgs(ctx context.Context, logger *logging.Logger, rt *Runtime, runtimeLookup RuntimeLookup) error { // Check runtime's key manager, if key manager ID is set. if rt.KeyManager != nil { - km, err := runtimeLookup.AnyRuntime(*rt.KeyManager) + km, err := runtimeLookup.AnyRuntime(ctx, *rt.KeyManager) if err != nil { logger.Error("RegisterRuntime: error when fetching the runtime's key manager from registry", "runtime", rt.ID, diff --git a/go/registry/api/sanity_check.go b/go/registry/api/sanity_check.go index efa35e6bdff..9fa6acd9a61 100644 --- a/go/registry/api/sanity_check.go +++ b/go/registry/api/sanity_check.go @@ -1,6 +1,7 @@ package api import ( + "context" "fmt" "time" @@ -59,7 +60,8 @@ func SanityCheckEntities(logger *logging.Logger, entities []*entity.SignedEntity } // SanityCheckRuntimes examines the runtimes table. -func SanityCheckRuntimes(logger *logging.Logger, +func SanityCheckRuntimes( + logger *logging.Logger, params *ConsensusParameters, runtimes []*SignedRuntime, suspendedRuntimes []*SignedRuntime, @@ -95,7 +97,7 @@ func SanityCheckRuntimes(logger *logging.Logger, if rt.Kind != KindCompute { continue } - if err := VerifyRegisterComputeRuntimeArgs(logger, rt, lookup); err != nil { + if err := VerifyRegisterComputeRuntimeArgs(context.Background(), logger, rt, lookup); err != nil { return nil, fmt.Errorf("compute runtime sanity check failed: %w", err) } } @@ -136,7 +138,9 @@ func SanityCheckNodes( return fmt.Errorf("registry: sanity check failed node: %s references a missing entity", n.ID.String()) } - node, _, err := VerifyRegisterNodeArgs(params, + node, _, err := VerifyRegisterNodeArgs( + context.Background(), + params, logger, sn, entity, @@ -186,7 +190,7 @@ func newSanityCheckRuntimeLookup(runtimes []*Runtime, suspendedRuntimes []*Runti return &sanityCheckRuntimeLookup{rtsMap, sRtsMap}, nil } -func (r *sanityCheckRuntimeLookup) Runtime(id common.Namespace) (*Runtime, error) { +func (r *sanityCheckRuntimeLookup) Runtime(ctx context.Context, id common.Namespace) (*Runtime, error) { rt, ok := r.runtimes[id] if !ok { return nil, fmt.Errorf("runtime not found") @@ -194,7 +198,7 @@ func (r *sanityCheckRuntimeLookup) Runtime(id common.Namespace) (*Runtime, error return rt, nil } -func (r *sanityCheckRuntimeLookup) SuspendedRuntime(id common.Namespace) (*Runtime, error) { +func (r *sanityCheckRuntimeLookup) SuspendedRuntime(ctx context.Context, id common.Namespace) (*Runtime, error) { srt, ok := r.suspendedRuntimes[id] if !ok { return nil, ErrNoSuchRuntime @@ -202,7 +206,7 @@ func (r *sanityCheckRuntimeLookup) SuspendedRuntime(id common.Namespace) (*Runti return srt, nil } -func (r *sanityCheckRuntimeLookup) AnyRuntime(id common.Namespace) (*Runtime, error) { +func (r *sanityCheckRuntimeLookup) AnyRuntime(ctx context.Context, id common.Namespace) (*Runtime, error) { rt, ok := r.runtimes[id] if !ok { srt, ok := r.suspendedRuntimes[id] @@ -220,7 +224,7 @@ type sanityCheckNodeLookup struct { nodesCertHashes map[hash.Hash]*node.Node } -func (n *sanityCheckNodeLookup) NodeByConsensusOrP2PKey(key signature.PublicKey) (*node.Node, error) { +func (n *sanityCheckNodeLookup) NodeByConsensusOrP2PKey(ctx context.Context, key signature.PublicKey) (*node.Node, error) { node, ok := n.nodes[key] if !ok { return nil, ErrNoSuchNode @@ -228,7 +232,7 @@ func (n *sanityCheckNodeLookup) NodeByConsensusOrP2PKey(key signature.PublicKey) return node, nil } -func (n *sanityCheckNodeLookup) NodeByCertificate(cert []byte) (*node.Node, error) { +func (n *sanityCheckNodeLookup) NodeByCertificate(ctx context.Context, cert []byte) (*node.Node, error) { var h = hash.Hash{} h.FromBytes(cert) diff --git a/go/roothash/api/commitment/pool.go b/go/roothash/api/commitment/pool.go index 7cf0389bec0..d602d86f6b7 100644 --- a/go/roothash/api/commitment/pool.go +++ b/go/roothash/api/commitment/pool.go @@ -1,6 +1,7 @@ package commitment import ( + "context" "time" "github.com/oasislabs/oasis-core/go/common/cbor" @@ -47,7 +48,7 @@ type SignatureVerifier interface { // NodeLookup is an interface for looking up registry node descriptors. type NodeLookup interface { // Node looks up a node descriptor. - Node(id signature.PublicKey) (*node.Node, error) + Node(ctx context.Context, id signature.PublicKey) (*node.Node, error) } // Pool is a serializable pool of commitments that can be used to perform @@ -173,7 +174,7 @@ func (p *Pool) addOpenExecutorCommitment(blk *block.Block, sv SignatureVerifier, // Verify RAK-attestation. if p.Runtime.TEEHardware != node.TEEHardwareInvalid { - n, err := nl.Node(id) + n, err := nl.Node(context.TODO(), id) if err != nil { // This should never happen as nodes cannot disappear mid-epoch. logger.Warn("unable to fetch node descriptor to verify RAK-attestation", diff --git a/go/roothash/api/commitment/pool_test.go b/go/roothash/api/commitment/pool_test.go index faa4fe12839..e473ab19734 100644 --- a/go/roothash/api/commitment/pool_test.go +++ b/go/roothash/api/commitment/pool_test.go @@ -1,6 +1,7 @@ package commitment import ( + "context" "crypto/rand" "errors" "testing" @@ -58,7 +59,7 @@ type staticNodeLookup struct { runtime *node.Runtime } -func (n *staticNodeLookup) Node(id signature.PublicKey) (*node.Node, error) { +func (n *staticNodeLookup) Node(ctx context.Context, id signature.PublicKey) (*node.Node, error) { return &node.Node{ ID: id, Runtimes: []*node.Runtime{n.runtime}, diff --git a/go/storage/api/api.go b/go/storage/api/api.go index 4e04012695c..cb75953bb9c 100644 --- a/go/storage/api/api.go +++ b/go/storage/api/api.go @@ -75,7 +75,7 @@ var ( ) // Config is the storage backend configuration. -type Config struct { +type Config struct { // nolint: maligned // Backend is the database backend. Backend string @@ -96,14 +96,22 @@ type Config struct { // MaxCacheSize is the maximum in-memory cache size for the database. MaxCacheSize int64 + + // DiscardWriteLogs will cause all write logs to be discarded. + DiscardWriteLogs bool + + // NoFsync will disable fsync() where possible. + NoFsync bool } // ToNodeDB converts from a Config to a node DB Config. func (cfg *Config) ToNodeDB() *nodedb.Config { return &nodedb.Config{ - DB: cfg.DB, - Namespace: cfg.Namespace, - MaxCacheSize: cfg.MaxCacheSize, + DB: cfg.DB, + Namespace: cfg.Namespace, + MaxCacheSize: cfg.MaxCacheSize, + NoFsync: cfg.NoFsync, + DiscardWriteLogs: cfg.DiscardWriteLogs, } } @@ -205,6 +213,9 @@ type ProofResponse = syncer.ProofResponse // Proof is a Merkle proof for a subtree. type Proof = syncer.Proof +// NodeDB is a node database. +type NodeDB = nodedb.NodeDB + // ApplyOp is an apply operation within a batch of apply operations. type ApplyOp struct { // SrcRound is the source root round. @@ -327,21 +338,11 @@ type Backend interface { type LocalBackend interface { Backend - // HasRoot checks if the storage backend contains the specified storage root. - HasRoot(root Root) bool - - // Finalize finalizes the specified round. The passed list of roots are the - // roots within the round that have been finalized. All non-finalized roots - // can be discarded. - Finalize(ctx context.Context, namespace common.Namespace, round uint64, roots []hash.Hash) error - - // Prune removes all roots recorded under the given namespace and round. - // - // Returns the number of pruned nodes. - Prune(ctx context.Context, namespace common.Namespace, round uint64) (int, error) - // Checkpointer returns the checkpoint creator/restorer for this storage backend. Checkpointer() checkpoint.CreateRestorer + + // NodeDB returns the underlying node database. + NodeDB() nodedb.NodeDB } // ClientBackend is a storage client backend implementation. diff --git a/go/storage/client/tests/tests.go b/go/storage/client/tests/tests.go index d3b43192f5b..6dbc209730b 100644 --- a/go/storage/client/tests/tests.go +++ b/go/storage/client/tests/tests.go @@ -6,8 +6,6 @@ import ( "time" "github.com/stretchr/testify/require" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" "github.com/oasislabs/oasis-core/go/common/crypto/hash" "github.com/oasislabs/oasis-core/go/common/identity" @@ -114,6 +112,8 @@ recvLoop: // Try getting path. // TimeOut is expected, as test nodes do not actually start storage worker. + ctx, cancel = context.WithTimeout(ctx, 1*time.Second) + defer cancel() r, err = client.SyncGet(ctx, &api.GetRequest{ Tree: api.TreeID{ Root: root, @@ -121,7 +121,6 @@ recvLoop: }, }) require.Error(err, "storage client should error") - require.Equal(codes.Unavailable, status.Code(err), "storage client should timeout") require.Nil(r, "result should be nil") rt.Cleanup(t, consensus.Registry(), consensus) diff --git a/go/storage/database/database.go b/go/storage/database/database.go index f215422c828..5ea347aa3d5 100644 --- a/go/storage/database/database.go +++ b/go/storage/database/database.go @@ -8,7 +8,6 @@ import ( "io" "path/filepath" - "github.com/oasislabs/oasis-core/go/common" "github.com/oasislabs/oasis-core/go/common/crypto/hash" "github.com/oasislabs/oasis-core/go/common/crypto/signature" "github.com/oasislabs/oasis-core/go/storage/api" @@ -202,18 +201,10 @@ func (ba *databaseBackend) GetCheckpointChunk(ctx context.Context, chunk *checkp return ba.checkpointer.GetCheckpointChunk(ctx, chunk, w) } -func (ba *databaseBackend) HasRoot(root api.Root) bool { - return ba.nodedb.HasRoot(root) -} - -func (ba *databaseBackend) Finalize(ctx context.Context, namespace common.Namespace, round uint64, roots []hash.Hash) error { - return ba.nodedb.Finalize(ctx, namespace, round, roots) -} - -func (ba *databaseBackend) Prune(ctx context.Context, namespace common.Namespace, round uint64) (int, error) { - return ba.nodedb.Prune(ctx, namespace, round) -} - func (ba *databaseBackend) Checkpointer() checkpoint.CreateRestorer { return ba.checkpointer } + +func (ba *databaseBackend) NodeDB() nodedb.NodeDB { + return ba.nodedb +} diff --git a/go/storage/database/database_test.go b/go/storage/database/database_test.go index 4798e00fa64..23fe28dd393 100644 --- a/go/storage/database/database_test.go +++ b/go/storage/database/database_test.go @@ -36,6 +36,7 @@ func doTestImpl(t *testing.T, backend string) { ApplyLockLRUSlots: 100, Namespace: testNs, MaxCacheSize: 16 * 1024 * 1024, + NoFsync: true, } err error ) diff --git a/go/storage/metrics.go b/go/storage/metrics.go index a5d14c66609..adce8d054f6 100644 --- a/go/storage/metrics.go +++ b/go/storage/metrics.go @@ -7,8 +7,6 @@ import ( "github.com/prometheus/client_golang/prometheus" - "github.com/oasislabs/oasis-core/go/common" - "github.com/oasislabs/oasis-core/go/common/crypto/hash" "github.com/oasislabs/oasis-core/go/common/node" "github.com/oasislabs/oasis-core/go/storage/api" "github.com/oasislabs/oasis-core/go/storage/mkvs/urkel/checkpoint" @@ -43,26 +41,12 @@ var ( }, []string{"call"}, ) - storagePrunedCount = prometheus.NewCounter( - prometheus.CounterOpts{ - Name: "oasis_storage_pruned", - Help: "Number of pruned nodes.", - }, - ) - storageFinalizedCount = prometheus.NewCounter( - prometheus.CounterOpts{ - Name: "oasis_storage_finalized", - Help: "Number of finalized rounds.", - }, - ) storageCollectors = []prometheus.Collector{ storageFailures, storageCalls, storageLatency, storageValueSize, - storagePrunedCount, - storageFinalizedCount, } labelApply = prometheus.Labels{"call": "apply"} @@ -72,9 +56,6 @@ var ( labelSyncGet = prometheus.Labels{"call": "sync_get"} labelSyncGetPrefixes = prometheus.Labels{"call": "sync_get_prefixes"} labelSyncIterate = prometheus.Labels{"call": "sync_iterate"} - labelHasRoot = prometheus.Labels{"call": "has_root"} - labelFinalize = prometheus.Labels{"call": "finalize"} - labelPrune = prometheus.Labels{"call": "prune"} _ api.LocalBackend = (*metricsWrapper)(nil) _ api.ClientBackend = (*metricsWrapper)(nil) @@ -198,52 +179,20 @@ func (w *metricsWrapper) SyncIterate(ctx context.Context, request *api.IterateRe return res, err } -func (w *metricsWrapper) HasRoot(root api.Root) bool { - localBackend, ok := w.Backend.(api.LocalBackend) - if !ok { - return false - } - start := time.Now() - flag := localBackend.HasRoot(root) - storageLatency.With(labelHasRoot).Observe(time.Since(start).Seconds()) - storageCalls.With(labelHasRoot).Inc() - return flag -} - -func (w *metricsWrapper) Finalize(ctx context.Context, namespace common.Namespace, round uint64, roots []hash.Hash) error { - localBackend, ok := w.Backend.(api.LocalBackend) - if !ok { - return api.ErrUnsupported - } - start := time.Now() - err := localBackend.Finalize(ctx, namespace, round, roots) - storageLatency.With(labelFinalize).Observe(time.Since(start).Seconds()) - storageCalls.With(labelFinalize).Inc() - if err == nil { - storageFinalizedCount.Inc() - } - return err -} - -func (w *metricsWrapper) Prune(ctx context.Context, namespace common.Namespace, round uint64) (int, error) { +func (w *metricsWrapper) Checkpointer() checkpoint.CreateRestorer { localBackend, ok := w.Backend.(api.LocalBackend) if !ok { - return 0, api.ErrUnsupported + return nil } - start := time.Now() - pruned, err := localBackend.Prune(ctx, namespace, round) - storageLatency.With(labelPrune).Observe(time.Since(start).Seconds()) - storageCalls.With(labelPrune).Inc() - storagePrunedCount.Add(float64(pruned)) - return pruned, err + return localBackend.Checkpointer() } -func (w *metricsWrapper) Checkpointer() checkpoint.CreateRestorer { +func (w *metricsWrapper) NodeDB() api.NodeDB { localBackend, ok := w.Backend.(api.LocalBackend) if !ok { return nil } - return localBackend.Checkpointer() + return localBackend.NodeDB() } func newMetricsWrapper(base api.Backend) api.Backend { diff --git a/go/storage/mkvs/urkel/db/api/api.go b/go/storage/mkvs/urkel/db/api/api.go index 23371c6ef6e..634d1b0dd86 100644 --- a/go/storage/mkvs/urkel/db/api/api.go +++ b/go/storage/mkvs/urkel/db/api/api.go @@ -44,18 +44,21 @@ var ( ) // Config is the node database backend configuration. -type Config struct { +type Config struct { // nolint: maligned // DB is the path to the database. DB string - // DebugNoFsync will disable fsync() where possible. - DebugNoFsync bool + // NoFsync will disable fsync() where possible. + NoFsync bool // Namespace is the namespace contained within the database. Namespace common.Namespace // MaxCacheSize is the maximum in-memory cache size for the database. MaxCacheSize int64 + + // DiscardWriteLogs will cause all write logs to be discarded. + DiscardWriteLogs bool } // NodeDB is the persistence layer used for persisting the in-memory tree. @@ -66,6 +69,15 @@ type NodeDB interface { // GetWriteLog retrieves a write log between two storage instances from the database. GetWriteLog(ctx context.Context, startRoot node.Root, endRoot node.Root) (writelog.Iterator, error) + // GetLatestRound returns the most recent round in the node database. + GetLatestRound(ctx context.Context) (uint64, error) + + // GetEarliestRound returns the earliest round in the node database. + GetEarliestRound(ctx context.Context) (uint64, error) + + // GetRootsForRound returns a list of roots stored under the given round. + GetRootsForRound(ctx context.Context, round uint64) ([]hash.Hash, error) + // NewBatch starts a new batch. // // The chunk argument specifies whether the given batch is being used to import a chunk of an @@ -87,6 +99,9 @@ type NodeDB interface { // Returns the number of pruned nodes. Prune(ctx context.Context, namespace common.Namespace, round uint64) (int, error) + // Size returns the size of the database in bytes. + Size() (int64, error) + // Close closes the database. Close() } @@ -168,6 +183,18 @@ func (d *nopNodeDB) GetWriteLog(ctx context.Context, startRoot node.Root, endRoo return nil, ErrWriteLogNotFound } +func (d *nopNodeDB) GetLatestRound(ctx context.Context) (uint64, error) { + return 0, nil +} + +func (d *nopNodeDB) GetEarliestRound(ctx context.Context) (uint64, error) { + return 0, nil +} + +func (d *nopNodeDB) GetRootsForRound(ctx context.Context, round uint64) ([]hash.Hash, error) { + return nil, nil +} + func (d *nopNodeDB) HasRoot(root node.Root) bool { return false } @@ -180,7 +207,10 @@ func (d *nopNodeDB) Prune(ctx context.Context, namespace common.Namespace, round return 0, nil } -// Close is a no-op. +func (d *nopNodeDB) Size() (int64, error) { + return 0, nil +} + func (d *nopNodeDB) Close() { } diff --git a/go/storage/mkvs/urkel/db/badger/badger.go b/go/storage/mkvs/urkel/db/badger/badger.go index a5a473563ee..eda02510306 100644 --- a/go/storage/mkvs/urkel/db/badger/badger.go +++ b/go/storage/mkvs/urkel/db/badger/badger.go @@ -118,14 +118,15 @@ func (m *metadata) setLastFinalizedRound(round uint64) { // New creates a new BadgerDB-backed node database. func New(cfg *api.Config) (api.NodeDB, error) { db := &badgerNodeDB{ - logger: logging.GetLogger("urkel/db/badger"), - namespace: cfg.Namespace, + logger: logging.GetLogger("urkel/db/badger"), + namespace: cfg.Namespace, + discardWriteLogs: cfg.DiscardWriteLogs, } opts := badger.DefaultOptions(cfg.DB) opts = opts.WithLogger(cmnBadger.NewLogAdapter(db.logger)) - opts = opts.WithSyncWrites(!cfg.DebugNoFsync) - opts = opts.WithCompression(options.None) + opts = opts.WithSyncWrites(!cfg.NoFsync) + opts = opts.WithCompression(options.Snappy) opts = opts.WithMaxCacheSize(cfg.MaxCacheSize) var err error @@ -144,11 +145,13 @@ func New(cfg *api.Config) (api.NodeDB, error) { return db, nil } -type badgerNodeDB struct { +type badgerNodeDB struct { // nolint: maligned logger *logging.Logger namespace common.Namespace + discardWriteLogs bool + db *badger.DB gc *cmnBadger.GCWorker meta metadata @@ -253,6 +256,9 @@ func (d *badgerNodeDB) GetNode(root node.Root, ptr *node.Pointer) (node.Node, er } func (d *badgerNodeDB) GetWriteLog(ctx context.Context, startRoot node.Root, endRoot node.Root) (writelog.Iterator, error) { + if d.discardWriteLogs { + return nil, api.ErrWriteLogNotFound + } if !endRoot.Follows(&startRoot) { return nil, api.ErrRootMustFollowOld } @@ -382,6 +388,50 @@ func (d *badgerNodeDB) GetWriteLog(ctx context.Context, startRoot node.Root, end return nil, api.ErrWriteLogNotFound } +func (d *badgerNodeDB) GetLatestRound(ctx context.Context) (uint64, error) { + round, _ := d.meta.getLastFinalizedRound() + return round, nil +} + +func (d *badgerNodeDB) GetEarliestRound(ctx context.Context) (round uint64, err error) { + err = d.db.View(func(tx *badger.Txn) error { + it := tx.NewIterator(badger.IteratorOptions{Prefix: rootLinkKeyFmt.Encode()}) + defer it.Close() + + if it.Rewind(); it.Valid() && !rootLinkKeyFmt.Decode(it.Item().Key(), &round) { + // This should not happen as the Badger iterator should take care of it. + panic("urkel/db/badger: bad iterator") + } + return nil + }) + return +} + +func (d *badgerNodeDB) GetRootsForRound(ctx context.Context, round uint64) (roots []hash.Hash, err error) { + err = d.db.View(func(tx *badger.Txn) error { + prefix := rootLinkKeyFmt.Encode(round) + it := tx.NewIterator(badger.IteratorOptions{Prefix: prefix}) + defer it.Close() + + for it.Rewind(); it.Valid(); it.Next() { + var decRound uint64 + var rootHash hash.Hash + + if !rootLinkKeyFmt.Decode(it.Item().Key(), &decRound, &rootHash) { + // This should not happen as the Badger iterator should take care of it. + panic("urkel/db/badger: bad iterator") + } + + // Multiple entries for the same root may exist due to multiple root link records. + if len(roots) == 0 || !rootHash.Equal(&roots[0]) { + roots = append(roots, rootHash) + } + } + return nil + }) + return +} + func (d *badgerNodeDB) HasRoot(root node.Root) bool { if err := d.sanityCheckNamespace(root.Namespace); err != nil { return false @@ -497,6 +547,7 @@ func (d *badgerNodeDB) Finalize(ctx context.Context, namespace common.Namespace, } continue } + // TODO: Consider removing empty root links in case an actual link is present. rootGcUpdatesKey := rootGcUpdatesKeyFmt.Encode(round, &rootHash) rootAddedNodesKey := rootAddedNodesKeyFmt.Encode(round, &rootHash) @@ -761,6 +812,11 @@ func (d *badgerNodeDB) NewBatch(oldRoot node.Root, chunk bool) api.Batch { } } +func (d *badgerNodeDB) Size() (int64, error) { + lsm, vlog := d.db.Size() + return lsm + vlog, nil +} + func (d *badgerNodeDB) Close() { d.closeOnce.Do(func() { d.gc.Close() @@ -842,6 +898,9 @@ func (ba *badgerBatch) PutWriteLog(writeLog writelog.WriteLog, annotations write if ba.chunk { return fmt.Errorf("urkel/db/badger: cannot put write log in chunk mode") } + if ba.db.discardWriteLogs { + return nil + } ba.writeLog = writeLog ba.annotations = annotations diff --git a/go/storage/mkvs/urkel/insert.go b/go/storage/mkvs/urkel/insert.go index b332f779752..1870c1893ed 100644 --- a/go/storage/mkvs/urkel/insert.go +++ b/go/storage/mkvs/urkel/insert.go @@ -27,16 +27,18 @@ func (t *tree) Insert(ctx context.Context, key []byte, value []byte) error { } // Update the pending write log. - entry := t.pendingWriteLog[node.ToMapKey(key)] - if entry == nil { - t.pendingWriteLog[node.ToMapKey(key)] = &pendingEntry{ - key: key, - value: value, - existed: result.existed, - insertedLeaf: result.insertedLeaf, + if !t.withoutWriteLog { + entry := t.pendingWriteLog[node.ToMapKey(key)] + if entry == nil { + t.pendingWriteLog[node.ToMapKey(key)] = &pendingEntry{ + key: key, + value: value, + existed: result.existed, + insertedLeaf: result.insertedLeaf, + } + } else { + entry.value = value } - } else { - entry.value = value } t.cache.setPendingRoot(result.newRoot) diff --git a/go/storage/mkvs/urkel/urkel.go b/go/storage/mkvs/urkel/urkel.go index 2791ef089e4..02385b4b9c9 100644 --- a/go/storage/mkvs/urkel/urkel.go +++ b/go/storage/mkvs/urkel/urkel.go @@ -17,6 +17,7 @@ type tree struct { // NOTE: This can be a map as updates are commutative. pendingWriteLog map[string]*pendingEntry + withoutWriteLog bool // pendingRemovedNodes are the nodes that have been removed from the // in-memory tree and should be marked for garbage collection if this // tree is committed to the node database. @@ -58,6 +59,13 @@ func PersistEverythingFromSyncer(doit bool) Option { } } +// WithoutWriteLog disables building a write log when performing operations. +func WithoutWriteLog() Option { + return func(t *tree) { + t.withoutWriteLog = true + } +} + // New creates a new empty Urkel tree backed by the given node database. func New(rs syncer.ReadSyncer, ndb db.NodeDB, options ...Option) Tree { if rs == nil { @@ -70,6 +78,7 @@ func New(rs syncer.ReadSyncer, ndb db.NodeDB, options ...Option) Tree { t := &tree{ cache: newCache(ndb, rs), pendingWriteLog: make(map[string]*pendingEntry), + withoutWriteLog: false, } for _, v := range options { diff --git a/go/storage/mkvs/urkel/urkel_test.go b/go/storage/mkvs/urkel/urkel_test.go index d56fb0d47d1..57693415072 100644 --- a/go/storage/mkvs/urkel/urkel_test.go +++ b/go/storage/mkvs/urkel/urkel_test.go @@ -1146,12 +1146,26 @@ func testPruneBasic(t *testing.T, ndb db.NodeDB) { err = ndb.Finalize(ctx, testNs, 2, []hash.Hash{rootHash3}) require.NoError(t, err, "Finalize") + earliestRound, err := ndb.GetEarliestRound(ctx) + require.NoError(t, err, "GetEarliestRound") + require.EqualValues(t, 0, earliestRound, "earliest round should be correct") + latestRound, err := ndb.GetLatestRound(ctx) + require.NoError(t, err, "GetLatestRound") + require.EqualValues(t, 2, latestRound, "latest round should be correct") + // Prune round 0. pruned, err := ndb.Prune(ctx, testNs, 0) require.NoError(t, err, "Prune") // Two nodes should have been pruned (root and left child). require.EqualValues(t, 2, pruned) + earliestRound, err = ndb.GetEarliestRound(ctx) + require.NoError(t, err, "GetEarliestRound") + require.EqualValues(t, 1, earliestRound, "earliest round should be correct") + latestRound, err = ndb.GetLatestRound(ctx) + require.NoError(t, err, "GetLatestRound") + require.EqualValues(t, 2, latestRound, "latest round should be correct") + // Keys must still be available in round 2. tree = NewWithRoot(nil, ndb, node.Root{Namespace: testNs, Round: 2, Hash: rootHash3}) value, err := tree.Get(ctx, []byte("blah")) @@ -1885,7 +1899,7 @@ func TestUrkelBadgerBackend(t *testing.T) { // Create a Badger-backed Node DB. ndb, err := badgerDb.New(&db.Config{ DB: dir, - DebugNoFsync: true, + NoFsync: true, Namespace: testNs, MaxCacheSize: 16 * 1024 * 1024, }) diff --git a/go/upgrade/migrations/dummy.go b/go/upgrade/migrations/dummy.go index 839dc2daba4..381700bdd24 100644 --- a/go/upgrade/migrations/dummy.go +++ b/go/upgrade/migrations/dummy.go @@ -1,6 +1,8 @@ package migrations import ( + "fmt" + "github.com/oasislabs/oasis-core/go/common/crypto/signature" "github.com/oasislabs/oasis-core/go/common/crypto/signature/signers/memory" "github.com/oasislabs/oasis-core/go/common/entity" @@ -45,15 +47,17 @@ func (th *dummyMigrationHandler) ConsensusUpgrade(ctx *Context, privateCtx inter sigEntity, err := entity.SignEntity(entitySigner, registry.RegisterEntitySignatureContext, &TestEntity) if err != nil { - return err + return fmt.Errorf("failed to sign entity: %w", err) } // Add a new entity to the registry. The test runner will check for its presence to verify // the migration ran successfully. - regState.SetEntity(&TestEntity, sigEntity) + if err = regState.SetEntity(abciCtx, &TestEntity, sigEntity); err != nil { + return fmt.Errorf("failed to set entity: %w", err) + } // Set this entity's staking properly. - stakeState.SetAccount(TestEntity.ID, &staking.Account{ + err = stakeState.SetAccount(abciCtx, TestEntity.ID, &staking.Account{ Escrow: staking.EscrowAccount{ StakeAccumulator: staking.StakeAccumulator{ Claims: map[staking.StakeClaim][]staking.ThresholdKind{ @@ -64,6 +68,9 @@ func (th *dummyMigrationHandler) ConsensusUpgrade(ctx *Context, privateCtx inter }, }, }) + if err != nil { + return fmt.Errorf("failed to set account: %w", err) + } return nil } diff --git a/go/worker/common/committee/group.go b/go/worker/common/committee/group.go index a3c94312d10..0ebe60a36c6 100644 --- a/go/worker/common/committee/group.go +++ b/go/worker/common/committee/group.go @@ -205,7 +205,7 @@ func (e *EpochSnapshot) Nodes() committee.NodeDescriptorLookup { // Node looks up a node descriptor. // // Implements commitment.NodeLookup. -func (e *EpochSnapshot) Node(id signature.PublicKey) (*node.Node, error) { +func (e *EpochSnapshot) Node(ctx context.Context, id signature.PublicKey) (*node.Node, error) { n := e.nodes.Lookup(id) if n == nil { return nil, registry.ErrNoSuchNode diff --git a/go/worker/storage/committee/node.go b/go/worker/storage/committee/node.go index d6592c8c2d1..b43c9315e6a 100644 --- a/go/worker/storage/committee/node.go +++ b/go/worker/storage/committee/node.go @@ -379,7 +379,7 @@ func (n *Node) ForceFinalize(ctx context.Context, round uint64) error { if err != nil { return err } - return n.localStorage.Finalize(ctx, block.Header.Namespace, round, []hash.Hash{ + return n.localStorage.NodeDB().Finalize(ctx, block.Header.Namespace, round, []hash.Hash{ block.Header.IORoot, block.Header.StateRoot, }) @@ -397,7 +397,7 @@ func (n *Node) fetchDiff(round uint64, prevRoot *urkelNode.Root, thisRoot *urkel n.diffCh <- result }() // Check if the new root doesn't already exist. - if !n.localStorage.HasRoot(*thisRoot) { + if !n.localStorage.NodeDB().HasRoot(*thisRoot) { result.fetched = true if thisRoot.Hash.Equal(&prevRoot.Hash) { // Even if HasRoot returns false the root can still exist if it is equal @@ -441,7 +441,7 @@ func (n *Node) fetchDiff(round uint64, prevRoot *urkelNode.Root, thisRoot *urkel } func (n *Node) finalize(summary *blockSummary) { - err := n.localStorage.Finalize(n.ctx, summary.Namespace, summary.Round, []hash.Hash{ + err := n.localStorage.NodeDB().Finalize(n.ctx, summary.Namespace, summary.Round, []hash.Hash{ summary.IORoot.Hash, summary.StateRoot.Hash, }) @@ -753,7 +753,7 @@ func (p *pruneHandler) Prune(ctx context.Context, rounds []uint64) error { p.logger.Debug("pruning storage for round", "round", round) // Prune given block. - if _, err := p.node.localStorage.Prune(ctx, p.namespace, round); err != nil { + if _, err := p.node.localStorage.NodeDB().Prune(ctx, p.namespace, round); err != nil { p.logger.Error("failed to prune block", "err", err, )