store: Remove Amino (#6984)

* Update kv pair to proto

* updates

* fix LastCommitID

* lint++

Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
This commit is contained in:
Alexander Bezobchuk 2020-08-11 06:09:16 -04:00 committed by GitHub
parent 69cd552260
commit 0f44d1af23
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
33 changed files with 1442 additions and 391 deletions

View File

@ -1,9 +1,16 @@
syntax = "proto3"; syntax = "proto3";
package cosmos.kv; package cosmos.kv;
import "gogoproto/gogo.proto";
option go_package = "github.com/cosmos/cosmos-sdk/types/kv"; option go_package = "github.com/cosmos/cosmos-sdk/types/kv";
// Key-Value Pair // Pairs defines a repeated slice of Pair objects.
message Pairs {
repeated Pair pairs = 1 [(gogoproto.nullable) = false];
}
// Pair defines a key/value bytes tuple.
message Pair { message Pair {
bytes key = 1; bytes key = 1;
bytes value = 2; bytes value = 2;

View File

@ -0,0 +1,29 @@
syntax = "proto3";
package cosmos.store;
import "gogoproto/gogo.proto";
option go_package = "github.com/cosmos/cosmos-sdk/store/types";
// CommitInfo defines commit information used by the multi-store when committing
// a version/height.
message CommitInfo {
int64 version = 1;
repeated StoreInfo store_infos = 2 [(gogoproto.nullable) = false];
}
// StoreInfo defines store-specific commit information. It contains a reference
// between a store name and the commit ID.
message StoreInfo {
string name = 1;
CommitID commit_id = 2 [(gogoproto.nullable) = false, (gogoproto.customname) = "CommitID"];
}
// CommitID defines the committment information when a specific store is
// committed.
message CommitID {
option (gogoproto.goproto_stringer) = false;
int64 version = 1;
bytes hash = 2;
}

View File

@ -270,18 +270,25 @@ func (st *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) {
res.Proof = getProofFromTree(mtree, req.Data, res.Value != nil) res.Proof = getProofFromTree(mtree, req.Data, res.Value != nil)
case "/subspace": case "/subspace":
var KVs []types.KVPair pairs := kv.Pairs{
Pairs: make([]kv.Pair, 0),
}
subspace := req.Data subspace := req.Data
res.Key = subspace res.Key = subspace
iterator := types.KVStorePrefixIterator(st, subspace) iterator := types.KVStorePrefixIterator(st, subspace)
for ; iterator.Valid(); iterator.Next() { for ; iterator.Valid(); iterator.Next() {
KVs = append(KVs, types.KVPair{Key: iterator.Key(), Value: iterator.Value()}) pairs.Pairs = append(pairs.Pairs, kv.Pair{Key: iterator.Key(), Value: iterator.Value()})
}
iterator.Close()
bz, err := pairs.Marshal()
if err != nil {
panic(fmt.Errorf("failed to marshal KV pairs: %w", err))
} }
iterator.Close() res.Value = bz
res.Value = cdc.MustMarshalBinaryBare(KVs)
default: default:
return sdkerrors.QueryResult(sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "unexpected query path: %v", req.Path)) return sdkerrors.QueryResult(sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "unexpected query path: %v", req.Path))

View File

@ -11,6 +11,7 @@ import (
dbm "github.com/tendermint/tm-db" dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/store/types" "github.com/cosmos/cosmos-sdk/store/types"
"github.com/cosmos/cosmos-sdk/types/kv"
) )
var ( var (
@ -408,18 +409,28 @@ func TestIAVLStoreQuery(t *testing.T) {
v3 := []byte("val3") v3 := []byte("val3")
ksub := []byte("key") ksub := []byte("key")
KVs0 := []types.KVPair{} KVs0 := kv.Pairs{}
KVs1 := []types.KVPair{ KVs1 := kv.Pairs{
{Key: k1, Value: v1}, Pairs: []kv.Pair{
{Key: k2, Value: v2}, {Key: k1, Value: v1},
{Key: k2, Value: v2},
},
} }
KVs2 := []types.KVPair{ KVs2 := kv.Pairs{
{Key: k1, Value: v3}, Pairs: []kv.Pair{
{Key: k2, Value: v2}, {Key: k1, Value: v3},
{Key: k2, Value: v2},
},
} }
valExpSubEmpty := cdc.MustMarshalBinaryBare(KVs0)
valExpSub1 := cdc.MustMarshalBinaryBare(KVs1) valExpSubEmpty, err := KVs0.Marshal()
valExpSub2 := cdc.MustMarshalBinaryBare(KVs2) require.NoError(t, err)
valExpSub1, err := KVs1.Marshal()
require.NoError(t, err)
valExpSub2, err := KVs2.Marshal()
require.NoError(t, err)
cid := iavlStore.Commit() cid := iavlStore.Commit()
ver := cid.Version ver := cid.Version

View File

@ -1,7 +0,0 @@
package iavl
import (
"github.com/cosmos/cosmos-sdk/codec"
)
var cdc = codec.New()

View File

@ -6,7 +6,6 @@ import (
"github.com/tendermint/tendermint/crypto/merkle" "github.com/tendermint/tendermint/crypto/merkle"
"github.com/tendermint/tendermint/crypto/tmhash" "github.com/tendermint/tendermint/crypto/tmhash"
"github.com/cosmos/cosmos-sdk/store/types"
"github.com/cosmos/cosmos-sdk/types/kv" "github.com/cosmos/cosmos-sdk/types/kv"
) )
@ -19,7 +18,7 @@ type merkleMap struct {
func newMerkleMap() *merkleMap { func newMerkleMap() *merkleMap {
return &merkleMap{ return &merkleMap{
kvs: nil, kvs: kv.Pairs{},
sorted: false, sorted: false,
} }
} }
@ -29,7 +28,7 @@ func newMerkleMap() *merkleMap {
// of kv.Pairs. Whenever called, the MerkleMap must be resorted. // of kv.Pairs. Whenever called, the MerkleMap must be resorted.
func (sm *merkleMap) set(key string, value []byte) { func (sm *merkleMap) set(key string, value []byte) {
byteKey := []byte(key) byteKey := []byte(key)
types.AssertValidKey(byteKey) assertValidKey(byteKey)
sm.sorted = false sm.sorted = false
@ -37,7 +36,7 @@ func (sm *merkleMap) set(key string, value []byte) {
// and make a determination to fetch or not. // and make a determination to fetch or not.
vhash := tmhash.Sum(value) vhash := tmhash.Sum(value)
sm.kvs = append(sm.kvs, kv.Pair{ sm.kvs.Pairs = append(sm.kvs.Pairs, kv.Pair{
Key: byteKey, Key: byteKey,
Value: vhash, Value: vhash,
}) })
@ -61,8 +60,8 @@ func (sm *merkleMap) sort() {
// hashKVPairs hashes a kvPair and creates a merkle tree where the leaves are // hashKVPairs hashes a kvPair and creates a merkle tree where the leaves are
// byte slices. // byte slices.
func hashKVPairs(kvs kv.Pairs) []byte { func hashKVPairs(kvs kv.Pairs) []byte {
kvsH := make([][]byte, len(kvs)) kvsH := make([][]byte, len(kvs.Pairs))
for i, kvp := range kvs { for i, kvp := range kvs.Pairs {
kvsH[i] = KVPair(kvp).Bytes() kvsH[i] = KVPair(kvp).Bytes()
} }
@ -81,7 +80,7 @@ type simpleMap struct {
func newSimpleMap() *simpleMap { func newSimpleMap() *simpleMap {
return &simpleMap{ return &simpleMap{
Kvs: nil, Kvs: kv.Pairs{},
sorted: false, sorted: false,
} }
} }
@ -90,7 +89,7 @@ func newSimpleMap() *simpleMap {
// and then appends it to SimpleMap's kv pairs. // and then appends it to SimpleMap's kv pairs.
func (sm *simpleMap) Set(key string, value []byte) { func (sm *simpleMap) Set(key string, value []byte) {
byteKey := []byte(key) byteKey := []byte(key)
types.AssertValidKey(byteKey) assertValidKey(byteKey)
sm.sorted = false sm.sorted = false
// The value is hashed, so you can // The value is hashed, so you can
@ -98,7 +97,7 @@ func (sm *simpleMap) Set(key string, value []byte) {
// and make a determination to fetch or not. // and make a determination to fetch or not.
vhash := tmhash.Sum(value) vhash := tmhash.Sum(value)
sm.Kvs = append(sm.Kvs, kv.Pair{ sm.Kvs.Pairs = append(sm.Kvs.Pairs, kv.Pair{
Key: byteKey, Key: byteKey,
Value: vhash, Value: vhash,
}) })
@ -123,8 +122,11 @@ func (sm *simpleMap) Sort() {
// NOTE these contain the hashed key and value. // NOTE these contain the hashed key and value.
func (sm *simpleMap) KVPairs() kv.Pairs { func (sm *simpleMap) KVPairs() kv.Pairs {
sm.Sort() sm.Sort()
kvs := make(kv.Pairs, len(sm.Kvs)) kvs := kv.Pairs{
copy(kvs, sm.Kvs) Pairs: make([]kv.Pair, len(sm.Kvs.Pairs)),
}
copy(kvs.Pairs, sm.Kvs.Pairs)
return kvs return kvs
} }
@ -188,18 +190,25 @@ func SimpleProofsFromMap(m map[string][]byte) ([]byte, map[string]*merkle.Simple
sm.Sort() sm.Sort()
kvs := sm.Kvs kvs := sm.Kvs
kvsBytes := make([][]byte, len(kvs)) kvsBytes := make([][]byte, len(kvs.Pairs))
for i, kvp := range kvs { for i, kvp := range kvs.Pairs {
kvsBytes[i] = KVPair(kvp).Bytes() kvsBytes[i] = KVPair(kvp).Bytes()
} }
rootHash, proofList := merkle.SimpleProofsFromByteSlices(kvsBytes) rootHash, proofList := merkle.SimpleProofsFromByteSlices(kvsBytes)
proofs := make(map[string]*merkle.SimpleProof) proofs := make(map[string]*merkle.SimpleProof)
keys := make([]string, len(proofList)) keys := make([]string, len(proofList))
for i, kvp := range kvs {
for i, kvp := range kvs.Pairs {
proofs[string(kvp.Key)] = proofList[i] proofs[string(kvp.Key)] = proofList[i]
keys[i] = string(kvp.Key) keys[i] = string(kvp.Key)
} }
return rootHash, proofs, keys return rootHash, proofs, keys
} }
func assertValidKey(key []byte) {
if len(key) == 0 {
panic("key is nil")
}
}

View File

@ -6,7 +6,7 @@ import (
ics23 "github.com/confio/ics23/go" ics23 "github.com/confio/ics23/go"
sdkmaps "github.com/cosmos/cosmos-sdk/store/rootmulti/internal/maps" sdkmaps "github.com/cosmos/cosmos-sdk/store/internal/maps"
) )
// TendermintSpec constrains the format from ics23-tendermint (crypto/merkle SimpleProof) // TendermintSpec constrains the format from ics23-tendermint (crypto/merkle SimpleProof)

View File

@ -1,24 +1,21 @@
package rootmulti package rootmulti
import ( import (
"encoding/binary"
"fmt" "fmt"
"io" "io"
"strings" "strings"
ics23 "github.com/confio/ics23/go" gogotypes "github.com/gogo/protobuf/types"
"github.com/pkg/errors" "github.com/pkg/errors"
iavltree "github.com/tendermint/iavl" iavltree "github.com/tendermint/iavl"
abci "github.com/tendermint/tendermint/abci/types" abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/merkle"
dbm "github.com/tendermint/tm-db" dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/codec"
"github.com/cosmos/cosmos-sdk/store/cachemulti" "github.com/cosmos/cosmos-sdk/store/cachemulti"
"github.com/cosmos/cosmos-sdk/store/dbadapter" "github.com/cosmos/cosmos-sdk/store/dbadapter"
"github.com/cosmos/cosmos-sdk/store/iavl" "github.com/cosmos/cosmos-sdk/store/iavl"
"github.com/cosmos/cosmos-sdk/store/mem" "github.com/cosmos/cosmos-sdk/store/mem"
sdkmaps "github.com/cosmos/cosmos-sdk/store/rootmulti/internal/maps"
sdkproofs "github.com/cosmos/cosmos-sdk/store/rootmulti/internal/proofs"
"github.com/cosmos/cosmos-sdk/store/tracekv" "github.com/cosmos/cosmos-sdk/store/tracekv"
"github.com/cosmos/cosmos-sdk/store/transient" "github.com/cosmos/cosmos-sdk/store/transient"
"github.com/cosmos/cosmos-sdk/store/types" "github.com/cosmos/cosmos-sdk/store/types"
@ -31,14 +28,12 @@ const (
commitInfoKeyFmt = "s/%d" // s/<version> commitInfoKeyFmt = "s/%d" // s/<version>
) )
var cdc = codec.New()
// Store is composed of many CommitStores. Name contrasts with // Store is composed of many CommitStores. Name contrasts with
// cacheMultiStore which is for cache-wrapping other MultiStores. It implements // cacheMultiStore which is for cache-wrapping other MultiStores. It implements
// the CommitMultiStore interface. // the CommitMultiStore interface.
type Store struct { type Store struct {
db dbm.DB db dbm.DB
lastCommitInfo commitInfo lastCommitInfo *types.CommitInfo
pruningOpts types.PruningOptions pruningOpts types.PruningOptions
storesParams map[types.StoreKey]storeParams storesParams map[types.StoreKey]storeParams
stores map[types.StoreKey]types.CommitKVStore stores map[types.StoreKey]types.CommitKVStore
@ -152,8 +147,9 @@ func (rs *Store) LoadVersion(ver int64) error {
} }
func (rs *Store) loadVersion(ver int64, upgrades *types.StoreUpgrades) error { func (rs *Store) loadVersion(ver int64, upgrades *types.StoreUpgrades) error {
infos := make(map[string]storeInfo) infos := make(map[string]types.StoreInfo)
var cInfo commitInfo
cInfo := &types.CommitInfo{}
// load old data if we are not version 0 // load old data if we are not version 0
if ver != 0 { if ver != 0 {
@ -217,12 +213,13 @@ func (rs *Store) loadVersion(ver int64, upgrades *types.StoreUpgrades) error {
return nil return nil
} }
func (rs *Store) getCommitID(infos map[string]storeInfo, name string) types.CommitID { func (rs *Store) getCommitID(infos map[string]types.StoreInfo, name string) types.CommitID {
info, ok := infos[name] info, ok := infos[name]
if !ok { if !ok {
return types.CommitID{} return types.CommitID{}
} }
return info.Core.CommitID
return info.CommitID
} }
func deleteKVStore(kv types.KVStore) error { func deleteKVStore(kv types.KVStore) error {
@ -290,11 +287,12 @@ func (rs *Store) TracingEnabled() bool {
return rs.traceWriter != nil return rs.traceWriter != nil
} }
//----------------------------------------
// +CommitStore
// LastCommitID implements Committer/CommitStore. // LastCommitID implements Committer/CommitStore.
func (rs *Store) LastCommitID() types.CommitID { func (rs *Store) LastCommitID() types.CommitID {
if rs.lastCommitInfo == nil {
return types.CommitID{}
}
return rs.lastCommitInfo.CommitID() return rs.lastCommitInfo.CommitID()
} }
@ -365,9 +363,6 @@ func (rs *Store) CacheWrapWithTrace(_ io.Writer, _ types.TraceContext) types.Cac
return rs.CacheWrap() return rs.CacheWrap()
} }
//----------------------------------------
// +MultiStore
// CacheMultiStore cache-wraps the multi-store and returns a CacheMultiStore. // CacheMultiStore cache-wraps the multi-store and returns a CacheMultiStore.
// It implements the MultiStore interface. // It implements the MultiStore interface.
func (rs *Store) CacheMultiStore() types.CacheMultiStore { func (rs *Store) CacheMultiStore() types.CacheMultiStore {
@ -453,8 +448,6 @@ func (rs *Store) getStoreByName(name string) types.Store {
return rs.GetCommitKVStore(key) return rs.GetCommitKVStore(key)
} }
//---------------------- Query ------------------
// Query calls substore.Query with the same `req` where `req.Path` is // Query calls substore.Query with the same `req` where `req.Path` is
// modified to remove the substore prefix. // modified to remove the substore prefix.
// Ie. `req.Path` here is `/<substore>/<path>`, and trimmed to `/<path>` for the substore. // Ie. `req.Path` here is `/<substore>/<path>`, and trimmed to `/<path>` for the substore.
@ -491,7 +484,7 @@ func (rs *Store) Query(req abci.RequestQuery) abci.ResponseQuery {
// If the request's height is the latest height we've committed, then utilize // If the request's height is the latest height we've committed, then utilize
// the store's lastCommitInfo as this commit info may not be flushed to disk. // the store's lastCommitInfo as this commit info may not be flushed to disk.
// Otherwise, we query for the commit info from disk. // Otherwise, we query for the commit info from disk.
var commitInfo commitInfo var commitInfo *types.CommitInfo
if res.Height == rs.lastCommitInfo.Version { if res.Height == rs.lastCommitInfo.Version {
commitInfo = rs.lastCommitInfo commitInfo = rs.lastCommitInfo
@ -578,125 +571,32 @@ func (rs *Store) loadCommitStoreFromParams(key types.StoreKey, id types.CommitID
} }
} }
//----------------------------------------
// storeParams
type storeParams struct { type storeParams struct {
key types.StoreKey key types.StoreKey
db dbm.DB db dbm.DB
typ types.StoreType typ types.StoreType
} }
//----------------------------------------
// commitInfo
// NOTE: Keep commitInfo a simple immutable struct.
type commitInfo struct {
// Version
Version int64
// Store info for
StoreInfos []storeInfo
}
func (ci commitInfo) toMap() map[string][]byte {
m := make(map[string][]byte, len(ci.StoreInfos))
for _, storeInfo := range ci.StoreInfos {
m[storeInfo.Name] = storeInfo.GetHash()
}
return m
}
// Hash returns the simple merkle root hash of the stores sorted by name.
func (ci commitInfo) Hash() []byte {
// we need a special case for empty set, as SimpleProofsFromMap requires at least one entry
if len(ci.StoreInfos) == 0 {
return nil
}
rootHash, _, _ := sdkmaps.SimpleProofsFromMap(ci.toMap())
return rootHash
}
func (ci commitInfo) ProofOp(storeName string) merkle.ProofOp {
cmap := ci.toMap()
_, proofs, _ := sdkmaps.SimpleProofsFromMap(cmap)
proof := proofs[storeName]
if proof == nil {
panic(fmt.Sprintf("ProofOp for %s but not registered store name", storeName))
}
// convert merkle.SimpleProof to CommitmentProof
existProof, err := sdkproofs.ConvertExistenceProof(proof, []byte(storeName), cmap[storeName])
if err != nil {
panic(fmt.Errorf("could not convert simple proof to existence proof: %w", err))
}
commitmentProof := &ics23.CommitmentProof{
Proof: &ics23.CommitmentProof_Exist{
Exist: existProof,
},
}
return types.NewSimpleMerkleCommitmentOp([]byte(storeName), commitmentProof).ProofOp()
}
func (ci commitInfo) CommitID() types.CommitID {
return types.CommitID{
Version: ci.Version,
Hash: ci.Hash(),
}
}
//----------------------------------------
// storeInfo
// storeInfo contains the name and core reference for an
// underlying store. It is the leaf of the Stores top
// level simple merkle tree.
type storeInfo struct {
Name string
Core storeCore
}
type storeCore struct {
// StoreType StoreType
CommitID types.CommitID
// ... maybe add more state
}
// GetHash returns the GetHash from the CommitID.
// This is used in CommitInfo.Hash()
//
// When we commit to this in a merkle proof, we create a map of storeInfo.Name -> storeInfo.GetHash()
// and build a merkle proof from that.
// This is then chained with the substore proof, so we prove the root hash from the substore before this
// and need to pass that (unmodified) as the leaf value of the multistore proof.
func (si storeInfo) GetHash() []byte {
return si.Core.CommitID.Hash
}
//----------------------------------------
// Misc.
func getLatestVersion(db dbm.DB) int64 { func getLatestVersion(db dbm.DB) int64 {
var latest int64 bz, err := db.Get([]byte(latestVersionKey))
latestBytes, err := db.Get([]byte(latestVersionKey))
if err != nil { if err != nil {
panic(err) panic(err)
} else if latestBytes == nil { } else if bz == nil {
return 0 return 0
} }
err = cdc.UnmarshalBinaryBare(latestBytes, &latest) var latestVersion int64
if err != nil {
if err := gogotypes.StdInt64Unmarshal(&latestVersion, bz); err != nil {
panic(err) panic(err)
} }
return latest return latestVersion
} }
// Commits each store and returns a new commitInfo. // Commits each store and returns a new commitInfo.
func commitStores(version int64, storeMap map[types.StoreKey]types.CommitKVStore) commitInfo { func commitStores(version int64, storeMap map[types.StoreKey]types.CommitKVStore) *types.CommitInfo {
storeInfos := make([]storeInfo, 0, len(storeMap)) storeInfos := make([]types.StoreInfo, 0, len(storeMap))
for key, store := range storeMap { for key, store := range storeMap {
commitID := store.Commit() commitID := store.Commit()
@ -705,52 +605,64 @@ func commitStores(version int64, storeMap map[types.StoreKey]types.CommitKVStore
continue continue
} }
si := storeInfo{} si := types.StoreInfo{}
si.Name = key.Name() si.Name = key.Name()
si.Core.CommitID = commitID si.CommitID = commitID
storeInfos = append(storeInfos, si) storeInfos = append(storeInfos, si)
} }
return commitInfo{ return &types.CommitInfo{
Version: version, Version: version,
StoreInfos: storeInfos, StoreInfos: storeInfos,
} }
} }
// Gets commitInfo from disk. // Gets commitInfo from disk.
func getCommitInfo(db dbm.DB, ver int64) (commitInfo, error) { func getCommitInfo(db dbm.DB, ver int64) (*types.CommitInfo, error) {
cInfoKey := fmt.Sprintf(commitInfoKeyFmt, ver) cInfoKey := fmt.Sprintf(commitInfoKeyFmt, ver)
cInfoBytes, err := db.Get([]byte(cInfoKey)) bz, err := db.Get([]byte(cInfoKey))
if err != nil { if err != nil {
return commitInfo{}, errors.Wrap(err, "failed to get commit info") return nil, errors.Wrap(err, "failed to get commit info")
} else if cInfoBytes == nil { } else if bz == nil {
return commitInfo{}, errors.New("failed to get commit info: no data") return nil, errors.New("no commit info found")
} }
var cInfo commitInfo cInfo := &types.CommitInfo{}
if err = cInfo.Unmarshal(bz); err != nil {
err = cdc.UnmarshalBinaryBare(cInfoBytes, &cInfo) return nil, errors.Wrap(err, "failed unmarshal commit info")
if err != nil {
return commitInfo{}, errors.Wrap(err, "failed to get store")
} }
return cInfo, nil return cInfo, nil
} }
func setCommitInfo(batch dbm.Batch, version int64, cInfo commitInfo) { func setCommitInfo(batch dbm.Batch, version int64, cInfo *types.CommitInfo) {
cInfoBytes := cdc.MustMarshalBinaryBare(cInfo) bz, err := cInfo.Marshal()
if err != nil {
panic(err)
}
cInfoKey := fmt.Sprintf(commitInfoKeyFmt, version) cInfoKey := fmt.Sprintf(commitInfoKeyFmt, version)
batch.Set([]byte(cInfoKey), cInfoBytes) batch.Set([]byte(cInfoKey), bz)
} }
func setLatestVersion(batch dbm.Batch, version int64) { func setLatestVersion(batch dbm.Batch, version int64) {
latestBytes := cdc.MustMarshalBinaryBare(version) bz, err := gogotypes.StdInt64Marshal(version)
batch.Set([]byte(latestVersionKey), latestBytes) if err != nil {
panic(err)
}
batch.Set([]byte(latestVersionKey), bz)
} }
func setPruningHeights(batch dbm.Batch, pruneHeights []int64) { func setPruningHeights(batch dbm.Batch, pruneHeights []int64) {
bz := cdc.MustMarshalBinaryBare(pruneHeights) bz := make([]byte, 0)
for _, ph := range pruneHeights {
buf := make([]byte, 8)
binary.BigEndian.PutUint64(buf, uint64(ph))
bz = append(bz, buf...)
}
batch.Set([]byte(pruneHeightsKey), bz) batch.Set([]byte(pruneHeightsKey), bz)
} }
@ -763,15 +675,18 @@ func getPruningHeights(db dbm.DB) ([]int64, error) {
return nil, errors.New("no pruned heights found") return nil, errors.New("no pruned heights found")
} }
var prunedHeights []int64 prunedHeights := make([]int64, len(bz)/8)
if err := cdc.UnmarshalBinaryBare(bz, &prunedHeights); err != nil { i, offset := 0, 0
return nil, fmt.Errorf("failed to unmarshal pruned heights: %w", err) for offset < len(bz) {
prunedHeights[i] = int64(binary.BigEndian.Uint64(bz[offset : offset+8]))
i++
offset += 8
} }
return prunedHeights, nil return prunedHeights, nil
} }
func flushMetadata(db dbm.DB, version int64, cInfo commitInfo, pruneHeights []int64) { func flushMetadata(db dbm.DB, version int64, cInfo *types.CommitInfo, pruneHeights []int64) {
batch := db.NewBatch() batch := db.NewBatch()
defer batch.Close() defer batch.Close()

View File

@ -9,7 +9,7 @@ import (
dbm "github.com/tendermint/tm-db" dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/store/iavl" "github.com/cosmos/cosmos-sdk/store/iavl"
sdkmaps "github.com/cosmos/cosmos-sdk/store/rootmulti/internal/maps" sdkmaps "github.com/cosmos/cosmos-sdk/store/internal/maps"
"github.com/cosmos/cosmos-sdk/store/types" "github.com/cosmos/cosmos-sdk/store/types"
sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors"
) )
@ -557,7 +557,7 @@ func checkStore(t *testing.T, store *Store, expect, got types.CommitID) {
} }
func checkContains(t testing.TB, info []storeInfo, wanted []string) { func checkContains(t testing.TB, info []types.StoreInfo, wanted []string) {
t.Helper() t.Helper()
for _, want := range wanted { for _, want := range wanted {
@ -565,7 +565,7 @@ func checkContains(t testing.TB, info []storeInfo, wanted []string) {
} }
} }
func checkHas(t testing.TB, info []storeInfo, want string) { func checkHas(t testing.TB, info []types.StoreInfo, want string) {
t.Helper() t.Helper()
for _, i := range info { for _, i := range info {
if i.Name == want { if i.Name == want {
@ -586,12 +586,9 @@ func hashStores(stores map[types.StoreKey]types.CommitKVStore) []byte {
m := make(map[string][]byte, len(stores)) m := make(map[string][]byte, len(stores))
for key, store := range stores { for key, store := range stores {
name := key.Name() name := key.Name()
m[name] = storeInfo{ m[name] = types.StoreInfo{
Name: name, Name: name,
Core: storeCore{ CommitID: store.LastCommitID(),
CommitID: store.LastCommitID(),
// StoreType: store.GetStoreType(),
},
}.GetHash() }.GetHash()
} }
return sdkmaps.SimpleHashFromMap(m) return sdkmaps.SimpleHashFromMap(m)

View File

@ -0,0 +1,73 @@
package types
import (
fmt "fmt"
ics23 "github.com/confio/ics23/go"
"github.com/tendermint/tendermint/crypto/merkle"
sdkmaps "github.com/cosmos/cosmos-sdk/store/internal/maps"
sdkproofs "github.com/cosmos/cosmos-sdk/store/internal/proofs"
)
// GetHash returns the GetHash from the CommitID.
// This is used in CommitInfo.Hash()
//
// When we commit to this in a merkle proof, we create a map of storeInfo.Name -> storeInfo.GetHash()
// and build a merkle proof from that.
// This is then chained with the substore proof, so we prove the root hash from the substore before this
// and need to pass that (unmodified) as the leaf value of the multistore proof.
func (si StoreInfo) GetHash() []byte {
return si.CommitID.Hash
}
func (ci CommitInfo) toMap() map[string][]byte {
m := make(map[string][]byte, len(ci.StoreInfos))
for _, storeInfo := range ci.StoreInfos {
m[storeInfo.Name] = storeInfo.GetHash()
}
return m
}
// Hash returns the simple merkle root hash of the stores sorted by name.
func (ci CommitInfo) Hash() []byte {
// we need a special case for empty set, as SimpleProofsFromMap requires at least one entry
if len(ci.StoreInfos) == 0 {
return nil
}
rootHash, _, _ := sdkmaps.SimpleProofsFromMap(ci.toMap())
return rootHash
}
func (ci CommitInfo) ProofOp(storeName string) merkle.ProofOp {
cmap := ci.toMap()
_, proofs, _ := sdkmaps.SimpleProofsFromMap(cmap)
proof := proofs[storeName]
if proof == nil {
panic(fmt.Sprintf("ProofOp for %s but not registered store name", storeName))
}
// convert merkle.SimpleProof to CommitmentProof
existProof, err := sdkproofs.ConvertExistenceProof(proof, []byte(storeName), cmap[storeName])
if err != nil {
panic(fmt.Errorf("could not convert simple proof to existence proof: %w", err))
}
commitmentProof := &ics23.CommitmentProof{
Proof: &ics23.CommitmentProof_Exist{
Exist: existProof,
},
}
return NewSimpleMerkleCommitmentOp([]byte(storeName), commitmentProof).ProofOp()
}
func (ci CommitInfo) CommitID() CommitID {
return CommitID{
Version: ci.Version,
Hash: ci.Hash(),
}
}

View File

@ -0,0 +1,814 @@
// Code generated by protoc-gen-gogo. DO NOT EDIT.
// source: cosmos/store/commit_info.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
// CommitInfo defines commit information used by the multi-store when committing
// a version/height.
type CommitInfo struct {
Version int64 `protobuf:"varint,1,opt,name=version,proto3" json:"version,omitempty"`
StoreInfos []StoreInfo `protobuf:"bytes,2,rep,name=store_infos,json=storeInfos,proto3" json:"store_infos"`
}
func (m *CommitInfo) Reset() { *m = CommitInfo{} }
func (m *CommitInfo) String() string { return proto.CompactTextString(m) }
func (*CommitInfo) ProtoMessage() {}
func (*CommitInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3e3b3900c32110a1, []int{0}
}
func (m *CommitInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *CommitInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_CommitInfo.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 *CommitInfo) XXX_Merge(src proto.Message) {
xxx_messageInfo_CommitInfo.Merge(m, src)
}
func (m *CommitInfo) XXX_Size() int {
return m.Size()
}
func (m *CommitInfo) XXX_DiscardUnknown() {
xxx_messageInfo_CommitInfo.DiscardUnknown(m)
}
var xxx_messageInfo_CommitInfo proto.InternalMessageInfo
func (m *CommitInfo) GetVersion() int64 {
if m != nil {
return m.Version
}
return 0
}
func (m *CommitInfo) GetStoreInfos() []StoreInfo {
if m != nil {
return m.StoreInfos
}
return nil
}
// StoreInfo defines store-specific commit information. It contains a reference
// between a store name and the commit ID.
type StoreInfo struct {
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
CommitID CommitID `protobuf:"bytes,2,opt,name=commit_id,json=commitId,proto3" json:"commit_id"`
}
func (m *StoreInfo) Reset() { *m = StoreInfo{} }
func (m *StoreInfo) String() string { return proto.CompactTextString(m) }
func (*StoreInfo) ProtoMessage() {}
func (*StoreInfo) Descriptor() ([]byte, []int) {
return fileDescriptor_3e3b3900c32110a1, []int{1}
}
func (m *StoreInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *StoreInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_StoreInfo.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 *StoreInfo) XXX_Merge(src proto.Message) {
xxx_messageInfo_StoreInfo.Merge(m, src)
}
func (m *StoreInfo) XXX_Size() int {
return m.Size()
}
func (m *StoreInfo) XXX_DiscardUnknown() {
xxx_messageInfo_StoreInfo.DiscardUnknown(m)
}
var xxx_messageInfo_StoreInfo proto.InternalMessageInfo
func (m *StoreInfo) GetName() string {
if m != nil {
return m.Name
}
return ""
}
func (m *StoreInfo) GetCommitID() CommitID {
if m != nil {
return m.CommitID
}
return CommitID{}
}
// CommitID defines the committment information when a specific store is
// committed.
type CommitID struct {
Version int64 `protobuf:"varint,1,opt,name=version,proto3" json:"version,omitempty"`
Hash []byte `protobuf:"bytes,2,opt,name=hash,proto3" json:"hash,omitempty"`
}
func (m *CommitID) Reset() { *m = CommitID{} }
func (*CommitID) ProtoMessage() {}
func (*CommitID) Descriptor() ([]byte, []int) {
return fileDescriptor_3e3b3900c32110a1, []int{2}
}
func (m *CommitID) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *CommitID) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_CommitID.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 *CommitID) XXX_Merge(src proto.Message) {
xxx_messageInfo_CommitID.Merge(m, src)
}
func (m *CommitID) XXX_Size() int {
return m.Size()
}
func (m *CommitID) XXX_DiscardUnknown() {
xxx_messageInfo_CommitID.DiscardUnknown(m)
}
var xxx_messageInfo_CommitID proto.InternalMessageInfo
func (m *CommitID) GetVersion() int64 {
if m != nil {
return m.Version
}
return 0
}
func (m *CommitID) GetHash() []byte {
if m != nil {
return m.Hash
}
return nil
}
func init() {
proto.RegisterType((*CommitInfo)(nil), "cosmos.store.CommitInfo")
proto.RegisterType((*StoreInfo)(nil), "cosmos.store.StoreInfo")
proto.RegisterType((*CommitID)(nil), "cosmos.store.CommitID")
}
func init() { proto.RegisterFile("cosmos/store/commit_info.proto", fileDescriptor_3e3b3900c32110a1) }
var fileDescriptor_3e3b3900c32110a1 = []byte{
// 289 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x4b, 0xce, 0x2f, 0xce,
0xcd, 0x2f, 0xd6, 0x2f, 0x2e, 0xc9, 0x2f, 0x4a, 0xd5, 0x4f, 0xce, 0xcf, 0xcd, 0xcd, 0x2c, 0x89,
0xcf, 0xcc, 0x4b, 0xcb, 0xd7, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0xe2, 0x81, 0xc8, 0xeb, 0x81,
0xe5, 0xa5, 0x44, 0xd2, 0xf3, 0xd3, 0xf3, 0xc1, 0x12, 0xfa, 0x20, 0x16, 0x44, 0x8d, 0x52, 0x1a,
0x17, 0x97, 0x33, 0x58, 0xa3, 0x67, 0x5e, 0x5a, 0xbe, 0x90, 0x04, 0x17, 0x7b, 0x59, 0x6a, 0x51,
0x71, 0x66, 0x7e, 0x9e, 0x04, 0xa3, 0x02, 0xa3, 0x06, 0x73, 0x10, 0x8c, 0x2b, 0x64, 0xc7, 0xc5,
0x0d, 0x36, 0x06, 0x6c, 0x7e, 0xb1, 0x04, 0x93, 0x02, 0xb3, 0x06, 0xb7, 0x91, 0xb8, 0x1e, 0xb2,
0x0d, 0x7a, 0xc1, 0x20, 0x12, 0x64, 0x8e, 0x13, 0xcb, 0x89, 0x7b, 0xf2, 0x0c, 0x41, 0x5c, 0xc5,
0x30, 0x81, 0x62, 0xa5, 0x34, 0x2e, 0x4e, 0xb8, 0xb4, 0x90, 0x10, 0x17, 0x4b, 0x5e, 0x62, 0x6e,
0x2a, 0xd8, 0x0e, 0xce, 0x20, 0x30, 0x5b, 0xc8, 0x95, 0x8b, 0x13, 0xe6, 0x83, 0x14, 0x09, 0x26,
0x05, 0x46, 0x0d, 0x6e, 0x23, 0x31, 0x54, 0xe3, 0xa1, 0xee, 0x74, 0x71, 0x12, 0x00, 0x99, 0xfe,
0xe8, 0x9e, 0x3c, 0x07, 0x4c, 0x24, 0x88, 0x03, 0xa2, 0xd5, 0x33, 0x45, 0xc9, 0x8e, 0x0b, 0x2e,
0x8a, 0xc7, 0x37, 0x42, 0x5c, 0x2c, 0x19, 0x89, 0xc5, 0x19, 0x60, 0x7b, 0x78, 0x82, 0xc0, 0x6c,
0x2b, 0x96, 0x19, 0x0b, 0xe4, 0x19, 0x9c, 0x9c, 0x4e, 0x3c, 0x92, 0x63, 0xbc, 0xf0, 0x48, 0x8e,
0xf1, 0xc1, 0x23, 0x39, 0xc6, 0x09, 0x8f, 0xe5, 0x18, 0x2e, 0x3c, 0x96, 0x63, 0xb8, 0xf1, 0x58,
0x8e, 0x21, 0x4a, 0x23, 0x3d, 0xb3, 0x24, 0xa3, 0x34, 0x49, 0x2f, 0x39, 0x3f, 0x57, 0x1f, 0x1a,
0xf0, 0x10, 0x4a, 0xb7, 0x38, 0x25, 0x1b, 0x1a, 0x07, 0x25, 0x95, 0x05, 0xa9, 0xc5, 0x49, 0x6c,
0xe0, 0xa0, 0x35, 0x06, 0x04, 0x00, 0x00, 0xff, 0xff, 0x2f, 0x20, 0xcc, 0x8d, 0xa0, 0x01, 0x00,
0x00,
}
func (m *CommitInfo) 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 *CommitInfo) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *CommitInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
if len(m.StoreInfos) > 0 {
for iNdEx := len(m.StoreInfos) - 1; iNdEx >= 0; iNdEx-- {
{
size, err := m.StoreInfos[iNdEx].MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintCommitInfo(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x12
}
}
if m.Version != 0 {
i = encodeVarintCommitInfo(dAtA, i, uint64(m.Version))
i--
dAtA[i] = 0x8
}
return len(dAtA) - i, nil
}
func (m *StoreInfo) 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 *StoreInfo) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *StoreInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
{
size, err := m.CommitID.MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintCommitInfo(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0x12
if len(m.Name) > 0 {
i -= len(m.Name)
copy(dAtA[i:], m.Name)
i = encodeVarintCommitInfo(dAtA, i, uint64(len(m.Name)))
i--
dAtA[i] = 0xa
}
return len(dAtA) - i, nil
}
func (m *CommitID) 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 *CommitID) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *CommitID) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
if len(m.Hash) > 0 {
i -= len(m.Hash)
copy(dAtA[i:], m.Hash)
i = encodeVarintCommitInfo(dAtA, i, uint64(len(m.Hash)))
i--
dAtA[i] = 0x12
}
if m.Version != 0 {
i = encodeVarintCommitInfo(dAtA, i, uint64(m.Version))
i--
dAtA[i] = 0x8
}
return len(dAtA) - i, nil
}
func encodeVarintCommitInfo(dAtA []byte, offset int, v uint64) int {
offset -= sovCommitInfo(v)
base := offset
for v >= 1<<7 {
dAtA[offset] = uint8(v&0x7f | 0x80)
v >>= 7
offset++
}
dAtA[offset] = uint8(v)
return base
}
func (m *CommitInfo) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if m.Version != 0 {
n += 1 + sovCommitInfo(uint64(m.Version))
}
if len(m.StoreInfos) > 0 {
for _, e := range m.StoreInfos {
l = e.Size()
n += 1 + l + sovCommitInfo(uint64(l))
}
}
return n
}
func (m *StoreInfo) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
l = len(m.Name)
if l > 0 {
n += 1 + l + sovCommitInfo(uint64(l))
}
l = m.CommitID.Size()
n += 1 + l + sovCommitInfo(uint64(l))
return n
}
func (m *CommitID) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if m.Version != 0 {
n += 1 + sovCommitInfo(uint64(m.Version))
}
l = len(m.Hash)
if l > 0 {
n += 1 + l + sovCommitInfo(uint64(l))
}
return n
}
func sovCommitInfo(x uint64) (n int) {
return (math_bits.Len64(x|1) + 6) / 7
}
func sozCommitInfo(x uint64) (n int) {
return sovCommitInfo(uint64((x << 1) ^ uint64((int64(x) >> 63))))
}
func (m *CommitInfo) 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 ErrIntOverflowCommitInfo
}
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: CommitInfo: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: CommitInfo: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
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 ErrIntOverflowCommitInfo
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
m.Version |= int64(b&0x7F) << shift
if b < 0x80 {
break
}
}
case 2:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field StoreInfos", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowCommitInfo
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthCommitInfo
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthCommitInfo
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.StoreInfos = append(m.StoreInfos, StoreInfo{})
if err := m.StoreInfos[len(m.StoreInfos)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipCommitInfo(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthCommitInfo
}
if (iNdEx + skippy) < 0 {
return ErrInvalidLengthCommitInfo
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *StoreInfo) 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 ErrIntOverflowCommitInfo
}
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: StoreInfo: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: StoreInfo: 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 ErrIntOverflowCommitInfo
}
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 ErrInvalidLengthCommitInfo
}
postIndex := iNdEx + intStringLen
if postIndex < 0 {
return ErrInvalidLengthCommitInfo
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Name = string(dAtA[iNdEx:postIndex])
iNdEx = postIndex
case 2:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field CommitID", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowCommitInfo
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthCommitInfo
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthCommitInfo
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
if err := m.CommitID.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipCommitInfo(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthCommitInfo
}
if (iNdEx + skippy) < 0 {
return ErrInvalidLengthCommitInfo
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *CommitID) 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 ErrIntOverflowCommitInfo
}
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: CommitID: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: CommitID: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
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 ErrIntOverflowCommitInfo
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
m.Version |= int64(b&0x7F) << shift
if b < 0x80 {
break
}
}
case 2:
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 ErrIntOverflowCommitInfo
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
byteLen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if byteLen < 0 {
return ErrInvalidLengthCommitInfo
}
postIndex := iNdEx + byteLen
if postIndex < 0 {
return ErrInvalidLengthCommitInfo
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Hash = append(m.Hash[:0], dAtA[iNdEx:postIndex]...)
if m.Hash == nil {
m.Hash = []byte{}
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipCommitInfo(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthCommitInfo
}
if (iNdEx + skippy) < 0 {
return ErrInvalidLengthCommitInfo
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func skipCommitInfo(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, ErrIntOverflowCommitInfo
}
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, ErrIntOverflowCommitInfo
}
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, ErrIntOverflowCommitInfo
}
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, ErrInvalidLengthCommitInfo
}
iNdEx += length
case 3:
depth++
case 4:
if depth == 0 {
return 0, ErrUnexpectedEndOfGroupCommitInfo
}
depth--
case 5:
iNdEx += 4
default:
return 0, fmt.Errorf("proto: illegal wireType %d", wireType)
}
if iNdEx < 0 {
return 0, ErrInvalidLengthCommitInfo
}
if depth == 0 {
return iNdEx, nil
}
}
return 0, io.ErrUnexpectedEOF
}
var (
ErrInvalidLengthCommitInfo = fmt.Errorf("proto: negative length found during unmarshaling")
ErrIntOverflowCommitInfo = fmt.Errorf("proto: integer overflow")
ErrUnexpectedEndOfGroupCommitInfo = fmt.Errorf("proto: unexpected end of group")
)

View File

@ -247,15 +247,6 @@ type CacheWrapper interface {
CacheWrapWithTrace(w io.Writer, tc TraceContext) CacheWrap CacheWrapWithTrace(w io.Writer, tc TraceContext) CacheWrap
} }
//----------------------------------------
// CommitID
// CommitID contains the tree version number and its merkle root.
type CommitID struct {
Version int64
Hash []byte
}
func (cid CommitID) IsZero() bool { func (cid CommitID) IsZero() bool {
return cid.Version == 0 && len(cid.Hash) == 0 return cid.Version == 0 && len(cid.Hash) == 0
} }

View File

@ -5,32 +5,22 @@ import (
"sort" "sort"
) )
//---------------------------------------- func (kvs Pairs) Len() int { return len(kvs.Pairs) }
// KVPair
/*
Defined in types.proto
type Pair struct {
Key []byte
Value []byte
}
*/
type Pairs []Pair
// Sorting
func (kvs Pairs) Len() int { return len(kvs) }
func (kvs Pairs) Less(i, j int) bool { func (kvs Pairs) Less(i, j int) bool {
switch bytes.Compare(kvs[i].Key, kvs[j].Key) { switch bytes.Compare(kvs.Pairs[i].Key, kvs.Pairs[j].Key) {
case -1: case -1:
return true return true
case 0: case 0:
return bytes.Compare(kvs[i].Value, kvs[j].Value) < 0 return bytes.Compare(kvs.Pairs[i].Value, kvs.Pairs[j].Value) < 0
case 1: case 1:
return false return false
default: default:
panic("invalid comparison result") panic("invalid comparison result")
} }
} }
func (kvs Pairs) Swap(i, j int) { kvs[i], kvs[j] = kvs[j], kvs[i] }
func (kvs Pairs) Swap(i, j int) { kvs.Pairs[i], kvs.Pairs[j] = kvs.Pairs[j], kvs.Pairs[i] }
func (kvs Pairs) Sort() { sort.Sort(kvs) } func (kvs Pairs) Sort() { sort.Sort(kvs) }

View File

@ -5,6 +5,7 @@ package kv
import ( import (
fmt "fmt" fmt "fmt"
_ "github.com/gogo/protobuf/gogoproto"
proto "github.com/gogo/protobuf/proto" proto "github.com/gogo/protobuf/proto"
io "io" io "io"
math "math" math "math"
@ -22,7 +23,52 @@ var _ = math.Inf
// proto package needs to be updated. // proto package needs to be updated.
const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package
// Key-Value Pair // Pairs defines a repeated slice of Pair objects.
type Pairs struct {
Pairs []Pair `protobuf:"bytes,1,rep,name=pairs,proto3" json:"pairs"`
}
func (m *Pairs) Reset() { *m = Pairs{} }
func (m *Pairs) String() string { return proto.CompactTextString(m) }
func (*Pairs) ProtoMessage() {}
func (*Pairs) Descriptor() ([]byte, []int) {
return fileDescriptor_23371bd43b515c6e, []int{0}
}
func (m *Pairs) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *Pairs) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_Pairs.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 *Pairs) XXX_Merge(src proto.Message) {
xxx_messageInfo_Pairs.Merge(m, src)
}
func (m *Pairs) XXX_Size() int {
return m.Size()
}
func (m *Pairs) XXX_DiscardUnknown() {
xxx_messageInfo_Pairs.DiscardUnknown(m)
}
var xxx_messageInfo_Pairs proto.InternalMessageInfo
func (m *Pairs) GetPairs() []Pair {
if m != nil {
return m.Pairs
}
return nil
}
// Pair defines a key/value bytes tuple.
type Pair struct { type Pair struct {
Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
@ -32,7 +78,7 @@ func (m *Pair) Reset() { *m = Pair{} }
func (m *Pair) String() string { return proto.CompactTextString(m) } func (m *Pair) String() string { return proto.CompactTextString(m) }
func (*Pair) ProtoMessage() {} func (*Pair) ProtoMessage() {}
func (*Pair) Descriptor() ([]byte, []int) { func (*Pair) Descriptor() ([]byte, []int) {
return fileDescriptor_23371bd43b515c6e, []int{0} return fileDescriptor_23371bd43b515c6e, []int{1}
} }
func (m *Pair) XXX_Unmarshal(b []byte) error { func (m *Pair) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b) return m.Unmarshal(b)
@ -76,23 +122,64 @@ func (m *Pair) GetValue() []byte {
} }
func init() { func init() {
proto.RegisterType((*Pairs)(nil), "cosmos.kv.Pairs")
proto.RegisterType((*Pair)(nil), "cosmos.kv.Pair") proto.RegisterType((*Pair)(nil), "cosmos.kv.Pair")
} }
func init() { proto.RegisterFile("cosmos/kv/kv.proto", fileDescriptor_23371bd43b515c6e) } func init() { proto.RegisterFile("cosmos/kv/kv.proto", fileDescriptor_23371bd43b515c6e) }
var fileDescriptor_23371bd43b515c6e = []byte{ var fileDescriptor_23371bd43b515c6e = []byte{
// 150 bytes of a gzipped FileDescriptorProto // 200 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x12, 0x4a, 0xce, 0x2f, 0xce, 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x12, 0x4a, 0xce, 0x2f, 0xce,
0xcd, 0x2f, 0xd6, 0xcf, 0x2e, 0xd3, 0xcf, 0x2e, 0xd3, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0xe2, 0xcd, 0x2f, 0xd6, 0xcf, 0x2e, 0xd3, 0xcf, 0x2e, 0xd3, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0xe2,
0x84, 0x88, 0xe9, 0x65, 0x97, 0x29, 0xe9, 0x71, 0xb1, 0x04, 0x24, 0x66, 0x16, 0x09, 0x09, 0x70, 0x84, 0x88, 0xe9, 0x65, 0x97, 0x49, 0x89, 0xa4, 0xe7, 0xa7, 0xe7, 0x83, 0x45, 0xf5, 0x41, 0x2c,
0x31, 0x67, 0xa7, 0x56, 0x4a, 0x30, 0x2a, 0x30, 0x6a, 0xf0, 0x04, 0x81, 0x98, 0x42, 0x22, 0x5c, 0x88, 0x02, 0x25, 0x13, 0x2e, 0xd6, 0x80, 0xc4, 0xcc, 0xa2, 0x62, 0x21, 0x6d, 0x2e, 0xd6, 0x02,
0xac, 0x65, 0x89, 0x39, 0xa5, 0xa9, 0x12, 0x4c, 0x60, 0x31, 0x08, 0xc7, 0xc9, 0xfe, 0xc4, 0x23, 0x10, 0x43, 0x82, 0x51, 0x81, 0x59, 0x83, 0xdb, 0x88, 0x5f, 0x0f, 0xae, 0x53, 0x0f, 0xa4, 0xc0,
0x39, 0xc6, 0x0b, 0x8f, 0xe4, 0x18, 0x1f, 0x3c, 0x92, 0x63, 0x9c, 0xf0, 0x58, 0x8e, 0xe1, 0xc2, 0x89, 0xe5, 0xc4, 0x3d, 0x79, 0x86, 0x20, 0x88, 0x1a, 0x25, 0x3d, 0x2e, 0x16, 0x90, 0xa0, 0x90,
0x63, 0x39, 0x86, 0x1b, 0x8f, 0xe5, 0x18, 0xa2, 0x54, 0xd3, 0x33, 0x4b, 0x32, 0x4a, 0x93, 0xf4, 0x00, 0x17, 0x73, 0x76, 0x6a, 0xa5, 0x04, 0xa3, 0x02, 0xa3, 0x06, 0x4f, 0x10, 0x88, 0x29, 0x24,
0x92, 0xf3, 0x73, 0xf5, 0xa1, 0x76, 0x42, 0x28, 0xdd, 0xe2, 0x94, 0x6c, 0xfd, 0x92, 0xca, 0x82, 0xc2, 0xc5, 0x5a, 0x96, 0x98, 0x53, 0x9a, 0x2a, 0xc1, 0x04, 0x16, 0x83, 0x70, 0x9c, 0xec, 0x4f,
0x54, 0x90, 0x23, 0x92, 0xd8, 0xc0, 0x4e, 0x30, 0x06, 0x04, 0x00, 0x00, 0xff, 0xff, 0xae, 0xee, 0x3c, 0x92, 0x63, 0xbc, 0xf0, 0x48, 0x8e, 0xf1, 0xc1, 0x23, 0x39, 0xc6, 0x09, 0x8f, 0xe5, 0x18,
0xa2, 0x4c, 0x98, 0x00, 0x00, 0x00, 0x2e, 0x3c, 0x96, 0x63, 0xb8, 0xf1, 0x58, 0x8e, 0x21, 0x4a, 0x35, 0x3d, 0xb3, 0x24, 0xa3, 0x34,
0x49, 0x2f, 0x39, 0x3f, 0x57, 0x1f, 0xea, 0x7e, 0x08, 0xa5, 0x5b, 0x9c, 0x92, 0xad, 0x5f, 0x52,
0x59, 0x90, 0x0a, 0xf2, 0x50, 0x12, 0x1b, 0xd8, 0xb5, 0xc6, 0x80, 0x00, 0x00, 0x00, 0xff, 0xff,
0xce, 0x49, 0x74, 0x20, 0xe4, 0x00, 0x00, 0x00,
}
func (m *Pairs) 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 *Pairs) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *Pairs) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
if len(m.Pairs) > 0 {
for iNdEx := len(m.Pairs) - 1; iNdEx >= 0; iNdEx-- {
{
size, err := m.Pairs[iNdEx].MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintKv(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0xa
}
}
return len(dAtA) - i, nil
} }
func (m *Pair) Marshal() (dAtA []byte, err error) { func (m *Pair) Marshal() (dAtA []byte, err error) {
@ -143,6 +230,21 @@ func encodeVarintKv(dAtA []byte, offset int, v uint64) int {
dAtA[offset] = uint8(v) dAtA[offset] = uint8(v)
return base return base
} }
func (m *Pairs) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if len(m.Pairs) > 0 {
for _, e := range m.Pairs {
l = e.Size()
n += 1 + l + sovKv(uint64(l))
}
}
return n
}
func (m *Pair) Size() (n int) { func (m *Pair) Size() (n int) {
if m == nil { if m == nil {
return 0 return 0
@ -166,6 +268,93 @@ func sovKv(x uint64) (n int) {
func sozKv(x uint64) (n int) { func sozKv(x uint64) (n int) {
return sovKv(uint64((x << 1) ^ uint64((int64(x) >> 63)))) return sovKv(uint64((x << 1) ^ uint64((int64(x) >> 63))))
} }
func (m *Pairs) 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 ErrIntOverflowKv
}
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: Pairs: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: Pairs: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Pairs", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowKv
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthKv
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthKv
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Pairs = append(m.Pairs, Pair{})
if err := m.Pairs[len(m.Pairs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipKv(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthKv
}
if (iNdEx + skippy) < 0 {
return ErrInvalidLengthKv
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *Pair) Unmarshal(dAtA []byte) error { func (m *Pair) Unmarshal(dAtA []byte) error {
l := len(dAtA) l := len(dAtA)
iNdEx := 0 iNdEx := 0

View File

@ -32,17 +32,19 @@ func TestDecodeStore(t *testing.T) {
globalAccNumber := gogotypes.UInt64Value{Value: 10} globalAccNumber := gogotypes.UInt64Value{Value: 10}
kvPairs := kv.Pairs{ kvPairs := kv.Pairs{
kv.Pair{ Pairs: []kv.Pair{
Key: types.AddressStoreKey(delAddr1), {
Value: accBz, Key: types.AddressStoreKey(delAddr1),
}, Value: accBz,
kv.Pair{ },
Key: types.GlobalAccountNumberKey, {
Value: cdc.MustMarshalBinaryBare(&globalAccNumber), Key: types.GlobalAccountNumberKey,
}, Value: cdc.MustMarshalBinaryBare(&globalAccNumber),
kv.Pair{ },
Key: []byte{0x99}, {
Value: []byte{0x99}, Key: []byte{0x99},
Value: []byte{0x99},
},
}, },
} }
tests := []struct { tests := []struct {
@ -59,9 +61,9 @@ func TestDecodeStore(t *testing.T) {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
switch i { switch i {
case len(tests) - 1: case len(tests) - 1:
require.Panics(t, func() { dec(kvPairs[i], kvPairs[i]) }, tt.name) require.Panics(t, func() { dec(kvPairs.Pairs[i], kvPairs.Pairs[i]) }, tt.name)
default: default:
require.Equal(t, tt.expectedLog, dec(kvPairs[i], kvPairs[i]), tt.name) require.Equal(t, tt.expectedLog, dec(kvPairs.Pairs[i], kvPairs.Pairs[i]), tt.name)
} }
}) })
} }

View File

@ -23,8 +23,10 @@ func TestDecodeStore(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
kvPairs := kv.Pairs{ kvPairs := kv.Pairs{
kv.Pair{Key: types.SupplyKey, Value: supplyBz}, Pairs: []kv.Pair{
kv.Pair{Key: []byte{0x99}, Value: []byte{0x99}}, {Key: types.SupplyKey, Value: supplyBz},
{Key: []byte{0x99}, Value: []byte{0x99}},
},
} }
tests := []struct { tests := []struct {
@ -40,9 +42,9 @@ func TestDecodeStore(t *testing.T) {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
switch i { switch i {
case len(tests) - 1: case len(tests) - 1:
require.Panics(t, func() { dec(kvPairs[i], kvPairs[i]) }, tt.name) require.Panics(t, func() { dec(kvPairs.Pairs[i], kvPairs.Pairs[i]) }, tt.name)
default: default:
require.Equal(t, tt.expectedLog, dec(kvPairs[i], kvPairs[i]), tt.name) require.Equal(t, tt.expectedLog, dec(kvPairs.Pairs[i], kvPairs.Pairs[i]), tt.name)
} }
}) })
} }

View File

@ -22,17 +22,19 @@ func TestDecodeStore(t *testing.T) {
} }
kvPairs := kv.Pairs{ kvPairs := kv.Pairs{
kv.Pair{ Pairs: []kv.Pair{
Key: types.KeyIndex, {
Value: sdk.Uint64ToBigEndian(10), Key: types.KeyIndex,
}, Value: sdk.Uint64ToBigEndian(10),
kv.Pair{ },
Key: types.KeyPrefixIndexCapability, {
Value: cdc.MustMarshalBinaryBare(&capOwners), Key: types.KeyPrefixIndexCapability,
}, Value: cdc.MustMarshalBinaryBare(&capOwners),
kv.Pair{ },
Key: []byte{0x99}, {
Value: []byte{0x99}, Key: []byte{0x99},
Value: []byte{0x99},
},
}, },
} }
tests := []struct { tests := []struct {
@ -49,9 +51,9 @@ func TestDecodeStore(t *testing.T) {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
switch i { switch i {
case len(tests) - 1: case len(tests) - 1:
require.Panics(t, func() { dec(kvPairs[i], kvPairs[i]) }, tt.name) require.Panics(t, func() { dec(kvPairs.Pairs[i], kvPairs.Pairs[i]) }, tt.name)
default: default:
require.Equal(t, tt.expectedLog, dec(kvPairs[i], kvPairs[i]), tt.name) require.Equal(t, tt.expectedLog, dec(kvPairs.Pairs[i], kvPairs.Pairs[i]), tt.name)
} }
}) })
} }

View File

@ -36,16 +36,18 @@ func TestDecodeDistributionStore(t *testing.T) {
slashEvent := types.NewValidatorSlashEvent(10, sdk.OneDec()) slashEvent := types.NewValidatorSlashEvent(10, sdk.OneDec())
kvPairs := kv.Pairs{ kvPairs := kv.Pairs{
kv.Pair{Key: types.FeePoolKey, Value: cdc.MustMarshalBinaryBare(&feePool)}, Pairs: []kv.Pair{
kv.Pair{Key: types.ProposerKey, Value: consAddr1.Bytes()}, {Key: types.FeePoolKey, Value: cdc.MustMarshalBinaryBare(&feePool)},
kv.Pair{Key: types.GetValidatorOutstandingRewardsKey(valAddr1), Value: cdc.MustMarshalBinaryBare(&outstanding)}, {Key: types.ProposerKey, Value: consAddr1.Bytes()},
kv.Pair{Key: types.GetDelegatorWithdrawAddrKey(delAddr1), Value: delAddr1.Bytes()}, {Key: types.GetValidatorOutstandingRewardsKey(valAddr1), Value: cdc.MustMarshalBinaryBare(&outstanding)},
kv.Pair{Key: types.GetDelegatorStartingInfoKey(valAddr1, delAddr1), Value: cdc.MustMarshalBinaryBare(&info)}, {Key: types.GetDelegatorWithdrawAddrKey(delAddr1), Value: delAddr1.Bytes()},
kv.Pair{Key: types.GetValidatorHistoricalRewardsKey(valAddr1, 100), Value: cdc.MustMarshalBinaryBare(&historicalRewards)}, {Key: types.GetDelegatorStartingInfoKey(valAddr1, delAddr1), Value: cdc.MustMarshalBinaryBare(&info)},
kv.Pair{Key: types.GetValidatorCurrentRewardsKey(valAddr1), Value: cdc.MustMarshalBinaryBare(&currentRewards)}, {Key: types.GetValidatorHistoricalRewardsKey(valAddr1, 100), Value: cdc.MustMarshalBinaryBare(&historicalRewards)},
kv.Pair{Key: types.GetValidatorAccumulatedCommissionKey(valAddr1), Value: cdc.MustMarshalBinaryBare(&commission)}, {Key: types.GetValidatorCurrentRewardsKey(valAddr1), Value: cdc.MustMarshalBinaryBare(&currentRewards)},
kv.Pair{Key: types.GetValidatorSlashEventKeyPrefix(valAddr1, 13), Value: cdc.MustMarshalBinaryBare(&slashEvent)}, {Key: types.GetValidatorAccumulatedCommissionKey(valAddr1), Value: cdc.MustMarshalBinaryBare(&commission)},
kv.Pair{Key: []byte{0x99}, Value: []byte{0x99}}, {Key: types.GetValidatorSlashEventKeyPrefix(valAddr1, 13), Value: cdc.MustMarshalBinaryBare(&slashEvent)},
{Key: []byte{0x99}, Value: []byte{0x99}},
},
} }
tests := []struct { tests := []struct {
@ -68,9 +70,9 @@ func TestDecodeDistributionStore(t *testing.T) {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
switch i { switch i {
case len(tests) - 1: case len(tests) - 1:
require.Panics(t, func() { dec(kvPairs[i], kvPairs[i]) }, tt.name) require.Panics(t, func() { dec(kvPairs.Pairs[i], kvPairs.Pairs[i]) }, tt.name)
default: default:
require.Equal(t, tt.expectedLog, dec(kvPairs[i], kvPairs[i]), tt.name) require.Equal(t, tt.expectedLog, dec(kvPairs.Pairs[i], kvPairs.Pairs[i]), tt.name)
} }
}) })
} }

View File

@ -32,13 +32,15 @@ func TestDecodeStore(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
kvPairs := kv.Pairs{ kvPairs := kv.Pairs{
kv.Pair{ Pairs: []kv.Pair{
Key: types.KeyPrefixEvidence, {
Value: evBz, Key: types.KeyPrefixEvidence,
}, Value: evBz,
kv.Pair{ },
Key: []byte{0x99}, {
Value: []byte{0x99}, Key: []byte{0x99},
Value: []byte{0x99},
},
}, },
} }
tests := []struct { tests := []struct {
@ -54,9 +56,9 @@ func TestDecodeStore(t *testing.T) {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
switch i { switch i {
case len(tests) - 1: case len(tests) - 1:
require.Panics(t, func() { dec(kvPairs[i], kvPairs[i]) }, tt.name) require.Panics(t, func() { dec(kvPairs.Pairs[i], kvPairs.Pairs[i]) }, tt.name)
default: default:
require.Equal(t, tt.expectedLog, dec(kvPairs[i], kvPairs[i]), tt.name) require.Equal(t, tt.expectedLog, dec(kvPairs.Pairs[i], kvPairs.Pairs[i]), tt.name)
} }
}) })
} }

View File

@ -40,11 +40,13 @@ func TestDecodeStore(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
kvPairs := kv.Pairs{ kvPairs := kv.Pairs{
kv.Pair{Key: types.ProposalKey(1), Value: proposalBz}, Pairs: []kv.Pair{
kv.Pair{Key: types.InactiveProposalQueueKey(1, endTime), Value: proposalIDBz}, {Key: types.ProposalKey(1), Value: proposalBz},
kv.Pair{Key: types.DepositKey(1, delAddr1), Value: cdc.MustMarshalBinaryBare(&deposit)}, {Key: types.InactiveProposalQueueKey(1, endTime), Value: proposalIDBz},
kv.Pair{Key: types.VoteKey(1, delAddr1), Value: cdc.MustMarshalBinaryBare(&vote)}, {Key: types.DepositKey(1, delAddr1), Value: cdc.MustMarshalBinaryBare(&deposit)},
kv.Pair{Key: []byte{0x99}, Value: []byte{0x99}}, {Key: types.VoteKey(1, delAddr1), Value: cdc.MustMarshalBinaryBare(&vote)},
{Key: []byte{0x99}, Value: []byte{0x99}},
},
} }
tests := []struct { tests := []struct {
@ -63,9 +65,9 @@ func TestDecodeStore(t *testing.T) {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
switch i { switch i {
case len(tests) - 1: case len(tests) - 1:
require.Panics(t, func() { dec(kvPairs[i], kvPairs[i]) }, tt.name) require.Panics(t, func() { dec(kvPairs.Pairs[i], kvPairs.Pairs[i]) }, tt.name)
default: default:
require.Equal(t, tt.expectedLog, dec(kvPairs[i], kvPairs[i]), tt.name) require.Equal(t, tt.expectedLog, dec(kvPairs.Pairs[i], kvPairs.Pairs[i]), tt.name)
} }
}) })
} }

