From 7c619b8c328007934ad3a72d5488126cad7a3ac5 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Tue, 7 Jul 2020 17:26:35 +0200 Subject: [PATCH 01/19] Add state sync support --- CHANGELOG.md | 3 + baseapp/abci.go | 137 ++++- baseapp/baseapp.go | 22 + baseapp/baseapp_test.go | 245 +++++++++ baseapp/options.go | 40 ++ server/config/config.go | 20 + server/config/toml.go | 15 + server/mock/store.go | 8 + server/start.go | 9 + simapp/simd/cmd/root.go | 15 + snapshots/alias.go | 20 + snapshots/helpers_test.go | 144 +++++ snapshots/manager.go | 223 ++++++++ snapshots/manager_test.go | 220 ++++++++ snapshots/store.go | 370 +++++++++++++ snapshots/store_test.go | 366 +++++++++++++ snapshots/types/convert.go | 39 ++ snapshots/types/errors.go | 16 + snapshots/types/format.go | 6 + snapshots/types/snapshotter.go | 14 + snapshots/types/types.pb.go | 664 +++++++++++++++++++++++ snapshots/types/types.proto | 20 + snapshots/util.go | 162 ++++++ snapshots/util_test.go | 166 ++++++ store/iavl/store.go | 23 + store/rootmulti/store.go | 254 +++++++++ store/rootmulti/store_test.go | 276 ++++++++++ store/types/store.go | 2 + store/types/types.pb.go | 951 +++++++++++++++++++++++++++++++++ store/types/types.proto | 28 + 30 files changed, 4462 insertions(+), 16 deletions(-) create mode 100644 snapshots/alias.go create mode 100644 snapshots/helpers_test.go create mode 100644 snapshots/manager.go create mode 100644 snapshots/manager_test.go create mode 100644 snapshots/store.go create mode 100644 snapshots/store_test.go create mode 100644 snapshots/types/convert.go create mode 100644 snapshots/types/errors.go create mode 100644 snapshots/types/format.go create mode 100644 snapshots/types/snapshotter.go create mode 100644 snapshots/types/types.pb.go create mode 100644 snapshots/types/types.proto create mode 100644 snapshots/util.go create mode 100644 snapshots/util_test.go create mode 100644 store/types/types.pb.go create mode 100644 store/types/types.proto diff --git a/CHANGELOG.md b/CHANGELOG.md index 46b7dcf58305..c179b1921d68 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -157,6 +157,7 @@ be used to retrieve the actual proposal `Content`. Also the `NewMsgSubmitProposa * (modules) [\#6734](https://github.com/cosmos/cosmos-sdk/issues/6834) Add `TxEncodingConfig` parameter to `AppModuleBasic.ValidateGenesis` command to support JSON tx decoding in `genutil`. * (genesis) [\#7000](https://github.com/cosmos/cosmos-sdk/pull/7000) The root `GenesisState` is now decoded using `encoding/json` instead of amino so `int64` and `uint64` types are now encoded as integers as opposed to strings. * (types) [\#7032](https://github.com/cosmos/cosmos-sdk/pull/7032) All types ending with `ID` (e.g. `ProposalID`) now end with `Id` (e.g. `ProposalId`), to match default Protobuf generated format. Also see [\#7033](https://github.com/cosmos/cosmos-sdk/pull/7033) for more details. +* (store) [\#5803](https://github.com/cosmos/cosmos-sdk/pull/5803) The `store.CommitMultiStore` interface now includes the new `snapshots.Snapshotter` interface as well. ### Features @@ -166,6 +167,8 @@ be used to retrieve the actual proposal `Content`. Also the `NewMsgSubmitProposa * (crypto/multisig) [\#6241](https://github.com/cosmos/cosmos-sdk/pull/6241) Add Multisig type directly to the repo. Previously this was in tendermint. * (rest) [\#6167](https://github.com/cosmos/cosmos-sdk/pull/6167) Support `max-body-bytes` CLI flag for the REST service. * (x/ibc) [\#5588](https://github.com/cosmos/cosmos-sdk/pull/5588) Add [ICS 024 - Host State Machine Requirements](https://github.com/cosmos/ics/tree/master/spec/ics-024-host-requirements) subpackage to `x/ibc` module. +* (baseapp) [\#5803](https://github.com/cosmos/cosmos-sdk/pull/5803) Added support for taking state snapshots at regular height intervals, via options `snapshot-interval` and `snapshot-retention`. +* (store) [\#5803](https://github.com/cosmos/cosmos-sdk/pull/5803) Added `rootmulti.Store` methods for taking and restoring snapshots, based on `iavl.Store` export/import. * (x/ibc) [\#5277](https://github.com/cosmos/cosmos-sdk/pull/5277) `x/ibc` changes from IBC alpha. For more details check the the [`x/ibc/spec`](https://github.com/cosmos/tree/master/x/ibc/spec) directory: * [ICS 002 - Client Semantics](https://github.com/cosmos/ics/tree/master/spec/ics-002-client-semantics) subpackage * [ICS 003 - Connection Semantics](https://github.com/cosmos/ics/blob/master/spec/ics-003-connection-semantics) subpackage diff --git a/baseapp/abci.go b/baseapp/abci.go index dc64fb96058c..6057907eebab 100644 --- a/baseapp/abci.go +++ b/baseapp/abci.go @@ -1,6 +1,7 @@ package baseapp import ( + "errors" "fmt" "os" "sort" @@ -15,6 +16,7 @@ import ( grpcstatus "google.golang.org/grpc/status" "github.com/cosmos/cosmos-sdk/codec" + "github.com/cosmos/cosmos-sdk/snapshots" "github.com/cosmos/cosmos-sdk/telemetry" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" @@ -297,6 +299,10 @@ func (app *BaseApp) Commit() (res abci.ResponseCommit) { app.halt() } + if app.snapshotInterval > 0 && uint64(header.Height)%app.snapshotInterval == 0 { + go app.snapshot(header.Height) + } + return abci.ResponseCommit{ Data: commitID.Hash, } @@ -324,6 +330,27 @@ func (app *BaseApp) halt() { os.Exit(0) } +// snapshot takes a snapshot of the current state and prunes any old snapshots. +func (app *BaseApp) snapshot(height int64) { + app.logger.Info("Taking state snapshot", "height", height) + snapshot, err := app.snapshotManager.Take(uint64(height)) + if err != nil { + app.logger.Error("Failed to take state snapshot", "height", height, "err", err) + return + } + app.logger.Info("Completed state snapshot", "height", height, "format", snapshot.Format) + + if app.snapshotKeepRecent > 0 { + app.logger.Debug("Pruning state snapshots") + pruned, err := app.snapshotManager.Prune(app.snapshotKeepRecent) + if err != nil { + app.logger.Error("Failed to prune state snapshots", "err", err.Error()) + return + } + app.logger.Debug("Pruned state snapshots", "pruned", pruned) + } +} + // Query implements the ABCI interface. It delegates to CommitMultiStore if it // implements Queryable. func (app *BaseApp) Query(req abci.RequestQuery) abci.ResponseQuery { @@ -358,6 +385,100 @@ func (app *BaseApp) Query(req abci.RequestQuery) abci.ResponseQuery { return sdkerrors.QueryResult(sdkerrors.Wrap(sdkerrors.ErrUnknownRequest, "unknown query path")) } +// ListSnapshots implements the ABCI interface. It delegates to app.snapshotManager if set. +func (app *BaseApp) ListSnapshots(req abci.RequestListSnapshots) abci.ResponseListSnapshots { + resp := abci.ResponseListSnapshots{Snapshots: []*abci.Snapshot{}} + if app.snapshotManager == nil { + return resp + } + + snapshots, err := app.snapshotManager.List() + if err != nil { + app.logger.Error("Failed to list snapshots", "err", err.Error()) + return resp + } + for _, snapshot := range snapshots { + abciSnapshot, err := snapshot.ToABCI() + if err != nil { + app.logger.Error("Failed to list snapshots", "err", err.Error()) + return resp + } + resp.Snapshots = append(resp.Snapshots, &abciSnapshot) + } + + return resp +} + +// LoadSnapshotChunk implements the ABCI interface. It delegates to app.snapshotManager if set. +func (app *BaseApp) LoadSnapshotChunk(req abci.RequestLoadSnapshotChunk) abci.ResponseLoadSnapshotChunk { + if app.snapshotManager == nil { + return abci.ResponseLoadSnapshotChunk{} + } + chunk, err := app.snapshotManager.LoadChunk(req.Height, req.Format, req.Chunk) + if err != nil { + app.logger.Error("Failed to load snapshot chunk", "height", req.Height, "format", req.Format, + "chunk", req.Chunk, "err", err.Error()) + return abci.ResponseLoadSnapshotChunk{} + } + return abci.ResponseLoadSnapshotChunk{Chunk: chunk} +} + +// OfferSnapshot implements the ABCI interface. It delegates to app.snapshotManager if set. +func (app *BaseApp) OfferSnapshot(req abci.RequestOfferSnapshot) abci.ResponseOfferSnapshot { + if req.Snapshot == nil { + app.logger.Error("Received nil snapshot") + return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT} + } + + snapshot, err := snapshots.SnapshotFromABCI(req.Snapshot) + if err != nil { + app.logger.Error("Failed to decode snapshot metadata", "err", err) + return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT} + } + err = app.snapshotManager.Restore(snapshot) + switch { + case err == nil: + return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ACCEPT} + + case errors.Is(err, snapshots.ErrUnknownFormat): + return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT_FORMAT} + + case errors.Is(err, snapshots.ErrInvalidMetadata): + app.logger.Error("Rejecting invalid snapshot", "height", req.Snapshot.Height, + "format", req.Snapshot.Format, "err", err.Error()) + return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT} + + default: + app.logger.Error("Failed to restore snapshot", "height", req.Snapshot.Height, + "format", req.Snapshot.Format, "err", err.Error()) + // We currently don't support resetting the IAVL stores and retrying a different snapshot, + // so we ask Tendermint to abort all snapshot restoration. + return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ABORT} + } +} + +// ApplySnapshotChunk implements the ABCI interface. It delegates to app.snapshotManager if set. +func (app *BaseApp) ApplySnapshotChunk(req abci.RequestApplySnapshotChunk) abci.ResponseApplySnapshotChunk { + _, err := app.snapshotManager.RestoreChunk(req.Chunk) + switch { + case err == nil: + return abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT} + + case errors.Is(err, snapshots.ErrChunkHashMismatch): + app.logger.Error("Chunk checksum mismatch, rejecting sender and requesting refetch", + "chunk", req.Index, "sender", req.Sender, "err", err) + return abci.ResponseApplySnapshotChunk{ + Result: abci.ResponseApplySnapshotChunk_RETRY, + RefetchChunks: []uint32{req.Index}, + RejectSenders: []string{req.Sender}, + } + + default: + app.logger.Error("Failed to restore snapshot", "err", err.Error()) + return abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ABORT} + } +} + func (app *BaseApp) handleQueryGRPC(handler GRPCQueryHandler, req abci.RequestQuery) abci.ResponseQuery { ctx, err := app.createQueryContext(req.Height, req.Prove) if err != nil { @@ -584,19 +705,3 @@ func splitPath(requestPath string) (path []string) { return path } - -func (app *BaseApp) ListSnapshots(abci.RequestListSnapshots) abci.ResponseListSnapshots { - return abci.ResponseListSnapshots{} -} - -func (app *BaseApp) OfferSnapshot(abci.RequestOfferSnapshot) abci.ResponseOfferSnapshot { - return abci.ResponseOfferSnapshot{} -} - -func (app *BaseApp) LoadSnapshotChunk(abci.RequestLoadSnapshotChunk) abci.ResponseLoadSnapshotChunk { - return abci.ResponseLoadSnapshotChunk{} -} - -func (app *BaseApp) ApplySnapshotChunk(abci.RequestApplySnapshotChunk) abci.ResponseApplySnapshotChunk { - return abci.ResponseApplySnapshotChunk{} -} diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 4455696afb61..2868ad7996cc 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -1,6 +1,7 @@ package baseapp import ( + "errors" "fmt" "reflect" "strings" @@ -12,7 +13,9 @@ import ( tmproto "github.com/tendermint/tendermint/proto/tendermint/types" dbm "github.com/tendermint/tm-db" + "github.com/cosmos/cosmos-sdk/snapshots" "github.com/cosmos/cosmos-sdk/store" + "github.com/cosmos/cosmos-sdk/store/rootmulti" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" ) @@ -60,6 +63,11 @@ type BaseApp struct { // nolint: maligned idPeerFilter sdk.PeerFilter // filter peers by node ID fauxMerkleMode bool // if true, IAVL MountStores uses MountStoresDB for simulation speed. + // manages snapshots, i.e. dumps of app state at certain intervals + snapshotManager *snapshots.Manager + snapshotInterval uint64 // block interval between state sync snapshots + snapshotKeepRecent uint32 // recent state sync snapshots to keep + // volatile states: // // checkState is set on InitChain and reset on Commit @@ -264,6 +272,20 @@ func (app *BaseApp) init() error { app.setCheckState(tmproto.Header{}) app.Seal() + // make sure the snapshot interval is a multiple of the pruning KeepEvery interval + if app.snapshotManager != nil && app.snapshotInterval > 0 { + rms, ok := app.cms.(*rootmulti.Store) + if !ok { + return errors.New("state sync snapshots require a rootmulti store") + } + pruningOpts := rms.GetPruning() + if pruningOpts.KeepEvery > 0 && app.snapshotInterval%pruningOpts.KeepEvery != 0 { + return fmt.Errorf( + "state sync snapshot interval %v must be a multiple of pruning keep every interval %v", + app.snapshotInterval, pruningOpts.KeepEvery) + } + } + return nil } diff --git a/baseapp/baseapp_test.go b/baseapp/baseapp_test.go index 05038ce2c36f..025be865b70c 100644 --- a/baseapp/baseapp_test.go +++ b/baseapp/baseapp_test.go @@ -5,10 +5,12 @@ import ( "encoding/binary" "encoding/json" "fmt" + "math/rand" "os" "strings" "sync" "testing" + "time" "github.com/gogo/protobuf/jsonpb" "github.com/stretchr/testify/assert" @@ -19,6 +21,7 @@ import ( dbm "github.com/tendermint/tm-db" "github.com/cosmos/cosmos-sdk/codec" + "github.com/cosmos/cosmos-sdk/snapshots" "github.com/cosmos/cosmos-sdk/store/rootmulti" store "github.com/cosmos/cosmos-sdk/store/types" "github.com/cosmos/cosmos-sdk/testutil/testdata" @@ -88,6 +91,7 @@ func registerTestCodec(cdc *codec.LegacyAmino) { cdc.RegisterConcrete(&txTest{}, "cosmos-sdk/baseapp/txTest", nil) cdc.RegisterConcrete(&msgCounter{}, "cosmos-sdk/baseapp/msgCounter", nil) cdc.RegisterConcrete(&msgCounter2{}, "cosmos-sdk/baseapp/msgCounter2", nil) + cdc.RegisterConcrete(&msgKeyValue{}, "cosmos-sdk/baseapp/msgKeyValue", nil) cdc.RegisterConcrete(&msgNoRoute{}, "cosmos-sdk/baseapp/msgNoRoute", nil) } @@ -105,6 +109,63 @@ func setupBaseApp(t *testing.T, options ...func(*BaseApp)) *BaseApp { return app } +// simple one store baseapp with data and snapshots. Each tx is 1 MB in size (uncompressed). +func setupBaseAppWithSnapshots(t *testing.T, blocks uint, blockTxs int, options ...func(*BaseApp)) (*BaseApp, func()) { + codec := codec.New() + registerTestCodec(codec) + routerOpt := func(bapp *BaseApp) { + bapp.Router().AddRoute(sdk.NewRoute(routeMsgKeyValue, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { + kv := msg.(*msgKeyValue) + bapp.cms.GetCommitKVStore(capKey2).Set(kv.Key, kv.Value) + return &sdk.Result{}, nil + })) + } + + snapshotDir := os.TempDir() + snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), snapshotDir) + require.NoError(t, err) + teardown := func() { + os.RemoveAll(snapshotDir) + } + + app := setupBaseApp(t, append(options, + SetSnapshotStore(snapshotStore), + SetSnapshotInterval(2), + SetPruning(sdk.PruningOptions{KeepEvery: 1}), + routerOpt)...) + + app.InitChain(abci.RequestInitChain{}) + + r := rand.New(rand.NewSource(3920758213583)) + keyCounter := 0 + for height := int64(1); height <= int64(blocks); height++ { + app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: height}}) + for txNum := 0; txNum < blockTxs; txNum++ { + tx := txTest{Msgs: []sdk.Msg{}} + for msgNum := 0; msgNum < 100; msgNum++ { + key := []byte(fmt.Sprintf("%v", keyCounter)) + value := make([]byte, 10000) + _, err := r.Read(value) + require.NoError(t, err) + tx.Msgs = append(tx.Msgs, msgKeyValue{Key: key, Value: value}) + keyCounter++ + } + txBytes, err := codec.MarshalBinaryBare(tx) + require.NoError(t, err) + resp := app.DeliverTx(abci.RequestDeliverTx{Tx: txBytes}) + require.True(t, resp.IsOK(), "%v", resp.String()) + } + app.EndBlock(abci.RequestEndBlock{Height: height}) + app.Commit() + + // Wait for snapshot to be taken, since it happens asynchronously. This + // heuristic is likely to be flaky on low-IO machines. + time.Sleep(time.Duration(int(height)*blockTxs) * 200 * time.Millisecond) + } + + return app, teardown +} + func TestMountStores(t *testing.T) { app := setupBaseApp(t) @@ -559,6 +620,7 @@ func (tx txTest) ValidateBasic() error { return nil } const ( routeMsgCounter = "msgCounter" routeMsgCounter2 = "msgCounter2" + routeMsgKeyValue = "msgKeyValue" ) // ValidateBasic() fails on negative counters. @@ -630,6 +692,29 @@ func (msg msgCounter2) ValidateBasic() error { return sdkerrors.Wrap(sdkerrors.ErrInvalidSequence, "counter should be a non-negative integer") } +// A msg that sets a key/value pair. +type msgKeyValue struct { + Key []byte + Value []byte +} + +func (msg msgKeyValue) Reset() {} +func (msg msgKeyValue) String() string { return "TODO" } +func (msg msgKeyValue) ProtoMessage() {} +func (msg msgKeyValue) Route() string { return routeMsgKeyValue } +func (msg msgKeyValue) Type() string { return "keyValue" } +func (msg msgKeyValue) GetSignBytes() []byte { return nil } +func (msg msgKeyValue) GetSigners() []sdk.AccAddress { return nil } +func (msg msgKeyValue) ValidateBasic() error { + if msg.Key == nil { + return sdkerrors.Wrap(sdkerrors.ErrInvalidRequest, "key cannot be nil") + } + if msg.Value == nil { + return sdkerrors.Wrap(sdkerrors.ErrInvalidRequest, "value cannot be nil") + } + return nil +} + // amino decode func testTxDecoder(cdc *codec.LegacyAmino) sdk.TxDecoder { return func(txBytes []byte) (sdk.Tx, error) { @@ -1617,6 +1702,166 @@ func TestGetMaximumBlockGas(t *testing.T) { require.Panics(t, func() { app.getMaximumBlockGas(ctx) }) } +func TestListSnapshots(t *testing.T) { + app, teardown := setupBaseAppWithSnapshots(t, 5, 4) + defer teardown() + + resp := app.ListSnapshots(abci.RequestListSnapshots{}) + for _, s := range resp.Snapshots { + assert.NotEmpty(t, s.Hash) + assert.NotEmpty(t, s.Metadata) + s.Hash = nil + s.Metadata = nil + } + assert.Equal(t, abci.ResponseListSnapshots{Snapshots: []*abci.Snapshot{ + {Height: 4, Format: 1, Chunks: 2}, + {Height: 2, Format: 1, Chunks: 1}, + }}, resp) +} + +func TestLoadSnapshotChunk(t *testing.T) { + app, teardown := setupBaseAppWithSnapshots(t, 2, 5) + defer teardown() + + testcases := map[string]struct { + height uint64 + format uint32 + chunk uint32 + expectEmpty bool + }{ + "Existing snapshot": {2, 1, 1, false}, + "Missing height": {100, 1, 1, true}, + "Missing format": {2, 2, 1, true}, + "Missing chunk": {2, 1, 9, true}, + "Zero height": {0, 1, 1, true}, + "Zero format": {2, 0, 1, true}, + "Zero chunk": {2, 1, 0, false}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + resp := app.LoadSnapshotChunk(abci.RequestLoadSnapshotChunk{ + Height: tc.height, + Format: tc.format, + Chunk: tc.chunk, + }) + if tc.expectEmpty { + assert.Equal(t, abci.ResponseLoadSnapshotChunk{}, resp) + return + } + assert.NotEmpty(t, resp.Chunk) + }) + } +} + +func TestOfferSnapshot_Errors(t *testing.T) { + // Set up app before test cases, since it's fairly expensive. + app, teardown := setupBaseAppWithSnapshots(t, 0, 0) + defer teardown() + + m := snapshots.Metadata{ChunkHashes: [][]byte{{1}, {2}, {3}}} + metadata, err := m.Marshal() + require.NoError(t, err) + hash := []byte{1, 2, 3} + + testcases := map[string]struct { + snapshot *abci.Snapshot + result abci.ResponseOfferSnapshot_Result + }{ + "nil snapshot": {nil, abci.ResponseOfferSnapshot_REJECT}, + "invalid format": {&abci.Snapshot{ + Height: 1, Format: 9, Chunks: 3, Hash: hash, Metadata: metadata, + }, abci.ResponseOfferSnapshot_REJECT_FORMAT}, + "incorrect chunk count": {&abci.Snapshot{ + Height: 1, Format: 1, Chunks: 2, Hash: hash, Metadata: metadata, + }, abci.ResponseOfferSnapshot_REJECT}, + "no chunks": {&abci.Snapshot{ + Height: 1, Format: 1, Chunks: 0, Hash: hash, Metadata: metadata, + }, abci.ResponseOfferSnapshot_REJECT}, + "invalid metadata serialization": {&abci.Snapshot{ + Height: 1, Format: 1, Chunks: 0, Hash: hash, Metadata: []byte{3, 1, 4}, + }, abci.ResponseOfferSnapshot_REJECT}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + resp := app.OfferSnapshot(abci.RequestOfferSnapshot{Snapshot: tc.snapshot}) + assert.Equal(t, tc.result, resp.Result) + }) + } + + // Offering a snapshot after one has been accepted should error + resp := app.OfferSnapshot(abci.RequestOfferSnapshot{Snapshot: &abci.Snapshot{ + Height: 1, + Format: snapshots.CurrentFormat, + Chunks: 3, + Hash: []byte{1, 2, 3}, + Metadata: metadata, + }}) + require.Equal(t, abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ACCEPT}, resp) + + resp = app.OfferSnapshot(abci.RequestOfferSnapshot{Snapshot: &abci.Snapshot{ + Height: 2, + Format: snapshots.CurrentFormat, + Chunks: 3, + Hash: []byte{1, 2, 3}, + Metadata: metadata, + }}) + require.Equal(t, abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ABORT}, resp) +} + +func TestApplySnapshotChunk(t *testing.T) { + source, teardown := setupBaseAppWithSnapshots(t, 4, 10) + defer teardown() + + target, teardown := setupBaseAppWithSnapshots(t, 0, 0) + defer teardown() + + // Fetch latest snapshot to restore + respList := source.ListSnapshots(abci.RequestListSnapshots{}) + require.NotEmpty(t, respList.Snapshots) + snapshot := respList.Snapshots[0] + + // Make sure the snapshot has at least 3 chunks + require.GreaterOrEqual(t, snapshot.Chunks, uint32(3), "Not enough snapshot chunks") + + // Begin a snapshot restoration in the target + respOffer := target.OfferSnapshot(abci.RequestOfferSnapshot{Snapshot: snapshot}) + require.Equal(t, abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ACCEPT}, respOffer) + + // We should be able to pass an invalid chunk and get a verify failure, before reapplying it. + respApply := target.ApplySnapshotChunk(abci.RequestApplySnapshotChunk{ + Index: 0, + Chunk: []byte{9}, + Sender: "sender", + }) + require.Equal(t, abci.ResponseApplySnapshotChunk{ + Result: abci.ResponseApplySnapshotChunk_RETRY, + RefetchChunks: []uint32{0}, + RejectSenders: []string{"sender"}, + }, respApply) + + // Fetch each chunk from the source and apply it to the target + for index := uint32(0); index < snapshot.Chunks; index++ { + respChunk := source.LoadSnapshotChunk(abci.RequestLoadSnapshotChunk{ + Height: snapshot.Height, + Format: snapshot.Format, + Chunk: index, + }) + require.NotNil(t, respChunk.Chunk) + respApply := target.ApplySnapshotChunk(abci.RequestApplySnapshotChunk{ + Index: index, + Chunk: respChunk.Chunk, + }) + require.Equal(t, abci.ResponseApplySnapshotChunk{ + Result: abci.ResponseApplySnapshotChunk_ACCEPT, + }, respApply) + } + + // The target should now have the same hash as the source + assert.Equal(t, source.LastCommitID(), target.LastCommitID()) +} + // NOTE: represents a new custom router for testing purposes of WithRouter() type testCustomRouter struct { routes sync.Map diff --git a/baseapp/options.go b/baseapp/options.go index a80745fc45fc..22483415fec7 100644 --- a/baseapp/options.go +++ b/baseapp/options.go @@ -6,6 +6,7 @@ import ( dbm "github.com/tendermint/tm-db" + "github.com/cosmos/cosmos-sdk/snapshots" "github.com/cosmos/cosmos-sdk/store" sdk "github.com/cosmos/cosmos-sdk/types" ) @@ -54,6 +55,21 @@ func SetInterBlockCache(cache sdk.MultiStorePersistentCache) func(*BaseApp) { return func(app *BaseApp) { app.setInterBlockCache(cache) } } +// SetSnapshotInterval sets the snapshot interval. +func SetSnapshotInterval(interval uint64) func(*BaseApp) { + return func(app *BaseApp) { app.SetSnapshotInterval(interval) } +} + +// SetSnapshotKeepRecent sets the recent snapshots to keep. +func SetSnapshotKeepRecent(keepRecent uint32) func(*BaseApp) { + return func(app *BaseApp) { app.SetSnapshotKeepRecent(keepRecent) } +} + +// SetSnapshotStore sets the snapshot store. +func SetSnapshotStore(snapshotStore *snapshots.Store) func(*BaseApp) { + return func(app *BaseApp) { app.SetSnapshotStore(snapshotStore) } +} + func (app *BaseApp) SetName(name string) { if app.sealed { panic("SetName() on sealed BaseApp") @@ -174,3 +190,27 @@ func (app *BaseApp) SetRouter(router sdk.Router) { } app.router = router } + +// SetSnapshotStore sets the snapshot store. +func (app *BaseApp) SetSnapshotStore(snapshotStore *snapshots.Store) { + if app.sealed { + panic("SetSnapshotStore() on sealed BaseApp") + } + app.snapshotManager = snapshots.NewManager(snapshotStore, app.cms) +} + +// SetSnapshotInterval sets the snapshot interval. +func (app *BaseApp) SetSnapshotInterval(snapshotInterval uint64) { + if app.sealed { + panic("SetSnapshotInterval() on sealed BaseApp") + } + app.snapshotInterval = snapshotInterval +} + +// SetSnapshotKeepRecent sets the number of recent snapshots to keep. +func (app *BaseApp) SetSnapshotKeepRecent(snapshotKeepRecent uint32) { + if app.sealed { + panic("SetSnapshotKeepRecent() on sealed BaseApp") + } + app.snapshotKeepRecent = snapshotKeepRecent +} diff --git a/server/config/config.go b/server/config/config.go index 40c6af03c4d6..3baac8164453 100644 --- a/server/config/config.go +++ b/server/config/config.go @@ -91,6 +91,17 @@ type GRPCConfig struct { Address string `mapstructure:"address"` } +// StateSyncConfig defines the state sync snapshot configuration. +type StateSyncConfig struct { + // SnapshotInterval sets the interval at which state sync snapshots are taken. + // 0 disables snapshots. Must be a multiple of PruningKeepEvery. + SnapshotInterval uint64 `mapstructure:"snapshot-interval"` + + // SnapshotKeepRecent sets the number of recent state sync snapshots to keep. + // 0 keeps all snapshots. + SnapshotKeepRecent uint32 `mapstructure:"snapshot-keep-recent"` +} + // Config defines the server's top level configuration type Config struct { BaseConfig `mapstructure:",squash"` @@ -99,6 +110,7 @@ type Config struct { Telemetry telemetry.Config `mapstructure:"telemetry"` API APIConfig `mapstructure:"api"` GRPC GRPCConfig `mapstructure:"grpc"` + StateSync StateSyncConfig `mapstructure:"state-sync"` } // SetMinGasPrices sets the validator's minimum gas prices. @@ -156,6 +168,10 @@ func DefaultConfig() *Config { Enable: true, Address: DefaultGRPCAddress, }, + StateSync: StateSyncConfig{ + SnapshotInterval: 0, + SnapshotKeepRecent: 2, + }, } } @@ -204,5 +220,9 @@ func GetConfig(v *viper.Viper) Config { Enable: v.GetBool("grpc.enable"), Address: v.GetString("grpc.address"), }, + StateSync: StateSyncConfig{ + SnapshotInterval: v.GetUint64("state-sync.snapshot-interval"), + SnapshotKeepRecent: v.GetUint32("state-sync.snapshot-keep-recent"), + }, } } diff --git a/server/config/toml.go b/server/config/toml.go index a0b6a3681f75..41497dc497c5 100644 --- a/server/config/toml.go +++ b/server/config/toml.go @@ -130,6 +130,21 @@ enable = {{ .GRPC.Enable }} # Address defines the gRPC server address to bind to. address = "{{ .GRPC.Address }}" + +############################################################################### +### State Sync Configuration ### +############################################################################### + +# State sync snapshots allow other nodes to rapidly join the network without replaying historical +# blocks, instead downloading and applying a snapshot of the application state at a given height. +[state-sync] + +# snapshot-interval specifies the block interval at which local state sync snapshots are +# taken (0 to disable). Must be a multiple of pruning-keep-every. +snapshot-interval = {{ .StateSync.SnapshotInterval }} + +# snapshot-keep-recent specifies the number of recent snapshots to keep and serve (0 to keep all). +snapshot-keep-recent = {{ .StateSync.SnapshotKeepRecent }} ` var configTemplate *template.Template diff --git a/server/mock/store.go b/server/mock/store.go index d8e9cee1c200..bceb73a92c39 100644 --- a/server/mock/store.go +++ b/server/mock/store.go @@ -99,6 +99,14 @@ func (ms multiStore) SetInterBlockCache(_ sdk.MultiStorePersistentCache) { panic("not implemented") } +func (ms multiStore) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { + panic("not implemented") +} + +func (ms multiStore) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser) error { + panic("not implemented") +} + var _ sdk.KVStore = kvStore{} type kvStore struct { diff --git a/server/start.go b/server/start.go index f4571e06e6cc..ab8b97b534ec 100644 --- a/server/start.go +++ b/server/start.go @@ -57,6 +57,12 @@ const ( flagGRPCAddress = "grpc.address" ) +// State sync-related flags. +const ( + FlagStateSyncSnapshotInterval = "state-sync.snapshot-interval" + FlagStateSyncSnapshotKeepRecent = "state-sync.snapshot-keep-recent" +) + // StartCmd runs the service passed in, either stand-alone or in-process with // Tendermint. func StartCmd(appCreator types.AppCreator, defaultNodeHome string) *cobra.Command { @@ -134,6 +140,9 @@ which accepts a path for the resulting pprof file. cmd.Flags().Bool(flagGRPCEnable, true, "Define if the gRPC server should be enabled") cmd.Flags().String(flagGRPCAddress, config.DefaultGRPCAddress, "the gRPC server address to listen on") + cmd.Flags().Uint64(FlagStateSyncSnapshotInterval, 0, "State sync snapshot interval") + cmd.Flags().Uint32(FlagStateSyncSnapshotKeepRecent, 2, "State sync snapshot to keep") + // add support for all Tendermint-specific command line options tcmd.AddNodeFlags(cmd) return cmd diff --git a/simapp/simd/cmd/root.go b/simapp/simd/cmd/root.go index ab961123e724..9db7d4c5be99 100644 --- a/simapp/simd/cmd/root.go +++ b/simapp/simd/cmd/root.go @@ -5,9 +5,11 @@ import ( "encoding/json" "io" "os" + "path/filepath" "github.com/cosmos/cosmos-sdk/codec" "github.com/cosmos/cosmos-sdk/simapp/params" + "github.com/cosmos/cosmos-sdk/snapshots" "github.com/spf13/cast" "github.com/spf13/cobra" @@ -176,6 +178,16 @@ func newApp(logger log.Logger, db dbm.DB, traceStore io.Writer, appOpts serverty panic(err) } + snapshotDir := filepath.Join(cast.ToString(appOpts.Get(flags.FlagHome)), "data", "snapshots") + snapshotDB, err := sdk.NewLevelDB("metadata", snapshotDir) + if err != nil { + panic(err) + } + snapshotStore, err := snapshots.NewStore(snapshotDB, snapshotDir) + if err != nil { + panic(err) + } + return simapp.NewSimApp( logger, db, traceStore, true, skipUpgradeHeights, cast.ToString(appOpts.Get(flags.FlagHome)), @@ -188,6 +200,9 @@ func newApp(logger log.Logger, db dbm.DB, traceStore io.Writer, appOpts serverty baseapp.SetInterBlockCache(cache), baseapp.SetTrace(cast.ToBool(appOpts.Get(server.FlagTrace))), baseapp.SetIndexEvents(cast.ToStringSlice(appOpts.Get(server.FlagIndexEvents))), + baseapp.SetSnapshotStore(snapshotStore), + baseapp.SetSnapshotInterval(cast.ToUint64(appOpts.Get(server.FlagStateSyncSnapshotInterval))), + baseapp.SetSnapshotKeepRecent(cast.ToUint32(appOpts.Get(server.FlagStateSyncSnapshotKeepRecent))), ) } diff --git a/snapshots/alias.go b/snapshots/alias.go new file mode 100644 index 000000000000..9b292f547733 --- /dev/null +++ b/snapshots/alias.go @@ -0,0 +1,20 @@ +package snapshots + +import "github.com/cosmos/cosmos-sdk/snapshots/types" + +const ( + CurrentFormat = types.CurrentFormat +) + +var ( + ErrInvalidMetadata = types.ErrInvalidMetadata + ErrChunkHashMismatch = types.ErrChunkHashMismatch + ErrUnknownFormat = types.ErrUnknownFormat + SnapshotFromABCI = types.SnapshotFromABCI +) + +type ( + Snapshotter = types.Snapshotter + Snapshot = types.Snapshot + Metadata = types.Metadata +) diff --git a/snapshots/helpers_test.go b/snapshots/helpers_test.go new file mode 100644 index 000000000000..061d9c12a868 --- /dev/null +++ b/snapshots/helpers_test.go @@ -0,0 +1,144 @@ +package snapshots_test + +import ( + "bytes" + "crypto/sha256" + "errors" + "io" + "io/ioutil" + "os" + "testing" + "time" + + "github.com/stretchr/testify/require" + db "github.com/tendermint/tm-db" + + "github.com/cosmos/cosmos-sdk/snapshots" +) + +func checksum(b []byte) []byte { + hash := sha256.Sum256(b) + return hash[:] +} + +func checksums(slice [][]byte) [][]byte { + checksums := [][]byte{} + for _, chunk := range slice { + checksums = append(checksums, checksum(chunk)) + } + return checksums +} + +func hash(chunks [][]byte) []byte { + hasher := sha256.New() + for _, chunk := range chunks { + hasher.Write(chunk) + } + return hasher.Sum(nil) +} + +func makeChunks(chunks [][]byte) <-chan io.ReadCloser { + ch := make(chan io.ReadCloser, len(chunks)) + for _, chunk := range chunks { + ch <- ioutil.NopCloser(bytes.NewReader(chunk)) + } + close(ch) + return ch +} + +func readChunks(chunks <-chan io.ReadCloser) [][]byte { + bodies := [][]byte{} + for chunk := range chunks { + body, err := ioutil.ReadAll(chunk) + if err != nil { + panic(err) + } + bodies = append(bodies, body) + } + return bodies +} + +type mockSnapshotter struct { + chunks [][]byte +} + +func (m *mockSnapshotter) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser) error { + if format == 0 { + return snapshots.ErrUnknownFormat + } + if m.chunks != nil { + return errors.New("already has contents") + } + + m.chunks = [][]byte{} + for reader := range chunks { + chunk, err := ioutil.ReadAll(reader) + if err != nil { + return err + } + m.chunks = append(m.chunks, chunk) + } + + return nil +} + +func (m *mockSnapshotter) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { + if format == 0 { + return nil, snapshots.ErrUnknownFormat + } + ch := make(chan io.ReadCloser, len(m.chunks)) + for _, chunk := range m.chunks { + ch <- ioutil.NopCloser(bytes.NewReader(chunk)) + } + close(ch) + return ch, nil +} + +// setupBusyManager creates a manager with an empty store that is busy taking a snapshot at height 1. +// The snapshot will complete when the returned closer is called. +func setupBusyManager(t *testing.T) (*snapshots.Manager, func()) { + tempdir, err := ioutil.TempDir("", "") + require.NoError(t, err) + store, err := snapshots.NewStore(db.NewMemDB(), tempdir) + require.NoError(t, err) + hung := newHungSnapshotter() + mgr := snapshots.NewManager(store, hung) + + go func() { + _, err := mgr.Take(1) + require.NoError(t, err) + }() + time.Sleep(10 * time.Millisecond) + + closer := func() { + hung.Close() + os.RemoveAll(tempdir) + } + return mgr, closer +} + +// hungSnapshotter can be used to test operations in progress. Call close to end the snapshot. +type hungSnapshotter struct { + ch chan struct{} +} + +func newHungSnapshotter() *hungSnapshotter { + return &hungSnapshotter{ + ch: make(chan struct{}), + } +} + +func (m *hungSnapshotter) Close() { + close(m.ch) +} + +func (m *hungSnapshotter) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { + <-m.ch + ch := make(chan io.ReadCloser, 1) + ch <- ioutil.NopCloser(bytes.NewReader([]byte{})) + return ch, nil +} + +func (m *hungSnapshotter) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser) error { + panic("not implemented") +} diff --git a/snapshots/manager.go b/snapshots/manager.go new file mode 100644 index 000000000000..93adc43e2cca --- /dev/null +++ b/snapshots/manager.go @@ -0,0 +1,223 @@ +package snapshots + +import ( + "bytes" + "crypto/sha256" + "errors" + "fmt" + "io" + "io/ioutil" + "sync" + "time" +) + +const ( + opNone operation = "" + opSnapshot operation = "snapshot" + opPrune operation = "prune" + opRestore operation = "restore" + + chunkBufferSize = 4 +) + +// operation represents a Manager operation. Only one operation can be in progress at a time. +type operation string + +// Manager manages snapshot and restore operations for an app, making sure only a single +// long-running operation is in progress at any given time, and provides convenience methods +// mirroring the ABCI interface. +type Manager struct { + store *Store + target Snapshotter + + mtx sync.Mutex + operation operation + chRestore chan<- io.ReadCloser + chRestoreDone <-chan error + restorePending [][]byte // pending chunk hashes +} + +// NewManager creates a new manager. +func NewManager(store *Store, target Snapshotter) *Manager { + return &Manager{ + store: store, + target: target, + } +} + +// begin starts an operation, or errors if one is in progress. It manages the mutex itself. +func (m *Manager) begin(op operation) error { + m.mtx.Lock() + defer m.mtx.Unlock() + return m.beginLocked(op) +} + +// beginLocked begins an operation while already holding the mutex. +func (m *Manager) beginLocked(op operation) error { + if op == opNone { + return errors.New("can't begin a none operation") + } + if m.operation != opNone { + return fmt.Errorf("a %v operation is in progress", m.operation) + } + m.operation = op + return nil +} + +// end ends the current operation. +func (m *Manager) end() { + m.mtx.Lock() + defer m.mtx.Unlock() + m.endLocked() +} + +// endLocked ends the current operation while already holding the mutex. +func (m *Manager) endLocked() { + m.operation = opNone + if m.chRestore != nil { + close(m.chRestore) + m.chRestore = nil + } + m.chRestoreDone = nil + m.restorePending = nil +} + +// List lists snapshots, mirroring ABCI ListSnapshots. It can be concurrent with other operations. +func (m *Manager) List() ([]*Snapshot, error) { + return m.store.List() +} + +// LoadChunk loads a chunk into a byte slice, mirroring ABCI LoadChunk. It can be called +// concurrently with other operations. If the chunk does not exist, nil is returned. +func (m *Manager) LoadChunk(height uint64, format uint32, chunk uint32) ([]byte, error) { + reader, err := m.store.LoadChunk(height, format, chunk) + if err != nil { + return nil, err + } + if reader == nil { + return nil, nil + } + defer reader.Close() + + return ioutil.ReadAll(reader) +} + +// Take takes a snapshot and returns its metadata. +func (m *Manager) Take(height uint64) (*Snapshot, error) { + if m == nil { + return nil, errors.New("no snapshot store configured") + } + err := m.begin(opSnapshot) + if err != nil { + return nil, err + } + defer m.end() + + latest, err := m.store.GetLatest() + if err != nil { + return nil, fmt.Errorf("failed to examine latest snapshot: %w", err) + } + if latest != nil && latest.Height >= height { + return nil, fmt.Errorf("a more recent snapshot already exists at height %v", latest.Height) + } + + chunks, err := m.target.Snapshot(height, CurrentFormat) + if err != nil { + return nil, err + } + return m.store.Save(height, CurrentFormat, chunks) +} + +// Prune prunes snapshots, if no other operations are in progress. +func (m *Manager) Prune(retain uint32) (uint64, error) { + err := m.begin(opPrune) + if err != nil { + return 0, err + } + defer m.end() + return m.store.Prune(retain) +} + +// Restore begins an async snapshot restoration, mirroring ABCI OfferSnapshot. Chunks must be fed +// via RestoreChunk() until the restore is complete or a chunk fails. +func (m *Manager) Restore(snapshot Snapshot) error { + if snapshot.Chunks == 0 { + return fmt.Errorf("%w: no chunks", ErrInvalidMetadata) + } + if uint32(len(snapshot.Metadata.ChunkHashes)) != snapshot.Chunks { + return fmt.Errorf("%w: snapshot has %v chunk hashes, but %v chunks", + ErrInvalidMetadata, + uint32(len(snapshot.Metadata.ChunkHashes)), + snapshot.Chunks) + } + m.mtx.Lock() + defer m.mtx.Unlock() + err := m.beginLocked(opRestore) + if err != nil { + return err + } + + // Start an asynchronous snapshot restoration, passing chunks and completion status via channels. + chChunks := make(chan io.ReadCloser, chunkBufferSize) + chDone := make(chan error, 1) + go func() { + chDone <- m.target.Restore(snapshot.Height, snapshot.Format, chChunks) + close(chDone) + }() + + // Check for any initial errors from the restore, before any chunks are fed. + select { + case err := <-chDone: + if err == nil { + err = errors.New("restore ended unexpectedly") + } + m.endLocked() + return err + case <-time.After(20 * time.Millisecond): + } + + m.chRestore = chChunks + m.chRestoreDone = chDone + m.restorePending = snapshot.Metadata.ChunkHashes + return nil +} + +// RestoreChunk adds a chunk to an active snapshot restoration, mirroring ABCI ApplySnapshotChunk. +// Chunks must be given until the restore is complete, returning true, or a chunk errors. +func (m *Manager) RestoreChunk(chunk []byte) (bool, error) { + m.mtx.Lock() + defer m.mtx.Unlock() + if m.operation != opRestore { + return false, fmt.Errorf("no restore operation in progress") + } + + // Check if any errors have occurred yet. + select { + case err := <-m.chRestoreDone: + if err == nil { + err = errors.New("restore ended unexpectedly") + } + m.endLocked() + return false, err + default: + } + + // Verify the chunk hash. + hash := sha256.Sum256(chunk) + if !bytes.Equal(hash[:], m.restorePending[0]) { + return false, fmt.Errorf("%w (expected %x, got %x)", ErrChunkHashMismatch, hash, m.restorePending[0]) + } + + // Pass the chunk to the restore, and wait for completion if it was the final one. + m.chRestore <- ioutil.NopCloser(bytes.NewReader(chunk)) + m.restorePending = m.restorePending[1:] + + if len(m.restorePending) == 0 { + close(m.chRestore) + m.chRestore = nil + err := <-m.chRestoreDone + m.endLocked() + return true, err + } + return false, nil +} diff --git a/snapshots/manager_test.go b/snapshots/manager_test.go new file mode 100644 index 000000000000..2ccf72833c6a --- /dev/null +++ b/snapshots/manager_test.go @@ -0,0 +1,220 @@ +package snapshots_test + +import ( + "errors" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/cosmos/cosmos-sdk/snapshots" +) + +func TestManager_List(t *testing.T) { + store, teardown := setupStore(t) + defer teardown() + manager := snapshots.NewManager(store, nil) + + mgrList, err := manager.List() + require.NoError(t, err) + storeList, err := store.List() + require.NoError(t, err) + + require.NotEmpty(t, storeList) + assert.Equal(t, storeList, mgrList) + + // list should not block or error on busy managers + manager, teardown = setupBusyManager(t) + defer teardown() + list, err := manager.List() + require.NoError(t, err) + assert.Equal(t, []*snapshots.Snapshot{}, list) +} + +func TestManager_LoadChunk(t *testing.T) { + store, teardown := setupStore(t) + defer teardown() + manager := snapshots.NewManager(store, nil) + + // Existing chunk should return body + chunk, err := manager.LoadChunk(2, 1, 1) + require.NoError(t, err) + assert.Equal(t, []byte{2, 1, 1}, chunk) + + // Missing chunk should return nil + chunk, err = manager.LoadChunk(2, 1, 9) + require.NoError(t, err) + assert.Nil(t, chunk) + + // LoadChunk should not block or error on busy managers + manager, teardown = setupBusyManager(t) + defer teardown() + chunk, err = manager.LoadChunk(2, 1, 0) + require.NoError(t, err) + assert.Nil(t, chunk) +} + +func TestManager_Take(t *testing.T) { + store, teardown := setupStore(t) + defer teardown() + snapshotter := &mockSnapshotter{ + chunks: [][]byte{ + {1, 2, 3}, + {4, 5, 6}, + {7, 8, 9}, + }, + } + manager := snapshots.NewManager(store, snapshotter) + + // nil manager should return error + _, err := (*snapshots.Manager)(nil).Take(1) + require.Error(t, err) + + // taking a snapshot at a lower height than the latest should error + _, err = manager.Take(3) + require.Error(t, err) + + // taking a snapshot at a higher height should be fine, and should return it + snapshot, err := manager.Take(5) + require.NoError(t, err) + assert.Equal(t, &snapshots.Snapshot{ + Height: 5, + Format: snapshots.CurrentFormat, + Chunks: 3, + Hash: []uint8{0x47, 0xe4, 0xee, 0x7f, 0x21, 0x1f, 0x73, 0x26, 0x5d, 0xd1, 0x76, 0x58, 0xf6, 0xe2, 0x1c, 0x13, 0x18, 0xbd, 0x6c, 0x81, 0xf3, 0x75, 0x98, 0xe2, 0xa, 0x27, 0x56, 0x29, 0x95, 0x42, 0xef, 0xcf}, + Metadata: snapshots.Metadata{ + ChunkHashes: [][]byte{ + checksum([]byte{1, 2, 3}), + checksum([]byte{4, 5, 6}), + checksum([]byte{7, 8, 9}), + }, + }, + }, snapshot) + + storeSnapshot, chunks, err := store.Load(snapshot.Height, snapshot.Format) + require.NoError(t, err) + assert.Equal(t, snapshot, storeSnapshot) + assert.Equal(t, [][]byte{{1, 2, 3}, {4, 5, 6}, {7, 8, 9}}, readChunks(chunks)) + + // taking a snapshot while a different snapshot is being taken should error + manager, teardown = setupBusyManager(t) + defer teardown() + _, err = manager.Take(9) + require.Error(t, err) +} + +func TestManager_Prune(t *testing.T) { + store, teardown := setupStore(t) + defer teardown() + manager := snapshots.NewManager(store, nil) + + pruned, err := manager.Prune(2) + require.NoError(t, err) + assert.EqualValues(t, 1, pruned) + + list, err := manager.List() + require.NoError(t, err) + assert.Len(t, list, 3) + + // Prune should error while a snapshot is being taken + manager, teardown = setupBusyManager(t) + defer teardown() + _, err = manager.Prune(2) + require.Error(t, err) +} + +func TestManager_Restore(t *testing.T) { + store, teardown := setupStore(t) + defer teardown() + target := &mockSnapshotter{} + manager := snapshots.NewManager(store, target) + + chunks := [][]byte{ + {1, 2, 3}, + {4, 5, 6}, + {7, 8, 9}, + } + + // Restore errors on invalid format + err := manager.Restore(snapshots.Snapshot{ + Height: 3, + Format: 0, + Hash: []byte{1, 2, 3}, + Chunks: uint32(len(chunks)), + Metadata: snapshots.Metadata{ChunkHashes: checksums(chunks)}, + }) + require.Error(t, err) + require.Equal(t, err, snapshots.ErrUnknownFormat) + + // Restore errors on no chunks + err = manager.Restore(snapshots.Snapshot{Height: 3, Format: 1, Hash: []byte{1, 2, 3}}) + require.Error(t, err) + + // Restore errors on chunk and chunkhashes mismatch + err = manager.Restore(snapshots.Snapshot{ + Height: 3, + Format: 1, + Hash: []byte{1, 2, 3}, + Chunks: 4, + Metadata: snapshots.Metadata{ChunkHashes: checksums(chunks)}, + }) + require.Error(t, err) + + // Starting a restore works + err = manager.Restore(snapshots.Snapshot{ + Height: 3, + Format: 1, + Hash: []byte{1, 2, 3}, + Chunks: 3, + Metadata: snapshots.Metadata{ChunkHashes: checksums(chunks)}, + }) + require.NoError(t, err) + + // While the restore is in progress, any other operations fail + _, err = manager.Take(4) + require.Error(t, err) + + _, err = manager.Prune(1) + require.Error(t, err) + + // Feeding an invalid chunk should error due to invalid checksum, but not abort restoration. + _, err = manager.RestoreChunk([]byte{9, 9, 9}) + require.Error(t, err) + require.True(t, errors.Is(err, snapshots.ErrChunkHashMismatch)) + + // Feeding the chunks should work + for i, chunk := range chunks { + done, err := manager.RestoreChunk(chunk) + require.NoError(t, err) + if i == len(chunks)-1 { + assert.True(t, done) + } else { + assert.False(t, done) + } + } + + assert.Equal(t, chunks, target.chunks) + + // Starting a new restore should fail now, because the target already has contents. + err = manager.Restore(snapshots.Snapshot{ + Height: 3, + Format: 1, + Hash: []byte{1, 2, 3}, + Chunks: 3, + Metadata: snapshots.Metadata{ChunkHashes: checksums(chunks)}, + }) + require.Error(t, err) + + // But if we clear out the target we should be able to start a new restore. This time we'll + // fail it with a checksum error. That error should stop the operation, so that we can do + // a prune operation right after. + target.chunks = nil + err = manager.Restore(snapshots.Snapshot{ + Height: 3, + Format: 1, + Hash: []byte{1, 2, 3}, + Chunks: 3, + Metadata: snapshots.Metadata{ChunkHashes: checksums(chunks)}, + }) + require.NoError(t, err) +} diff --git a/snapshots/store.go b/snapshots/store.go new file mode 100644 index 000000000000..b566f0b645e0 --- /dev/null +++ b/snapshots/store.go @@ -0,0 +1,370 @@ +package snapshots + +import ( + "crypto/sha256" + "encoding/binary" + "errors" + "fmt" + "io" + "math" + "os" + "path/filepath" + "strconv" + "sync" + + "github.com/gogo/protobuf/proto" + db "github.com/tendermint/tm-db" + + "github.com/cosmos/cosmos-sdk/snapshots/types" +) + +const ( + // keyPrefixSnapshot is the prefix for snapshot database keys + keyPrefixSnapshot byte = 0x01 +) + +// Store is a snapshot store, containing snapshot metadata and binary chunks. +type Store struct { + db db.DB + dir string + + mtx sync.Mutex + saving map[uint64]bool // heights currently being saved +} + +// NewStore creates a new snapshot store. +func NewStore(db db.DB, dir string) (*Store, error) { + if dir == "" { + return nil, errors.New("snapshot directory not given") + } + err := os.MkdirAll(dir, 0755) + if err != nil { + return nil, fmt.Errorf("failed to create snapshot directory %q: %w", dir, err) + } + + return &Store{ + db: db, + dir: dir, + saving: make(map[uint64]bool), + }, nil +} + +// Delete deletes a snapshot. +func (s *Store) Delete(height uint64, format uint32) error { + s.mtx.Lock() + saving := s.saving[height] + s.mtx.Unlock() + if saving { + return fmt.Errorf("snapshot for height %v format %v is currently being saved", height, format) + } + err := s.db.DeleteSync(encodeKey(height, format)) + if err != nil { + return fmt.Errorf("failed to delete snapshot for height %v format %v: %w", + height, format, err) + } + err = os.RemoveAll(s.pathSnapshot(height, format)) + if err != nil { + return fmt.Errorf("failed to delete snapshot chunks for height %v format %v: %w", + height, format, err) + } + return nil +} + +// Get fetches snapshot info from the database. +func (s *Store) Get(height uint64, format uint32) (*types.Snapshot, error) { + bytes, err := s.db.Get(encodeKey(height, format)) + if err != nil { + return nil, fmt.Errorf("failed to fetch snapshot metadata for height %v format %v: %w", + height, format, err) + } + if bytes == nil { + return nil, nil + } + snapshot := &types.Snapshot{} + err = proto.Unmarshal(bytes, snapshot) + if err != nil { + return nil, fmt.Errorf("failed to decode snapshot metadata for height %v format %v: %w", + height, format, err) + } + if snapshot.Metadata.ChunkHashes == nil { + snapshot.Metadata.ChunkHashes = [][]byte{} + } + return snapshot, nil +} + +// Get fetches the latest snapshot from the database, if any. +func (s *Store) GetLatest() (*types.Snapshot, error) { + iter, err := s.db.ReverseIterator(encodeKey(0, 0), encodeKey(math.MaxUint64, math.MaxUint32)) + if err != nil { + return nil, fmt.Errorf("failed to find latest snapshot: %w", err) + } + defer iter.Close() + + var snapshot *types.Snapshot + if iter.Valid() { + snapshot = &types.Snapshot{} + err := proto.Unmarshal(iter.Value(), snapshot) + if err != nil { + return nil, fmt.Errorf("failed to decode latest snapshot: %w", err) + } + } + err = iter.Error() + if err != nil { + return nil, fmt.Errorf("failed to find latest snapshot: %w", err) + } + return snapshot, nil +} + +// List lists snapshots, in reverse order (newest first). +func (s *Store) List() ([]*types.Snapshot, error) { + iter, err := s.db.ReverseIterator(encodeKey(0, 0), encodeKey(math.MaxUint64, math.MaxUint32)) + if err != nil { + return nil, fmt.Errorf("failed to list snapshots: %w", err) + } + defer iter.Close() + + snapshots := make([]*types.Snapshot, 0) + for ; iter.Valid(); iter.Next() { + snapshot := &types.Snapshot{} + err := proto.Unmarshal(iter.Value(), snapshot) + if err != nil { + return nil, fmt.Errorf("failed to decode snapshot info: %w", err) + } + snapshots = append(snapshots, snapshot) + } + err = iter.Error() + if err != nil { + return nil, err + } + return snapshots, nil +} + +// Load loads a snapshot (both metadata and binary chunks). The chunks must be consumed and closed. +// Returns nil if the snapshot does not exist. +func (s *Store) Load(height uint64, format uint32) (*types.Snapshot, <-chan io.ReadCloser, error) { + snapshot, err := s.Get(height, format) + if err != nil { + return nil, nil, err + } + if snapshot == nil { + return nil, nil, nil + } + + ch := make(chan io.ReadCloser) + go func() { + defer close(ch) + for i := uint32(0); i < snapshot.Chunks; i++ { + pr, pw := io.Pipe() + ch <- pr + chunk, err := s.loadChunkFile(height, format, i) + if err != nil { + pw.CloseWithError(err) + return + } + defer chunk.Close() + _, err = io.Copy(pw, chunk) + if err != nil { + pw.CloseWithError(err) + return + } + chunk.Close() + pw.Close() + } + }() + + return snapshot, ch, nil +} + +// LoadChunk loads a chunk from disk, or returns nil if it does not exist. The caller must call +// Close() on it when done. +func (s *Store) LoadChunk(height uint64, format uint32, chunk uint32) (io.ReadCloser, error) { + path := s.pathChunk(height, format, chunk) + file, err := os.Open(path) + if os.IsNotExist(err) { + return nil, nil + } + return file, err +} + +// loadChunkFile loads a chunk from disk, and errors if it does not exist. +func (s *Store) loadChunkFile(height uint64, format uint32, chunk uint32) (io.ReadCloser, error) { + path := s.pathChunk(height, format, chunk) + file, err := os.Open(path) + if err != nil { + return nil, err + } + return file, nil +} + +// Prune removes old snapshots. The given number of most recent heights (regardless of format) are retained. +func (s *Store) Prune(retain uint32) (uint64, error) { + iter, err := s.db.ReverseIterator(encodeKey(0, 0), encodeKey(math.MaxUint64, math.MaxUint32)) + if err != nil { + return 0, fmt.Errorf("failed to prune snapshots: %w", err) + } + defer iter.Close() + + pruned := uint64(0) + prunedHeights := make(map[uint64]bool) + skip := make(map[uint64]bool) + for ; iter.Valid(); iter.Next() { + height, format, err := decodeKey(iter.Key()) + if err != nil { + return 0, fmt.Errorf("failed to prune snapshots: %w", err) + } + if skip[height] || uint32(len(skip)) < retain { + skip[height] = true + continue + } + err = s.Delete(height, format) + if err != nil { + return 0, fmt.Errorf("failed to prune snapshots: %w", err) + } + pruned++ + prunedHeights[height] = true + } + // Since Delete() deletes a specific format, while we want to prune a height, we clean up + // the height directory as well + for height, ok := range prunedHeights { + if ok { + err = os.Remove(s.pathHeight(height)) + if err != nil { + return 0, fmt.Errorf("failed to remove snapshot directory for height %v", height) + } + } + } + err = iter.Error() + if err != nil { + return 0, err + } + return pruned, nil +} + +// Save saves a snapshot to disk, returning it. +func (s *Store) Save(height uint64, format uint32, chunks <-chan io.ReadCloser) (*Snapshot, error) { + defer DrainChunks(chunks) + if height == 0 { + return nil, errors.New("snapshot height cannot be 0") + } + + s.mtx.Lock() + saving := s.saving[height] + s.saving[height] = true + s.mtx.Unlock() + if saving { + return nil, fmt.Errorf("a snapshot for height %v is already being saved", height) + } + defer func() { + s.mtx.Lock() + delete(s.saving, height) + s.mtx.Unlock() + }() + + exists, err := s.db.Has(encodeKey(height, format)) + if err != nil { + return nil, err + } + if exists { + return nil, fmt.Errorf("snapshot already exists for height %v format %v", height, format) + } + + snapshot := &types.Snapshot{ + Height: height, + Format: format, + } + index := uint32(0) + snapshotHasher := sha256.New() + for chunkBody := range chunks { + defer chunkBody.Close() // nolint: staticcheck + chunkHasher := sha256.New() + dir := s.pathSnapshot(height, format) + err = os.MkdirAll(dir, 0755) + if err != nil { + return nil, fmt.Errorf("failed to create snapshot directory %q: %w", dir, err) + } + path := s.pathChunk(height, format, index) + file, err := os.Create(path) + if err != nil { + return nil, fmt.Errorf("failed to create snapshot chunk file %q: %w", path, err) + } + defer file.Close() // nolint: staticcheck + _, err = io.Copy(io.MultiWriter(file, chunkHasher, snapshotHasher), chunkBody) + if err != nil { + return nil, fmt.Errorf("failed to generate snapshot chunk %v: %w", index, err) + } + err = file.Close() + if err != nil { + return nil, fmt.Errorf("failed to close snapshot chunk %v: %w", index, err) + } + err = chunkBody.Close() + if err != nil { + return nil, fmt.Errorf("failed to close snapshot chunk %v: %w", index, err) + } + snapshot.Metadata.ChunkHashes = append(snapshot.Metadata.ChunkHashes, chunkHasher.Sum(nil)) + index++ + } + snapshot.Chunks = index + snapshot.Hash = snapshotHasher.Sum(nil) + err = s.saveSnapshot(snapshot) + if err != nil { + return nil, err + } + return snapshot, nil +} + +// saveSnapshot saves snapshot metadata to the database. +func (s *Store) saveSnapshot(snapshot *types.Snapshot) error { + value, err := proto.Marshal(snapshot) + if err != nil { + return fmt.Errorf("failed to encode snapshot metadata: %w", err) + } + err = s.db.SetSync(encodeKey(snapshot.Height, snapshot.Format), value) + if err != nil { + return fmt.Errorf("failed to store snapshot: %w", err) + } + return nil +} + +// pathHeight generates the path to a height, containing multiple snapshot formats. +func (s *Store) pathHeight(height uint64) string { + return filepath.Join(s.dir, strconv.FormatUint(height, 10)) +} + +// pathSnapshot generates a snapshot path, as a specific format under a height. +func (s *Store) pathSnapshot(height uint64, format uint32) string { + return filepath.Join(s.pathHeight(height), strconv.FormatUint(uint64(format), 10)) +} + +// pathChunk generates a snapshot chunk path. +func (s *Store) pathChunk(height uint64, format uint32, chunk uint32) string { + return filepath.Join(s.pathSnapshot(height, format), strconv.FormatUint(uint64(chunk), 10)) +} + +// decodeKey decodes a snapshot key. +func decodeKey(k []byte) (uint64, uint32, error) { + if len(k) != 13 { + return 0, 0, fmt.Errorf("invalid snapshot key with length %v", len(k)) + } + if k[0] != keyPrefixSnapshot { + return 0, 0, fmt.Errorf("invalid snapshot key prefix %x", k[0]) + } + height := binary.BigEndian.Uint64(k[1:9]) + format := binary.BigEndian.Uint32(k[9:13]) + return height, format, nil +} + +// encodeKey encodes a snapshot key. +func encodeKey(height uint64, format uint32) []byte { + k := make([]byte, 0, 13) + k = append(k, keyPrefixSnapshot) + + bHeight := make([]byte, 8) + binary.BigEndian.PutUint64(bHeight, height) + k = append(k, bHeight...) + + bFormat := make([]byte, 4) + binary.BigEndian.PutUint32(bFormat, format) + k = append(k, bFormat...) + + return k +} diff --git a/snapshots/store_test.go b/snapshots/store_test.go new file mode 100644 index 000000000000..7d9e0bba7fa1 --- /dev/null +++ b/snapshots/store_test.go @@ -0,0 +1,366 @@ +package snapshots_test + +import ( + "bytes" + "errors" + "io" + "io/ioutil" + "os" + "path/filepath" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + db "github.com/tendermint/tm-db" + + "github.com/cosmos/cosmos-sdk/snapshots" + "github.com/cosmos/cosmos-sdk/snapshots/types" +) + +func setupStore(t *testing.T) (*snapshots.Store, func()) { + tempdir, err := ioutil.TempDir("", "snapshots") + require.NoError(t, err) + + store, err := snapshots.NewStore(db.NewMemDB(), tempdir) + require.NoError(t, err) + + _, err = store.Save(1, 1, makeChunks([][]byte{ + {1, 1, 0}, {1, 1, 1}, + })) + require.NoError(t, err) + _, err = store.Save(2, 1, makeChunks([][]byte{ + {2, 1, 0}, {2, 1, 1}, + })) + require.NoError(t, err) + _, err = store.Save(2, 2, makeChunks([][]byte{ + {2, 2, 0}, {2, 2, 1}, {2, 2, 2}, + })) + require.NoError(t, err) + _, err = store.Save(3, 2, makeChunks([][]byte{ + {3, 2, 0}, {3, 2, 1}, {3, 2, 2}, + })) + require.NoError(t, err) + + teardown := func() { + err := os.RemoveAll(tempdir) + if err != nil { + t.Logf("Failed to remove tempdir %q: %v", tempdir, err) + } + } + return store, teardown +} + +func TestNewStore(t *testing.T) { + tempdir, err := ioutil.TempDir("", "snapshots") + require.NoError(t, err) + defer os.RemoveAll(tempdir) + + _, err = snapshots.NewStore(db.NewMemDB(), tempdir) + require.NoError(t, err) +} + +func TestNewStore_ErrNoDir(t *testing.T) { + _, err := snapshots.NewStore(db.NewMemDB(), "") + require.Error(t, err) +} + +func TestNewStore_ErrDirFailure(t *testing.T) { + tempfile, err := ioutil.TempFile("", "snapshots") + require.NoError(t, err) + defer func() { + os.RemoveAll(tempfile.Name()) + tempfile.Close() + }() + tempdir := filepath.Join(tempfile.Name(), "subdir") + + _, err = snapshots.NewStore(db.NewMemDB(), tempdir) + require.Error(t, err) +} + +func TestStore_Delete(t *testing.T) { + store, teardown := setupStore(t) + defer teardown() + + // Deleting a snapshot should remove it + err := store.Delete(2, 2) + require.NoError(t, err) + + snapshot, err := store.Get(2, 2) + require.NoError(t, err) + assert.Nil(t, snapshot) + + snapshots, err := store.List() + require.NoError(t, err) + assert.Len(t, snapshots, 3) + + // Deleting it again should not error + err = store.Delete(2, 2) + require.NoError(t, err) + + // Deleting a snapshot being saved should error + ch := make(chan io.ReadCloser) + go store.Save(9, 1, ch) + + time.Sleep(10 * time.Millisecond) + err = store.Delete(9, 1) + require.Error(t, err) + + // But after it's saved it should work + close(ch) + time.Sleep(10 * time.Millisecond) + err = store.Delete(9, 1) + require.NoError(t, err) +} + +func TestStore_Get(t *testing.T) { + store, teardown := setupStore(t) + defer teardown() + + // Loading a missing snapshot should return nil + snapshot, err := store.Get(9, 9) + require.NoError(t, err) + assert.Nil(t, snapshot) + + // Loading a snapshot should returns its metadata + snapshot, err = store.Get(2, 1) + require.NoError(t, err) + assert.Equal(t, &types.Snapshot{ + Height: 2, + Format: 1, + Chunks: 2, + Hash: hash([][]byte{{2, 1, 0}, {2, 1, 1}}), + Metadata: types.Metadata{ + ChunkHashes: [][]byte{ + checksum([]byte{2, 1, 0}), + checksum([]byte{2, 1, 1}), + }, + }, + }, snapshot) +} + +func TestStore_GetLatest(t *testing.T) { + store, teardown := setupStore(t) + defer teardown() + + // Loading a missing snapshot should return nil + snapshot, err := store.GetLatest() + require.NoError(t, err) + assert.Equal(t, &types.Snapshot{ + Height: 3, + Format: 2, + Chunks: 3, + Hash: hash([][]byte{ + {3, 2, 0}, + {3, 2, 1}, + {3, 2, 2}, + }), + Metadata: types.Metadata{ + ChunkHashes: checksums([][]byte{ + {3, 2, 0}, + {3, 2, 1}, + {3, 2, 2}, + }), + }, + }, snapshot) +} + +func TestStore_List(t *testing.T) { + store, teardown := setupStore(t) + defer teardown() + + snapshots, err := store.List() + require.NoError(t, err) + + require.Equal(t, []*types.Snapshot{ + {Height: 3, Format: 2, Chunks: 3, Hash: hash([][]byte{{3, 2, 0}, {3, 2, 1}, {3, 2, 2}}), + Metadata: types.Metadata{ChunkHashes: checksums([][]byte{{3, 2, 0}, {3, 2, 1}, {3, 2, 2}})}, + }, + {Height: 2, Format: 2, Chunks: 3, Hash: hash([][]byte{{2, 2, 0}, {2, 2, 1}, {2, 2, 2}}), + Metadata: types.Metadata{ChunkHashes: checksums([][]byte{{2, 2, 0}, {2, 2, 1}, {2, 2, 2}})}, + }, + {Height: 2, Format: 1, Chunks: 2, Hash: hash([][]byte{{2, 1, 0}, {2, 1, 1}}), + Metadata: types.Metadata{ChunkHashes: checksums([][]byte{{2, 1, 0}, {2, 1, 1}})}, + }, + {Height: 1, Format: 1, Chunks: 2, Hash: hash([][]byte{{1, 1, 0}, {1, 1, 1}}), + Metadata: types.Metadata{ChunkHashes: checksums([][]byte{{1, 1, 0}, {1, 1, 1}})}, + }, + }, snapshots) +} + +func TestStore_Load(t *testing.T) { + store, teardown := setupStore(t) + defer teardown() + + // Loading a missing snapshot should return nil + snapshot, chunks, err := store.Load(9, 9) + require.NoError(t, err) + assert.Nil(t, snapshot) + assert.Nil(t, chunks) + + // Loading a snapshot should returns its metadata and chunks + snapshot, chunks, err = store.Load(2, 1) + require.NoError(t, err) + assert.Equal(t, &types.Snapshot{ + Height: 2, + Format: 1, + Chunks: 2, + Hash: hash([][]byte{{2, 1, 0}, {2, 1, 1}}), + Metadata: types.Metadata{ + ChunkHashes: [][]byte{ + checksum([]byte{2, 1, 0}), + checksum([]byte{2, 1, 1}), + }, + }, + }, snapshot) + + for i := uint32(0); i < snapshot.Chunks; i++ { + reader, ok := <-chunks + require.True(t, ok) + chunk, err := ioutil.ReadAll(reader) + require.NoError(t, err) + err = reader.Close() + require.NoError(t, err) + assert.Equal(t, []byte{2, 1, byte(i)}, chunk) + } + assert.Empty(t, chunks) +} + +func TestStore_LoadChunk(t *testing.T) { + store, teardown := setupStore(t) + defer teardown() + + // Loading a missing snapshot should return nil + chunk, err := store.LoadChunk(9, 9, 0) + require.NoError(t, err) + assert.Nil(t, chunk) + + // Loading a missing chunk index should return nil + chunk, err = store.LoadChunk(2, 1, 2) + require.NoError(t, err) + require.Nil(t, chunk) + + // Loading a chunk should returns a content reader + chunk, err = store.LoadChunk(2, 1, 0) + require.NoError(t, err) + require.NotNil(t, chunk) + body, err := ioutil.ReadAll(chunk) + require.NoError(t, err) + assert.Equal(t, []byte{2, 1, 0}, body) + err = chunk.Close() + require.NoError(t, err) +} + +func TestStore_Prune(t *testing.T) { + store, teardown := setupStore(t) + defer teardown() + + // Pruning too many snapshots should be fine + pruned, err := store.Prune(4) + require.NoError(t, err) + assert.EqualValues(t, 0, pruned) + + snapshots, err := store.List() + require.NoError(t, err) + assert.Len(t, snapshots, 4) + + // Pruning until the last two heights should leave three snapshots (for two heights) + pruned, err = store.Prune(2) + require.NoError(t, err) + assert.EqualValues(t, 1, pruned) + + snapshots, err = store.List() + require.NoError(t, err) + require.Equal(t, []*types.Snapshot{ + {Height: 3, Format: 2, Chunks: 3, Hash: hash([][]byte{{3, 2, 0}, {3, 2, 1}, {3, 2, 2}}), + Metadata: types.Metadata{ChunkHashes: checksums([][]byte{{3, 2, 0}, {3, 2, 1}, {3, 2, 2}})}, + }, + {Height: 2, Format: 2, Chunks: 3, Hash: hash([][]byte{{2, 2, 0}, {2, 2, 1}, {2, 2, 2}}), + Metadata: types.Metadata{ChunkHashes: checksums([][]byte{{2, 2, 0}, {2, 2, 1}, {2, 2, 2}})}, + }, + {Height: 2, Format: 1, Chunks: 2, Hash: hash([][]byte{{2, 1, 0}, {2, 1, 1}}), + Metadata: types.Metadata{ChunkHashes: checksums([][]byte{{2, 1, 0}, {2, 1, 1}})}, + }, + }, snapshots) + + // Pruning all heights should also be fine + pruned, err = store.Prune(0) + require.NoError(t, err) + assert.EqualValues(t, 3, pruned) + + snapshots, err = store.List() + require.NoError(t, err) + assert.Empty(t, snapshots) +} + +func TestStore_Save(t *testing.T) { + store, teardown := setupStore(t) + defer teardown() + + // Saving a snapshot should work + snapshot, err := store.Save(4, 1, makeChunks([][]byte{{1}, {2}})) + require.NoError(t, err) + assert.Equal(t, &snapshots.Snapshot{ + Height: 4, + Format: 1, + Chunks: 2, + Hash: hash([][]byte{{1}, {2}}), + Metadata: snapshots.Metadata{ + ChunkHashes: [][]byte{ + checksum([]byte{1}), + checksum([]byte{2}), + }, + }, + }, snapshot) + loaded, err := store.Get(snapshot.Height, snapshot.Format) + require.NoError(t, err) + assert.Equal(t, snapshot, loaded) + + // Saving an existing snapshot should error + _, err = store.Save(4, 1, makeChunks([][]byte{{1}, {2}})) + require.Error(t, err) + + // Saving at height 0 should error + _, err = store.Save(0, 1, makeChunks([][]byte{{1}, {2}})) + require.Error(t, err) + + // Saving at format 0 should be fine + _, err = store.Save(1, 0, makeChunks([][]byte{{1}, {2}})) + require.NoError(t, err) + + // Saving a snapshot with no chunks should be fine, as should loading it + _, err = store.Save(5, 1, makeChunks([][]byte{})) + require.NoError(t, err) + snapshot, chunks, err := store.Load(5, 1) + require.NoError(t, err) + assert.Equal(t, &types.Snapshot{Height: 5, Format: 1, Hash: hash([][]byte{}), Metadata: types.Metadata{ChunkHashes: [][]byte{}}}, snapshot) + assert.Empty(t, chunks) + + // Saving a snapshot should error if a chunk reader returns an error, and it should empty out + // the channel + someErr := errors.New("boom") + pr, pw := io.Pipe() + err = pw.CloseWithError(someErr) + require.NoError(t, err) + + ch := make(chan io.ReadCloser, 2) + ch <- pr + ch <- ioutil.NopCloser(bytes.NewBuffer([]byte{0xff})) + close(ch) + + _, err = store.Save(6, 1, ch) + require.Error(t, err) + require.True(t, errors.Is(err, someErr)) + assert.Empty(t, ch) + + // Saving a snapshot should error if a snapshot is already in progress for the same height, + // regardless of format. However, a different height should succeed. + ch = make(chan io.ReadCloser) + go store.Save(7, 1, ch) + time.Sleep(10 * time.Millisecond) + _, err = store.Save(7, 2, makeChunks(nil)) + require.Error(t, err) + _, err = store.Save(8, 1, makeChunks(nil)) + require.NoError(t, err) + close(ch) +} diff --git a/snapshots/types/convert.go b/snapshots/types/convert.go new file mode 100644 index 000000000000..33e315ee032c --- /dev/null +++ b/snapshots/types/convert.go @@ -0,0 +1,39 @@ +package types + +import ( + fmt "fmt" + + proto "github.com/gogo/protobuf/proto" + abci "github.com/tendermint/tendermint/abci/types" +) + +// Converts an ABCI snapshot to a snapshot. Mainly to decode the SDK metadata. +func SnapshotFromABCI(in *abci.Snapshot) (Snapshot, error) { + snapshot := Snapshot{ + Height: in.Height, + Format: in.Format, + Chunks: in.Chunks, + Hash: in.Hash, + } + err := proto.Unmarshal(in.Metadata, &snapshot.Metadata) + if err != nil { + return Snapshot{}, fmt.Errorf("failed to unmarshal snapshot metadata: %w", err) + } + return snapshot, nil +} + +// Converts a Snapshot to its ABCI representation. Mainly to encode the SDK metadata. +func (s Snapshot) ToABCI() (abci.Snapshot, error) { + out := abci.Snapshot{ + Height: s.Height, + Format: s.Format, + Chunks: s.Chunks, + Hash: s.Hash, + } + var err error + out.Metadata, err = proto.Marshal(&s.Metadata) + if err != nil { + return abci.Snapshot{}, fmt.Errorf("failed to marshal snapshot metadata: %w", err) + } + return out, nil +} diff --git a/snapshots/types/errors.go b/snapshots/types/errors.go new file mode 100644 index 000000000000..c98dd055ed88 --- /dev/null +++ b/snapshots/types/errors.go @@ -0,0 +1,16 @@ +package types + +import ( + "errors" +) + +var ( + // ErrUnknownFormat is returned when an unknown format is used. + ErrUnknownFormat = errors.New("unknown snapshot format") + + // ErrChunkHashMismatch is returned when chunk hash verification failed. + ErrChunkHashMismatch = errors.New("chunk hash verification failed") + + // ErrInvalidMetadata is returned when the snapshot metadata is invalid. + ErrInvalidMetadata = errors.New("invalid snapshot metadata") +) diff --git a/snapshots/types/format.go b/snapshots/types/format.go new file mode 100644 index 000000000000..edfdb36d7bfc --- /dev/null +++ b/snapshots/types/format.go @@ -0,0 +1,6 @@ +package types + +// CurrentFormat is the currently used format for snapshots. Snapshots using the same format +// must be identical across all nodes for a given height, so this must be bumped when the binary +// snapshot output changes. +const CurrentFormat uint32 = 1 diff --git a/snapshots/types/snapshotter.go b/snapshots/types/snapshotter.go new file mode 100644 index 000000000000..af9c2bb5bc13 --- /dev/null +++ b/snapshots/types/snapshotter.go @@ -0,0 +1,14 @@ +package types + +import "io" + +// Snapshotter is something that can take and restore snapshots, consisting of streamed binary +// chunks - all of which must be read from the channel and closed. If an unsupported format is +// given, it must return ErrUnknownFormat (possibly wrapped with fmt.Errorf). +type Snapshotter interface { + // Snapshot takes a state snapshot. + Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) + + // Restore restores a state snapshot. + Restore(height uint64, format uint32, chunks <-chan io.ReadCloser) error +} diff --git a/snapshots/types/types.pb.go b/snapshots/types/types.pb.go new file mode 100644 index 000000000000..4835f02de6b5 --- /dev/null +++ b/snapshots/types/types.pb.go @@ -0,0 +1,664 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: snapshots/types/types.proto + +package types + +import ( + fmt "fmt" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + io "io" + math "math" + math_bits "math/bits" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +// Snapshot contains snapshot metadata. +type Snapshot struct { + Height uint64 `protobuf:"varint,1,opt,name=height,proto3" json:"height,omitempty"` + Format uint32 `protobuf:"varint,2,opt,name=format,proto3" json:"format,omitempty"` + Chunks uint32 `protobuf:"varint,3,opt,name=chunks,proto3" json:"chunks,omitempty"` + Hash []byte `protobuf:"bytes,4,opt,name=hash,proto3" json:"hash,omitempty"` + Metadata Metadata `protobuf:"bytes,5,opt,name=metadata,proto3" json:"metadata"` +} + +func (m *Snapshot) Reset() { *m = Snapshot{} } +func (m *Snapshot) String() string { return proto.CompactTextString(m) } +func (*Snapshot) ProtoMessage() {} +func (*Snapshot) Descriptor() ([]byte, []int) { + return fileDescriptor_03ed7742cffb8ebd, []int{0} +} +func (m *Snapshot) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Snapshot) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Snapshot.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Snapshot) XXX_Merge(src proto.Message) { + xxx_messageInfo_Snapshot.Merge(m, src) +} +func (m *Snapshot) XXX_Size() int { + return m.Size() +} +func (m *Snapshot) XXX_DiscardUnknown() { + xxx_messageInfo_Snapshot.DiscardUnknown(m) +} + +var xxx_messageInfo_Snapshot proto.InternalMessageInfo + +func (m *Snapshot) GetHeight() uint64 { + if m != nil { + return m.Height + } + return 0 +} + +func (m *Snapshot) GetFormat() uint32 { + if m != nil { + return m.Format + } + return 0 +} + +func (m *Snapshot) GetChunks() uint32 { + if m != nil { + return m.Chunks + } + return 0 +} + +func (m *Snapshot) GetHash() []byte { + if m != nil { + return m.Hash + } + return nil +} + +func (m *Snapshot) GetMetadata() Metadata { + if m != nil { + return m.Metadata + } + return Metadata{} +} + +type Metadata struct { + ChunkHashes [][]byte `protobuf:"bytes,1,rep,name=chunk_hashes,json=chunkHashes,proto3" json:"chunk_hashes,omitempty"` +} + +func (m *Metadata) Reset() { *m = Metadata{} } +func (m *Metadata) String() string { return proto.CompactTextString(m) } +func (*Metadata) ProtoMessage() {} +func (*Metadata) Descriptor() ([]byte, []int) { + return fileDescriptor_03ed7742cffb8ebd, []int{1} +} +func (m *Metadata) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Metadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Metadata.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Metadata) XXX_Merge(src proto.Message) { + xxx_messageInfo_Metadata.Merge(m, src) +} +func (m *Metadata) XXX_Size() int { + return m.Size() +} +func (m *Metadata) XXX_DiscardUnknown() { + xxx_messageInfo_Metadata.DiscardUnknown(m) +} + +var xxx_messageInfo_Metadata proto.InternalMessageInfo + +func (m *Metadata) GetChunkHashes() [][]byte { + if m != nil { + return m.ChunkHashes + } + return nil +} + +func init() { + proto.RegisterType((*Snapshot)(nil), "cosmos_sdk.snapshots.v1.Snapshot") + proto.RegisterType((*Metadata)(nil), "cosmos_sdk.snapshots.v1.Metadata") +} + +func init() { proto.RegisterFile("snapshots/types/types.proto", fileDescriptor_03ed7742cffb8ebd) } + +var fileDescriptor_03ed7742cffb8ebd = []byte{ + // 288 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2e, 0xce, 0x4b, 0x2c, + 0x28, 0xce, 0xc8, 0x2f, 0x29, 0xd6, 0x2f, 0xa9, 0x2c, 0x48, 0x85, 0x92, 0x7a, 0x05, 0x45, 0xf9, + 0x25, 0xf9, 0x42, 0xe2, 0xc9, 0xf9, 0xc5, 0xb9, 0xf9, 0xc5, 0xf1, 0xc5, 0x29, 0xd9, 0x7a, 0x70, + 0x75, 0x7a, 0x65, 0x86, 0x52, 0x6a, 0x25, 0x19, 0x99, 0x45, 0x29, 0xf1, 0x05, 0x89, 0x45, 0x25, + 0x95, 0xfa, 0x60, 0xb5, 0xfa, 0xe9, 0xf9, 0xe9, 0xf9, 0x08, 0x16, 0xc4, 0x00, 0xa5, 0xd5, 0x8c, + 0x5c, 0x1c, 0xc1, 0x50, 0x8d, 0x42, 0x62, 0x5c, 0x6c, 0x19, 0xa9, 0x99, 0xe9, 0x19, 0x25, 0x12, + 0x8c, 0x0a, 0x8c, 0x1a, 0x2c, 0x41, 0x50, 0x1e, 0x48, 0x3c, 0x2d, 0xbf, 0x28, 0x37, 0xb1, 0x44, + 0x82, 0x49, 0x81, 0x51, 0x83, 0x37, 0x08, 0xca, 0x03, 0x89, 0x27, 0x67, 0x94, 0xe6, 0x65, 0x17, + 0x4b, 0x30, 0x43, 0xc4, 0x21, 0x3c, 0x21, 0x21, 0x2e, 0x96, 0x8c, 0xc4, 0xe2, 0x0c, 0x09, 0x16, + 0x05, 0x46, 0x0d, 0x9e, 0x20, 0x30, 0x5b, 0xc8, 0x99, 0x8b, 0x23, 0x37, 0xb5, 0x24, 0x31, 0x25, + 0xb1, 0x24, 0x51, 0x82, 0x55, 0x81, 0x51, 0x83, 0xdb, 0x48, 0x51, 0x0f, 0x87, 0xe3, 0xf5, 0x7c, + 0xa1, 0x0a, 0x9d, 0x58, 0x4e, 0xdc, 0x93, 0x67, 0x08, 0x82, 0x6b, 0x54, 0xd2, 0xe5, 0xe2, 0x80, + 0xc9, 0x09, 0x29, 0x72, 0xf1, 0x80, 0xad, 0x8b, 0x07, 0x19, 0x9f, 0x5a, 0x2c, 0xc1, 0xa8, 0xc0, + 0xac, 0xc1, 0x13, 0xc4, 0x0d, 0x16, 0xf3, 0x00, 0x0b, 0x39, 0xb9, 0x9d, 0x78, 0x24, 0xc7, 0x78, + 0xe1, 0x91, 0x1c, 0xe3, 0x83, 0x47, 0x72, 0x8c, 0x13, 0x1e, 0xcb, 0x31, 0x5c, 0x78, 0x2c, 0xc7, + 0x70, 0xe3, 0xb1, 0x1c, 0x43, 0x94, 0x4e, 0x7a, 0x66, 0x49, 0x46, 0x69, 0x92, 0x5e, 0x72, 0x7e, + 0xae, 0x3e, 0xc4, 0x15, 0x50, 0x4a, 0xb7, 0x38, 0x25, 0x5b, 0x1f, 0x2d, 0xc4, 0x93, 0xd8, 0xc0, + 0x61, 0x65, 0x0c, 0x08, 0x00, 0x00, 0xff, 0xff, 0x15, 0xba, 0x89, 0x65, 0x8b, 0x01, 0x00, 0x00, +} + +func (m *Snapshot) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Snapshot) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Snapshot) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + { + size, err := m.Metadata.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + if len(m.Hash) > 0 { + i -= len(m.Hash) + copy(dAtA[i:], m.Hash) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Hash))) + i-- + dAtA[i] = 0x22 + } + if m.Chunks != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Chunks)) + i-- + dAtA[i] = 0x18 + } + if m.Format != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Format)) + i-- + dAtA[i] = 0x10 + } + if m.Height != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Height)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Metadata) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Metadata) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Metadata) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.ChunkHashes) > 0 { + for iNdEx := len(m.ChunkHashes) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.ChunkHashes[iNdEx]) + copy(dAtA[i:], m.ChunkHashes[iNdEx]) + i = encodeVarintTypes(dAtA, i, uint64(len(m.ChunkHashes[iNdEx]))) + i-- + dAtA[i] = 0xa + } + } + return len(dAtA) - i, nil +} + +func encodeVarintTypes(dAtA []byte, offset int, v uint64) int { + offset -= sovTypes(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *Snapshot) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Height != 0 { + n += 1 + sovTypes(uint64(m.Height)) + } + if m.Format != 0 { + n += 1 + sovTypes(uint64(m.Format)) + } + if m.Chunks != 0 { + n += 1 + sovTypes(uint64(m.Chunks)) + } + l = len(m.Hash) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + l = m.Metadata.Size() + n += 1 + l + sovTypes(uint64(l)) + return n +} + +func (m *Metadata) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.ChunkHashes) > 0 { + for _, b := range m.ChunkHashes { + l = len(b) + n += 1 + l + sovTypes(uint64(l)) + } + } + return n +} + +func sovTypes(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozTypes(x uint64) (n int) { + return sovTypes(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *Snapshot) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Snapshot: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Snapshot: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Height", wireType) + } + m.Height = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Height |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Format", wireType) + } + m.Format = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Format |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Chunks", wireType) + } + m.Chunks = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Chunks |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hash", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Hash = append(m.Hash[:0], dAtA[iNdEx:postIndex]...) + if m.Hash == nil { + m.Hash = []byte{} + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Metadata.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Metadata) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Metadata: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Metadata: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ChunkHashes", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ChunkHashes = append(m.ChunkHashes, make([]byte, postIndex-iNdEx)) + copy(m.ChunkHashes[len(m.ChunkHashes)-1], dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipTypes(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + depth := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + case 1: + iNdEx += 8 + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthTypes + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupTypes + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthTypes + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthTypes = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowTypes = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupTypes = fmt.Errorf("proto: unexpected end of group") +) diff --git a/snapshots/types/types.proto b/snapshots/types/types.proto new file mode 100644 index 000000000000..d39abe4e0b50 --- /dev/null +++ b/snapshots/types/types.proto @@ -0,0 +1,20 @@ +syntax = "proto3"; +package cosmos_sdk.snapshots.v1; + +import "third_party/proto/gogoproto/gogo.proto"; + +option go_package = "github.com/cosmos/cosmos-sdk/snapshots/types"; + +// Snapshot contains snapshot info. +message Snapshot { + uint64 height = 1; + uint32 format = 2; + uint32 chunks = 3; + bytes hash = 4; + Metadata metadata = 5 [(gogoproto.nullable) = false]; +} + +// Metadata contains SDK-specific snapshot metadata. +message Metadata { + repeated bytes chunk_hashes = 1; // SHA-256 chunk hashes +} \ No newline at end of file diff --git a/snapshots/util.go b/snapshots/util.go new file mode 100644 index 000000000000..19bc87c1aab6 --- /dev/null +++ b/snapshots/util.go @@ -0,0 +1,162 @@ +package snapshots + +import ( + "errors" + "io" +) + +// ChunkWriter reads an input stream, splits it into fixed-size chunks, and writes them to a +// sequence of io.ReadClosers via a channel. +type ChunkWriter struct { + ch chan<- io.ReadCloser + pipe *io.PipeWriter + chunkSize uint64 + written uint64 + closed bool +} + +// NewChunkWriter creates a new ChunkWriter. If chunkSize is 0, no chunking will be done. +func NewChunkWriter(ch chan<- io.ReadCloser, chunkSize uint64) *ChunkWriter { + return &ChunkWriter{ + ch: ch, + chunkSize: chunkSize, + } +} + +// chunk creates a new chunk. +func (w *ChunkWriter) chunk() error { + if w.pipe != nil { + err := w.pipe.Close() + if err != nil { + return err + } + } + pr, pw := io.Pipe() + w.ch <- pr + w.pipe = pw + w.written = 0 + return nil +} + +// Close implements io.Closer. +func (w *ChunkWriter) Close() error { + if !w.closed { + w.closed = true + close(w.ch) + var err error + if w.pipe != nil { + err = w.pipe.Close() + } + return err + } + return nil +} + +// CloseWithError closes the writer and sends an error to the reader. +func (w *ChunkWriter) CloseWithError(err error) { + if !w.closed { + w.closed = true + close(w.ch) + if w.pipe != nil { + w.pipe.CloseWithError(err) + } + } +} + +// Write implements io.Writer. +func (w *ChunkWriter) Write(data []byte) (int, error) { + if w.closed { + return 0, errors.New("cannot write to closed ChunkWriter") + } + nTotal := 0 + for len(data) > 0 { + if w.pipe == nil || (w.written >= w.chunkSize && w.chunkSize > 0) { + err := w.chunk() + if err != nil { + return nTotal, err + } + } + + var writeSize uint64 + if w.chunkSize == 0 { + writeSize = uint64(len(data)) + } else { + writeSize = w.chunkSize - w.written + } + if writeSize > uint64(len(data)) { + writeSize = uint64(len(data)) + } + + n, err := w.pipe.Write(data[:writeSize]) + w.written += uint64(n) + nTotal += n + if err != nil { + return nTotal, err + } + data = data[writeSize:] + } + return nTotal, nil +} + +// ChunkReader reads chunks from a channel of io.ReadClosers and outputs them as an io.Reader +type ChunkReader struct { + ch <-chan io.ReadCloser + reader io.ReadCloser +} + +// NewChunkReader creates a new ChunkReader. +func NewChunkReader(ch <-chan io.ReadCloser) *ChunkReader { + return &ChunkReader{ch: ch} +} + +// next fetches the next chunk from the channel, or returns io.EOF if there are no more chunks. +func (r *ChunkReader) next() error { + reader, ok := <-r.ch + if !ok { + return io.EOF + } + r.reader = reader + return nil +} + +// Close implements io.ReadCloser. +func (r *ChunkReader) Close() error { + var err error + if r.reader != nil { + err = r.reader.Close() + r.reader = nil + } + for reader := range r.ch { + if e := reader.Close(); e != nil && err == nil { + err = e + } + } + return err +} + +// Read implements io.Reader. +func (r *ChunkReader) Read(p []byte) (int, error) { + if r.reader == nil { + err := r.next() + if err != nil { + return 0, err + } + } + n, err := r.reader.Read(p) + if err == io.EOF { + err = r.reader.Close() + r.reader = nil + if err != nil { + return 0, err + } + return r.Read(p) + } + return n, err +} + +// DrainChunks drains and closes all remaining chunks from a chunk channel. +func DrainChunks(chunks <-chan io.ReadCloser) { + for chunk := range chunks { + _ = chunk.Close() + } +} diff --git a/snapshots/util_test.go b/snapshots/util_test.go new file mode 100644 index 000000000000..72b236dafcac --- /dev/null +++ b/snapshots/util_test.go @@ -0,0 +1,166 @@ +package snapshots_test + +import ( + "bytes" + "errors" + "io" + "io/ioutil" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/cosmos/cosmos-sdk/snapshots" +) + +func TestChunkWriter(t *testing.T) { + ch := make(chan io.ReadCloser, 100) + go func() { + chunkWriter := snapshots.NewChunkWriter(ch, 2) + + n, err := chunkWriter.Write([]byte{1, 2, 3}) + require.NoError(t, err) + assert.Equal(t, 3, n) + + n, err = chunkWriter.Write([]byte{4, 5, 6}) + require.NoError(t, err) + assert.Equal(t, 3, n) + + n, err = chunkWriter.Write([]byte{7, 8, 9}) + require.NoError(t, err) + assert.Equal(t, 3, n) + + err = chunkWriter.Close() + require.NoError(t, err) + + // closed writer should error + _, err = chunkWriter.Write([]byte{10}) + require.Error(t, err) + + // closing again should be fine + err = chunkWriter.Close() + require.NoError(t, err) + }() + + assert.Equal(t, [][]byte{{1, 2}, {3, 4}, {5, 6}, {7, 8}, {9}}, readChunks(ch)) + + // 0-sized chunks should return the whole body as one chunk + ch = make(chan io.ReadCloser, 100) + go func() { + chunkWriter := snapshots.NewChunkWriter(ch, 0) + _, err := chunkWriter.Write([]byte{1, 2, 3}) + require.NoError(t, err) + _, err = chunkWriter.Write([]byte{4, 5, 6}) + require.NoError(t, err) + err = chunkWriter.Close() + require.NoError(t, err) + }() + assert.Equal(t, [][]byte{{1, 2, 3, 4, 5, 6}}, readChunks(ch)) + + // closing with error should return the error + theErr := errors.New("boom") + ch = make(chan io.ReadCloser, 100) + go func() { + chunkWriter := snapshots.NewChunkWriter(ch, 2) + _, err := chunkWriter.Write([]byte{1, 2, 3}) + require.NoError(t, err) + chunkWriter.CloseWithError(theErr) + }() + chunk, err := ioutil.ReadAll(<-ch) + require.NoError(t, err) + assert.Equal(t, []byte{1, 2}, chunk) + _, err = ioutil.ReadAll(<-ch) + require.Error(t, err) + assert.Equal(t, theErr, err) + assert.Empty(t, ch) + + // closing immediately should return no chunks + ch = make(chan io.ReadCloser, 100) + chunkWriter := snapshots.NewChunkWriter(ch, 2) + err = chunkWriter.Close() + require.NoError(t, err) + assert.Empty(t, ch) +} + +func TestChunkReader(t *testing.T) { + + ch := makeChunks([][]byte{ + {1, 2, 3}, + {4}, + {}, + {5, 6}, + }) + chunkReader := snapshots.NewChunkReader(ch) + + buf := []byte{0, 0, 0, 0} + n, err := chunkReader.Read(buf) + require.NoError(t, err) + assert.Equal(t, 3, n) + assert.Equal(t, []byte{1, 2, 3, 0}, buf) + + buf = []byte{0, 0, 0, 0} + n, err = chunkReader.Read(buf) + require.NoError(t, err) + assert.Equal(t, 1, n) + assert.Equal(t, []byte{4, 0, 0, 0}, buf) + + buf = []byte{0, 0, 0, 0} + n, err = chunkReader.Read(buf) + require.NoError(t, err) + assert.Equal(t, 2, n) + assert.Equal(t, []byte{5, 6, 0, 0}, buf) + + buf = []byte{0, 0, 0, 0} + _, err = chunkReader.Read(buf) + require.Error(t, err) + assert.Equal(t, io.EOF, err) + + err = chunkReader.Close() + require.NoError(t, err) + + err = chunkReader.Close() // closing twice should be fine + require.NoError(t, err) + + // Empty channel should be fine + ch = makeChunks(nil) + chunkReader = snapshots.NewChunkReader(ch) + buf = make([]byte, 4) + _, err = chunkReader.Read(buf) + require.Error(t, err) + assert.Equal(t, io.EOF, err) + + // Using a pipe that closes with an error should return the error + theErr := errors.New("boom") + pr, pw := io.Pipe() + pch := make(chan io.ReadCloser, 1) + pch <- pr + pw.CloseWithError(theErr) + + chunkReader = snapshots.NewChunkReader(pch) + buf = make([]byte, 4) + _, err = chunkReader.Read(buf) + require.Error(t, err) + assert.Equal(t, theErr, err) + + // Closing the reader should close the writer + pr, pw = io.Pipe() + pch = make(chan io.ReadCloser, 2) + pch <- ioutil.NopCloser(bytes.NewBuffer([]byte{1, 2, 3})) + pch <- pr + close(pch) + + go func() { + chunkReader = snapshots.NewChunkReader(pch) + buf = []byte{0, 0, 0, 0} + _, err = chunkReader.Read(buf) + require.NoError(t, err) + assert.Equal(t, []byte{1, 2, 3, 0}, buf) + + err = chunkReader.Close() + require.NoError(t, err) + }() + + _, err = pw.Write([]byte{9, 9, 9}) + require.Error(t, err) + assert.Equal(t, err, io.ErrClosedPipe) +} diff --git a/store/iavl/store.go b/store/iavl/store.go index 2e389f68978b..eb5ce398c7fa 100644 --- a/store/iavl/store.go +++ b/store/iavl/store.go @@ -1,6 +1,7 @@ package iavl import ( + "errors" "fmt" "io" "sync" @@ -205,6 +206,28 @@ func (st *Store) ReverseIterator(start, end []byte) types.Iterator { return newIAVLIterator(iTree, start, end, false) } +// Exports the IAVL store at the given version, returning an iavl.Exporter for the tree. +func (st *Store) Export(version int64) (*iavl.Exporter, error) { + istore, err := st.GetImmutable(version) + if err != nil { + return nil, fmt.Errorf("iavl export failed for version %v: %w", version, err) + } + tree, ok := istore.tree.(*immutableTree) + if !ok || tree == nil { + return nil, fmt.Errorf("iavl export failed: unable to fetch tree for version %v", version) + } + return tree.Export(), nil +} + +// Import imports an IAVL tree at the given version, returning an iavl.Importer for importing. +func (st *Store) Import(version int64) (*iavl.Importer, error) { + tree, ok := st.tree.(*iavl.MutableTree) + if !ok { + return nil, errors.New("iavl import failed: unable to find mutable tree") + } + return tree.Import(version) +} + // Handle gatest the latest height, if height is 0 func getHeight(tree Tree, req abci.RequestQuery) int64 { height := req.Height diff --git a/store/rootmulti/store.go b/store/rootmulti/store.go index a07c8257e6b1..0a9807f060e5 100644 --- a/store/rootmulti/store.go +++ b/store/rootmulti/store.go @@ -1,17 +1,23 @@ package rootmulti import ( + "bufio" + "compress/zlib" "encoding/binary" "fmt" "io" + "math" + "sort" "strings" iavltree "github.com/cosmos/iavl" + protoio "github.com/gogo/protobuf/io" gogotypes "github.com/gogo/protobuf/types" "github.com/pkg/errors" abci "github.com/tendermint/tendermint/abci/types" dbm "github.com/tendermint/tm-db" + "github.com/cosmos/cosmos-sdk/snapshots" "github.com/cosmos/cosmos-sdk/store/cachemulti" "github.com/cosmos/cosmos-sdk/store/dbadapter" "github.com/cosmos/cosmos-sdk/store/iavl" @@ -26,6 +32,11 @@ const ( latestVersionKey = "s/latest" pruneHeightsKey = "s/pruneheights" commitInfoKeyFmt = "s/%d" // s/ + + // Do not change chunk size without new snapshot format (must be uniform across nodes) + snapshotChunkSize = uint64(10e6) + snapshotBufferSize = int(snapshotChunkSize) + snapshotMaxItemSize = int(64e6) // SDK has no key/value size limit, so we set an arbitrary limit ) // Store is composed of many CommitStores. Name contrasts with @@ -67,6 +78,11 @@ func NewStore(db dbm.DB) *Store { } } +// GetPruning fetches the pruning strategy from the root store. +func (rs *Store) GetPruning() types.PruningOptions { + return rs.pruningOpts +} + // SetPruning sets the pruning strategy on the root store and all the sub-stores. // Note, calling SetPruning on the root store prior to LoadVersion or // LoadLatestVersion performs a no-op as the stores aren't mounted yet. @@ -519,6 +535,227 @@ func parsePath(path string) (storeName string, subpath string, err error) { return storeName, subpath, nil } +//---------------------- Snapshotting ------------------ + +// Snapshot implements snapshots.Snapshotter. The snapshot output for a given format must be +// identical across nodes such that chunks from different sources fit together. If the output for a +// given format changes (at the byte level), the snapshot format must be bumped - see +// TestMultistoreSnapshot_Checksum test. +func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { + if format != snapshots.CurrentFormat { + return nil, fmt.Errorf("%w %v", snapshots.ErrUnknownFormat, format) + } + if height == 0 { + return nil, errors.New("cannot snapshot height 0") + } + if height > uint64(rs.LastCommitID().Version) { + return nil, fmt.Errorf("cannot snapshot future height %v", height) + } + + // Collect stores to snapshot (only IAVL stores are supported) + type namedStore struct { + *iavl.Store + name string + } + stores := []namedStore{} + for key := range rs.stores { + switch store := rs.GetCommitKVStore(key).(type) { + case *iavl.Store: + stores = append(stores, namedStore{name: key.Name(), Store: store}) + case *transient.Store, *mem.Store: + // Non-persisted stores shouldn't be snapshotted + continue + default: + return nil, errors.Errorf("don't know how to snapshot store %q of type %T", key.Name(), store) + } + } + sort.Slice(stores, func(i, j int) bool { + return strings.Compare(stores[i].name, stores[j].name) == -1 + }) + + // Spawn goroutine to generate snapshot chunks and pass their io.ReadClosers through a channel + ch := make(chan io.ReadCloser) + go func() { + // Set up a stream pipeline to serialize snapshot nodes: + // ExportNode -> delimited Protobuf -> zlib -> buffer -> chunkWriter -> chan io.ReadCloser + chunkWriter := snapshots.NewChunkWriter(ch, snapshotChunkSize) + defer chunkWriter.Close() + bufWriter := bufio.NewWriterSize(chunkWriter, snapshotBufferSize) + defer func() { + if err := bufWriter.Flush(); err != nil { + chunkWriter.CloseWithError(err) + } + }() + zWriter, err := zlib.NewWriterLevel(bufWriter, 7) + if err != nil { + chunkWriter.CloseWithError(fmt.Errorf("zlib error: %w", err)) + return + } + defer func() { + if err := zWriter.Close(); err != nil { + chunkWriter.CloseWithError(err) + } + }() + protoWriter := protoio.NewDelimitedWriter(zWriter) + defer func() { + if err := protoWriter.Close(); err != nil { + chunkWriter.CloseWithError(err) + } + }() + + // Export each IAVL store. Stores are serialized as a stream of SnapshotItem Protobuf + // messages. The first item contains a SnapshotStore with store metadata (i.e. name), + // and the following messages contain a SnapshotNode (i.e. an ExportNode). Store changes + // are demarcated by new SnapshotStore items. + for _, store := range stores { + exporter, err := store.Export(int64(height)) + if err != nil { + chunkWriter.CloseWithError(err) + return + } + defer exporter.Close() + err = protoWriter.WriteMsg(&types.SnapshotItem{ + Item: &types.SnapshotItem_Store{ + Store: &types.SnapshotStoreItem{ + Name: store.name, + }, + }, + }) + if err != nil { + chunkWriter.CloseWithError(err) + return + } + + for { + node, err := exporter.Next() + if err == iavltree.ExportDone { + break + } else if err != nil { + chunkWriter.CloseWithError(err) + return + } + err = protoWriter.WriteMsg(&types.SnapshotItem{ + Item: &types.SnapshotItem_IAVL{ + IAVL: &types.SnapshotIAVLItem{ + Key: node.Key, + Value: node.Value, + Height: int32(node.Height), + Version: node.Version, + }, + }, + }) + if err != nil { + chunkWriter.CloseWithError(err) + return + } + } + exporter.Close() + } + }() + + return ch, nil +} + +// Restore implements snapshots.Snapshotter. +func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser) error { + if format != snapshots.CurrentFormat { + return fmt.Errorf("%w %v", snapshots.ErrUnknownFormat, format) + } + if height == 0 { + return fmt.Errorf("%w: cannot restore snapshot at height 0", snapshots.ErrInvalidMetadata) + } + if height > math.MaxInt64 { + return fmt.Errorf("%w: snapshot height %v cannot exceed %v", snapshots.ErrInvalidMetadata, + height, math.MaxInt64) + } + + // Set up a restore stream pipeline + // chan io.ReadCloser -> chunkReader -> zlib -> delimited Protobuf -> ExportNode + chunkReader := snapshots.NewChunkReader(chunks) + defer chunkReader.Close() + zReader, err := zlib.NewReader(chunkReader) + if err != nil { + return fmt.Errorf("zlib error: %w", err) + } + defer zReader.Close() + protoReader := protoio.NewDelimitedReader(zReader, snapshotMaxItemSize) + defer protoReader.Close() + + // Import nodes into stores. The first item is expected to be a SnapshotItem containing + // a SnapshotStoreItem, telling us which store to import into. The following items will contain + // SnapshotNodeItem (i.e. ExportNode) until we reach the next SnapshotStoreItem or EOF. + var importer *iavltree.Importer + for { + item := &types.SnapshotItem{} + err := protoReader.ReadMsg(item) + if err == io.EOF { + break + } else if err != nil { + return fmt.Errorf("invalid protobuf message: %w", err) + } + + switch item := item.Item.(type) { + case *types.SnapshotItem_Store: + if importer != nil { + err = importer.Commit() + if err != nil { + return fmt.Errorf("IAVL commit failed: %w", err) + } + importer.Close() + } + store, ok := rs.getStoreByName(item.Store.Name).(*iavl.Store) + if !ok || store == nil { + return fmt.Errorf("cannot import into non-IAVL store %q", item.Store.Name) + } + importer, err = store.Import(int64(height)) + if err != nil { + return fmt.Errorf("import failed: %w", err) + } + defer importer.Close() + + case *types.SnapshotItem_IAVL: + if importer == nil { + return fmt.Errorf("received IAVL node item before store item") + } + if item.IAVL.Height > math.MaxInt8 { + return fmt.Errorf("node height %v cannot exceed %v", item.IAVL.Height, math.MaxInt8) + } + node := &iavltree.ExportNode{ + Key: item.IAVL.Key, + Value: item.IAVL.Value, + Height: int8(item.IAVL.Height), + Version: item.IAVL.Version, + } + // Protobuf does not differentiate between []byte{} as nil, but fortunately IAVL does + // not allow nil keys nor nil values for leaf nodes, so we can always set them to empty. + if node.Key == nil { + node.Key = []byte{} + } + if node.Height == 0 && node.Value == nil { + node.Value = []byte{} + } + err := importer.Add(node) + if err != nil { + return fmt.Errorf("IAVL node import failed: %w", err) + } + + default: + return fmt.Errorf("unknown snapshot item %T", item) + } + } + + if importer != nil { + err := importer.Commit() + if err != nil { + return fmt.Errorf("IAVL commit failed: %w", err) + } + importer.Close() + } + + flushMetadata(rs.db, int64(height), rs.buildCommitInfo(int64(height)), []int64{}) + return rs.LoadLatestVersion() +} + func (rs *Store) loadCommitStoreFromParams(key types.StoreKey, id types.CommitID, params storeParams) (types.CommitKVStore, error) { var db dbm.DB @@ -571,6 +808,23 @@ func (rs *Store) loadCommitStoreFromParams(key types.StoreKey, id types.CommitID } } +func (rs *Store) buildCommitInfo(version int64) *types.CommitInfo { + storeInfos := []types.StoreInfo{} + for key, store := range rs.stores { + if store.GetStoreType() == types.StoreTypeTransient { + continue + } + storeInfos = append(storeInfos, types.StoreInfo{ + Name: key.Name(), + CommitId: store.LastCommitID(), + }) + } + return &types.CommitInfo{ + Version: version, + StoreInfos: storeInfos, + } +} + type storeParams struct { key types.StoreKey db dbm.DB diff --git a/store/rootmulti/store_test.go b/store/rootmulti/store_test.go index 1166adcc64ab..c5dc0510efb0 100644 --- a/store/rootmulti/store_test.go +++ b/store/rootmulti/store_test.go @@ -1,13 +1,22 @@ package rootmulti import ( + "crypto/sha256" + "encoding/binary" + "encoding/hex" + "errors" "fmt" + "io" + "io/ioutil" + "math/rand" "testing" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" abci "github.com/tendermint/tendermint/abci/types" dbm "github.com/tendermint/tm-db" + "github.com/cosmos/cosmos-sdk/snapshots" "github.com/cosmos/cosmos-sdk/store/iavl" sdkmaps "github.com/cosmos/cosmos-sdk/store/internal/maps" "github.com/cosmos/cosmos-sdk/store/types" @@ -505,6 +514,185 @@ func TestMultiStore_PruningRestart(t *testing.T) { } } +func TestMultistoreSnapshot_Checksum(t *testing.T) { + // Chunks from different nodes must fit together, so all nodes must produce identical chunks. + // This checksum test makes sure that the byte stream remains identical. If the test fails + // without having changed the data (e.g. because the Protobuf or zlib encoding changes), + // snapshots.CurrentFormat must be bumped. + store := newMultiStoreWithGeneratedData(dbm.NewMemDB(), 5, 10000) + version := uint64(store.LastCommitID().Version) + + testcases := []struct { + format uint32 + chunkHashes []string + }{ + {1, []string{ + "503e5b51b657055b77e88169fadae543619368744ad15f1de0736c0a20482f24", + "e1a0daaa738eeb43e778aefd2805e3dd720798288a410b06da4b8459c4d8f72e", + "aa048b4ee0f484965d7b3b06822cf0772cdcaad02f3b1b9055e69f2cb365ef3c", + "7921eaa3ed4921341e504d9308a9877986a879fe216a099c86e8db66fcba4c63", + "a4a864e6c02c9fca5837ec80dc84f650b25276ed7e4820cf7516ced9f9901b86", + "ca2879ac6e7205d257440131ba7e72bef784cd61642e32b847729e543c1928b9", + }}, + } + for _, tc := range testcases { + tc := tc + t.Run(fmt.Sprintf("Format %v", tc.format), func(t *testing.T) { + chunks, err := store.Snapshot(version, tc.format) + require.NoError(t, err) + hashes := []string{} + for chunk := range chunks { + hasher := sha256.New() + _, err := io.Copy(hasher, chunk) + require.NoError(t, err) + hashes = append(hashes, hex.EncodeToString(hasher.Sum(nil))) + } + assert.Equal(t, tc.chunkHashes, hashes, + "Snapshot output for format %v has changed", tc.format) + }) + } +} + +func TestMultistoreSnapshot_Errors(t *testing.T) { + store := newMultiStoreWithMixedMountsAndBasicData(dbm.NewMemDB()) + + testcases := map[string]struct { + height uint64 + format uint32 + expectType error + }{ + "0 height": {0, snapshots.CurrentFormat, nil}, + "0 format": {1, 0, snapshots.ErrUnknownFormat}, + "unknown height": {9, snapshots.CurrentFormat, nil}, + "unknown format": {1, 9, snapshots.ErrUnknownFormat}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + _, err := store.Snapshot(tc.height, tc.format) + require.Error(t, err) + if tc.expectType != nil { + assert.True(t, errors.Is(err, tc.expectType)) + } + }) + } +} + +func TestMultistoreRestore_Errors(t *testing.T) { + store := newMultiStoreWithMixedMounts(dbm.NewMemDB()) + + testcases := map[string]struct { + height uint64 + format uint32 + expectType error + }{ + "0 height": {0, snapshots.CurrentFormat, nil}, + "0 format": {1, 0, snapshots.ErrUnknownFormat}, + "unknown format": {1, 9, snapshots.ErrUnknownFormat}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + err := store.Restore(tc.height, tc.format, nil) + require.Error(t, err) + if tc.expectType != nil { + assert.True(t, errors.Is(err, tc.expectType)) + } + }) + } +} + +func TestMultistoreSnapshotRestore(t *testing.T) { + source := newMultiStoreWithMixedMountsAndBasicData(dbm.NewMemDB()) + target := newMultiStoreWithMixedMounts(dbm.NewMemDB()) + version := uint64(source.LastCommitID().Version) + require.EqualValues(t, 3, version) + + chunks, err := source.Snapshot(version, snapshots.CurrentFormat) + require.NoError(t, err) + err = target.Restore(version, snapshots.CurrentFormat, chunks) + require.NoError(t, err) + + assert.Equal(t, source.LastCommitID(), target.LastCommitID()) + for key, sourceStore := range source.stores { + targetStore := target.getStoreByName(key.Name()).(types.CommitKVStore) + switch sourceStore.GetStoreType() { + case types.StoreTypeTransient: + assert.False(t, targetStore.Iterator(nil, nil).Valid(), + "transient store %v not empty", key.Name()) + default: + assertStoresEqual(t, sourceStore, targetStore, "store %q not equal", key.Name()) + } + } +} + +func BenchmarkMultistoreSnapshot100K(b *testing.B) { + benchmarkMultistoreSnapshot(b, 10, 10000) +} + +func BenchmarkMultistoreSnapshot1M(b *testing.B) { + benchmarkMultistoreSnapshot(b, 10, 100000) +} + +func BenchmarkMultistoreSnapshotRestore100K(b *testing.B) { + benchmarkMultistoreSnapshotRestore(b, 10, 10000) +} + +func BenchmarkMultistoreSnapshotRestore1M(b *testing.B) { + benchmarkMultistoreSnapshotRestore(b, 10, 100000) +} + +func benchmarkMultistoreSnapshot(b *testing.B, stores uint8, storeKeys uint64) { + b.StopTimer() + source := newMultiStoreWithGeneratedData(dbm.NewMemDB(), stores, storeKeys) + version := source.LastCommitID().Version + require.EqualValues(b, 1, version) + b.StartTimer() + + for i := 0; i < b.N; i++ { + target := NewStore(dbm.NewMemDB()) + for key := range source.stores { + target.MountStoreWithDB(key, types.StoreTypeIAVL, nil) + } + err := target.LoadLatestVersion() + require.NoError(b, err) + require.EqualValues(b, 0, target.LastCommitID().Version) + + chunks, err := source.Snapshot(uint64(version), snapshots.CurrentFormat) + require.NoError(b, err) + for reader := range chunks { + _, err := io.Copy(ioutil.Discard, reader) + require.NoError(b, err) + err = reader.Close() + require.NoError(b, err) + } + } +} + +func benchmarkMultistoreSnapshotRestore(b *testing.B, stores uint8, storeKeys uint64) { + b.StopTimer() + source := newMultiStoreWithGeneratedData(dbm.NewMemDB(), stores, storeKeys) + version := uint64(source.LastCommitID().Version) + require.EqualValues(b, 1, version) + b.StartTimer() + + for i := 0; i < b.N; i++ { + target := NewStore(dbm.NewMemDB()) + for key := range source.stores { + target.MountStoreWithDB(key, types.StoreTypeIAVL, nil) + } + err := target.LoadLatestVersion() + require.NoError(b, err) + require.EqualValues(b, 0, target.LastCommitID().Version) + + chunks, err := source.Snapshot(version, snapshots.CurrentFormat) + require.NoError(b, err) + err = target.Restore(version, snapshots.CurrentFormat, chunks) + require.NoError(b, err) + require.Equal(b, source.LastCommitID(), target.LastCommitID()) + } +} + //----------------------------------------------------------------------- // utils @@ -519,6 +707,75 @@ func newMultiStoreWithMounts(db dbm.DB, pruningOpts types.PruningOptions) *Store return store } +func newMultiStoreWithMixedMounts(db dbm.DB) *Store { + store := NewStore(db) + store.MountStoreWithDB(types.NewKVStoreKey("iavl1"), types.StoreTypeIAVL, nil) + store.MountStoreWithDB(types.NewKVStoreKey("iavl2"), types.StoreTypeIAVL, nil) + store.MountStoreWithDB(types.NewKVStoreKey("iavl3"), types.StoreTypeIAVL, nil) + store.MountStoreWithDB(types.NewTransientStoreKey("trans1"), types.StoreTypeTransient, nil) + store.LoadLatestVersion() + + return store +} + +func newMultiStoreWithMixedMountsAndBasicData(db dbm.DB) *Store { + store := newMultiStoreWithMixedMounts(db) + store1 := store.getStoreByName("iavl1").(types.CommitKVStore) + store2 := store.getStoreByName("iavl2").(types.CommitKVStore) + trans1 := store.getStoreByName("trans1").(types.KVStore) + + store1.Set([]byte("a"), []byte{1}) + store1.Set([]byte("b"), []byte{1}) + store2.Set([]byte("X"), []byte{255}) + store2.Set([]byte("A"), []byte{101}) + trans1.Set([]byte("x1"), []byte{91}) + store.Commit() + + store1.Set([]byte("b"), []byte{2}) + store1.Set([]byte("c"), []byte{3}) + store2.Set([]byte("B"), []byte{102}) + store.Commit() + + store2.Set([]byte("C"), []byte{103}) + store2.Delete([]byte("X")) + trans1.Set([]byte("x2"), []byte{92}) + store.Commit() + + return store +} + +func newMultiStoreWithGeneratedData(db dbm.DB, stores uint8, storeKeys uint64) *Store { + multiStore := NewStore(db) + r := rand.New(rand.NewSource(49872768940)) // Fixed seed for deterministic tests + + keys := []*types.KVStoreKey{} + for i := uint8(0); i < stores; i++ { + key := types.NewKVStoreKey(fmt.Sprintf("store%v", i)) + multiStore.MountStoreWithDB(key, types.StoreTypeIAVL, nil) + keys = append(keys, key) + } + multiStore.LoadLatestVersion() + + for _, key := range keys { + store := multiStore.stores[key].(*iavl.Store) + for i := uint64(0); i < storeKeys; i++ { + k := make([]byte, 8) + v := make([]byte, 1024) + binary.BigEndian.PutUint64(k, i) + _, err := r.Read(v) + if err != nil { + panic(err) + } + store.Set(k, v) + } + } + + multiStore.Commit() + multiStore.LoadLatestVersion() + + return multiStore +} + func newMultiStoreWithModifiedMounts(db dbm.DB, pruningOpts types.PruningOptions) (*Store, *types.StoreUpgrades) { store := NewStore(db) store.pruningOpts = pruningOpts @@ -538,6 +795,25 @@ func newMultiStoreWithModifiedMounts(db dbm.DB, pruningOpts types.PruningOptions return store, upgrades } +func assertStoresEqual(t *testing.T, expect, actual types.CommitKVStore, msgAndArgs ...interface{}) { + assert.Equal(t, expect.LastCommitID(), actual.LastCommitID()) + expectIter := expect.Iterator(nil, nil) + expectMap := map[string][]byte{} + for ; expectIter.Valid(); expectIter.Next() { + expectMap[string(expectIter.Key())] = expectIter.Value() + } + require.NoError(t, expectIter.Error()) + + actualIter := expect.Iterator(nil, nil) + actualMap := map[string][]byte{} + for ; actualIter.Valid(); actualIter.Next() { + actualMap[string(actualIter.Key())] = actualIter.Value() + } + require.NoError(t, actualIter.Error()) + + assert.Equal(t, expectMap, actualMap, msgAndArgs...) +} + func checkStore(t *testing.T, store *Store, expect, got types.CommitID) { require.Equal(t, expect, got) require.Equal(t, expect, store.LastCommitID()) diff --git a/store/types/store.go b/store/types/store.go index 4ca8f442c05d..6a60e6a5054b 100644 --- a/store/types/store.go +++ b/store/types/store.go @@ -4,6 +4,7 @@ import ( "fmt" "io" + "github.com/cosmos/cosmos-sdk/snapshots" abci "github.com/tendermint/tendermint/abci/types" dbm "github.com/tendermint/tm-db" @@ -131,6 +132,7 @@ type CacheMultiStore interface { type CommitMultiStore interface { Committer MultiStore + snapshots.Snapshotter // Mount a store of type using the given db. // If db == nil, the new store will use the CommitMultiStore db. diff --git a/store/types/types.pb.go b/store/types/types.pb.go new file mode 100644 index 000000000000..2f484f9284b6 --- /dev/null +++ b/store/types/types.pb.go @@ -0,0 +1,951 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: store/types/types.proto + +package types + +import ( + fmt "fmt" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + io "io" + math "math" + math_bits "math/bits" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +// SnapshotItem is an item contained in a rootmulti.Store snapshot +type SnapshotItem struct { + // item gives the specific type of snapshot item + // + // Types that are valid to be assigned to Item: + // *SnapshotItem_Store + // *SnapshotItem_IAVL + Item isSnapshotItem_Item `protobuf_oneof:"item"` +} + +func (m *SnapshotItem) Reset() { *m = SnapshotItem{} } +func (m *SnapshotItem) String() string { return proto.CompactTextString(m) } +func (*SnapshotItem) ProtoMessage() {} +func (*SnapshotItem) Descriptor() ([]byte, []int) { + return fileDescriptor_8a5c5d9c27fabc9c, []int{0} +} +func (m *SnapshotItem) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SnapshotItem) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SnapshotItem.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SnapshotItem) XXX_Merge(src proto.Message) { + xxx_messageInfo_SnapshotItem.Merge(m, src) +} +func (m *SnapshotItem) XXX_Size() int { + return m.Size() +} +func (m *SnapshotItem) XXX_DiscardUnknown() { + xxx_messageInfo_SnapshotItem.DiscardUnknown(m) +} + +var xxx_messageInfo_SnapshotItem proto.InternalMessageInfo + +type isSnapshotItem_Item interface { + isSnapshotItem_Item() + MarshalTo([]byte) (int, error) + Size() int +} + +type SnapshotItem_Store struct { + Store *SnapshotStoreItem `protobuf:"bytes,1,opt,name=store,proto3,oneof" json:"store,omitempty"` +} +type SnapshotItem_IAVL struct { + IAVL *SnapshotIAVLItem `protobuf:"bytes,2,opt,name=iavl,proto3,oneof" json:"iavl,omitempty"` +} + +func (*SnapshotItem_Store) isSnapshotItem_Item() {} +func (*SnapshotItem_IAVL) isSnapshotItem_Item() {} + +func (m *SnapshotItem) GetItem() isSnapshotItem_Item { + if m != nil { + return m.Item + } + return nil +} + +func (m *SnapshotItem) GetStore() *SnapshotStoreItem { + if x, ok := m.GetItem().(*SnapshotItem_Store); ok { + return x.Store + } + return nil +} + +func (m *SnapshotItem) GetIAVL() *SnapshotIAVLItem { + if x, ok := m.GetItem().(*SnapshotItem_IAVL); ok { + return x.IAVL + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*SnapshotItem) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*SnapshotItem_Store)(nil), + (*SnapshotItem_IAVL)(nil), + } +} + +// SnapshotStore contains metadata about a snapshotted store +type SnapshotStoreItem struct { + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` +} + +func (m *SnapshotStoreItem) Reset() { *m = SnapshotStoreItem{} } +func (m *SnapshotStoreItem) String() string { return proto.CompactTextString(m) } +func (*SnapshotStoreItem) ProtoMessage() {} +func (*SnapshotStoreItem) Descriptor() ([]byte, []int) { + return fileDescriptor_8a5c5d9c27fabc9c, []int{1} +} +func (m *SnapshotStoreItem) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SnapshotStoreItem) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SnapshotStoreItem.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SnapshotStoreItem) XXX_Merge(src proto.Message) { + xxx_messageInfo_SnapshotStoreItem.Merge(m, src) +} +func (m *SnapshotStoreItem) XXX_Size() int { + return m.Size() +} +func (m *SnapshotStoreItem) XXX_DiscardUnknown() { + xxx_messageInfo_SnapshotStoreItem.DiscardUnknown(m) +} + +var xxx_messageInfo_SnapshotStoreItem proto.InternalMessageInfo + +func (m *SnapshotStoreItem) GetName() string { + if m != nil { + return m.Name + } + return "" +} + +// SnapshotIAVLItem is an exported IAVL node +type SnapshotIAVLItem struct { + Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + Version int64 `protobuf:"varint,3,opt,name=version,proto3" json:"version,omitempty"` + Height int32 `protobuf:"varint,4,opt,name=height,proto3" json:"height,omitempty"` +} + +func (m *SnapshotIAVLItem) Reset() { *m = SnapshotIAVLItem{} } +func (m *SnapshotIAVLItem) String() string { return proto.CompactTextString(m) } +func (*SnapshotIAVLItem) ProtoMessage() {} +func (*SnapshotIAVLItem) Descriptor() ([]byte, []int) { + return fileDescriptor_8a5c5d9c27fabc9c, []int{2} +} +func (m *SnapshotIAVLItem) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SnapshotIAVLItem) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SnapshotIAVLItem.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SnapshotIAVLItem) XXX_Merge(src proto.Message) { + xxx_messageInfo_SnapshotIAVLItem.Merge(m, src) +} +func (m *SnapshotIAVLItem) XXX_Size() int { + return m.Size() +} +func (m *SnapshotIAVLItem) XXX_DiscardUnknown() { + xxx_messageInfo_SnapshotIAVLItem.DiscardUnknown(m) +} + +var xxx_messageInfo_SnapshotIAVLItem proto.InternalMessageInfo + +func (m *SnapshotIAVLItem) GetKey() []byte { + if m != nil { + return m.Key + } + return nil +} + +func (m *SnapshotIAVLItem) GetValue() []byte { + if m != nil { + return m.Value + } + return nil +} + +func (m *SnapshotIAVLItem) GetVersion() int64 { + if m != nil { + return m.Version + } + return 0 +} + +func (m *SnapshotIAVLItem) GetHeight() int32 { + if m != nil { + return m.Height + } + return 0 +} + +func init() { + proto.RegisterType((*SnapshotItem)(nil), "cosmos_sdk.store.v1.SnapshotItem") + proto.RegisterType((*SnapshotStoreItem)(nil), "cosmos_sdk.store.v1.SnapshotStoreItem") + proto.RegisterType((*SnapshotIAVLItem)(nil), "cosmos_sdk.store.v1.SnapshotIAVLItem") +} + +func init() { proto.RegisterFile("store/types/types.proto", fileDescriptor_8a5c5d9c27fabc9c) } + +var fileDescriptor_8a5c5d9c27fabc9c = []byte{ + // 325 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x91, 0x41, 0x4b, 0xfb, 0x30, + 0x18, 0xc6, 0x9b, 0xff, 0xba, 0xfd, 0xf5, 0x75, 0x87, 0x19, 0x45, 0x8b, 0x87, 0x38, 0x06, 0xce, + 0x5e, 0x4c, 0x51, 0xef, 0x82, 0x05, 0x61, 0x03, 0x4f, 0x19, 0x78, 0xf0, 0x32, 0xba, 0x2d, 0xb4, + 0x61, 0xeb, 0x52, 0x9a, 0xac, 0xb0, 0x6f, 0xe1, 0x17, 0xf0, 0xfb, 0x78, 0xdc, 0xd1, 0x93, 0x48, + 0xf7, 0x45, 0xa4, 0xc9, 0x86, 0xa2, 0xe2, 0xa5, 0x7d, 0x9e, 0xf0, 0xfc, 0x9e, 0xbc, 0xe1, 0x85, + 0x63, 0xa5, 0x65, 0xce, 0x03, 0xbd, 0xcc, 0xb8, 0xb2, 0x5f, 0x9a, 0xe5, 0x52, 0x4b, 0x7c, 0x30, + 0x96, 0x2a, 0x95, 0x6a, 0xa8, 0x26, 0x53, 0x6a, 0x32, 0xb4, 0xb8, 0x3c, 0xe9, 0xea, 0x44, 0xe4, + 0x93, 0x61, 0x16, 0xe5, 0x7a, 0x19, 0x98, 0x5c, 0x10, 0xcb, 0x58, 0x7e, 0x2a, 0x0b, 0x77, 0x9e, + 0x11, 0x34, 0x07, 0xf3, 0x28, 0x53, 0x89, 0xd4, 0x7d, 0xcd, 0x53, 0x7c, 0x03, 0x75, 0x53, 0xe2, + 0xa1, 0x36, 0xf2, 0xf7, 0xae, 0xba, 0xf4, 0x97, 0x76, 0xba, 0x25, 0x06, 0xd5, 0x41, 0x85, 0xf5, + 0x1c, 0x66, 0x31, 0x7c, 0x07, 0xae, 0x88, 0x8a, 0x99, 0xf7, 0xcf, 0xe0, 0x67, 0x7f, 0xe2, 0xfd, + 0xdb, 0x87, 0xfb, 0x8a, 0x0e, 0x77, 0xca, 0xb7, 0x53, 0xb7, 0x72, 0x3d, 0x87, 0x19, 0x3c, 0x6c, + 0x80, 0x2b, 0x34, 0x4f, 0x3b, 0xe7, 0xb0, 0xff, 0xe3, 0x32, 0x8c, 0xc1, 0x9d, 0x47, 0xa9, 0x1d, + 0x71, 0x97, 0x19, 0xdd, 0x99, 0x41, 0xeb, 0x7b, 0x2d, 0x6e, 0x41, 0x6d, 0xca, 0x97, 0x26, 0xd6, + 0x64, 0x95, 0xc4, 0x87, 0x50, 0x2f, 0xa2, 0xd9, 0x82, 0x9b, 0xf1, 0x9a, 0xcc, 0x1a, 0xec, 0xc1, + 0xff, 0x82, 0xe7, 0x4a, 0xc8, 0xb9, 0x57, 0x6b, 0x23, 0xbf, 0xc6, 0xb6, 0x16, 0x1f, 0x41, 0x23, + 0xe1, 0x22, 0x4e, 0xb4, 0xe7, 0xb6, 0x91, 0x5f, 0x67, 0x1b, 0x17, 0x86, 0x2f, 0x25, 0x41, 0xab, + 0x92, 0xa0, 0xf7, 0x92, 0xa0, 0xa7, 0x35, 0x71, 0x56, 0x6b, 0xe2, 0xbc, 0xae, 0x89, 0xf3, 0xe8, + 0xc7, 0x42, 0x27, 0x8b, 0x11, 0x1d, 0xcb, 0x34, 0xb0, 0x6f, 0xdf, 0xfc, 0x2e, 0xd4, 0x64, 0x1a, + 0x7c, 0xd9, 0xe1, 0xa8, 0x61, 0x36, 0x70, 0xfd, 0x11, 0x00, 0x00, 0xff, 0xff, 0x11, 0x49, 0x37, + 0xf2, 0xd9, 0x01, 0x00, 0x00, +} + +func (m *SnapshotItem) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SnapshotItem) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SnapshotItem) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Item != nil { + { + size := m.Item.Size() + i -= size + if _, err := m.Item.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *SnapshotItem_Store) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SnapshotItem_Store) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Store != nil { + { + size, err := m.Store.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} +func (m *SnapshotItem_IAVL) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SnapshotItem_IAVL) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.IAVL != nil { + { + size, err := m.IAVL.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} +func (m *SnapshotStoreItem) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SnapshotStoreItem) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SnapshotStoreItem) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Name) > 0 { + i -= len(m.Name) + copy(dAtA[i:], m.Name) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Name))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SnapshotIAVLItem) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SnapshotIAVLItem) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SnapshotIAVLItem) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Height != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Height)) + i-- + dAtA[i] = 0x20 + } + if m.Version != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Version)) + i-- + dAtA[i] = 0x18 + } + if len(m.Value) > 0 { + i -= len(m.Value) + copy(dAtA[i:], m.Value) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Value))) + i-- + dAtA[i] = 0x12 + } + if len(m.Key) > 0 { + i -= len(m.Key) + copy(dAtA[i:], m.Key) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Key))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func encodeVarintTypes(dAtA []byte, offset int, v uint64) int { + offset -= sovTypes(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *SnapshotItem) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Item != nil { + n += m.Item.Size() + } + return n +} + +func (m *SnapshotItem_Store) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Store != nil { + l = m.Store.Size() + n += 1 + l + sovTypes(uint64(l)) + } + return n +} +func (m *SnapshotItem_IAVL) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.IAVL != nil { + l = m.IAVL.Size() + n += 1 + l + sovTypes(uint64(l)) + } + return n +} +func (m *SnapshotStoreItem) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Name) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + return n +} + +func (m *SnapshotIAVLItem) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Key) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + if m.Version != 0 { + n += 1 + sovTypes(uint64(m.Version)) + } + if m.Height != 0 { + n += 1 + sovTypes(uint64(m.Height)) + } + return n +} + +func sovTypes(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozTypes(x uint64) (n int) { + return sovTypes(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *SnapshotItem) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SnapshotItem: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SnapshotItem: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Store", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &SnapshotStoreItem{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Item = &SnapshotItem_Store{v} + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field IAVL", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &SnapshotIAVLItem{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Item = &SnapshotItem_IAVL{v} + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SnapshotStoreItem) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SnapshotStoreItem: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SnapshotStoreItem: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SnapshotIAVLItem) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SnapshotIAVLItem: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SnapshotIAVLItem: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) + if m.Key == nil { + m.Key = []byte{} + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) + if m.Value == nil { + m.Value = []byte{} + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + m.Version = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Version |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Height", wireType) + } + m.Height = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Height |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipTypes(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + depth := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + case 1: + iNdEx += 8 + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthTypes + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupTypes + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthTypes + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthTypes = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowTypes = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupTypes = fmt.Errorf("proto: unexpected end of group") +) diff --git a/store/types/types.proto b/store/types/types.proto new file mode 100644 index 000000000000..e39c467dd918 --- /dev/null +++ b/store/types/types.proto @@ -0,0 +1,28 @@ +syntax = "proto3"; +package cosmos_sdk.store.v1; + +import "third_party/proto/gogoproto/gogo.proto"; + +option go_package = "github.com/cosmos/cosmos-sdk/store/types"; + +// SnapshotItem is an item contained in a rootmulti.Store snapshot +message SnapshotItem { + // item gives the specific type of snapshot item + oneof item { + SnapshotStoreItem store = 1; + SnapshotIAVLItem iavl = 2 [(gogoproto.customname) = "IAVL"]; + } +} + +// SnapshotStore contains metadata about a snapshotted store +message SnapshotStoreItem { + string name = 1; +} + +// SnapshotIAVLItem is an exported IAVL node +message SnapshotIAVLItem { + bytes key = 1; + bytes value = 2; + int64 version = 3; + int32 height = 4; +} \ No newline at end of file From 5298d3cfc723725ce9a5ba64cec4c36b7db3cd60 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Tue, 25 Aug 2020 22:00:14 +0200 Subject: [PATCH 02/19] fix incorrect test tempdir --- baseapp/baseapp_test.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/baseapp/baseapp_test.go b/baseapp/baseapp_test.go index 025be865b70c..53a829f4fddb 100644 --- a/baseapp/baseapp_test.go +++ b/baseapp/baseapp_test.go @@ -5,6 +5,7 @@ import ( "encoding/binary" "encoding/json" "fmt" + "io/ioutil" "math/rand" "os" "strings" @@ -121,7 +122,8 @@ func setupBaseAppWithSnapshots(t *testing.T, blocks uint, blockTxs int, options })) } - snapshotDir := os.TempDir() + snapshotDir, err := ioutil.TempDir("", "baseapp") + require.NoError(t, err) snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), snapshotDir) require.NoError(t, err) teardown := func() { From 5026979927704513819671240d7dfc5b57f72d86 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Wed, 26 Aug 2020 11:21:15 +0200 Subject: [PATCH 03/19] proto: move and update Protobuf schemas --- .../base/snapshots/v1beta1/snapshot.proto | 6 +- .../cosmos/base/store/v1beta1/snapshot.proto | 27 +++ .../types/{types.pb.go => snapshot.pb.go} | 149 +++++++------- store/types/{types.pb.go => snapshot.pb.go} | 186 +++++++++--------- store/types/types.proto | 28 --- 5 files changed, 199 insertions(+), 197 deletions(-) rename snapshots/types/types.proto => proto/cosmos/base/snapshots/v1beta1/snapshot.proto (75%) create mode 100644 proto/cosmos/base/store/v1beta1/snapshot.proto rename snapshots/types/{types.pb.go => snapshot.pb.go} (72%) rename store/types/{types.pb.go => snapshot.pb.go} (77%) delete mode 100644 store/types/types.proto diff --git a/snapshots/types/types.proto b/proto/cosmos/base/snapshots/v1beta1/snapshot.proto similarity index 75% rename from snapshots/types/types.proto rename to proto/cosmos/base/snapshots/v1beta1/snapshot.proto index d39abe4e0b50..9ac5a7c31be4 100644 --- a/snapshots/types/types.proto +++ b/proto/cosmos/base/snapshots/v1beta1/snapshot.proto @@ -1,11 +1,11 @@ syntax = "proto3"; -package cosmos_sdk.snapshots.v1; +package cosmos.base.snapshots.v1beta1; -import "third_party/proto/gogoproto/gogo.proto"; +import "gogoproto/gogo.proto"; option go_package = "github.com/cosmos/cosmos-sdk/snapshots/types"; -// Snapshot contains snapshot info. +// Snapshot contains Tendermint state sync snapshot info. message Snapshot { uint64 height = 1; uint32 format = 2; diff --git a/proto/cosmos/base/store/v1beta1/snapshot.proto b/proto/cosmos/base/store/v1beta1/snapshot.proto new file mode 100644 index 000000000000..fd65b16bfdd7 --- /dev/null +++ b/proto/cosmos/base/store/v1beta1/snapshot.proto @@ -0,0 +1,27 @@ +syntax = "proto3"; +package cosmos.base.store.v1betav1; + +import "gogoproto/gogo.proto"; + +option go_package = "github.com/cosmos/cosmos-sdk/store/types"; + +// SnapshotItem is an item contained in a rootmulti.Store snapshot. +message SnapshotItem { + oneof item { + SnapshotStoreItem store = 1; + SnapshotIAVLItem iavl = 2 [(gogoproto.customname) = "IAVL"]; + } +} + +// SnapshotStoreItem contains metadata about a snapshotted store. +message SnapshotStoreItem { + string name = 1; +} + +// SnapshotIAVLItem is an exported IAVL node. +message SnapshotIAVLItem { + bytes key = 1; + bytes value = 2; + int64 version = 3; + int32 height = 4; +} \ No newline at end of file diff --git a/snapshots/types/types.pb.go b/snapshots/types/snapshot.pb.go similarity index 72% rename from snapshots/types/types.pb.go rename to snapshots/types/snapshot.pb.go index 4835f02de6b5..92fcb8151603 100644 --- a/snapshots/types/types.pb.go +++ b/snapshots/types/snapshot.pb.go @@ -1,5 +1,5 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. -// source: snapshots/types/types.proto +// source: cosmos/base/snapshots/v1beta1/snapshot.proto package types @@ -23,7 +23,7 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package -// Snapshot contains snapshot metadata. +// Snapshot contains Tendermint state sync snapshot info. type Snapshot struct { Height uint64 `protobuf:"varint,1,opt,name=height,proto3" json:"height,omitempty"` Format uint32 `protobuf:"varint,2,opt,name=format,proto3" json:"format,omitempty"` @@ -36,7 +36,7 @@ func (m *Snapshot) Reset() { *m = Snapshot{} } func (m *Snapshot) String() string { return proto.CompactTextString(m) } func (*Snapshot) ProtoMessage() {} func (*Snapshot) Descriptor() ([]byte, []int) { - return fileDescriptor_03ed7742cffb8ebd, []int{0} + return fileDescriptor_dd7a3c9b0a19e1ee, []int{0} } func (m *Snapshot) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -100,6 +100,7 @@ func (m *Snapshot) GetMetadata() Metadata { return Metadata{} } +// Metadata contains SDK-specific snapshot metadata. type Metadata struct { ChunkHashes [][]byte `protobuf:"bytes,1,rep,name=chunk_hashes,json=chunkHashes,proto3" json:"chunk_hashes,omitempty"` } @@ -108,7 +109,7 @@ func (m *Metadata) Reset() { *m = Metadata{} } func (m *Metadata) String() string { return proto.CompactTextString(m) } func (*Metadata) ProtoMessage() {} func (*Metadata) Descriptor() ([]byte, []int) { - return fileDescriptor_03ed7742cffb8ebd, []int{1} + return fileDescriptor_dd7a3c9b0a19e1ee, []int{1} } func (m *Metadata) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -145,32 +146,34 @@ func (m *Metadata) GetChunkHashes() [][]byte { } func init() { - proto.RegisterType((*Snapshot)(nil), "cosmos_sdk.snapshots.v1.Snapshot") - proto.RegisterType((*Metadata)(nil), "cosmos_sdk.snapshots.v1.Metadata") + proto.RegisterType((*Snapshot)(nil), "cosmos.base.snapshots.v1beta1.Snapshot") + proto.RegisterType((*Metadata)(nil), "cosmos.base.snapshots.v1beta1.Metadata") } -func init() { proto.RegisterFile("snapshots/types/types.proto", fileDescriptor_03ed7742cffb8ebd) } +func init() { + proto.RegisterFile("cosmos/base/snapshots/v1beta1/snapshot.proto", fileDescriptor_dd7a3c9b0a19e1ee) +} -var fileDescriptor_03ed7742cffb8ebd = []byte{ +var fileDescriptor_dd7a3c9b0a19e1ee = []byte{ // 288 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2e, 0xce, 0x4b, 0x2c, - 0x28, 0xce, 0xc8, 0x2f, 0x29, 0xd6, 0x2f, 0xa9, 0x2c, 0x48, 0x85, 0x92, 0x7a, 0x05, 0x45, 0xf9, - 0x25, 0xf9, 0x42, 0xe2, 0xc9, 0xf9, 0xc5, 0xb9, 0xf9, 0xc5, 0xf1, 0xc5, 0x29, 0xd9, 0x7a, 0x70, - 0x75, 0x7a, 0x65, 0x86, 0x52, 0x6a, 0x25, 0x19, 0x99, 0x45, 0x29, 0xf1, 0x05, 0x89, 0x45, 0x25, - 0x95, 0xfa, 0x60, 0xb5, 0xfa, 0xe9, 0xf9, 0xe9, 0xf9, 0x08, 0x16, 0xc4, 0x00, 0xa5, 0xd5, 0x8c, - 0x5c, 0x1c, 0xc1, 0x50, 0x8d, 0x42, 0x62, 0x5c, 0x6c, 0x19, 0xa9, 0x99, 0xe9, 0x19, 0x25, 0x12, - 0x8c, 0x0a, 0x8c, 0x1a, 0x2c, 0x41, 0x50, 0x1e, 0x48, 0x3c, 0x2d, 0xbf, 0x28, 0x37, 0xb1, 0x44, - 0x82, 0x49, 0x81, 0x51, 0x83, 0x37, 0x08, 0xca, 0x03, 0x89, 0x27, 0x67, 0x94, 0xe6, 0x65, 0x17, - 0x4b, 0x30, 0x43, 0xc4, 0x21, 0x3c, 0x21, 0x21, 0x2e, 0x96, 0x8c, 0xc4, 0xe2, 0x0c, 0x09, 0x16, - 0x05, 0x46, 0x0d, 0x9e, 0x20, 0x30, 0x5b, 0xc8, 0x99, 0x8b, 0x23, 0x37, 0xb5, 0x24, 0x31, 0x25, - 0xb1, 0x24, 0x51, 0x82, 0x55, 0x81, 0x51, 0x83, 0xdb, 0x48, 0x51, 0x0f, 0x87, 0xe3, 0xf5, 0x7c, - 0xa1, 0x0a, 0x9d, 0x58, 0x4e, 0xdc, 0x93, 0x67, 0x08, 0x82, 0x6b, 0x54, 0xd2, 0xe5, 0xe2, 0x80, - 0xc9, 0x09, 0x29, 0x72, 0xf1, 0x80, 0xad, 0x8b, 0x07, 0x19, 0x9f, 0x5a, 0x2c, 0xc1, 0xa8, 0xc0, - 0xac, 0xc1, 0x13, 0xc4, 0x0d, 0x16, 0xf3, 0x00, 0x0b, 0x39, 0xb9, 0x9d, 0x78, 0x24, 0xc7, 0x78, - 0xe1, 0x91, 0x1c, 0xe3, 0x83, 0x47, 0x72, 0x8c, 0x13, 0x1e, 0xcb, 0x31, 0x5c, 0x78, 0x2c, 0xc7, - 0x70, 0xe3, 0xb1, 0x1c, 0x43, 0x94, 0x4e, 0x7a, 0x66, 0x49, 0x46, 0x69, 0x92, 0x5e, 0x72, 0x7e, - 0xae, 0x3e, 0xc4, 0x15, 0x50, 0x4a, 0xb7, 0x38, 0x25, 0x5b, 0x1f, 0x2d, 0xc4, 0x93, 0xd8, 0xc0, - 0x61, 0x65, 0x0c, 0x08, 0x00, 0x00, 0xff, 0xff, 0x15, 0xba, 0x89, 0x65, 0x8b, 0x01, 0x00, 0x00, + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x90, 0x31, 0x4e, 0xc3, 0x30, + 0x14, 0x86, 0x63, 0x1a, 0xaa, 0xc8, 0x0d, 0x8b, 0x85, 0x90, 0x85, 0x84, 0x09, 0x5d, 0xc8, 0xd0, + 0x3a, 0x2a, 0xdc, 0xa0, 0x03, 0x82, 0x81, 0x25, 0x6c, 0x2c, 0xc8, 0x49, 0x4d, 0x5c, 0x55, 0xa9, + 0xa3, 0x3e, 0x17, 0x89, 0x5b, 0x70, 0x15, 0x6e, 0xd1, 0xb1, 0x23, 0x13, 0x42, 0xc9, 0x45, 0x50, + 0x1c, 0x13, 0x31, 0x75, 0xca, 0xfb, 0xbf, 0x7c, 0x4f, 0xcf, 0xfa, 0xf1, 0x24, 0xd7, 0x50, 0x6a, + 0x48, 0x32, 0x01, 0x32, 0x81, 0xb5, 0xa8, 0x40, 0x69, 0x03, 0xc9, 0xdb, 0x2c, 0x93, 0x46, 0xcc, + 0x7a, 0xc2, 0xab, 0x8d, 0x36, 0x9a, 0x5c, 0x74, 0x36, 0x6f, 0x6d, 0xde, 0xdb, 0xdc, 0xd9, 0xe7, + 0xa7, 0x85, 0x2e, 0xb4, 0x35, 0x93, 0x76, 0xea, 0x96, 0xc6, 0x9f, 0x08, 0x07, 0x4f, 0xce, 0x25, + 0x67, 0x78, 0xa8, 0xe4, 0xb2, 0x50, 0x86, 0xa2, 0x08, 0xc5, 0x7e, 0xea, 0x52, 0xcb, 0x5f, 0xf5, + 0xa6, 0x14, 0x86, 0x1e, 0x45, 0x28, 0x3e, 0x49, 0x5d, 0x6a, 0x79, 0xae, 0xb6, 0xeb, 0x15, 0xd0, + 0x41, 0xc7, 0xbb, 0x44, 0x08, 0xf6, 0x95, 0x00, 0x45, 0xfd, 0x08, 0xc5, 0x61, 0x6a, 0x67, 0xf2, + 0x80, 0x83, 0x52, 0x1a, 0xb1, 0x10, 0x46, 0xd0, 0xe3, 0x08, 0xc5, 0xa3, 0x9b, 0x6b, 0x7e, 0xf0, + 0xc1, 0xfc, 0xd1, 0xe9, 0x73, 0x7f, 0xf7, 0x7d, 0xe9, 0xa5, 0xfd, 0xfa, 0x78, 0x8a, 0x83, 0xbf, + 0x7f, 0xe4, 0x0a, 0x87, 0xf6, 0xe8, 0x4b, 0x7b, 0x44, 0x02, 0x45, 0xd1, 0x20, 0x0e, 0xd3, 0x91, + 0x65, 0xf7, 0x16, 0xcd, 0xef, 0x76, 0x35, 0x43, 0xfb, 0x9a, 0xa1, 0x9f, 0x9a, 0xa1, 0x8f, 0x86, + 0x79, 0xfb, 0x86, 0x79, 0x5f, 0x0d, 0xf3, 0x9e, 0x27, 0xc5, 0xd2, 0xa8, 0x6d, 0xc6, 0x73, 0x5d, + 0x26, 0xae, 0xea, 0xee, 0x33, 0x85, 0xc5, 0xea, 0x5f, 0xe1, 0xe6, 0xbd, 0x92, 0x90, 0x0d, 0x6d, + 0x63, 0xb7, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0xb8, 0x89, 0x2e, 0x8f, 0x96, 0x01, 0x00, 0x00, } func (m *Snapshot) Marshal() (dAtA []byte, err error) { @@ -199,29 +202,29 @@ func (m *Snapshot) MarshalToSizedBuffer(dAtA []byte) (int, error) { return 0, err } i -= size - i = encodeVarintTypes(dAtA, i, uint64(size)) + i = encodeVarintSnapshot(dAtA, i, uint64(size)) } i-- dAtA[i] = 0x2a if len(m.Hash) > 0 { i -= len(m.Hash) copy(dAtA[i:], m.Hash) - i = encodeVarintTypes(dAtA, i, uint64(len(m.Hash))) + i = encodeVarintSnapshot(dAtA, i, uint64(len(m.Hash))) i-- dAtA[i] = 0x22 } if m.Chunks != 0 { - i = encodeVarintTypes(dAtA, i, uint64(m.Chunks)) + i = encodeVarintSnapshot(dAtA, i, uint64(m.Chunks)) i-- dAtA[i] = 0x18 } if m.Format != 0 { - i = encodeVarintTypes(dAtA, i, uint64(m.Format)) + i = encodeVarintSnapshot(dAtA, i, uint64(m.Format)) i-- dAtA[i] = 0x10 } if m.Height != 0 { - i = encodeVarintTypes(dAtA, i, uint64(m.Height)) + i = encodeVarintSnapshot(dAtA, i, uint64(m.Height)) i-- dAtA[i] = 0x8 } @@ -252,7 +255,7 @@ func (m *Metadata) MarshalToSizedBuffer(dAtA []byte) (int, error) { for iNdEx := len(m.ChunkHashes) - 1; iNdEx >= 0; iNdEx-- { i -= len(m.ChunkHashes[iNdEx]) copy(dAtA[i:], m.ChunkHashes[iNdEx]) - i = encodeVarintTypes(dAtA, i, uint64(len(m.ChunkHashes[iNdEx]))) + i = encodeVarintSnapshot(dAtA, i, uint64(len(m.ChunkHashes[iNdEx]))) i-- dAtA[i] = 0xa } @@ -260,8 +263,8 @@ func (m *Metadata) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func encodeVarintTypes(dAtA []byte, offset int, v uint64) int { - offset -= sovTypes(v) +func encodeVarintSnapshot(dAtA []byte, offset int, v uint64) int { + offset -= sovSnapshot(v) base := offset for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -278,20 +281,20 @@ func (m *Snapshot) Size() (n int) { var l int _ = l if m.Height != 0 { - n += 1 + sovTypes(uint64(m.Height)) + n += 1 + sovSnapshot(uint64(m.Height)) } if m.Format != 0 { - n += 1 + sovTypes(uint64(m.Format)) + n += 1 + sovSnapshot(uint64(m.Format)) } if m.Chunks != 0 { - n += 1 + sovTypes(uint64(m.Chunks)) + n += 1 + sovSnapshot(uint64(m.Chunks)) } l = len(m.Hash) if l > 0 { - n += 1 + l + sovTypes(uint64(l)) + n += 1 + l + sovSnapshot(uint64(l)) } l = m.Metadata.Size() - n += 1 + l + sovTypes(uint64(l)) + n += 1 + l + sovSnapshot(uint64(l)) return n } @@ -304,17 +307,17 @@ func (m *Metadata) Size() (n int) { if len(m.ChunkHashes) > 0 { for _, b := range m.ChunkHashes { l = len(b) - n += 1 + l + sovTypes(uint64(l)) + n += 1 + l + sovSnapshot(uint64(l)) } } return n } -func sovTypes(x uint64) (n int) { +func sovSnapshot(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } -func sozTypes(x uint64) (n int) { - return sovTypes(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +func sozSnapshot(x uint64) (n int) { + return sovSnapshot(uint64((x << 1) ^ uint64((int64(x) >> 63)))) } func (m *Snapshot) Unmarshal(dAtA []byte) error { l := len(dAtA) @@ -324,7 +327,7 @@ func (m *Snapshot) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -352,7 +355,7 @@ func (m *Snapshot) Unmarshal(dAtA []byte) error { m.Height = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -371,7 +374,7 @@ func (m *Snapshot) Unmarshal(dAtA []byte) error { m.Format = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -390,7 +393,7 @@ func (m *Snapshot) Unmarshal(dAtA []byte) error { m.Chunks = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -409,7 +412,7 @@ func (m *Snapshot) Unmarshal(dAtA []byte) error { var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -422,11 +425,11 @@ func (m *Snapshot) Unmarshal(dAtA []byte) error { } } if byteLen < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } postIndex := iNdEx + byteLen if postIndex < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if postIndex > l { return io.ErrUnexpectedEOF @@ -443,7 +446,7 @@ func (m *Snapshot) Unmarshal(dAtA []byte) error { var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -456,11 +459,11 @@ func (m *Snapshot) Unmarshal(dAtA []byte) error { } } if msglen < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } postIndex := iNdEx + msglen if postIndex < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if postIndex > l { return io.ErrUnexpectedEOF @@ -471,15 +474,15 @@ func (m *Snapshot) Unmarshal(dAtA []byte) error { iNdEx = postIndex default: iNdEx = preIndex - skippy, err := skipTypes(dAtA[iNdEx:]) + skippy, err := skipSnapshot(dAtA[iNdEx:]) if err != nil { return err } if skippy < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if (iNdEx + skippy) < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -501,7 +504,7 @@ func (m *Metadata) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -529,7 +532,7 @@ func (m *Metadata) Unmarshal(dAtA []byte) error { var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -542,11 +545,11 @@ func (m *Metadata) Unmarshal(dAtA []byte) error { } } if byteLen < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } postIndex := iNdEx + byteLen if postIndex < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if postIndex > l { return io.ErrUnexpectedEOF @@ -556,15 +559,15 @@ func (m *Metadata) Unmarshal(dAtA []byte) error { iNdEx = postIndex default: iNdEx = preIndex - skippy, err := skipTypes(dAtA[iNdEx:]) + skippy, err := skipSnapshot(dAtA[iNdEx:]) if err != nil { return err } if skippy < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if (iNdEx + skippy) < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -578,7 +581,7 @@ func (m *Metadata) Unmarshal(dAtA []byte) error { } return nil } -func skipTypes(dAtA []byte) (n int, err error) { +func skipSnapshot(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 depth := 0 @@ -586,7 +589,7 @@ func skipTypes(dAtA []byte) (n int, err error) { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return 0, ErrIntOverflowTypes + return 0, ErrIntOverflowSnapshot } if iNdEx >= l { return 0, io.ErrUnexpectedEOF @@ -603,7 +606,7 @@ func skipTypes(dAtA []byte) (n int, err error) { case 0: for shift := uint(0); ; shift += 7 { if shift >= 64 { - return 0, ErrIntOverflowTypes + return 0, ErrIntOverflowSnapshot } if iNdEx >= l { return 0, io.ErrUnexpectedEOF @@ -619,7 +622,7 @@ func skipTypes(dAtA []byte) (n int, err error) { var length int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return 0, ErrIntOverflowTypes + return 0, ErrIntOverflowSnapshot } if iNdEx >= l { return 0, io.ErrUnexpectedEOF @@ -632,14 +635,14 @@ func skipTypes(dAtA []byte) (n int, err error) { } } if length < 0 { - return 0, ErrInvalidLengthTypes + return 0, ErrInvalidLengthSnapshot } iNdEx += length case 3: depth++ case 4: if depth == 0 { - return 0, ErrUnexpectedEndOfGroupTypes + return 0, ErrUnexpectedEndOfGroupSnapshot } depth-- case 5: @@ -648,7 +651,7 @@ func skipTypes(dAtA []byte) (n int, err error) { return 0, fmt.Errorf("proto: illegal wireType %d", wireType) } if iNdEx < 0 { - return 0, ErrInvalidLengthTypes + return 0, ErrInvalidLengthSnapshot } if depth == 0 { return iNdEx, nil @@ -658,7 +661,7 @@ func skipTypes(dAtA []byte) (n int, err error) { } var ( - ErrInvalidLengthTypes = fmt.Errorf("proto: negative length found during unmarshaling") - ErrIntOverflowTypes = fmt.Errorf("proto: integer overflow") - ErrUnexpectedEndOfGroupTypes = fmt.Errorf("proto: unexpected end of group") + ErrInvalidLengthSnapshot = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowSnapshot = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupSnapshot = fmt.Errorf("proto: unexpected end of group") ) diff --git a/store/types/types.pb.go b/store/types/snapshot.pb.go similarity index 77% rename from store/types/types.pb.go rename to store/types/snapshot.pb.go index 2f484f9284b6..8ed0022ff40a 100644 --- a/store/types/types.pb.go +++ b/store/types/snapshot.pb.go @@ -1,5 +1,5 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. -// source: store/types/types.proto +// source: cosmos/base/store/v1beta1/snapshot.proto package types @@ -23,10 +23,8 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package -// SnapshotItem is an item contained in a rootmulti.Store snapshot +// SnapshotItem is an item contained in a rootmulti.Store snapshot. type SnapshotItem struct { - // item gives the specific type of snapshot item - // // Types that are valid to be assigned to Item: // *SnapshotItem_Store // *SnapshotItem_IAVL @@ -37,7 +35,7 @@ func (m *SnapshotItem) Reset() { *m = SnapshotItem{} } func (m *SnapshotItem) String() string { return proto.CompactTextString(m) } func (*SnapshotItem) ProtoMessage() {} func (*SnapshotItem) Descriptor() ([]byte, []int) { - return fileDescriptor_8a5c5d9c27fabc9c, []int{0} + return fileDescriptor_9c55879db4cc4502, []int{0} } func (m *SnapshotItem) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -111,7 +109,7 @@ func (*SnapshotItem) XXX_OneofWrappers() []interface{} { } } -// SnapshotStore contains metadata about a snapshotted store +// SnapshotStoreItem contains metadata about a snapshotted store. type SnapshotStoreItem struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` } @@ -120,7 +118,7 @@ func (m *SnapshotStoreItem) Reset() { *m = SnapshotStoreItem{} } func (m *SnapshotStoreItem) String() string { return proto.CompactTextString(m) } func (*SnapshotStoreItem) ProtoMessage() {} func (*SnapshotStoreItem) Descriptor() ([]byte, []int) { - return fileDescriptor_8a5c5d9c27fabc9c, []int{1} + return fileDescriptor_9c55879db4cc4502, []int{1} } func (m *SnapshotStoreItem) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -156,7 +154,7 @@ func (m *SnapshotStoreItem) GetName() string { return "" } -// SnapshotIAVLItem is an exported IAVL node +// SnapshotIAVLItem is an exported IAVL node. type SnapshotIAVLItem struct { Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` @@ -168,7 +166,7 @@ func (m *SnapshotIAVLItem) Reset() { *m = SnapshotIAVLItem{} } func (m *SnapshotIAVLItem) String() string { return proto.CompactTextString(m) } func (*SnapshotIAVLItem) ProtoMessage() {} func (*SnapshotIAVLItem) Descriptor() ([]byte, []int) { - return fileDescriptor_8a5c5d9c27fabc9c, []int{2} + return fileDescriptor_9c55879db4cc4502, []int{2} } func (m *SnapshotIAVLItem) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -226,36 +224,38 @@ func (m *SnapshotIAVLItem) GetHeight() int32 { } func init() { - proto.RegisterType((*SnapshotItem)(nil), "cosmos_sdk.store.v1.SnapshotItem") - proto.RegisterType((*SnapshotStoreItem)(nil), "cosmos_sdk.store.v1.SnapshotStoreItem") - proto.RegisterType((*SnapshotIAVLItem)(nil), "cosmos_sdk.store.v1.SnapshotIAVLItem") + proto.RegisterType((*SnapshotItem)(nil), "cosmos.base.store.v1betav1.SnapshotItem") + proto.RegisterType((*SnapshotStoreItem)(nil), "cosmos.base.store.v1betav1.SnapshotStoreItem") + proto.RegisterType((*SnapshotIAVLItem)(nil), "cosmos.base.store.v1betav1.SnapshotIAVLItem") } -func init() { proto.RegisterFile("store/types/types.proto", fileDescriptor_8a5c5d9c27fabc9c) } +func init() { + proto.RegisterFile("cosmos/base/store/v1beta1/snapshot.proto", fileDescriptor_9c55879db4cc4502) +} -var fileDescriptor_8a5c5d9c27fabc9c = []byte{ +var fileDescriptor_9c55879db4cc4502 = []byte{ // 325 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x91, 0x41, 0x4b, 0xfb, 0x30, - 0x18, 0xc6, 0x9b, 0xff, 0xba, 0xfd, 0xf5, 0x75, 0x87, 0x19, 0x45, 0x8b, 0x87, 0x38, 0x06, 0xce, - 0x5e, 0x4c, 0x51, 0xef, 0x82, 0x05, 0x61, 0x03, 0x4f, 0x19, 0x78, 0xf0, 0x32, 0xba, 0x2d, 0xb4, - 0x61, 0xeb, 0x52, 0x9a, 0xac, 0xb0, 0x6f, 0xe1, 0x17, 0xf0, 0xfb, 0x78, 0xdc, 0xd1, 0x93, 0x48, - 0xf7, 0x45, 0xa4, 0xc9, 0x86, 0xa2, 0xe2, 0xa5, 0x7d, 0x9e, 0xf0, 0xfc, 0x9e, 0xbc, 0xe1, 0x85, - 0x63, 0xa5, 0x65, 0xce, 0x03, 0xbd, 0xcc, 0xb8, 0xb2, 0x5f, 0x9a, 0xe5, 0x52, 0x4b, 0x7c, 0x30, - 0x96, 0x2a, 0x95, 0x6a, 0xa8, 0x26, 0x53, 0x6a, 0x32, 0xb4, 0xb8, 0x3c, 0xe9, 0xea, 0x44, 0xe4, - 0x93, 0x61, 0x16, 0xe5, 0x7a, 0x19, 0x98, 0x5c, 0x10, 0xcb, 0x58, 0x7e, 0x2a, 0x0b, 0x77, 0x9e, - 0x11, 0x34, 0x07, 0xf3, 0x28, 0x53, 0x89, 0xd4, 0x7d, 0xcd, 0x53, 0x7c, 0x03, 0x75, 0x53, 0xe2, - 0xa1, 0x36, 0xf2, 0xf7, 0xae, 0xba, 0xf4, 0x97, 0x76, 0xba, 0x25, 0x06, 0xd5, 0x41, 0x85, 0xf5, - 0x1c, 0x66, 0x31, 0x7c, 0x07, 0xae, 0x88, 0x8a, 0x99, 0xf7, 0xcf, 0xe0, 0x67, 0x7f, 0xe2, 0xfd, - 0xdb, 0x87, 0xfb, 0x8a, 0x0e, 0x77, 0xca, 0xb7, 0x53, 0xb7, 0x72, 0x3d, 0x87, 0x19, 0x3c, 0x6c, - 0x80, 0x2b, 0x34, 0x4f, 0x3b, 0xe7, 0xb0, 0xff, 0xe3, 0x32, 0x8c, 0xc1, 0x9d, 0x47, 0xa9, 0x1d, - 0x71, 0x97, 0x19, 0xdd, 0x99, 0x41, 0xeb, 0x7b, 0x2d, 0x6e, 0x41, 0x6d, 0xca, 0x97, 0x26, 0xd6, - 0x64, 0x95, 0xc4, 0x87, 0x50, 0x2f, 0xa2, 0xd9, 0x82, 0x9b, 0xf1, 0x9a, 0xcc, 0x1a, 0xec, 0xc1, - 0xff, 0x82, 0xe7, 0x4a, 0xc8, 0xb9, 0x57, 0x6b, 0x23, 0xbf, 0xc6, 0xb6, 0x16, 0x1f, 0x41, 0x23, - 0xe1, 0x22, 0x4e, 0xb4, 0xe7, 0xb6, 0x91, 0x5f, 0x67, 0x1b, 0x17, 0x86, 0x2f, 0x25, 0x41, 0xab, - 0x92, 0xa0, 0xf7, 0x92, 0xa0, 0xa7, 0x35, 0x71, 0x56, 0x6b, 0xe2, 0xbc, 0xae, 0x89, 0xf3, 0xe8, - 0xc7, 0x42, 0x27, 0x8b, 0x11, 0x1d, 0xcb, 0x34, 0xb0, 0x6f, 0xdf, 0xfc, 0x2e, 0xd4, 0x64, 0x1a, - 0x7c, 0xd9, 0xe1, 0xa8, 0x61, 0x36, 0x70, 0xfd, 0x11, 0x00, 0x00, 0xff, 0xff, 0x11, 0x49, 0x37, - 0xf2, 0xd9, 0x01, 0x00, 0x00, + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x91, 0xc1, 0x4a, 0xc3, 0x30, + 0x18, 0xc7, 0x1b, 0xd7, 0x4d, 0xfd, 0xdc, 0x61, 0x86, 0x21, 0x65, 0x87, 0x3a, 0x76, 0xb1, 0x07, + 0x97, 0x30, 0x7d, 0x02, 0x0b, 0xc2, 0x06, 0x3b, 0x65, 0xe0, 0xc1, 0x5b, 0x3a, 0x43, 0x5b, 0xb6, + 0x2e, 0x63, 0xc9, 0x0a, 0x7b, 0x0b, 0xdf, 0xc3, 0x17, 0xf1, 0xb8, 0xa3, 0x27, 0x91, 0xee, 0x45, + 0x24, 0x49, 0x77, 0x51, 0x04, 0x4f, 0xfd, 0xfe, 0xe5, 0xf7, 0xff, 0xe5, 0x83, 0x0f, 0xa2, 0xb9, + 0x54, 0x85, 0x54, 0x34, 0xe1, 0x4a, 0x50, 0xa5, 0xe5, 0x46, 0xd0, 0x72, 0x94, 0x08, 0xcd, 0x47, + 0x54, 0xad, 0xf8, 0x5a, 0x65, 0x52, 0x93, 0xf5, 0x46, 0x6a, 0x89, 0x7b, 0x8e, 0x24, 0x86, 0x24, + 0x96, 0x24, 0x8e, 0x2c, 0x47, 0xbd, 0x6e, 0x2a, 0x53, 0x69, 0x31, 0x6a, 0x26, 0xd7, 0x18, 0xbc, + 0x21, 0x68, 0xcf, 0x6a, 0xc9, 0x44, 0x8b, 0x02, 0x3f, 0x42, 0xd3, 0x16, 0x03, 0xd4, 0x47, 0xd1, + 0xc5, 0xdd, 0x90, 0xfc, 0xad, 0x24, 0xc7, 0xe2, 0xcc, 0xfc, 0x36, 0xed, 0xb1, 0xc7, 0x5c, 0x1b, + 0x4f, 0xc1, 0xcf, 0x79, 0xb9, 0x0c, 0x4e, 0xac, 0xe5, 0xf6, 0x3f, 0x96, 0xc9, 0xc3, 0xd3, 0xd4, + 0x48, 0xe2, 0xb3, 0xea, 0xf3, 0xda, 0x37, 0x69, 0xec, 0x31, 0x6b, 0x89, 0x5b, 0xe0, 0xe7, 0x5a, + 0x14, 0x83, 0x1b, 0xb8, 0xfc, 0xf5, 0x26, 0xc6, 0xe0, 0xaf, 0x78, 0xe1, 0x16, 0x3e, 0x67, 0x76, + 0x1e, 0x2c, 0xa1, 0xf3, 0x53, 0x8b, 0x3b, 0xd0, 0x58, 0x88, 0x9d, 0xc5, 0xda, 0xcc, 0x8c, 0xb8, + 0x0b, 0xcd, 0x92, 0x2f, 0xb7, 0xc2, 0x6e, 0xd9, 0x66, 0x2e, 0xe0, 0x00, 0x4e, 0x4b, 0xb1, 0x51, + 0xb9, 0x5c, 0x05, 0x8d, 0x3e, 0x8a, 0x1a, 0xec, 0x18, 0xf1, 0x15, 0xb4, 0x32, 0x91, 0xa7, 0x99, + 0x0e, 0xfc, 0x3e, 0x8a, 0x9a, 0xac, 0x4e, 0x71, 0xfc, 0x5e, 0x85, 0x68, 0x5f, 0x85, 0xe8, 0xab, + 0x0a, 0xd1, 0xeb, 0x21, 0xf4, 0xf6, 0x87, 0xd0, 0xfb, 0x38, 0x84, 0xde, 0x73, 0x94, 0xe6, 0x3a, + 0xdb, 0x26, 0x64, 0x2e, 0x0b, 0x5a, 0x5f, 0xd1, 0x7d, 0x86, 0xea, 0x65, 0x51, 0xdf, 0x52, 0xef, + 0xd6, 0x42, 0x25, 0x2d, 0x7b, 0x8f, 0xfb, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0x21, 0xc0, 0xab, + 0x06, 0xed, 0x01, 0x00, 0x00, } func (m *SnapshotItem) Marshal() (dAtA []byte, err error) { @@ -304,7 +304,7 @@ func (m *SnapshotItem_Store) MarshalToSizedBuffer(dAtA []byte) (int, error) { return 0, err } i -= size - i = encodeVarintTypes(dAtA, i, uint64(size)) + i = encodeVarintSnapshot(dAtA, i, uint64(size)) } i-- dAtA[i] = 0xa @@ -325,7 +325,7 @@ func (m *SnapshotItem_IAVL) MarshalToSizedBuffer(dAtA []byte) (int, error) { return 0, err } i -= size - i = encodeVarintTypes(dAtA, i, uint64(size)) + i = encodeVarintSnapshot(dAtA, i, uint64(size)) } i-- dAtA[i] = 0x12 @@ -355,7 +355,7 @@ func (m *SnapshotStoreItem) MarshalToSizedBuffer(dAtA []byte) (int, error) { if len(m.Name) > 0 { i -= len(m.Name) copy(dAtA[i:], m.Name) - i = encodeVarintTypes(dAtA, i, uint64(len(m.Name))) + i = encodeVarintSnapshot(dAtA, i, uint64(len(m.Name))) i-- dAtA[i] = 0xa } @@ -383,34 +383,34 @@ func (m *SnapshotIAVLItem) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if m.Height != 0 { - i = encodeVarintTypes(dAtA, i, uint64(m.Height)) + i = encodeVarintSnapshot(dAtA, i, uint64(m.Height)) i-- dAtA[i] = 0x20 } if m.Version != 0 { - i = encodeVarintTypes(dAtA, i, uint64(m.Version)) + i = encodeVarintSnapshot(dAtA, i, uint64(m.Version)) i-- dAtA[i] = 0x18 } if len(m.Value) > 0 { i -= len(m.Value) copy(dAtA[i:], m.Value) - i = encodeVarintTypes(dAtA, i, uint64(len(m.Value))) + i = encodeVarintSnapshot(dAtA, i, uint64(len(m.Value))) i-- dAtA[i] = 0x12 } if len(m.Key) > 0 { i -= len(m.Key) copy(dAtA[i:], m.Key) - i = encodeVarintTypes(dAtA, i, uint64(len(m.Key))) + i = encodeVarintSnapshot(dAtA, i, uint64(len(m.Key))) i-- dAtA[i] = 0xa } return len(dAtA) - i, nil } -func encodeVarintTypes(dAtA []byte, offset int, v uint64) int { - offset -= sovTypes(v) +func encodeVarintSnapshot(dAtA []byte, offset int, v uint64) int { + offset -= sovSnapshot(v) base := offset for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -440,7 +440,7 @@ func (m *SnapshotItem_Store) Size() (n int) { _ = l if m.Store != nil { l = m.Store.Size() - n += 1 + l + sovTypes(uint64(l)) + n += 1 + l + sovSnapshot(uint64(l)) } return n } @@ -452,7 +452,7 @@ func (m *SnapshotItem_IAVL) Size() (n int) { _ = l if m.IAVL != nil { l = m.IAVL.Size() - n += 1 + l + sovTypes(uint64(l)) + n += 1 + l + sovSnapshot(uint64(l)) } return n } @@ -464,7 +464,7 @@ func (m *SnapshotStoreItem) Size() (n int) { _ = l l = len(m.Name) if l > 0 { - n += 1 + l + sovTypes(uint64(l)) + n += 1 + l + sovSnapshot(uint64(l)) } return n } @@ -477,26 +477,26 @@ func (m *SnapshotIAVLItem) Size() (n int) { _ = l l = len(m.Key) if l > 0 { - n += 1 + l + sovTypes(uint64(l)) + n += 1 + l + sovSnapshot(uint64(l)) } l = len(m.Value) if l > 0 { - n += 1 + l + sovTypes(uint64(l)) + n += 1 + l + sovSnapshot(uint64(l)) } if m.Version != 0 { - n += 1 + sovTypes(uint64(m.Version)) + n += 1 + sovSnapshot(uint64(m.Version)) } if m.Height != 0 { - n += 1 + sovTypes(uint64(m.Height)) + n += 1 + sovSnapshot(uint64(m.Height)) } return n } -func sovTypes(x uint64) (n int) { +func sovSnapshot(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } -func sozTypes(x uint64) (n int) { - return sovTypes(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +func sozSnapshot(x uint64) (n int) { + return sovSnapshot(uint64((x << 1) ^ uint64((int64(x) >> 63)))) } func (m *SnapshotItem) Unmarshal(dAtA []byte) error { l := len(dAtA) @@ -506,7 +506,7 @@ func (m *SnapshotItem) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -534,7 +534,7 @@ func (m *SnapshotItem) Unmarshal(dAtA []byte) error { var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -547,11 +547,11 @@ func (m *SnapshotItem) Unmarshal(dAtA []byte) error { } } if msglen < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } postIndex := iNdEx + msglen if postIndex < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if postIndex > l { return io.ErrUnexpectedEOF @@ -569,7 +569,7 @@ func (m *SnapshotItem) Unmarshal(dAtA []byte) error { var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -582,11 +582,11 @@ func (m *SnapshotItem) Unmarshal(dAtA []byte) error { } } if msglen < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } postIndex := iNdEx + msglen if postIndex < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if postIndex > l { return io.ErrUnexpectedEOF @@ -599,15 +599,15 @@ func (m *SnapshotItem) Unmarshal(dAtA []byte) error { iNdEx = postIndex default: iNdEx = preIndex - skippy, err := skipTypes(dAtA[iNdEx:]) + skippy, err := skipSnapshot(dAtA[iNdEx:]) if err != nil { return err } if skippy < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if (iNdEx + skippy) < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -629,7 +629,7 @@ func (m *SnapshotStoreItem) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -657,7 +657,7 @@ func (m *SnapshotStoreItem) Unmarshal(dAtA []byte) error { var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -671,11 +671,11 @@ func (m *SnapshotStoreItem) Unmarshal(dAtA []byte) error { } intStringLen := int(stringLen) if intStringLen < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } postIndex := iNdEx + intStringLen if postIndex < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if postIndex > l { return io.ErrUnexpectedEOF @@ -684,15 +684,15 @@ func (m *SnapshotStoreItem) Unmarshal(dAtA []byte) error { iNdEx = postIndex default: iNdEx = preIndex - skippy, err := skipTypes(dAtA[iNdEx:]) + skippy, err := skipSnapshot(dAtA[iNdEx:]) if err != nil { return err } if skippy < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if (iNdEx + skippy) < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -714,7 +714,7 @@ func (m *SnapshotIAVLItem) Unmarshal(dAtA []byte) error { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -742,7 +742,7 @@ func (m *SnapshotIAVLItem) Unmarshal(dAtA []byte) error { var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -755,11 +755,11 @@ func (m *SnapshotIAVLItem) Unmarshal(dAtA []byte) error { } } if byteLen < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } postIndex := iNdEx + byteLen if postIndex < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if postIndex > l { return io.ErrUnexpectedEOF @@ -776,7 +776,7 @@ func (m *SnapshotIAVLItem) Unmarshal(dAtA []byte) error { var byteLen int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -789,11 +789,11 @@ func (m *SnapshotIAVLItem) Unmarshal(dAtA []byte) error { } } if byteLen < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } postIndex := iNdEx + byteLen if postIndex < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if postIndex > l { return io.ErrUnexpectedEOF @@ -810,7 +810,7 @@ func (m *SnapshotIAVLItem) Unmarshal(dAtA []byte) error { m.Version = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -829,7 +829,7 @@ func (m *SnapshotIAVLItem) Unmarshal(dAtA []byte) error { m.Height = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return ErrIntOverflowTypes + return ErrIntOverflowSnapshot } if iNdEx >= l { return io.ErrUnexpectedEOF @@ -843,15 +843,15 @@ func (m *SnapshotIAVLItem) Unmarshal(dAtA []byte) error { } default: iNdEx = preIndex - skippy, err := skipTypes(dAtA[iNdEx:]) + skippy, err := skipSnapshot(dAtA[iNdEx:]) if err != nil { return err } if skippy < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if (iNdEx + skippy) < 0 { - return ErrInvalidLengthTypes + return ErrInvalidLengthSnapshot } if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF @@ -865,7 +865,7 @@ func (m *SnapshotIAVLItem) Unmarshal(dAtA []byte) error { } return nil } -func skipTypes(dAtA []byte) (n int, err error) { +func skipSnapshot(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 depth := 0 @@ -873,7 +873,7 @@ func skipTypes(dAtA []byte) (n int, err error) { var wire uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { - return 0, ErrIntOverflowTypes + return 0, ErrIntOverflowSnapshot } if iNdEx >= l { return 0, io.ErrUnexpectedEOF @@ -890,7 +890,7 @@ func skipTypes(dAtA []byte) (n int, err error) { case 0: for shift := uint(0); ; shift += 7 { if shift >= 64 { - return 0, ErrIntOverflowTypes + return 0, ErrIntOverflowSnapshot } if iNdEx >= l { return 0, io.ErrUnexpectedEOF @@ -906,7 +906,7 @@ func skipTypes(dAtA []byte) (n int, err error) { var length int for shift := uint(0); ; shift += 7 { if shift >= 64 { - return 0, ErrIntOverflowTypes + return 0, ErrIntOverflowSnapshot } if iNdEx >= l { return 0, io.ErrUnexpectedEOF @@ -919,14 +919,14 @@ func skipTypes(dAtA []byte) (n int, err error) { } } if length < 0 { - return 0, ErrInvalidLengthTypes + return 0, ErrInvalidLengthSnapshot } iNdEx += length case 3: depth++ case 4: if depth == 0 { - return 0, ErrUnexpectedEndOfGroupTypes + return 0, ErrUnexpectedEndOfGroupSnapshot } depth-- case 5: @@ -935,7 +935,7 @@ func skipTypes(dAtA []byte) (n int, err error) { return 0, fmt.Errorf("proto: illegal wireType %d", wireType) } if iNdEx < 0 { - return 0, ErrInvalidLengthTypes + return 0, ErrInvalidLengthSnapshot } if depth == 0 { return iNdEx, nil @@ -945,7 +945,7 @@ func skipTypes(dAtA []byte) (n int, err error) { } var ( - ErrInvalidLengthTypes = fmt.Errorf("proto: negative length found during unmarshaling") - ErrIntOverflowTypes = fmt.Errorf("proto: integer overflow") - ErrUnexpectedEndOfGroupTypes = fmt.Errorf("proto: unexpected end of group") + ErrInvalidLengthSnapshot = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowSnapshot = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupSnapshot = fmt.Errorf("proto: unexpected end of group") ) diff --git a/store/types/types.proto b/store/types/types.proto deleted file mode 100644 index e39c467dd918..000000000000 --- a/store/types/types.proto +++ /dev/null @@ -1,28 +0,0 @@ -syntax = "proto3"; -package cosmos_sdk.store.v1; - -import "third_party/proto/gogoproto/gogo.proto"; - -option go_package = "github.com/cosmos/cosmos-sdk/store/types"; - -// SnapshotItem is an item contained in a rootmulti.Store snapshot -message SnapshotItem { - // item gives the specific type of snapshot item - oneof item { - SnapshotStoreItem store = 1; - SnapshotIAVLItem iavl = 2 [(gogoproto.customname) = "IAVL"]; - } -} - -// SnapshotStore contains metadata about a snapshotted store -message SnapshotStoreItem { - string name = 1; -} - -// SnapshotIAVLItem is an exported IAVL node -message SnapshotIAVLItem { - bytes key = 1; - bytes value = 2; - int64 version = 3; - int32 height = 4; -} \ No newline at end of file From af07bf20a1b14ae7b41037672fc04f1606b7d11c Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Wed, 26 Aug 2020 11:26:20 +0200 Subject: [PATCH 04/19] proto: lint fixes --- .../cosmos/base/store/v1beta1/snapshot.proto | 3 +- store/types/snapshot.pb.go | 50 ++++++++++--------- 2 files changed, 28 insertions(+), 25 deletions(-) diff --git a/proto/cosmos/base/store/v1beta1/snapshot.proto b/proto/cosmos/base/store/v1beta1/snapshot.proto index fd65b16bfdd7..834855093b24 100644 --- a/proto/cosmos/base/store/v1beta1/snapshot.proto +++ b/proto/cosmos/base/store/v1beta1/snapshot.proto @@ -1,5 +1,5 @@ syntax = "proto3"; -package cosmos.base.store.v1betav1; +package cosmos.base.store.v1beta1; import "gogoproto/gogo.proto"; @@ -7,6 +7,7 @@ option go_package = "github.com/cosmos/cosmos-sdk/store/types"; // SnapshotItem is an item contained in a rootmulti.Store snapshot. message SnapshotItem { + // item is the specific type of snapshot item. oneof item { SnapshotStoreItem store = 1; SnapshotIAVLItem iavl = 2 [(gogoproto.customname) = "IAVL"]; diff --git a/store/types/snapshot.pb.go b/store/types/snapshot.pb.go index 8ed0022ff40a..6450eeebcb4e 100644 --- a/store/types/snapshot.pb.go +++ b/store/types/snapshot.pb.go @@ -25,6 +25,8 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package // SnapshotItem is an item contained in a rootmulti.Store snapshot. type SnapshotItem struct { + // item is the specific type of snapshot item. + // // Types that are valid to be assigned to Item: // *SnapshotItem_Store // *SnapshotItem_IAVL @@ -224,9 +226,9 @@ func (m *SnapshotIAVLItem) GetHeight() int32 { } func init() { - proto.RegisterType((*SnapshotItem)(nil), "cosmos.base.store.v1betav1.SnapshotItem") - proto.RegisterType((*SnapshotStoreItem)(nil), "cosmos.base.store.v1betav1.SnapshotStoreItem") - proto.RegisterType((*SnapshotIAVLItem)(nil), "cosmos.base.store.v1betav1.SnapshotIAVLItem") + proto.RegisterType((*SnapshotItem)(nil), "cosmos.base.store.v1beta1.SnapshotItem") + proto.RegisterType((*SnapshotStoreItem)(nil), "cosmos.base.store.v1beta1.SnapshotStoreItem") + proto.RegisterType((*SnapshotIAVLItem)(nil), "cosmos.base.store.v1beta1.SnapshotIAVLItem") } func init() { @@ -234,28 +236,28 @@ func init() { } var fileDescriptor_9c55879db4cc4502 = []byte{ - // 325 bytes of a gzipped FileDescriptorProto + // 324 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x91, 0xc1, 0x4a, 0xc3, 0x30, - 0x18, 0xc7, 0x1b, 0xd7, 0x4d, 0xfd, 0xdc, 0x61, 0x86, 0x21, 0x65, 0x87, 0x3a, 0x76, 0xb1, 0x07, - 0x97, 0x30, 0x7d, 0x02, 0x0b, 0xc2, 0x06, 0x3b, 0x65, 0xe0, 0xc1, 0x5b, 0x3a, 0x43, 0x5b, 0xb6, - 0x2e, 0x63, 0xc9, 0x0a, 0x7b, 0x0b, 0xdf, 0xc3, 0x17, 0xf1, 0xb8, 0xa3, 0x27, 0x91, 0xee, 0x45, - 0x24, 0x49, 0x77, 0x51, 0x04, 0x4f, 0xfd, 0xfe, 0xe5, 0xf7, 0xff, 0xe5, 0x83, 0x0f, 0xa2, 0xb9, - 0x54, 0x85, 0x54, 0x34, 0xe1, 0x4a, 0x50, 0xa5, 0xe5, 0x46, 0xd0, 0x72, 0x94, 0x08, 0xcd, 0x47, - 0x54, 0xad, 0xf8, 0x5a, 0x65, 0x52, 0x93, 0xf5, 0x46, 0x6a, 0x89, 0x7b, 0x8e, 0x24, 0x86, 0x24, - 0x96, 0x24, 0x8e, 0x2c, 0x47, 0xbd, 0x6e, 0x2a, 0x53, 0x69, 0x31, 0x6a, 0x26, 0xd7, 0x18, 0xbc, - 0x21, 0x68, 0xcf, 0x6a, 0xc9, 0x44, 0x8b, 0x02, 0x3f, 0x42, 0xd3, 0x16, 0x03, 0xd4, 0x47, 0xd1, - 0xc5, 0xdd, 0x90, 0xfc, 0xad, 0x24, 0xc7, 0xe2, 0xcc, 0xfc, 0x36, 0xed, 0xb1, 0xc7, 0x5c, 0x1b, - 0x4f, 0xc1, 0xcf, 0x79, 0xb9, 0x0c, 0x4e, 0xac, 0xe5, 0xf6, 0x3f, 0x96, 0xc9, 0xc3, 0xd3, 0xd4, - 0x48, 0xe2, 0xb3, 0xea, 0xf3, 0xda, 0x37, 0x69, 0xec, 0x31, 0x6b, 0x89, 0x5b, 0xe0, 0xe7, 0x5a, - 0x14, 0x83, 0x1b, 0xb8, 0xfc, 0xf5, 0x26, 0xc6, 0xe0, 0xaf, 0x78, 0xe1, 0x16, 0x3e, 0x67, 0x76, - 0x1e, 0x2c, 0xa1, 0xf3, 0x53, 0x8b, 0x3b, 0xd0, 0x58, 0x88, 0x9d, 0xc5, 0xda, 0xcc, 0x8c, 0xb8, - 0x0b, 0xcd, 0x92, 0x2f, 0xb7, 0xc2, 0x6e, 0xd9, 0x66, 0x2e, 0xe0, 0x00, 0x4e, 0x4b, 0xb1, 0x51, - 0xb9, 0x5c, 0x05, 0x8d, 0x3e, 0x8a, 0x1a, 0xec, 0x18, 0xf1, 0x15, 0xb4, 0x32, 0x91, 0xa7, 0x99, - 0x0e, 0xfc, 0x3e, 0x8a, 0x9a, 0xac, 0x4e, 0x71, 0xfc, 0x5e, 0x85, 0x68, 0x5f, 0x85, 0xe8, 0xab, - 0x0a, 0xd1, 0xeb, 0x21, 0xf4, 0xf6, 0x87, 0xd0, 0xfb, 0x38, 0x84, 0xde, 0x73, 0x94, 0xe6, 0x3a, - 0xdb, 0x26, 0x64, 0x2e, 0x0b, 0x5a, 0x5f, 0xd1, 0x7d, 0x86, 0xea, 0x65, 0x51, 0xdf, 0x52, 0xef, - 0xd6, 0x42, 0x25, 0x2d, 0x7b, 0x8f, 0xfb, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0x21, 0xc0, 0xab, - 0x06, 0xed, 0x01, 0x00, 0x00, + 0x18, 0xc7, 0x1b, 0xd7, 0x4d, 0xfd, 0xdc, 0x61, 0x86, 0x21, 0xd5, 0x43, 0x1d, 0xbb, 0x58, 0x50, + 0x13, 0xa6, 0x4f, 0x60, 0xf1, 0xb0, 0xa1, 0xa7, 0x0c, 0x3c, 0x78, 0x4b, 0x67, 0x68, 0xcb, 0xd6, + 0x65, 0x2c, 0x59, 0x61, 0x6f, 0xe1, 0x6b, 0xf8, 0x26, 0x1e, 0x77, 0xf4, 0x24, 0xd2, 0xbd, 0x88, + 0x24, 0xe9, 0x2e, 0x8a, 0xe0, 0xa9, 0xdf, 0xbf, 0xfc, 0xfe, 0xbf, 0x7c, 0xf0, 0x41, 0x34, 0x91, + 0xaa, 0x90, 0x8a, 0x26, 0x5c, 0x09, 0xaa, 0xb4, 0x5c, 0x0a, 0x5a, 0x0e, 0x12, 0xa1, 0xf9, 0x80, + 0xaa, 0x39, 0x5f, 0xa8, 0x4c, 0x6a, 0xb2, 0x58, 0x4a, 0x2d, 0xf1, 0xa9, 0x23, 0x89, 0x21, 0x89, + 0x25, 0x49, 0x4d, 0x9e, 0x75, 0x53, 0x99, 0x4a, 0x4b, 0x51, 0x33, 0xb9, 0x42, 0xff, 0x0d, 0x41, + 0x7b, 0x5c, 0x3b, 0x46, 0x5a, 0x14, 0xf8, 0x1e, 0x9a, 0xb6, 0x17, 0xa0, 0x1e, 0x8a, 0x8e, 0x6e, + 0xae, 0xc8, 0x9f, 0x46, 0xb2, 0xeb, 0x8d, 0xcd, 0x5f, 0x53, 0x1e, 0x7a, 0xcc, 0x95, 0xf1, 0x03, + 0xf8, 0x39, 0x2f, 0x67, 0xc1, 0x9e, 0x95, 0x5c, 0xfe, 0x43, 0x32, 0xba, 0x7b, 0x7a, 0x34, 0x8e, + 0xf8, 0xa0, 0xfa, 0x3c, 0xf7, 0x4d, 0x1a, 0x7a, 0xcc, 0x4a, 0xe2, 0x16, 0xf8, 0xb9, 0x16, 0x45, + 0xff, 0x02, 0x8e, 0x7f, 0x3d, 0x89, 0x31, 0xf8, 0x73, 0x5e, 0xb8, 0x75, 0x0f, 0x99, 0x9d, 0xfb, + 0x33, 0xe8, 0xfc, 0xd4, 0xe2, 0x0e, 0x34, 0xa6, 0x62, 0x6d, 0xb1, 0x36, 0x33, 0x23, 0xee, 0x42, + 0xb3, 0xe4, 0xb3, 0x95, 0xb0, 0x4b, 0xb6, 0x99, 0x0b, 0x38, 0x80, 0xfd, 0x52, 0x2c, 0x55, 0x2e, + 0xe7, 0x41, 0xa3, 0x87, 0xa2, 0x06, 0xdb, 0x45, 0x7c, 0x02, 0xad, 0x4c, 0xe4, 0x69, 0xa6, 0x03, + 0xbf, 0x87, 0xa2, 0x26, 0xab, 0x53, 0x1c, 0xbf, 0x57, 0x21, 0xda, 0x54, 0x21, 0xfa, 0xaa, 0x42, + 0xf4, 0xba, 0x0d, 0xbd, 0xcd, 0x36, 0xf4, 0x3e, 0xb6, 0xa1, 0xf7, 0x1c, 0xa5, 0xb9, 0xce, 0x56, + 0x09, 0x99, 0xc8, 0x82, 0xd6, 0x27, 0x74, 0x9f, 0x6b, 0xf5, 0x32, 0xad, 0x0f, 0xa9, 0xd7, 0x0b, + 0xa1, 0x92, 0x96, 0xbd, 0xc6, 0xed, 0x77, 0x00, 0x00, 0x00, 0xff, 0xff, 0x75, 0x87, 0x24, 0x7b, + 0xea, 0x01, 0x00, 0x00, } func (m *SnapshotItem) Marshal() (dAtA []byte, err error) { From 59ed84a5cb820e957ee07b1038d2e01e18f69f2e Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Fri, 28 Aug 2020 17:13:18 +0200 Subject: [PATCH 05/19] comment tweaks --- snapshots/types/snapshotter.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/snapshots/types/snapshotter.go b/snapshots/types/snapshotter.go index af9c2bb5bc13..d59a2a0874bb 100644 --- a/snapshots/types/snapshotter.go +++ b/snapshots/types/snapshotter.go @@ -2,13 +2,13 @@ package types import "io" -// Snapshotter is something that can take and restore snapshots, consisting of streamed binary +// Snapshotter is something that can create and restore snapshots, consisting of streamed binary // chunks - all of which must be read from the channel and closed. If an unsupported format is // given, it must return ErrUnknownFormat (possibly wrapped with fmt.Errorf). type Snapshotter interface { - // Snapshot takes a state snapshot. + // Snapshot creates a state snapshot, returning a channel of snapshot chunk readers. Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) - // Restore restores a state snapshot. + // Restore restores a state snapshot, taking snapshot chunk readers as input. Restore(height uint64, format uint32, chunks <-chan io.ReadCloser) error } From 03daa7845d09aafc3cdd9ae0a9e87b9361572606 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Fri, 28 Aug 2020 17:28:09 +0200 Subject: [PATCH 06/19] don't use type aliasing --- baseapp/abci.go | 12 ++++++------ baseapp/baseapp_test.go | 8 ++++---- snapshots/alias.go | 20 -------------------- snapshots/helpers_test.go | 5 +++-- snapshots/manager.go | 23 +++++++++++++---------- snapshots/manager_test.go | 35 ++++++++++++++++++----------------- snapshots/store.go | 4 +++- snapshots/store_test.go | 4 ++-- store/rootmulti/store.go | 17 +++++++++-------- store/rootmulti/store_test.go | 28 ++++++++++++++-------------- store/types/store.go | 4 ++-- 11 files changed, 74 insertions(+), 86 deletions(-) delete mode 100644 snapshots/alias.go diff --git a/baseapp/abci.go b/baseapp/abci.go index 6057907eebab..f0c66912e39b 100644 --- a/baseapp/abci.go +++ b/baseapp/abci.go @@ -16,7 +16,7 @@ import ( grpcstatus "google.golang.org/grpc/status" "github.com/cosmos/cosmos-sdk/codec" - "github.com/cosmos/cosmos-sdk/snapshots" + snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" "github.com/cosmos/cosmos-sdk/telemetry" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" @@ -330,7 +330,7 @@ func (app *BaseApp) halt() { os.Exit(0) } -// snapshot takes a snapshot of the current state and prunes any old snapshots. +// snapshot takes a snapshot of the current state and prunes any old snapshottypes. func (app *BaseApp) snapshot(height int64) { app.logger.Info("Taking state snapshot", "height", height) snapshot, err := app.snapshotManager.Take(uint64(height)) @@ -430,7 +430,7 @@ func (app *BaseApp) OfferSnapshot(req abci.RequestOfferSnapshot) abci.ResponseOf return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT} } - snapshot, err := snapshots.SnapshotFromABCI(req.Snapshot) + snapshot, err := snapshottypes.SnapshotFromABCI(req.Snapshot) if err != nil { app.logger.Error("Failed to decode snapshot metadata", "err", err) return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT} @@ -440,10 +440,10 @@ func (app *BaseApp) OfferSnapshot(req abci.RequestOfferSnapshot) abci.ResponseOf case err == nil: return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ACCEPT} - case errors.Is(err, snapshots.ErrUnknownFormat): + case errors.Is(err, snapshottypes.ErrUnknownFormat): return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT_FORMAT} - case errors.Is(err, snapshots.ErrInvalidMetadata): + case errors.Is(err, snapshottypes.ErrInvalidMetadata): app.logger.Error("Rejecting invalid snapshot", "height", req.Snapshot.Height, "format", req.Snapshot.Format, "err", err.Error()) return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT} @@ -464,7 +464,7 @@ func (app *BaseApp) ApplySnapshotChunk(req abci.RequestApplySnapshotChunk) abci. case err == nil: return abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT} - case errors.Is(err, snapshots.ErrChunkHashMismatch): + case errors.Is(err, snapshottypes.ErrChunkHashMismatch): app.logger.Error("Chunk checksum mismatch, rejecting sender and requesting refetch", "chunk", req.Index, "sender", req.Sender, "err", err) return abci.ResponseApplySnapshotChunk{ diff --git a/baseapp/baseapp_test.go b/baseapp/baseapp_test.go index 53a829f4fddb..ea1d79936ddd 100644 --- a/baseapp/baseapp_test.go +++ b/baseapp/baseapp_test.go @@ -23,6 +23,7 @@ import ( "github.com/cosmos/cosmos-sdk/codec" "github.com/cosmos/cosmos-sdk/snapshots" + snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" "github.com/cosmos/cosmos-sdk/store/rootmulti" store "github.com/cosmos/cosmos-sdk/store/types" "github.com/cosmos/cosmos-sdk/testutil/testdata" @@ -1379,7 +1380,6 @@ func TestCustomRunTxPanicHandler(t *testing.T) { anteOpt := func(bapp *BaseApp) { bapp.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, err error) { panic(sdkerrors.Wrap(anteErr, "anteHandler")) - return }) } routerOpt := func(bapp *BaseApp) { @@ -1761,7 +1761,7 @@ func TestOfferSnapshot_Errors(t *testing.T) { app, teardown := setupBaseAppWithSnapshots(t, 0, 0) defer teardown() - m := snapshots.Metadata{ChunkHashes: [][]byte{{1}, {2}, {3}}} + m := snapshottypes.Metadata{ChunkHashes: [][]byte{{1}, {2}, {3}}} metadata, err := m.Marshal() require.NoError(t, err) hash := []byte{1, 2, 3} @@ -1795,7 +1795,7 @@ func TestOfferSnapshot_Errors(t *testing.T) { // Offering a snapshot after one has been accepted should error resp := app.OfferSnapshot(abci.RequestOfferSnapshot{Snapshot: &abci.Snapshot{ Height: 1, - Format: snapshots.CurrentFormat, + Format: snapshottypes.CurrentFormat, Chunks: 3, Hash: []byte{1, 2, 3}, Metadata: metadata, @@ -1804,7 +1804,7 @@ func TestOfferSnapshot_Errors(t *testing.T) { resp = app.OfferSnapshot(abci.RequestOfferSnapshot{Snapshot: &abci.Snapshot{ Height: 2, - Format: snapshots.CurrentFormat, + Format: snapshottypes.CurrentFormat, Chunks: 3, Hash: []byte{1, 2, 3}, Metadata: metadata, diff --git a/snapshots/alias.go b/snapshots/alias.go deleted file mode 100644 index 9b292f547733..000000000000 --- a/snapshots/alias.go +++ /dev/null @@ -1,20 +0,0 @@ -package snapshots - -import "github.com/cosmos/cosmos-sdk/snapshots/types" - -const ( - CurrentFormat = types.CurrentFormat -) - -var ( - ErrInvalidMetadata = types.ErrInvalidMetadata - ErrChunkHashMismatch = types.ErrChunkHashMismatch - ErrUnknownFormat = types.ErrUnknownFormat - SnapshotFromABCI = types.SnapshotFromABCI -) - -type ( - Snapshotter = types.Snapshotter - Snapshot = types.Snapshot - Metadata = types.Metadata -) diff --git a/snapshots/helpers_test.go b/snapshots/helpers_test.go index 061d9c12a868..17711d151e60 100644 --- a/snapshots/helpers_test.go +++ b/snapshots/helpers_test.go @@ -14,6 +14,7 @@ import ( db "github.com/tendermint/tm-db" "github.com/cosmos/cosmos-sdk/snapshots" + "github.com/cosmos/cosmos-sdk/snapshots/types" ) func checksum(b []byte) []byte { @@ -64,7 +65,7 @@ type mockSnapshotter struct { func (m *mockSnapshotter) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser) error { if format == 0 { - return snapshots.ErrUnknownFormat + return types.ErrUnknownFormat } if m.chunks != nil { return errors.New("already has contents") @@ -84,7 +85,7 @@ func (m *mockSnapshotter) Restore(height uint64, format uint32, chunks <-chan io func (m *mockSnapshotter) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { if format == 0 { - return nil, snapshots.ErrUnknownFormat + return nil, types.ErrUnknownFormat } ch := make(chan io.ReadCloser, len(m.chunks)) for _, chunk := range m.chunks { diff --git a/snapshots/manager.go b/snapshots/manager.go index 93adc43e2cca..5e313e8cca29 100644 --- a/snapshots/manager.go +++ b/snapshots/manager.go @@ -9,6 +9,8 @@ import ( "io/ioutil" "sync" "time" + + "github.com/cosmos/cosmos-sdk/snapshots/types" ) const ( @@ -28,7 +30,7 @@ type operation string // mirroring the ABCI interface. type Manager struct { store *Store - target Snapshotter + target types.Snapshotter mtx sync.Mutex operation operation @@ -38,7 +40,7 @@ type Manager struct { } // NewManager creates a new manager. -func NewManager(store *Store, target Snapshotter) *Manager { +func NewManager(store *Store, target types.Snapshotter) *Manager { return &Manager{ store: store, target: target, @@ -83,7 +85,7 @@ func (m *Manager) endLocked() { } // List lists snapshots, mirroring ABCI ListSnapshots. It can be concurrent with other operations. -func (m *Manager) List() ([]*Snapshot, error) { +func (m *Manager) List() ([]*types.Snapshot, error) { return m.store.List() } @@ -103,7 +105,7 @@ func (m *Manager) LoadChunk(height uint64, format uint32, chunk uint32) ([]byte, } // Take takes a snapshot and returns its metadata. -func (m *Manager) Take(height uint64) (*Snapshot, error) { +func (m *Manager) Take(height uint64) (*types.Snapshot, error) { if m == nil { return nil, errors.New("no snapshot store configured") } @@ -121,11 +123,11 @@ func (m *Manager) Take(height uint64) (*Snapshot, error) { return nil, fmt.Errorf("a more recent snapshot already exists at height %v", latest.Height) } - chunks, err := m.target.Snapshot(height, CurrentFormat) + chunks, err := m.target.Snapshot(height, types.CurrentFormat) if err != nil { return nil, err } - return m.store.Save(height, CurrentFormat, chunks) + return m.store.Save(height, types.CurrentFormat, chunks) } // Prune prunes snapshots, if no other operations are in progress. @@ -140,13 +142,13 @@ func (m *Manager) Prune(retain uint32) (uint64, error) { // Restore begins an async snapshot restoration, mirroring ABCI OfferSnapshot. Chunks must be fed // via RestoreChunk() until the restore is complete or a chunk fails. -func (m *Manager) Restore(snapshot Snapshot) error { +func (m *Manager) Restore(snapshot types.Snapshot) error { if snapshot.Chunks == 0 { - return fmt.Errorf("%w: no chunks", ErrInvalidMetadata) + return fmt.Errorf("%w: no chunks", types.ErrInvalidMetadata) } if uint32(len(snapshot.Metadata.ChunkHashes)) != snapshot.Chunks { return fmt.Errorf("%w: snapshot has %v chunk hashes, but %v chunks", - ErrInvalidMetadata, + types.ErrInvalidMetadata, uint32(len(snapshot.Metadata.ChunkHashes)), snapshot.Chunks) } @@ -205,7 +207,8 @@ func (m *Manager) RestoreChunk(chunk []byte) (bool, error) { // Verify the chunk hash. hash := sha256.Sum256(chunk) if !bytes.Equal(hash[:], m.restorePending[0]) { - return false, fmt.Errorf("%w (expected %x, got %x)", ErrChunkHashMismatch, hash, m.restorePending[0]) + return false, fmt.Errorf("%w (expected %x, got %x)", + types.ErrChunkHashMismatch, hash, m.restorePending[0]) } // Pass the chunk to the restore, and wait for completion if it was the final one. diff --git a/snapshots/manager_test.go b/snapshots/manager_test.go index 2ccf72833c6a..7d47e11317a0 100644 --- a/snapshots/manager_test.go +++ b/snapshots/manager_test.go @@ -8,6 +8,7 @@ import ( "github.com/stretchr/testify/require" "github.com/cosmos/cosmos-sdk/snapshots" + "github.com/cosmos/cosmos-sdk/snapshots/types" ) func TestManager_List(t *testing.T) { @@ -28,7 +29,7 @@ func TestManager_List(t *testing.T) { defer teardown() list, err := manager.List() require.NoError(t, err) - assert.Equal(t, []*snapshots.Snapshot{}, list) + assert.Equal(t, []*types.Snapshot{}, list) } func TestManager_LoadChunk(t *testing.T) { @@ -77,12 +78,12 @@ func TestManager_Take(t *testing.T) { // taking a snapshot at a higher height should be fine, and should return it snapshot, err := manager.Take(5) require.NoError(t, err) - assert.Equal(t, &snapshots.Snapshot{ + assert.Equal(t, &types.Snapshot{ Height: 5, - Format: snapshots.CurrentFormat, + Format: types.CurrentFormat, Chunks: 3, Hash: []uint8{0x47, 0xe4, 0xee, 0x7f, 0x21, 0x1f, 0x73, 0x26, 0x5d, 0xd1, 0x76, 0x58, 0xf6, 0xe2, 0x1c, 0x13, 0x18, 0xbd, 0x6c, 0x81, 0xf3, 0x75, 0x98, 0xe2, 0xa, 0x27, 0x56, 0x29, 0x95, 0x42, 0xef, 0xcf}, - Metadata: snapshots.Metadata{ + Metadata: types.Metadata{ ChunkHashes: [][]byte{ checksum([]byte{1, 2, 3}), checksum([]byte{4, 5, 6}), @@ -136,37 +137,37 @@ func TestManager_Restore(t *testing.T) { } // Restore errors on invalid format - err := manager.Restore(snapshots.Snapshot{ + err := manager.Restore(types.Snapshot{ Height: 3, Format: 0, Hash: []byte{1, 2, 3}, Chunks: uint32(len(chunks)), - Metadata: snapshots.Metadata{ChunkHashes: checksums(chunks)}, + Metadata: types.Metadata{ChunkHashes: checksums(chunks)}, }) require.Error(t, err) - require.Equal(t, err, snapshots.ErrUnknownFormat) + require.Equal(t, err, types.ErrUnknownFormat) // Restore errors on no chunks - err = manager.Restore(snapshots.Snapshot{Height: 3, Format: 1, Hash: []byte{1, 2, 3}}) + err = manager.Restore(types.Snapshot{Height: 3, Format: 1, Hash: []byte{1, 2, 3}}) require.Error(t, err) // Restore errors on chunk and chunkhashes mismatch - err = manager.Restore(snapshots.Snapshot{ + err = manager.Restore(types.Snapshot{ Height: 3, Format: 1, Hash: []byte{1, 2, 3}, Chunks: 4, - Metadata: snapshots.Metadata{ChunkHashes: checksums(chunks)}, + Metadata: types.Metadata{ChunkHashes: checksums(chunks)}, }) require.Error(t, err) // Starting a restore works - err = manager.Restore(snapshots.Snapshot{ + err = manager.Restore(types.Snapshot{ Height: 3, Format: 1, Hash: []byte{1, 2, 3}, Chunks: 3, - Metadata: snapshots.Metadata{ChunkHashes: checksums(chunks)}, + Metadata: types.Metadata{ChunkHashes: checksums(chunks)}, }) require.NoError(t, err) @@ -180,7 +181,7 @@ func TestManager_Restore(t *testing.T) { // Feeding an invalid chunk should error due to invalid checksum, but not abort restoration. _, err = manager.RestoreChunk([]byte{9, 9, 9}) require.Error(t, err) - require.True(t, errors.Is(err, snapshots.ErrChunkHashMismatch)) + require.True(t, errors.Is(err, types.ErrChunkHashMismatch)) // Feeding the chunks should work for i, chunk := range chunks { @@ -196,12 +197,12 @@ func TestManager_Restore(t *testing.T) { assert.Equal(t, chunks, target.chunks) // Starting a new restore should fail now, because the target already has contents. - err = manager.Restore(snapshots.Snapshot{ + err = manager.Restore(types.Snapshot{ Height: 3, Format: 1, Hash: []byte{1, 2, 3}, Chunks: 3, - Metadata: snapshots.Metadata{ChunkHashes: checksums(chunks)}, + Metadata: types.Metadata{ChunkHashes: checksums(chunks)}, }) require.Error(t, err) @@ -209,12 +210,12 @@ func TestManager_Restore(t *testing.T) { // fail it with a checksum error. That error should stop the operation, so that we can do // a prune operation right after. target.chunks = nil - err = manager.Restore(snapshots.Snapshot{ + err = manager.Restore(types.Snapshot{ Height: 3, Format: 1, Hash: []byte{1, 2, 3}, Chunks: 3, - Metadata: snapshots.Metadata{ChunkHashes: checksums(chunks)}, + Metadata: types.Metadata{ChunkHashes: checksums(chunks)}, }) require.NoError(t, err) } diff --git a/snapshots/store.go b/snapshots/store.go index b566f0b645e0..ed262d8c71df 100644 --- a/snapshots/store.go +++ b/snapshots/store.go @@ -241,7 +241,9 @@ func (s *Store) Prune(retain uint32) (uint64, error) { } // Save saves a snapshot to disk, returning it. -func (s *Store) Save(height uint64, format uint32, chunks <-chan io.ReadCloser) (*Snapshot, error) { +func (s *Store) Save( + height uint64, format uint32, chunks <-chan io.ReadCloser, +) (*types.Snapshot, error) { defer DrainChunks(chunks) if height == 0 { return nil, errors.New("snapshot height cannot be 0") diff --git a/snapshots/store_test.go b/snapshots/store_test.go index 7d9e0bba7fa1..02e59d3fefc2 100644 --- a/snapshots/store_test.go +++ b/snapshots/store_test.go @@ -300,12 +300,12 @@ func TestStore_Save(t *testing.T) { // Saving a snapshot should work snapshot, err := store.Save(4, 1, makeChunks([][]byte{{1}, {2}})) require.NoError(t, err) - assert.Equal(t, &snapshots.Snapshot{ + assert.Equal(t, &types.Snapshot{ Height: 4, Format: 1, Chunks: 2, Hash: hash([][]byte{{1}, {2}}), - Metadata: snapshots.Metadata{ + Metadata: types.Metadata{ ChunkHashes: [][]byte{ checksum([]byte{1}), checksum([]byte{2}), diff --git a/store/rootmulti/store.go b/store/rootmulti/store.go index 0a9807f060e5..9cc4b3f3d634 100644 --- a/store/rootmulti/store.go +++ b/store/rootmulti/store.go @@ -18,6 +18,7 @@ import ( dbm "github.com/tendermint/tm-db" "github.com/cosmos/cosmos-sdk/snapshots" + snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" "github.com/cosmos/cosmos-sdk/store/cachemulti" "github.com/cosmos/cosmos-sdk/store/dbadapter" "github.com/cosmos/cosmos-sdk/store/iavl" @@ -537,13 +538,13 @@ func parsePath(path string) (storeName string, subpath string, err error) { //---------------------- Snapshotting ------------------ -// Snapshot implements snapshots.Snapshotter. The snapshot output for a given format must be +// Snapshot implements snapshottypes.Snapshotter. The snapshot output for a given format must be // identical across nodes such that chunks from different sources fit together. If the output for a // given format changes (at the byte level), the snapshot format must be bumped - see // TestMultistoreSnapshot_Checksum test. func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { - if format != snapshots.CurrentFormat { - return nil, fmt.Errorf("%w %v", snapshots.ErrUnknownFormat, format) + if format != snapshottypes.CurrentFormat { + return nil, fmt.Errorf("%w %v", snapshottypes.ErrUnknownFormat, format) } if height == 0 { return nil, errors.New("cannot snapshot height 0") @@ -656,16 +657,16 @@ func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, e return ch, nil } -// Restore implements snapshots.Snapshotter. +// Restore implements snapshottypes.Snapshotter. func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser) error { - if format != snapshots.CurrentFormat { - return fmt.Errorf("%w %v", snapshots.ErrUnknownFormat, format) + if format != snapshottypes.CurrentFormat { + return fmt.Errorf("%w %v", snapshottypes.ErrUnknownFormat, format) } if height == 0 { - return fmt.Errorf("%w: cannot restore snapshot at height 0", snapshots.ErrInvalidMetadata) + return fmt.Errorf("%w: cannot restore snapshot at height 0", snapshottypes.ErrInvalidMetadata) } if height > math.MaxInt64 { - return fmt.Errorf("%w: snapshot height %v cannot exceed %v", snapshots.ErrInvalidMetadata, + return fmt.Errorf("%w: snapshot height %v cannot exceed %v", snapshottypes.ErrInvalidMetadata, height, math.MaxInt64) } diff --git a/store/rootmulti/store_test.go b/store/rootmulti/store_test.go index c5dc0510efb0..55af45cc3aeb 100644 --- a/store/rootmulti/store_test.go +++ b/store/rootmulti/store_test.go @@ -16,7 +16,7 @@ import ( abci "github.com/tendermint/tendermint/abci/types" dbm "github.com/tendermint/tm-db" - "github.com/cosmos/cosmos-sdk/snapshots" + snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" "github.com/cosmos/cosmos-sdk/store/iavl" sdkmaps "github.com/cosmos/cosmos-sdk/store/internal/maps" "github.com/cosmos/cosmos-sdk/store/types" @@ -518,7 +518,7 @@ func TestMultistoreSnapshot_Checksum(t *testing.T) { // Chunks from different nodes must fit together, so all nodes must produce identical chunks. // This checksum test makes sure that the byte stream remains identical. If the test fails // without having changed the data (e.g. because the Protobuf or zlib encoding changes), - // snapshots.CurrentFormat must be bumped. + // snapshottypes.CurrentFormat must be bumped. store := newMultiStoreWithGeneratedData(dbm.NewMemDB(), 5, 10000) version := uint64(store.LastCommitID().Version) @@ -561,10 +561,10 @@ func TestMultistoreSnapshot_Errors(t *testing.T) { format uint32 expectType error }{ - "0 height": {0, snapshots.CurrentFormat, nil}, - "0 format": {1, 0, snapshots.ErrUnknownFormat}, - "unknown height": {9, snapshots.CurrentFormat, nil}, - "unknown format": {1, 9, snapshots.ErrUnknownFormat}, + "0 height": {0, snapshottypes.CurrentFormat, nil}, + "0 format": {1, 0, snapshottypes.ErrUnknownFormat}, + "unknown height": {9, snapshottypes.CurrentFormat, nil}, + "unknown format": {1, 9, snapshottypes.ErrUnknownFormat}, } for name, tc := range testcases { tc := tc @@ -586,9 +586,9 @@ func TestMultistoreRestore_Errors(t *testing.T) { format uint32 expectType error }{ - "0 height": {0, snapshots.CurrentFormat, nil}, - "0 format": {1, 0, snapshots.ErrUnknownFormat}, - "unknown format": {1, 9, snapshots.ErrUnknownFormat}, + "0 height": {0, snapshottypes.CurrentFormat, nil}, + "0 format": {1, 0, snapshottypes.ErrUnknownFormat}, + "unknown format": {1, 9, snapshottypes.ErrUnknownFormat}, } for name, tc := range testcases { tc := tc @@ -608,9 +608,9 @@ func TestMultistoreSnapshotRestore(t *testing.T) { version := uint64(source.LastCommitID().Version) require.EqualValues(t, 3, version) - chunks, err := source.Snapshot(version, snapshots.CurrentFormat) + chunks, err := source.Snapshot(version, snapshottypes.CurrentFormat) require.NoError(t, err) - err = target.Restore(version, snapshots.CurrentFormat, chunks) + err = target.Restore(version, snapshottypes.CurrentFormat, chunks) require.NoError(t, err) assert.Equal(t, source.LastCommitID(), target.LastCommitID()) @@ -658,7 +658,7 @@ func benchmarkMultistoreSnapshot(b *testing.B, stores uint8, storeKeys uint64) { require.NoError(b, err) require.EqualValues(b, 0, target.LastCommitID().Version) - chunks, err := source.Snapshot(uint64(version), snapshots.CurrentFormat) + chunks, err := source.Snapshot(uint64(version), snapshottypes.CurrentFormat) require.NoError(b, err) for reader := range chunks { _, err := io.Copy(ioutil.Discard, reader) @@ -685,9 +685,9 @@ func benchmarkMultistoreSnapshotRestore(b *testing.B, stores uint8, storeKeys ui require.NoError(b, err) require.EqualValues(b, 0, target.LastCommitID().Version) - chunks, err := source.Snapshot(version, snapshots.CurrentFormat) + chunks, err := source.Snapshot(version, snapshottypes.CurrentFormat) require.NoError(b, err) - err = target.Restore(version, snapshots.CurrentFormat, chunks) + err = target.Restore(version, snapshottypes.CurrentFormat, chunks) require.NoError(b, err) require.Equal(b, source.LastCommitID(), target.LastCommitID()) } diff --git a/store/types/store.go b/store/types/store.go index 6a60e6a5054b..92f11a562cf9 100644 --- a/store/types/store.go +++ b/store/types/store.go @@ -4,7 +4,7 @@ import ( "fmt" "io" - "github.com/cosmos/cosmos-sdk/snapshots" + snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" abci "github.com/tendermint/tendermint/abci/types" dbm "github.com/tendermint/tm-db" @@ -132,7 +132,7 @@ type CacheMultiStore interface { type CommitMultiStore interface { Committer MultiStore - snapshots.Snapshotter + snapshottypes.Snapshotter // Mount a store of type using the given db. // If db == nil, the new store will use the CommitMultiStore db. From d8590caaf459f876a87027910c86761232e8be74 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Fri, 28 Aug 2020 17:32:50 +0200 Subject: [PATCH 07/19] don't call .Error() when logging errors --- baseapp/abci.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/baseapp/abci.go b/baseapp/abci.go index f0c66912e39b..d25f034cff77 100644 --- a/baseapp/abci.go +++ b/baseapp/abci.go @@ -344,7 +344,7 @@ func (app *BaseApp) snapshot(height int64) { app.logger.Debug("Pruning state snapshots") pruned, err := app.snapshotManager.Prune(app.snapshotKeepRecent) if err != nil { - app.logger.Error("Failed to prune state snapshots", "err", err.Error()) + app.logger.Error("Failed to prune state snapshots", "err", err) return } app.logger.Debug("Pruned state snapshots", "pruned", pruned) @@ -394,13 +394,13 @@ func (app *BaseApp) ListSnapshots(req abci.RequestListSnapshots) abci.ResponseLi snapshots, err := app.snapshotManager.List() if err != nil { - app.logger.Error("Failed to list snapshots", "err", err.Error()) + app.logger.Error("Failed to list snapshots", "err", err) return resp } for _, snapshot := range snapshots { abciSnapshot, err := snapshot.ToABCI() if err != nil { - app.logger.Error("Failed to list snapshots", "err", err.Error()) + app.logger.Error("Failed to list snapshots", "err", err) return resp } resp.Snapshots = append(resp.Snapshots, &abciSnapshot) @@ -417,7 +417,7 @@ func (app *BaseApp) LoadSnapshotChunk(req abci.RequestLoadSnapshotChunk) abci.Re chunk, err := app.snapshotManager.LoadChunk(req.Height, req.Format, req.Chunk) if err != nil { app.logger.Error("Failed to load snapshot chunk", "height", req.Height, "format", req.Format, - "chunk", req.Chunk, "err", err.Error()) + "chunk", req.Chunk, "err") return abci.ResponseLoadSnapshotChunk{} } return abci.ResponseLoadSnapshotChunk{Chunk: chunk} @@ -445,12 +445,12 @@ func (app *BaseApp) OfferSnapshot(req abci.RequestOfferSnapshot) abci.ResponseOf case errors.Is(err, snapshottypes.ErrInvalidMetadata): app.logger.Error("Rejecting invalid snapshot", "height", req.Snapshot.Height, - "format", req.Snapshot.Format, "err", err.Error()) + "format", req.Snapshot.Format, "err", err) return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_REJECT} default: app.logger.Error("Failed to restore snapshot", "height", req.Snapshot.Height, - "format", req.Snapshot.Format, "err", err.Error()) + "format", req.Snapshot.Format, "err", err) // We currently don't support resetting the IAVL stores and retrying a different snapshot, // so we ask Tendermint to abort all snapshot restoration. return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ABORT} @@ -474,7 +474,7 @@ func (app *BaseApp) ApplySnapshotChunk(req abci.RequestApplySnapshotChunk) abci. } default: - app.logger.Error("Failed to restore snapshot", "err", err.Error()) + app.logger.Error("Failed to restore snapshot", "err", err) return abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ABORT} } } From 9d05a4c4a8a48f06f07ce13922e6e6b449637f35 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Fri, 28 Aug 2020 17:49:53 +0200 Subject: [PATCH 08/19] use create terminology instead of take for snapshots --- baseapp/abci.go | 6 +++--- snapshots/helpers_test.go | 4 ++-- snapshots/manager.go | 44 +++++++++++++++++++-------------------- snapshots/manager_test.go | 16 +++++++------- 4 files changed, 35 insertions(+), 35 deletions(-) diff --git a/baseapp/abci.go b/baseapp/abci.go index d25f034cff77..2172cb5bd597 100644 --- a/baseapp/abci.go +++ b/baseapp/abci.go @@ -332,10 +332,10 @@ func (app *BaseApp) halt() { // snapshot takes a snapshot of the current state and prunes any old snapshottypes. func (app *BaseApp) snapshot(height int64) { - app.logger.Info("Taking state snapshot", "height", height) - snapshot, err := app.snapshotManager.Take(uint64(height)) + app.logger.Info("Creating state snapshot", "height", height) + snapshot, err := app.snapshotManager.Create(uint64(height)) if err != nil { - app.logger.Error("Failed to take state snapshot", "height", height, "err", err) + app.logger.Error("Failed to create state snapshot", "height", height, "err", err) return } app.logger.Info("Completed state snapshot", "height", height, "format", snapshot.Format) diff --git a/snapshots/helpers_test.go b/snapshots/helpers_test.go index 17711d151e60..eede8b2d5857 100644 --- a/snapshots/helpers_test.go +++ b/snapshots/helpers_test.go @@ -95,7 +95,7 @@ func (m *mockSnapshotter) Snapshot(height uint64, format uint32) (<-chan io.Read return ch, nil } -// setupBusyManager creates a manager with an empty store that is busy taking a snapshot at height 1. +// setupBusyManager creates a manager with an empty store that is busy creating a snapshot at height 1. // The snapshot will complete when the returned closer is called. func setupBusyManager(t *testing.T) (*snapshots.Manager, func()) { tempdir, err := ioutil.TempDir("", "") @@ -106,7 +106,7 @@ func setupBusyManager(t *testing.T) (*snapshots.Manager, func()) { mgr := snapshots.NewManager(store, hung) go func() { - _, err := mgr.Take(1) + _, err := mgr.Create(1) require.NoError(t, err) }() time.Sleep(10 * time.Millisecond) diff --git a/snapshots/manager.go b/snapshots/manager.go index 5e313e8cca29..26019ee37661 100644 --- a/snapshots/manager.go +++ b/snapshots/manager.go @@ -84,28 +84,8 @@ func (m *Manager) endLocked() { m.restorePending = nil } -// List lists snapshots, mirroring ABCI ListSnapshots. It can be concurrent with other operations. -func (m *Manager) List() ([]*types.Snapshot, error) { - return m.store.List() -} - -// LoadChunk loads a chunk into a byte slice, mirroring ABCI LoadChunk. It can be called -// concurrently with other operations. If the chunk does not exist, nil is returned. -func (m *Manager) LoadChunk(height uint64, format uint32, chunk uint32) ([]byte, error) { - reader, err := m.store.LoadChunk(height, format, chunk) - if err != nil { - return nil, err - } - if reader == nil { - return nil, nil - } - defer reader.Close() - - return ioutil.ReadAll(reader) -} - -// Take takes a snapshot and returns its metadata. -func (m *Manager) Take(height uint64) (*types.Snapshot, error) { +// Create creates a snapshot and returns its metadata. +func (m *Manager) Create(height uint64) (*types.Snapshot, error) { if m == nil { return nil, errors.New("no snapshot store configured") } @@ -130,6 +110,26 @@ func (m *Manager) Take(height uint64) (*types.Snapshot, error) { return m.store.Save(height, types.CurrentFormat, chunks) } +// List lists snapshots, mirroring ABCI ListSnapshots. It can be concurrent with other operations. +func (m *Manager) List() ([]*types.Snapshot, error) { + return m.store.List() +} + +// LoadChunk loads a chunk into a byte slice, mirroring ABCI LoadChunk. It can be called +// concurrently with other operations. If the chunk does not exist, nil is returned. +func (m *Manager) LoadChunk(height uint64, format uint32, chunk uint32) ([]byte, error) { + reader, err := m.store.LoadChunk(height, format, chunk) + if err != nil { + return nil, err + } + if reader == nil { + return nil, nil + } + defer reader.Close() + + return ioutil.ReadAll(reader) +} + // Prune prunes snapshots, if no other operations are in progress. func (m *Manager) Prune(retain uint32) (uint64, error) { err := m.begin(opPrune) diff --git a/snapshots/manager_test.go b/snapshots/manager_test.go index 7d47e11317a0..16e010151664 100644 --- a/snapshots/manager_test.go +++ b/snapshots/manager_test.go @@ -68,15 +68,15 @@ func TestManager_Take(t *testing.T) { manager := snapshots.NewManager(store, snapshotter) // nil manager should return error - _, err := (*snapshots.Manager)(nil).Take(1) + _, err := (*snapshots.Manager)(nil).Create(1) require.Error(t, err) - // taking a snapshot at a lower height than the latest should error - _, err = manager.Take(3) + // creating a snapshot at a lower height than the latest should error + _, err = manager.Create(3) require.Error(t, err) - // taking a snapshot at a higher height should be fine, and should return it - snapshot, err := manager.Take(5) + // creating a snapshot at a higher height should be fine, and should return it + snapshot, err := manager.Create(5) require.NoError(t, err) assert.Equal(t, &types.Snapshot{ Height: 5, @@ -97,10 +97,10 @@ func TestManager_Take(t *testing.T) { assert.Equal(t, snapshot, storeSnapshot) assert.Equal(t, [][]byte{{1, 2, 3}, {4, 5, 6}, {7, 8, 9}}, readChunks(chunks)) - // taking a snapshot while a different snapshot is being taken should error + // creating a snapshot while a different snapshot is being created should error manager, teardown = setupBusyManager(t) defer teardown() - _, err = manager.Take(9) + _, err = manager.Create(9) require.Error(t, err) } @@ -172,7 +172,7 @@ func TestManager_Restore(t *testing.T) { require.NoError(t, err) // While the restore is in progress, any other operations fail - _, err = manager.Take(4) + _, err = manager.Create(4) require.Error(t, err) _, err = manager.Prune(1) From 6be3953ef3be3e90277fafc611bef5a6d9a55b18 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Fri, 28 Aug 2020 17:53:04 +0200 Subject: [PATCH 09/19] reuse chunk hasher --- snapshots/store.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/snapshots/store.go b/snapshots/store.go index ed262d8c71df..43ad1faf3d5a 100644 --- a/snapshots/store.go +++ b/snapshots/store.go @@ -276,9 +276,9 @@ func (s *Store) Save( } index := uint32(0) snapshotHasher := sha256.New() + chunkHasher := sha256.New() for chunkBody := range chunks { defer chunkBody.Close() // nolint: staticcheck - chunkHasher := sha256.New() dir := s.pathSnapshot(height, format) err = os.MkdirAll(dir, 0755) if err != nil { @@ -290,6 +290,7 @@ func (s *Store) Save( return nil, fmt.Errorf("failed to create snapshot chunk file %q: %w", path, err) } defer file.Close() // nolint: staticcheck + chunkHasher.Reset() _, err = io.Copy(io.MultiWriter(file, chunkHasher, snapshotHasher), chunkBody) if err != nil { return nil, fmt.Errorf("failed to generate snapshot chunk %v: %w", index, err) From 4811638d5090afba680269176d095a4163656618 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Fri, 28 Aug 2020 17:56:23 +0200 Subject: [PATCH 10/19] simplify key encoding code --- snapshots/store.go | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/snapshots/store.go b/snapshots/store.go index 43ad1faf3d5a..177478684acb 100644 --- a/snapshots/store.go +++ b/snapshots/store.go @@ -358,16 +358,9 @@ func decodeKey(k []byte) (uint64, uint32, error) { // encodeKey encodes a snapshot key. func encodeKey(height uint64, format uint32) []byte { - k := make([]byte, 0, 13) - k = append(k, keyPrefixSnapshot) - - bHeight := make([]byte, 8) - binary.BigEndian.PutUint64(bHeight, height) - k = append(k, bHeight...) - - bFormat := make([]byte, 4) - binary.BigEndian.PutUint32(bFormat, format) - k = append(k, bFormat...) - + k := make([]byte, 13) + k[0] = keyPrefixSnapshot + binary.BigEndian.PutUint64(k[1:], height) + binary.BigEndian.PutUint32(k[9:], format) return k } From 7cdf03db5c3b95eab26858fafedc0965e599abeb Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Mon, 31 Aug 2020 20:12:10 +0200 Subject: [PATCH 11/19] track chunk index in Manager --- snapshots/manager.go | 30 +++++++++++++++++++----------- 1 file changed, 19 insertions(+), 11 deletions(-) diff --git a/snapshots/manager.go b/snapshots/manager.go index 26019ee37661..1da67ab02a30 100644 --- a/snapshots/manager.go +++ b/snapshots/manager.go @@ -32,11 +32,12 @@ type Manager struct { store *Store target types.Snapshotter - mtx sync.Mutex - operation operation - chRestore chan<- io.ReadCloser - chRestoreDone <-chan error - restorePending [][]byte // pending chunk hashes + mtx sync.Mutex + operation operation + chRestore chan<- io.ReadCloser + chRestoreDone <-chan error + restoreChunkHashes [][]byte + restoreChunkIndex uint32 } // NewManager creates a new manager. @@ -81,7 +82,8 @@ func (m *Manager) endLocked() { m.chRestore = nil } m.chRestoreDone = nil - m.restorePending = nil + m.restoreChunkHashes = nil + m.restoreChunkIndex = 0 } // Create creates a snapshot and returns its metadata. @@ -180,7 +182,8 @@ func (m *Manager) Restore(snapshot types.Snapshot) error { m.chRestore = chChunks m.chRestoreDone = chDone - m.restorePending = snapshot.Metadata.ChunkHashes + m.restoreChunkHashes = snapshot.Metadata.ChunkHashes + m.restoreChunkIndex = 0 return nil } @@ -193,6 +196,10 @@ func (m *Manager) RestoreChunk(chunk []byte) (bool, error) { return false, fmt.Errorf("no restore operation in progress") } + if int(m.restoreChunkIndex) >= len(m.restoreChunkHashes) { + return false, errors.New("received unexpected chunk") + } + // Check if any errors have occurred yet. select { case err := <-m.chRestoreDone: @@ -206,16 +213,17 @@ func (m *Manager) RestoreChunk(chunk []byte) (bool, error) { // Verify the chunk hash. hash := sha256.Sum256(chunk) - if !bytes.Equal(hash[:], m.restorePending[0]) { + expected := m.restoreChunkHashes[m.restoreChunkIndex] + if !bytes.Equal(hash[:], expected) { return false, fmt.Errorf("%w (expected %x, got %x)", - types.ErrChunkHashMismatch, hash, m.restorePending[0]) + types.ErrChunkHashMismatch, hash, expected) } // Pass the chunk to the restore, and wait for completion if it was the final one. m.chRestore <- ioutil.NopCloser(bytes.NewReader(chunk)) - m.restorePending = m.restorePending[1:] + m.restoreChunkIndex++ - if len(m.restorePending) == 0 { + if int(m.restoreChunkIndex) >= len(m.restoreChunkHashes) { close(m.chRestore) m.chRestore = nil err := <-m.chRestoreDone From 39063521392b1190c8f5b651db05768c98f2b690 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Mon, 31 Aug 2020 20:30:04 +0200 Subject: [PATCH 12/19] add restoreDone message for Manager --- snapshots/manager.go | 46 ++++++++++++++++++++++++++------------- snapshots/manager_test.go | 2 +- 2 files changed, 32 insertions(+), 16 deletions(-) diff --git a/snapshots/manager.go b/snapshots/manager.go index 1da67ab02a30..3ccf502b3b21 100644 --- a/snapshots/manager.go +++ b/snapshots/manager.go @@ -25,6 +25,12 @@ const ( // operation represents a Manager operation. Only one operation can be in progress at a time. type operation string +// restoreDone represents the result of a restore operation. +type restoreDone struct { + complete bool // if true, restore completed successfully (not prematurely) + err error // if non-nil, restore errored +} + // Manager manages snapshot and restore operations for an app, making sure only a single // long-running operation is in progress at any given time, and provides convenience methods // mirroring the ABCI interface. @@ -35,7 +41,7 @@ type Manager struct { mtx sync.Mutex operation operation chRestore chan<- io.ReadCloser - chRestoreDone <-chan error + chRestoreDone <-chan restoreDone restoreChunkHashes [][]byte restoreChunkIndex uint32 } @@ -163,20 +169,24 @@ func (m *Manager) Restore(snapshot types.Snapshot) error { // Start an asynchronous snapshot restoration, passing chunks and completion status via channels. chChunks := make(chan io.ReadCloser, chunkBufferSize) - chDone := make(chan error, 1) + chDone := make(chan restoreDone, 1) go func() { - chDone <- m.target.Restore(snapshot.Height, snapshot.Format, chChunks) + err := m.target.Restore(snapshot.Height, snapshot.Format, chChunks) + chDone <- restoreDone{ + complete: err == nil, + err: err, + } close(chDone) }() // Check for any initial errors from the restore, before any chunks are fed. select { - case err := <-chDone: - if err == nil { - err = errors.New("restore ended unexpectedly") - } + case done := <-chDone: m.endLocked() - return err + if done.err != nil { + return done.err + } + return errors.New("restore ended unexpectedly") case <-time.After(20 * time.Millisecond): } @@ -202,12 +212,12 @@ func (m *Manager) RestoreChunk(chunk []byte) (bool, error) { // Check if any errors have occurred yet. select { - case err := <-m.chRestoreDone: - if err == nil { - err = errors.New("restore ended unexpectedly") - } + case done := <-m.chRestoreDone: m.endLocked() - return false, err + if done.err != nil { + return false, done.err + } + return false, errors.New("restore ended unexpectedly") default: } @@ -226,9 +236,15 @@ func (m *Manager) RestoreChunk(chunk []byte) (bool, error) { if int(m.restoreChunkIndex) >= len(m.restoreChunkHashes) { close(m.chRestore) m.chRestore = nil - err := <-m.chRestoreDone + done := <-m.chRestoreDone m.endLocked() - return true, err + if done.err != nil { + return false, done.err + } + if !done.complete { + return false, errors.New("restore ended prematurely") + } + return true, nil } return false, nil } diff --git a/snapshots/manager_test.go b/snapshots/manager_test.go index 16e010151664..c4f41b0e3e00 100644 --- a/snapshots/manager_test.go +++ b/snapshots/manager_test.go @@ -145,7 +145,7 @@ func TestManager_Restore(t *testing.T) { Metadata: types.Metadata{ChunkHashes: checksums(chunks)}, }) require.Error(t, err) - require.Equal(t, err, types.ErrUnknownFormat) + require.Equal(t, types.ErrUnknownFormat, err) // Restore errors on no chunks err = manager.Restore(types.Snapshot{Height: 3, Format: 1, Hash: []byte{1, 2, 3}}) From 659fcf5a3670d79c10d149e166ed9336ffa5844c Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Mon, 31 Aug 2020 21:17:13 +0200 Subject: [PATCH 13/19] add a ready channel to Snapshotter.Restore() --- server/mock/store.go | 4 +++- snapshots/helpers_test.go | 11 +++++++++-- snapshots/manager.go | 6 +++--- snapshots/types/snapshotter.go | 4 +++- store/rootmulti/store.go | 10 +++++++++- store/rootmulti/store_test.go | 8 +++++--- 6 files changed, 32 insertions(+), 11 deletions(-) diff --git a/server/mock/store.go b/server/mock/store.go index bceb73a92c39..460c2a677f43 100644 --- a/server/mock/store.go +++ b/server/mock/store.go @@ -103,7 +103,9 @@ func (ms multiStore) Snapshot(height uint64, format uint32) (<-chan io.ReadClose panic("not implemented") } -func (ms multiStore) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser) error { +func (ms multiStore) Restore( + height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}, +) error { panic("not implemented") } diff --git a/snapshots/helpers_test.go b/snapshots/helpers_test.go index eede8b2d5857..bda3c315545c 100644 --- a/snapshots/helpers_test.go +++ b/snapshots/helpers_test.go @@ -63,13 +63,18 @@ type mockSnapshotter struct { chunks [][]byte } -func (m *mockSnapshotter) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser) error { +func (m *mockSnapshotter) Restore( + height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}, +) error { if format == 0 { return types.ErrUnknownFormat } if m.chunks != nil { return errors.New("already has contents") } + if ready != nil { + close(ready) + } m.chunks = [][]byte{} for reader := range chunks { @@ -140,6 +145,8 @@ func (m *hungSnapshotter) Snapshot(height uint64, format uint32) (<-chan io.Read return ch, nil } -func (m *hungSnapshotter) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser) error { +func (m *hungSnapshotter) Restore( + height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}, +) error { panic("not implemented") } diff --git a/snapshots/manager.go b/snapshots/manager.go index 3ccf502b3b21..1341bcf958ac 100644 --- a/snapshots/manager.go +++ b/snapshots/manager.go @@ -8,7 +8,6 @@ import ( "io" "io/ioutil" "sync" - "time" "github.com/cosmos/cosmos-sdk/snapshots/types" ) @@ -169,9 +168,10 @@ func (m *Manager) Restore(snapshot types.Snapshot) error { // Start an asynchronous snapshot restoration, passing chunks and completion status via channels. chChunks := make(chan io.ReadCloser, chunkBufferSize) + chReady := make(chan struct{}, 1) chDone := make(chan restoreDone, 1) go func() { - err := m.target.Restore(snapshot.Height, snapshot.Format, chChunks) + err := m.target.Restore(snapshot.Height, snapshot.Format, chChunks, chReady) chDone <- restoreDone{ complete: err == nil, err: err, @@ -187,7 +187,7 @@ func (m *Manager) Restore(snapshot types.Snapshot) error { return done.err } return errors.New("restore ended unexpectedly") - case <-time.After(20 * time.Millisecond): + case <-chReady: } m.chRestore = chChunks diff --git a/snapshots/types/snapshotter.go b/snapshots/types/snapshotter.go index d59a2a0874bb..1ebd763b5d75 100644 --- a/snapshots/types/snapshotter.go +++ b/snapshots/types/snapshotter.go @@ -10,5 +10,7 @@ type Snapshotter interface { Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) // Restore restores a state snapshot, taking snapshot chunk readers as input. - Restore(height uint64, format uint32, chunks <-chan io.ReadCloser) error + // If the ready channel is non-nil, it returns a ready signal (by being closed) once the + // restorer is ready to accept chunks. + Restore(height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}) error } diff --git a/store/rootmulti/store.go b/store/rootmulti/store.go index 9cc4b3f3d634..4e55d531362f 100644 --- a/store/rootmulti/store.go +++ b/store/rootmulti/store.go @@ -658,7 +658,9 @@ func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, e } // Restore implements snapshottypes.Snapshotter. -func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser) error { +func (rs *Store) Restore( + height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}, +) error { if format != snapshottypes.CurrentFormat { return fmt.Errorf("%w %v", snapshottypes.ErrUnknownFormat, format) } @@ -670,6 +672,12 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos height, math.MaxInt64) } + // Signal readiness. Must be done before the readers below are set up, since the zlib + // reader reads from the stream on initialization, potentially causing deadlocks. + if ready != nil { + close(ready) + } + // Set up a restore stream pipeline // chan io.ReadCloser -> chunkReader -> zlib -> delimited Protobuf -> ExportNode chunkReader := snapshots.NewChunkReader(chunks) diff --git a/store/rootmulti/store_test.go b/store/rootmulti/store_test.go index 55af45cc3aeb..06509b0cacaf 100644 --- a/store/rootmulti/store_test.go +++ b/store/rootmulti/store_test.go @@ -593,7 +593,7 @@ func TestMultistoreRestore_Errors(t *testing.T) { for name, tc := range testcases { tc := tc t.Run(name, func(t *testing.T) { - err := store.Restore(tc.height, tc.format, nil) + err := store.Restore(tc.height, tc.format, nil, nil) require.Error(t, err) if tc.expectType != nil { assert.True(t, errors.Is(err, tc.expectType)) @@ -610,8 +610,10 @@ func TestMultistoreSnapshotRestore(t *testing.T) { chunks, err := source.Snapshot(version, snapshottypes.CurrentFormat) require.NoError(t, err) - err = target.Restore(version, snapshottypes.CurrentFormat, chunks) + ready := make(chan struct{}) + err = target.Restore(version, snapshottypes.CurrentFormat, chunks, ready) require.NoError(t, err) + assert.EqualValues(t, struct{}{}, <-ready) assert.Equal(t, source.LastCommitID(), target.LastCommitID()) for key, sourceStore := range source.stores { @@ -687,7 +689,7 @@ func benchmarkMultistoreSnapshotRestore(b *testing.B, stores uint8, storeKeys ui chunks, err := source.Snapshot(version, snapshottypes.CurrentFormat) require.NoError(b, err) - err = target.Restore(version, snapshottypes.CurrentFormat, chunks) + err = target.Restore(version, snapshottypes.CurrentFormat, chunks, nil) require.NoError(b, err) require.Equal(b, source.LastCommitID(), target.LastCommitID()) } From 6a80f57467274ae8ced09ac740283f615fbdaad4 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Tue, 1 Sep 2020 14:43:19 +0200 Subject: [PATCH 14/19] add comment on streaming IO API --- snapshots/manager.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/snapshots/manager.go b/snapshots/manager.go index 1341bcf958ac..003d6354fc31 100644 --- a/snapshots/manager.go +++ b/snapshots/manager.go @@ -33,6 +33,16 @@ type restoreDone struct { // Manager manages snapshot and restore operations for an app, making sure only a single // long-running operation is in progress at any given time, and provides convenience methods // mirroring the ABCI interface. +// +// Although the ABCI interface (and this manager) passes chunks as byte slices, the internal +// snapshot/restore APIs use IO streams (i.e. chan io.ReadCloser), for two reasons: +// +// 1) In the future, ABCI should support streaming. Consider e.g. InitChain during chain +// upgrades, which currently passes the entire chain state as an in-memory byte slice. +// https://github.com/tendermint/tendermint/issues/5184 +// +// 2) io.ReadCloser streams automatically propagate IO errors, and can pass arbitrary +// errors via io.Pipe.CloseWithError(). type Manager struct { store *Store target types.Snapshotter From d618638f2cb75b98039502d1fc065c75c855b95e Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Tue, 1 Sep 2020 15:37:39 +0200 Subject: [PATCH 15/19] use sdkerrors for error handling --- snapshots/manager.go | 33 +++++++++--------- snapshots/store.go | 70 ++++++++++++++++++++------------------ snapshots/types/convert.go | 7 ++-- snapshots/util.go | 5 +-- store/rootmulti/store.go | 40 +++++++++++----------- types/errors/errors.go | 8 +++++ 6 files changed, 87 insertions(+), 76 deletions(-) diff --git a/snapshots/manager.go b/snapshots/manager.go index 003d6354fc31..3cb96e65f42f 100644 --- a/snapshots/manager.go +++ b/snapshots/manager.go @@ -3,13 +3,12 @@ package snapshots import ( "bytes" "crypto/sha256" - "errors" - "fmt" "io" "io/ioutil" "sync" "github.com/cosmos/cosmos-sdk/snapshots/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" ) const ( @@ -73,10 +72,10 @@ func (m *Manager) begin(op operation) error { // beginLocked begins an operation while already holding the mutex. func (m *Manager) beginLocked(op operation) error { if op == opNone { - return errors.New("can't begin a none operation") + return sdkerrors.Wrap(sdkerrors.ErrLogic, "can't begin a none operation") } if m.operation != opNone { - return fmt.Errorf("a %v operation is in progress", m.operation) + return sdkerrors.Wrapf(sdkerrors.ErrConflict, "a %v operation is in progress", m.operation) } m.operation = op return nil @@ -104,7 +103,7 @@ func (m *Manager) endLocked() { // Create creates a snapshot and returns its metadata. func (m *Manager) Create(height uint64) (*types.Snapshot, error) { if m == nil { - return nil, errors.New("no snapshot store configured") + return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "no snapshot store configured") } err := m.begin(opSnapshot) if err != nil { @@ -114,10 +113,11 @@ func (m *Manager) Create(height uint64) (*types.Snapshot, error) { latest, err := m.store.GetLatest() if err != nil { - return nil, fmt.Errorf("failed to examine latest snapshot: %w", err) + return nil, sdkerrors.Wrap(err, "failed to examine latest snapshot") } if latest != nil && latest.Height >= height { - return nil, fmt.Errorf("a more recent snapshot already exists at height %v", latest.Height) + return nil, sdkerrors.Wrapf(sdkerrors.ErrConflict, + "a more recent snapshot already exists at height %v", latest.Height) } chunks, err := m.target.Snapshot(height, types.CurrentFormat) @@ -161,11 +161,10 @@ func (m *Manager) Prune(retain uint32) (uint64, error) { // via RestoreChunk() until the restore is complete or a chunk fails. func (m *Manager) Restore(snapshot types.Snapshot) error { if snapshot.Chunks == 0 { - return fmt.Errorf("%w: no chunks", types.ErrInvalidMetadata) + return sdkerrors.Wrap(types.ErrInvalidMetadata, "no chunks") } if uint32(len(snapshot.Metadata.ChunkHashes)) != snapshot.Chunks { - return fmt.Errorf("%w: snapshot has %v chunk hashes, but %v chunks", - types.ErrInvalidMetadata, + return sdkerrors.Wrapf(types.ErrInvalidMetadata, "snapshot has %v chunk hashes, but %v chunks", uint32(len(snapshot.Metadata.ChunkHashes)), snapshot.Chunks) } @@ -196,7 +195,7 @@ func (m *Manager) Restore(snapshot types.Snapshot) error { if done.err != nil { return done.err } - return errors.New("restore ended unexpectedly") + return sdkerrors.Wrap(sdkerrors.ErrLogic, "restore ended unexpectedly") case <-chReady: } @@ -213,11 +212,11 @@ func (m *Manager) RestoreChunk(chunk []byte) (bool, error) { m.mtx.Lock() defer m.mtx.Unlock() if m.operation != opRestore { - return false, fmt.Errorf("no restore operation in progress") + return false, sdkerrors.Wrap(sdkerrors.ErrLogic, "no restore operation in progress") } if int(m.restoreChunkIndex) >= len(m.restoreChunkHashes) { - return false, errors.New("received unexpected chunk") + return false, sdkerrors.Wrap(sdkerrors.ErrLogic, "received unexpected chunk") } // Check if any errors have occurred yet. @@ -227,7 +226,7 @@ func (m *Manager) RestoreChunk(chunk []byte) (bool, error) { if done.err != nil { return false, done.err } - return false, errors.New("restore ended unexpectedly") + return false, sdkerrors.Wrap(sdkerrors.ErrLogic, "restore ended unexpectedly") default: } @@ -235,8 +234,8 @@ func (m *Manager) RestoreChunk(chunk []byte) (bool, error) { hash := sha256.Sum256(chunk) expected := m.restoreChunkHashes[m.restoreChunkIndex] if !bytes.Equal(hash[:], expected) { - return false, fmt.Errorf("%w (expected %x, got %x)", - types.ErrChunkHashMismatch, hash, expected) + return false, sdkerrors.Wrapf(types.ErrChunkHashMismatch, + "expected %x, got %x", hash, expected) } // Pass the chunk to the restore, and wait for completion if it was the final one. @@ -252,7 +251,7 @@ func (m *Manager) RestoreChunk(chunk []byte) (bool, error) { return false, done.err } if !done.complete { - return false, errors.New("restore ended prematurely") + return false, sdkerrors.Wrap(sdkerrors.ErrLogic, "restore ended prematurely") } return true, nil } diff --git a/snapshots/store.go b/snapshots/store.go index 177478684acb..687653a220fd 100644 --- a/snapshots/store.go +++ b/snapshots/store.go @@ -3,8 +3,6 @@ package snapshots import ( "crypto/sha256" "encoding/binary" - "errors" - "fmt" "io" "math" "os" @@ -16,6 +14,7 @@ import ( db "github.com/tendermint/tm-db" "github.com/cosmos/cosmos-sdk/snapshots/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" ) const ( @@ -35,11 +34,11 @@ type Store struct { // NewStore creates a new snapshot store. func NewStore(db db.DB, dir string) (*Store, error) { if dir == "" { - return nil, errors.New("snapshot directory not given") + return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "snapshot directory not given") } err := os.MkdirAll(dir, 0755) if err != nil { - return nil, fmt.Errorf("failed to create snapshot directory %q: %w", dir, err) + return nil, sdkerrors.Wrapf(err, "failed to create snapshot directory %q", dir) } return &Store{ @@ -55,17 +54,18 @@ func (s *Store) Delete(height uint64, format uint32) error { saving := s.saving[height] s.mtx.Unlock() if saving { - return fmt.Errorf("snapshot for height %v format %v is currently being saved", height, format) + return sdkerrors.Wrapf(sdkerrors.ErrConflict, + "snapshot for height %v format %v is currently being saved", height, format) } err := s.db.DeleteSync(encodeKey(height, format)) if err != nil { - return fmt.Errorf("failed to delete snapshot for height %v format %v: %w", - height, format, err) + return sdkerrors.Wrapf(err, "failed to delete snapshot for height %v format %v", + height, format) } err = os.RemoveAll(s.pathSnapshot(height, format)) if err != nil { - return fmt.Errorf("failed to delete snapshot chunks for height %v format %v: %w", - height, format, err) + return sdkerrors.Wrapf(err, "failed to delete snapshot chunks for height %v format %v", + height, format) } return nil } @@ -74,8 +74,8 @@ func (s *Store) Delete(height uint64, format uint32) error { func (s *Store) Get(height uint64, format uint32) (*types.Snapshot, error) { bytes, err := s.db.Get(encodeKey(height, format)) if err != nil { - return nil, fmt.Errorf("failed to fetch snapshot metadata for height %v format %v: %w", - height, format, err) + return nil, sdkerrors.Wrapf(err, "failed to fetch snapshot metadata for height %v format %v", + height, format) } if bytes == nil { return nil, nil @@ -83,8 +83,8 @@ func (s *Store) Get(height uint64, format uint32) (*types.Snapshot, error) { snapshot := &types.Snapshot{} err = proto.Unmarshal(bytes, snapshot) if err != nil { - return nil, fmt.Errorf("failed to decode snapshot metadata for height %v format %v: %w", - height, format, err) + return nil, sdkerrors.Wrapf(err, "failed to decode snapshot metadata for height %v format %v", + height, format) } if snapshot.Metadata.ChunkHashes == nil { snapshot.Metadata.ChunkHashes = [][]byte{} @@ -96,7 +96,7 @@ func (s *Store) Get(height uint64, format uint32) (*types.Snapshot, error) { func (s *Store) GetLatest() (*types.Snapshot, error) { iter, err := s.db.ReverseIterator(encodeKey(0, 0), encodeKey(math.MaxUint64, math.MaxUint32)) if err != nil { - return nil, fmt.Errorf("failed to find latest snapshot: %w", err) + return nil, sdkerrors.Wrap(err, "failed to find latest snapshot") } defer iter.Close() @@ -105,12 +105,12 @@ func (s *Store) GetLatest() (*types.Snapshot, error) { snapshot = &types.Snapshot{} err := proto.Unmarshal(iter.Value(), snapshot) if err != nil { - return nil, fmt.Errorf("failed to decode latest snapshot: %w", err) + return nil, sdkerrors.Wrap(err, "failed to decode latest snapshot") } } err = iter.Error() if err != nil { - return nil, fmt.Errorf("failed to find latest snapshot: %w", err) + return nil, sdkerrors.Wrap(err, "failed to find latest snapshot") } return snapshot, nil } @@ -119,7 +119,7 @@ func (s *Store) GetLatest() (*types.Snapshot, error) { func (s *Store) List() ([]*types.Snapshot, error) { iter, err := s.db.ReverseIterator(encodeKey(0, 0), encodeKey(math.MaxUint64, math.MaxUint32)) if err != nil { - return nil, fmt.Errorf("failed to list snapshots: %w", err) + return nil, sdkerrors.Wrap(err, "failed to list snapshots") } defer iter.Close() @@ -128,7 +128,7 @@ func (s *Store) List() ([]*types.Snapshot, error) { snapshot := &types.Snapshot{} err := proto.Unmarshal(iter.Value(), snapshot) if err != nil { - return nil, fmt.Errorf("failed to decode snapshot info: %w", err) + return nil, sdkerrors.Wrap(err, "failed to decode snapshot info") } snapshots = append(snapshots, snapshot) } @@ -200,7 +200,7 @@ func (s *Store) loadChunkFile(height uint64, format uint32, chunk uint32) (io.Re func (s *Store) Prune(retain uint32) (uint64, error) { iter, err := s.db.ReverseIterator(encodeKey(0, 0), encodeKey(math.MaxUint64, math.MaxUint32)) if err != nil { - return 0, fmt.Errorf("failed to prune snapshots: %w", err) + return 0, sdkerrors.Wrap(err, "failed to prune snapshots") } defer iter.Close() @@ -210,7 +210,7 @@ func (s *Store) Prune(retain uint32) (uint64, error) { for ; iter.Valid(); iter.Next() { height, format, err := decodeKey(iter.Key()) if err != nil { - return 0, fmt.Errorf("failed to prune snapshots: %w", err) + return 0, sdkerrors.Wrap(err, "failed to prune snapshots") } if skip[height] || uint32(len(skip)) < retain { skip[height] = true @@ -218,7 +218,7 @@ func (s *Store) Prune(retain uint32) (uint64, error) { } err = s.Delete(height, format) if err != nil { - return 0, fmt.Errorf("failed to prune snapshots: %w", err) + return 0, sdkerrors.Wrap(err, "failed to prune snapshots") } pruned++ prunedHeights[height] = true @@ -229,7 +229,7 @@ func (s *Store) Prune(retain uint32) (uint64, error) { if ok { err = os.Remove(s.pathHeight(height)) if err != nil { - return 0, fmt.Errorf("failed to remove snapshot directory for height %v", height) + return 0, sdkerrors.Wrapf(err, "failed to remove snapshot directory for height %v", height) } } } @@ -246,7 +246,7 @@ func (s *Store) Save( ) (*types.Snapshot, error) { defer DrainChunks(chunks) if height == 0 { - return nil, errors.New("snapshot height cannot be 0") + return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "snapshot height cannot be 0") } s.mtx.Lock() @@ -254,7 +254,8 @@ func (s *Store) Save( s.saving[height] = true s.mtx.Unlock() if saving { - return nil, fmt.Errorf("a snapshot for height %v is already being saved", height) + return nil, sdkerrors.Wrapf(sdkerrors.ErrConflict, + "a snapshot for height %v is already being saved", height) } defer func() { s.mtx.Lock() @@ -267,7 +268,8 @@ func (s *Store) Save( return nil, err } if exists { - return nil, fmt.Errorf("snapshot already exists for height %v format %v", height, format) + return nil, sdkerrors.Wrapf(sdkerrors.ErrConflict, + "snapshot already exists for height %v format %v", height, format) } snapshot := &types.Snapshot{ @@ -282,26 +284,26 @@ func (s *Store) Save( dir := s.pathSnapshot(height, format) err = os.MkdirAll(dir, 0755) if err != nil { - return nil, fmt.Errorf("failed to create snapshot directory %q: %w", dir, err) + return nil, sdkerrors.Wrapf(err, "failed to create snapshot directory %q", dir) } path := s.pathChunk(height, format, index) file, err := os.Create(path) if err != nil { - return nil, fmt.Errorf("failed to create snapshot chunk file %q: %w", path, err) + return nil, sdkerrors.Wrapf(err, "failed to create snapshot chunk file %q", path) } defer file.Close() // nolint: staticcheck chunkHasher.Reset() _, err = io.Copy(io.MultiWriter(file, chunkHasher, snapshotHasher), chunkBody) if err != nil { - return nil, fmt.Errorf("failed to generate snapshot chunk %v: %w", index, err) + return nil, sdkerrors.Wrapf(err, "failed to generate snapshot chunk %v", index) } err = file.Close() if err != nil { - return nil, fmt.Errorf("failed to close snapshot chunk %v: %w", index, err) + return nil, sdkerrors.Wrapf(err, "failed to close snapshot chunk %v", index) } err = chunkBody.Close() if err != nil { - return nil, fmt.Errorf("failed to close snapshot chunk %v: %w", index, err) + return nil, sdkerrors.Wrapf(err, "failed to close snapshot chunk %v", index) } snapshot.Metadata.ChunkHashes = append(snapshot.Metadata.ChunkHashes, chunkHasher.Sum(nil)) index++ @@ -319,11 +321,11 @@ func (s *Store) Save( func (s *Store) saveSnapshot(snapshot *types.Snapshot) error { value, err := proto.Marshal(snapshot) if err != nil { - return fmt.Errorf("failed to encode snapshot metadata: %w", err) + return sdkerrors.Wrap(err, "failed to encode snapshot metadata") } err = s.db.SetSync(encodeKey(snapshot.Height, snapshot.Format), value) if err != nil { - return fmt.Errorf("failed to store snapshot: %w", err) + return sdkerrors.Wrap(err, "failed to store snapshot") } return nil } @@ -346,10 +348,10 @@ func (s *Store) pathChunk(height uint64, format uint32, chunk uint32) string { // decodeKey decodes a snapshot key. func decodeKey(k []byte) (uint64, uint32, error) { if len(k) != 13 { - return 0, 0, fmt.Errorf("invalid snapshot key with length %v", len(k)) + return 0, 0, sdkerrors.Wrapf(sdkerrors.ErrLogic, "invalid snapshot key with length %v", len(k)) } if k[0] != keyPrefixSnapshot { - return 0, 0, fmt.Errorf("invalid snapshot key prefix %x", k[0]) + return 0, 0, sdkerrors.Wrapf(sdkerrors.ErrLogic, "invalid snapshot key prefix %x", k[0]) } height := binary.BigEndian.Uint64(k[1:9]) format := binary.BigEndian.Uint32(k[9:13]) diff --git a/snapshots/types/convert.go b/snapshots/types/convert.go index 33e315ee032c..238e9623740e 100644 --- a/snapshots/types/convert.go +++ b/snapshots/types/convert.go @@ -1,8 +1,7 @@ package types import ( - fmt "fmt" - + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" proto "github.com/gogo/protobuf/proto" abci "github.com/tendermint/tendermint/abci/types" ) @@ -17,7 +16,7 @@ func SnapshotFromABCI(in *abci.Snapshot) (Snapshot, error) { } err := proto.Unmarshal(in.Metadata, &snapshot.Metadata) if err != nil { - return Snapshot{}, fmt.Errorf("failed to unmarshal snapshot metadata: %w", err) + return Snapshot{}, sdkerrors.Wrap(err, "failed to unmarshal snapshot metadata") } return snapshot, nil } @@ -33,7 +32,7 @@ func (s Snapshot) ToABCI() (abci.Snapshot, error) { var err error out.Metadata, err = proto.Marshal(&s.Metadata) if err != nil { - return abci.Snapshot{}, fmt.Errorf("failed to marshal snapshot metadata: %w", err) + return abci.Snapshot{}, sdkerrors.Wrap(err, "failed to marshal snapshot metadata") } return out, nil } diff --git a/snapshots/util.go b/snapshots/util.go index 19bc87c1aab6..674bd49d9278 100644 --- a/snapshots/util.go +++ b/snapshots/util.go @@ -1,8 +1,9 @@ package snapshots import ( - "errors" "io" + + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" ) // ChunkWriter reads an input stream, splits it into fixed-size chunks, and writes them to a @@ -66,7 +67,7 @@ func (w *ChunkWriter) CloseWithError(err error) { // Write implements io.Writer. func (w *ChunkWriter) Write(data []byte) (int, error) { if w.closed { - return 0, errors.New("cannot write to closed ChunkWriter") + return 0, sdkerrors.Wrap(sdkerrors.ErrLogic, "cannot write to closed ChunkWriter") } nTotal := 0 for len(data) > 0 { diff --git a/store/rootmulti/store.go b/store/rootmulti/store.go index 4e55d531362f..dee9065ceb90 100644 --- a/store/rootmulti/store.go +++ b/store/rootmulti/store.go @@ -544,13 +544,13 @@ func parsePath(path string) (storeName string, subpath string, err error) { // TestMultistoreSnapshot_Checksum test. func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { if format != snapshottypes.CurrentFormat { - return nil, fmt.Errorf("%w %v", snapshottypes.ErrUnknownFormat, format) + return nil, sdkerrors.Wrapf(snapshottypes.ErrUnknownFormat, "format %v", format) } if height == 0 { - return nil, errors.New("cannot snapshot height 0") + return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "cannot snapshot height 0") } if height > uint64(rs.LastCommitID().Version) { - return nil, fmt.Errorf("cannot snapshot future height %v", height) + return nil, sdkerrors.Wrapf(sdkerrors.ErrLogic, "cannot snapshot future height %v", height) } // Collect stores to snapshot (only IAVL stores are supported) @@ -567,7 +567,8 @@ func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, e // Non-persisted stores shouldn't be snapshotted continue default: - return nil, errors.Errorf("don't know how to snapshot store %q of type %T", key.Name(), store) + return nil, sdkerrors.Wrapf(sdkerrors.ErrLogic, + "don't know how to snapshot store %q of type %T", key.Name(), store) } } sort.Slice(stores, func(i, j int) bool { @@ -589,7 +590,7 @@ func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, e }() zWriter, err := zlib.NewWriterLevel(bufWriter, 7) if err != nil { - chunkWriter.CloseWithError(fmt.Errorf("zlib error: %w", err)) + chunkWriter.CloseWithError(sdkerrors.Wrap(err, "zlib failure")) return } defer func() { @@ -662,14 +663,14 @@ func (rs *Store) Restore( height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}, ) error { if format != snapshottypes.CurrentFormat { - return fmt.Errorf("%w %v", snapshottypes.ErrUnknownFormat, format) + return sdkerrors.Wrapf(snapshottypes.ErrUnknownFormat, "format %v", format) } if height == 0 { - return fmt.Errorf("%w: cannot restore snapshot at height 0", snapshottypes.ErrInvalidMetadata) + return sdkerrors.Wrap(sdkerrors.ErrLogic, "cannot restore snapshot at height 0") } if height > math.MaxInt64 { - return fmt.Errorf("%w: snapshot height %v cannot exceed %v", snapshottypes.ErrInvalidMetadata, - height, math.MaxInt64) + return sdkerrors.Wrapf(sdkerrors.ErrLogic, "snapshot height %v cannot exceed %v", + snapshottypes.ErrInvalidMetadata, height, math.MaxInt64) } // Signal readiness. Must be done before the readers below are set up, since the zlib @@ -684,7 +685,7 @@ func (rs *Store) Restore( defer chunkReader.Close() zReader, err := zlib.NewReader(chunkReader) if err != nil { - return fmt.Errorf("zlib error: %w", err) + return sdkerrors.Wrap(err, "zlib failure") } defer zReader.Close() protoReader := protoio.NewDelimitedReader(zReader, snapshotMaxItemSize) @@ -700,7 +701,7 @@ func (rs *Store) Restore( if err == io.EOF { break } else if err != nil { - return fmt.Errorf("invalid protobuf message: %w", err) + return sdkerrors.Wrap(err, "invalid protobuf message") } switch item := item.Item.(type) { @@ -708,26 +709,27 @@ func (rs *Store) Restore( if importer != nil { err = importer.Commit() if err != nil { - return fmt.Errorf("IAVL commit failed: %w", err) + return sdkerrors.Wrap(err, "IAVL commit failed") } importer.Close() } store, ok := rs.getStoreByName(item.Store.Name).(*iavl.Store) if !ok || store == nil { - return fmt.Errorf("cannot import into non-IAVL store %q", item.Store.Name) + return sdkerrors.Wrapf(sdkerrors.ErrLogic, "cannot import into non-IAVL store %q", item.Store.Name) } importer, err = store.Import(int64(height)) if err != nil { - return fmt.Errorf("import failed: %w", err) + return sdkerrors.Wrap(err, "import failed") } defer importer.Close() case *types.SnapshotItem_IAVL: if importer == nil { - return fmt.Errorf("received IAVL node item before store item") + return sdkerrors.Wrap(sdkerrors.ErrLogic, "received IAVL node item before store item") } if item.IAVL.Height > math.MaxInt8 { - return fmt.Errorf("node height %v cannot exceed %v", item.IAVL.Height, math.MaxInt8) + return sdkerrors.Wrapf(sdkerrors.ErrLogic, "node height %v cannot exceed %v", + item.IAVL.Height, math.MaxInt8) } node := &iavltree.ExportNode{ Key: item.IAVL.Key, @@ -745,18 +747,18 @@ func (rs *Store) Restore( } err := importer.Add(node) if err != nil { - return fmt.Errorf("IAVL node import failed: %w", err) + return sdkerrors.Wrap(err, "IAVL node import failed") } default: - return fmt.Errorf("unknown snapshot item %T", item) + return sdkerrors.Wrapf(sdkerrors.ErrLogic, "unknown snapshot item %T", item) } } if importer != nil { err := importer.Commit() if err != nil { - return fmt.Errorf("IAVL commit failed: %w", err) + return sdkerrors.Wrap(err, "IAVL commit failed") } importer.Close() } diff --git a/types/errors/errors.go b/types/errors/errors.go index ed0f62501084..1b103684c908 100644 --- a/types/errors/errors.go +++ b/types/errors/errors.go @@ -117,6 +117,14 @@ var ( // the signer info doesn't match the account's actual sequence number. ErrWrongSequence = Register(RootCodespace, 32, "incorrect account sequence") + // ErrLogic defines an internal logic error, e.g. an invariant or assertion + // that is violated. It is a programmer error, not a user-facing error. + ErrLogic = Register(RootCodespace, 33, "internal logic error") + + // ErrConflict defines a conflict error, e.g. when two goroutines try to access + // the same resource and one of them fails. + ErrConflict = Register(RootCodespace, 34, "conflict error") + // ErrPanic is only set when we recover from a panic, so we know to // redact potentially sensitive system info ErrPanic = Register(UndefinedCodespace, 111222, "panic") From 93a2089875dc055ba1bfc362c79587fe8ecf722d Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Wed, 2 Sep 2020 01:05:55 +0200 Subject: [PATCH 16/19] fix incorrect error --- store/rootmulti/store.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/store/rootmulti/store.go b/store/rootmulti/store.go index dee9065ceb90..fd6073529e01 100644 --- a/store/rootmulti/store.go +++ b/store/rootmulti/store.go @@ -669,8 +669,8 @@ func (rs *Store) Restore( return sdkerrors.Wrap(sdkerrors.ErrLogic, "cannot restore snapshot at height 0") } if height > math.MaxInt64 { - return sdkerrors.Wrapf(sdkerrors.ErrLogic, "snapshot height %v cannot exceed %v", - snapshottypes.ErrInvalidMetadata, height, math.MaxInt64) + return sdkerrors.Wrapf(snapshottypes.ErrInvalidMetadata, + "snapshot height %v cannot exceed %v", height, math.MaxInt64) } // Signal readiness. Must be done before the readers below are set up, since the zlib From 9e119b81835277251e09444b5aba50c09e0b6752 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Tue, 8 Sep 2020 10:36:48 +0200 Subject: [PATCH 17/19] tweak changelog --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index abd5090a100f..11601034c927 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -167,7 +167,7 @@ be used to retrieve the actual proposal `Content`. Also the `NewMsgSubmitProposa * (crypto/multisig) [\#6241](https://github.com/cosmos/cosmos-sdk/pull/6241) Add Multisig type directly to the repo. Previously this was in tendermint. * (rest) [\#6167](https://github.com/cosmos/cosmos-sdk/pull/6167) Support `max-body-bytes` CLI flag for the REST service. * (x/ibc) [\#5588](https://github.com/cosmos/cosmos-sdk/pull/5588) Add [ICS 024 - Host State Machine Requirements](https://github.com/cosmos/ics/tree/master/spec/ics-024-host-requirements) subpackage to `x/ibc` module. -* (baseapp) [\#5803](https://github.com/cosmos/cosmos-sdk/pull/5803) Added support for taking state snapshots at regular height intervals, via options `snapshot-interval` and `snapshot-retention`. +* (baseapp) [\#5803](https://github.com/cosmos/cosmos-sdk/pull/5803) Added support for taking state snapshots at regular height intervals, via options `snapshot-interval` and `snapshot-keep-recent`. * (store) [\#5803](https://github.com/cosmos/cosmos-sdk/pull/5803) Added `rootmulti.Store` methods for taking and restoring snapshots, based on `iavl.Store` export/import. * (x/ibc) [\#5277](https://github.com/cosmos/cosmos-sdk/pull/5277) `x/ibc` changes from IBC alpha. For more details check the the [`x/ibc/spec`](https://github.com/cosmos/tree/master/x/ibc/spec) directory: * [ICS 002 - Client Semantics](https://github.com/cosmos/ics/tree/master/spec/ics-002-client-semantics) subpackage From b9c97fa4b58a0395a59fb3eda9d718fc6d6852d0 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Tue, 8 Sep 2020 10:46:35 +0200 Subject: [PATCH 18/19] syntax fix --- server/mock/store.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/mock/store.go b/server/mock/store.go index 99cc45ebfe07..9e0e05e46722 100644 --- a/server/mock/store.go +++ b/server/mock/store.go @@ -110,6 +110,8 @@ func (ms multiStore) Snapshot(height uint64, format uint32) (<-chan io.ReadClose func (ms multiStore) Restore( height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}, ) error { + panic("not implemented") +} var _ sdk.KVStore = kvStore{} From 64a67e672ff811d569184caca297cc536a99b7be Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Tue, 8 Sep 2020 10:58:14 +0200 Subject: [PATCH 19/19] update test code after merge --- baseapp/baseapp_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/baseapp/baseapp_test.go b/baseapp/baseapp_test.go index 829e7352b04b..efe2b1528bb8 100644 --- a/baseapp/baseapp_test.go +++ b/baseapp/baseapp_test.go @@ -113,7 +113,7 @@ func setupBaseApp(t *testing.T, options ...func(*BaseApp)) *BaseApp { // simple one store baseapp with data and snapshots. Each tx is 1 MB in size (uncompressed). func setupBaseAppWithSnapshots(t *testing.T, blocks uint, blockTxs int, options ...func(*BaseApp)) (*BaseApp, func()) { - codec := codec.New() + codec := codec.NewLegacyAmino() registerTestCodec(codec) routerOpt := func(bapp *BaseApp) { bapp.Router().AddRoute(sdk.NewRoute(routeMsgKeyValue, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) {