View File

@ -29,21 +29,23 @@ func TestDecodeStore(t *testing.T) {
} }
kvPairs := kv.Pairs{ kvPairs := kv.Pairs{
kv.Pair{ Pairs: []kv.Pair{
Key: host.FullKeyClientPath(clientID, host.KeyClientState()), {
Value: app.IBCKeeper.ClientKeeper.MustMarshalClientState(clientState), Key: host.FullKeyClientPath(clientID, host.KeyClientState()),
}, Value: app.IBCKeeper.ClientKeeper.MustMarshalClientState(clientState),
kv.Pair{ },
Key: host.FullKeyClientPath(clientID, host.KeyClientType()), {
Value: []byte(exported.Tendermint.String()), Key: host.FullKeyClientPath(clientID, host.KeyClientType()),
}, Value: []byte(exported.Tendermint.String()),
kv.Pair{ },
Key: host.FullKeyClientPath(clientID, host.KeyConsensusState(10)), {
Value: app.IBCKeeper.ClientKeeper.MustMarshalConsensusState(consState), Key: host.FullKeyClientPath(clientID, host.KeyConsensusState(10)),
}, Value: app.IBCKeeper.ClientKeeper.MustMarshalConsensusState(consState),
kv.Pair{ },
Key: []byte{0x99}, {
Value: []byte{0x99}, Key: []byte{0x99},
Value: []byte{0x99},
},
}, },
} }
tests := []struct { tests := []struct {
@ -59,13 +61,13 @@ func TestDecodeStore(t *testing.T) {
for i, tt := range tests { for i, tt := range tests {
i, tt := i, tt i, tt := i, tt
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
res, found := simulation.NewDecodeStore(app.IBCKeeper.ClientKeeper, kvPairs[i], kvPairs[i]) res, found := simulation.NewDecodeStore(app.IBCKeeper.ClientKeeper, kvPairs.Pairs[i], kvPairs.Pairs[i])
if i == len(tests)-1 { if i == len(tests)-1 {
require.False(t, found, string(kvPairs[i].Key)) require.False(t, found, string(kvPairs.Pairs[i].Key))
require.Empty(t, res, string(kvPairs[i].Key)) require.Empty(t, res, string(kvPairs.Pairs[i].Key))
} else { } else {
require.True(t, found, string(kvPairs[i].Key)) require.True(t, found, string(kvPairs.Pairs[i].Key))
require.Equal(t, tt.expectedLog, res, string(kvPairs[i].Key)) require.Equal(t, tt.expectedLog, res, string(kvPairs.Pairs[i].Key))
} }
}) })
} }

View File

@ -29,17 +29,19 @@ func TestDecodeStore(t *testing.T) {
} }
kvPairs := kv.Pairs{ kvPairs := kv.Pairs{
kv.Pair{ Pairs: []kv.Pair{
Key: host.KeyClientConnections(connection.ClientID), {
Value: cdc.MustMarshalBinaryBare(&paths), Key: host.KeyClientConnections(connection.ClientID),
}, Value: cdc.MustMarshalBinaryBare(&paths),
kv.Pair{ },
Key: host.KeyConnection(connectionID), {
Value: cdc.MustMarshalBinaryBare(&connection), Key: host.KeyConnection(connectionID),
}, Value: cdc.MustMarshalBinaryBare(&connection),
kv.Pair{ },
Key: []byte{0x99}, {
Value: []byte{0x99}, Key: []byte{0x99},
Value: []byte{0x99},
},
}, },
} }
tests := []struct { tests := []struct {
@ -54,13 +56,13 @@ func TestDecodeStore(t *testing.T) {
for i, tt := range tests { for i, tt := range tests {
i, tt := i, tt i, tt := i, tt
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
res, found := simulation.NewDecodeStore(cdc, kvPairs[i], kvPairs[i]) res, found := simulation.NewDecodeStore(cdc, kvPairs.Pairs[i], kvPairs.Pairs[i])
if i == len(tests)-1 { if i == len(tests)-1 {
require.False(t, found, string(kvPairs[i].Key)) require.False(t, found, string(kvPairs.Pairs[i].Key))
require.Empty(t, res, string(kvPairs[i].Key)) require.Empty(t, res, string(kvPairs.Pairs[i].Key))
} else { } else {
require.True(t, found, string(kvPairs[i].Key)) require.True(t, found, string(kvPairs.Pairs[i].Key))
require.Equal(t, tt.expectedLog, res, string(kvPairs[i].Key)) require.Equal(t, tt.expectedLog, res, string(kvPairs.Pairs[i].Key))
} }
}) })
} }

View File

@ -29,33 +29,35 @@ func TestDecodeStore(t *testing.T) {
bz := []byte{0x1, 0x2, 0x3} bz := []byte{0x1, 0x2, 0x3}
kvPairs := kv.Pairs{ kvPairs := kv.Pairs{
kv.Pair{ Pairs: []kv.Pair{
Key: host.KeyChannel(portID, channelID), {
Value: cdc.MustMarshalBinaryBare(&channel), Key: host.KeyChannel(portID, channelID),
}, Value: cdc.MustMarshalBinaryBare(&channel),
kv.Pair{ },
Key: host.KeyNextSequenceSend(portID, channelID), {
Value: sdk.Uint64ToBigEndian(1), Key: host.KeyNextSequenceSend(portID, channelID),
}, Value: sdk.Uint64ToBigEndian(1),
kv.Pair{ },
Key: host.KeyNextSequenceRecv(portID, channelID), {
Value: sdk.Uint64ToBigEndian(1), Key: host.KeyNextSequenceRecv(portID, channelID),
}, Value: sdk.Uint64ToBigEndian(1),
kv.Pair{ },
Key: host.KeyNextSequenceAck(portID, channelID), {
Value: sdk.Uint64ToBigEndian(1), Key: host.KeyNextSequenceAck(portID, channelID),
}, Value: sdk.Uint64ToBigEndian(1),
kv.Pair{ },
Key: host.KeyPacketCommitment(portID, channelID, 1), {
Value: bz, Key: host.KeyPacketCommitment(portID, channelID, 1),
}, Value: bz,
kv.Pair{ },
Key: host.KeyPacketAcknowledgement(portID, channelID, 1), {
Value: bz, Key: host.KeyPacketAcknowledgement(portID, channelID, 1),
}, Value: bz,
kv.Pair{ },
Key: []byte{0x99}, {
Value: []byte{0x99}, Key: []byte{0x99},
Value: []byte{0x99},
},
}, },
} }
tests := []struct { tests := []struct {
@ -74,13 +76,13 @@ func TestDecodeStore(t *testing.T) {
for i, tt := range tests { for i, tt := range tests {
i, tt := i, tt i, tt := i, tt
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
res, found := simulation.NewDecodeStore(cdc, kvPairs[i], kvPairs[i]) res, found := simulation.NewDecodeStore(cdc, kvPairs.Pairs[i], kvPairs.Pairs[i])
if i == len(tests)-1 { if i == len(tests)-1 {
require.False(t, found, string(kvPairs[i].Key)) require.False(t, found, string(kvPairs.Pairs[i].Key))
require.Empty(t, res, string(kvPairs[i].Key)) require.Empty(t, res, string(kvPairs.Pairs[i].Key))
} else { } else {
require.True(t, found, string(kvPairs[i].Key)) require.True(t, found, string(kvPairs.Pairs[i].Key))
require.Equal(t, tt.expectedLog, res, string(kvPairs[i].Key)) require.Equal(t, tt.expectedLog, res, string(kvPairs.Pairs[i].Key))
} }
}) })
} }

View File

@ -37,21 +37,23 @@ func TestDecodeStore(t *testing.T) {
} }
kvPairs := kv.Pairs{ kvPairs := kv.Pairs{
kv.Pair{ Pairs: []kv.Pair{
Key: host.FullKeyClientPath(clientID, host.KeyClientState()), {
Value: app.IBCKeeper.ClientKeeper.MustMarshalClientState(clientState), Key: host.FullKeyClientPath(clientID, host.KeyClientState()),
}, Value: app.IBCKeeper.ClientKeeper.MustMarshalClientState(clientState),
kv.Pair{ },
Key: host.KeyConnection(connectionID), {
Value: app.IBCKeeper.Codec().MustMarshalBinaryBare(&connection), Key: host.KeyConnection(connectionID),
}, Value: app.IBCKeeper.Codec().MustMarshalBinaryBare(&connection),
kv.Pair{ },
Key: host.KeyChannel(portID, channelID), {
Value: app.IBCKeeper.Codec().MustMarshalBinaryBare(&channel), Key: host.KeyChannel(portID, channelID),
}, Value: app.IBCKeeper.Codec().MustMarshalBinaryBare(&channel),
kv.Pair{ },
Key: []byte{0x99}, {
Value: []byte{0x99}, Key: []byte{0x99},
Value: []byte{0x99},
},
}, },
} }
tests := []struct { tests := []struct {
@ -68,9 +70,9 @@ func TestDecodeStore(t *testing.T) {
i, tt := i, tt i, tt := i, tt
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
if i == len(tests)-1 { if i == len(tests)-1 {
require.Panics(t, func() { dec(kvPairs[i], kvPairs[i]) }, tt.name) require.Panics(t, func() { dec(kvPairs.Pairs[i], kvPairs.Pairs[i]) }, tt.name)
} else { } else {
require.Equal(t, tt.expectedLog, dec(kvPairs[i], kvPairs[i]), tt.name) require.Equal(t, tt.expectedLog, dec(kvPairs.Pairs[i], kvPairs.Pairs[i]), tt.name)
} }
}) })
} }

View File

@ -20,8 +20,10 @@ func TestDecodeStore(t *testing.T) {
minter := types.NewMinter(sdk.OneDec(), sdk.NewDec(15)) minter := types.NewMinter(sdk.OneDec(), sdk.NewDec(15))
kvPairs := kv.Pairs{ kvPairs := kv.Pairs{
kv.Pair{Key: types.MinterKey, Value: cdc.MustMarshalBinaryBare(&minter)}, Pairs: []kv.Pair{
kv.Pair{Key: []byte{0x99}, Value: []byte{0x99}}, {Key: types.MinterKey, Value: cdc.MustMarshalBinaryBare(&minter)},
{Key: []byte{0x99}, Value: []byte{0x99}},
},
} }
tests := []struct { tests := []struct {
name string name string
@ -36,9 +38,9 @@ func TestDecodeStore(t *testing.T) {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
switch i { switch i {
case len(tests) - 1: case len(tests) - 1:
require.Panics(t, func() { dec(kvPairs[i], kvPairs[i]) }, tt.name) require.Panics(t, func() { dec(kvPairs.Pairs[i], kvPairs.Pairs[i]) }, tt.name)
default: default:
require.Equal(t, tt.expectedLog, dec(kvPairs[i], kvPairs[i]), tt.name) require.Equal(t, tt.expectedLog, dec(kvPairs.Pairs[i], kvPairs.Pairs[i]), tt.name)
} }
}) })
} }

View File

@ -33,10 +33,12 @@ func TestDecodeStore(t *testing.T) {
missed := gogotypes.BoolValue{Value: true} missed := gogotypes.BoolValue{Value: true}
kvPairs := kv.Pairs{ kvPairs := kv.Pairs{
kv.Pair{Key: types.ValidatorSigningInfoKey(consAddr1), Value: cdc.MustMarshalBinaryBare(&info)}, Pairs: []kv.Pair{
kv.Pair{Key: types.ValidatorMissedBlockBitArrayKey(consAddr1, 6), Value: cdc.MustMarshalBinaryBare(&missed)}, {Key: types.ValidatorSigningInfoKey(consAddr1), Value: cdc.MustMarshalBinaryBare(&info)},
kv.Pair{Key: types.AddrPubkeyRelationKey(delAddr1), Value: cdc.MustMarshalBinaryBare(&gogotypes.StringValue{Value: bechPK})}, {Key: types.ValidatorMissedBlockBitArrayKey(consAddr1, 6), Value: cdc.MustMarshalBinaryBare(&missed)},
kv.Pair{Key: []byte{0x99}, Value: []byte{0x99}}, {Key: types.AddrPubkeyRelationKey(delAddr1), Value: cdc.MustMarshalBinaryBare(&gogotypes.StringValue{Value: bechPK})},
{Key: []byte{0x99}, Value: []byte{0x99}},
},
} }
tests := []struct { tests := []struct {
@ -53,9 +55,9 @@ func TestDecodeStore(t *testing.T) {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
switch i { switch i {
case len(tests) - 1: case len(tests) - 1:
require.Panics(t, func() { dec(kvPairs[i], kvPairs[i]) }, tt.name) require.Panics(t, func() { dec(kvPairs.Pairs[i], kvPairs.Pairs[i]) }, tt.name)
default: default:
require.Equal(t, tt.expectedLog, dec(kvPairs[i], kvPairs[i]), tt.name) require.Equal(t, tt.expectedLog, dec(kvPairs.Pairs[i], kvPairs.Pairs[i]), tt.name)
} }
}) })
} }

View File

@ -43,13 +43,15 @@ func TestDecodeStore(t *testing.T) {
red := types.NewRedelegation(delAddr1, valAddr1, valAddr1, 12, bondTime, sdk.OneInt(), sdk.OneDec()) red := types.NewRedelegation(delAddr1, valAddr1, valAddr1, 12, bondTime, sdk.OneInt(), sdk.OneDec())
kvPairs := kv.Pairs{ kvPairs := kv.Pairs{
kv.Pair{Key: types.LastTotalPowerKey, Value: cdc.MustMarshalBinaryBare(&sdk.IntProto{Int: sdk.OneInt()})}, Pairs: []kv.Pair{
kv.Pair{Key: types.GetValidatorKey(valAddr1), Value: cdc.MustMarshalBinaryBare(&val)}, {Key: types.LastTotalPowerKey, Value: cdc.MustMarshalBinaryBare(&sdk.IntProto{Int: sdk.OneInt()})},
kv.Pair{Key: types.LastValidatorPowerKey, Value: valAddr1.Bytes()}, {Key: types.GetValidatorKey(valAddr1), Value: cdc.MustMarshalBinaryBare(&val)},
kv.Pair{Key: types.GetDelegationKey(delAddr1, valAddr1), Value: cdc.MustMarshalBinaryBare(&del)}, {Key: types.LastValidatorPowerKey, Value: valAddr1.Bytes()},
kv.Pair{Key: types.GetUBDKey(delAddr1, valAddr1), Value: cdc.MustMarshalBinaryBare(&ubd)}, {Key: types.GetDelegationKey(delAddr1, valAddr1), Value: cdc.MustMarshalBinaryBare(&del)},
kv.Pair{Key: types.GetREDKey(delAddr1, valAddr1, valAddr1), Value: cdc.MustMarshalBinaryBare(&red)}, {Key: types.GetUBDKey(delAddr1, valAddr1), Value: cdc.MustMarshalBinaryBare(&ubd)},
kv.Pair{Key: []byte{0x99}, Value: []byte{0x99}}, {Key: types.GetREDKey(delAddr1, valAddr1, valAddr1), Value: cdc.MustMarshalBinaryBare(&red)},
{Key: []byte{0x99}, Value: []byte{0x99}},
},
} }
tests := []struct { tests := []struct {
@ -69,9 +71,9 @@ func TestDecodeStore(t *testing.T) {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
switch i { switch i {
case len(tests) - 1: case len(tests) - 1:
require.Panics(t, func() { dec(kvPairs[i], kvPairs[i]) }, tt.name) require.Panics(t, func() { dec(kvPairs.Pairs[i], kvPairs.Pairs[i]) }, tt.name)
default: default:
require.Equal(t, tt.expectedLog, dec(kvPairs[i], kvPairs[i]), tt.name) require.Equal(t, tt.expectedLog, dec(kvPairs.Pairs[i], kvPairs.Pairs[i]), tt.name)
} }
}) })
} }