Browse Source

e2e: integrate light clients (bp #6196)

integrate light clients (#6196)
fix e2e app test (#6223)
fix light client generator (#6236)
pull/6274/head
mergify[bot] 3 years ago
committed by GitHub
parent
commit
b2f01448be
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 243 additions and 59 deletions
  1. +2
    -2
      .github/workflows/lint.yaml
  2. +4
    -31
      cmd/tendermint/commands/light.go
  3. +5
    -1
      light/client.go
  4. +24
    -0
      light/proxy/proxy.go
  5. +22
    -0
      light/rpc/client.go
  6. +1
    -1
      test/e2e/app/app.go
  7. +1
    -0
      test/e2e/app/config.go
  8. +88
    -1
      test/e2e/app/main.go
  9. +29
    -4
      test/e2e/generator/generate.go
  10. +11
    -0
      test/e2e/networks/ci.toml
  11. +1
    -1
      test/e2e/networks/simple.toml
  12. +5
    -4
      test/e2e/pkg/manifest.go
  13. +11
    -2
      test/e2e/pkg/testnet.go
  14. +18
    -6
      test/e2e/runner/setup.go
  15. +1
    -1
      test/e2e/runner/start.go
  16. +20
    -5
      test/e2e/tests/app_test.go

+ 2
- 2
.github/workflows/lint.yaml View File

@ -20,10 +20,10 @@ jobs:
**/**.go
go.mod
go.sum
- uses: golangci/golangci-lint-action@v2.2.1
- uses: golangci/golangci-lint-action@v2.5.1
with:
# Required: the version of golangci-lint is required and must be specified without patch version: we always use the latest patch version.
version: v1.31
version: v1.38
args: --timeout 10m
github-token: ${{ secrets.github_token }}
if: env.GIT_DIFF

+ 4
- 31
cmd/tendermint/commands/light.go View File

@ -8,7 +8,6 @@ import (
"net/http"
"os"
"path/filepath"
"regexp"
"strings"
"time"
@ -16,7 +15,6 @@ import (
dbm "github.com/tendermint/tm-db"
"github.com/tendermint/tendermint/crypto/merkle"
"github.com/tendermint/tendermint/libs/log"
tmmath "github.com/tendermint/tendermint/libs/math"
tmos "github.com/tendermint/tendermint/libs/os"
@ -24,7 +22,6 @@ import (
lproxy "github.com/tendermint/tendermint/light/proxy"
lrpc "github.com/tendermint/tendermint/light/rpc"
dbs "github.com/tendermint/tendermint/light/store/db"
rpchttp "github.com/tendermint/tendermint/rpc/client/http"
rpcserver "github.com/tendermint/tendermint/rpc/jsonrpc/server"
)
@ -204,11 +201,6 @@ func runProxy(cmd *cobra.Command, args []string) error {
return err
}
rpcClient, err := rpchttp.New(primaryAddr, "/websocket")
if err != nil {
return fmt.Errorf("http client for %s: %w", primaryAddr, err)
}
cfg := rpcserver.DefaultConfig()
cfg.MaxBodyBytes = config.RPC.MaxBodyBytes
cfg.MaxHeaderBytes = config.RPC.MaxHeaderBytes
@ -220,12 +212,11 @@ func runProxy(cmd *cobra.Command, args []string) error {
cfg.WriteTimeout = config.RPC.TimeoutBroadcastTxCommit + 1*time.Second
}
p := lproxy.Proxy{
Addr: listenAddr,
Config: cfg,
Client: lrpc.NewClient(rpcClient, c, lrpc.KeyPathFn(defaultMerkleKeyPathFn())),
Logger: logger,
p, err := lproxy.NewProxy(c, listenAddr, primaryAddr, cfg, logger, lrpc.KeyPathFn(lrpc.DefaultMerkleKeyPathFn()))
if err != nil {
return err
}
// Stop upon receiving SIGTERM or CTRL-C.
tmos.TrapSignal(logger, func() {
p.Listener.Close()
@ -264,21 +255,3 @@ func saveProviders(db dbm.DB, primaryAddr, witnessesAddrs string) error {
}
return nil
}
func defaultMerkleKeyPathFn() lrpc.KeyPathFunc {
// regexp for extracting store name from /abci_query path
storeNameRegexp := regexp.MustCompile(`\/store\/(.+)\/key`)
return func(path string, key []byte) (merkle.KeyPath, error) {
matches := storeNameRegexp.FindStringSubmatch(path)
if len(matches) != 2 {
return nil, fmt.Errorf("can't find store name in %s using %s", path, storeNameRegexp)
}
storeName := matches[1]
kp := merkle.KeyPath{}
kp = kp.AppendKey([]byte(storeName), merkle.KeyEncodingURL)
kp = kp.AppendKey(key, merkle.KeyEncodingURL)
return kp, nil
}
}

+ 5
- 1
light/client.go View File

@ -891,6 +891,8 @@ func (c *Client) cleanupAfter(height int64) error {
}
func (c *Client) updateTrustedLightBlock(l *types.LightBlock) error {
c.logger.Debug("updating trusted light block", "light_block", l)
if err := c.trustedStore.SaveLightBlock(l); err != nil {
return fmt.Errorf("failed to save trusted header: %w", err)
}
@ -1033,10 +1035,12 @@ and remove witness. Otherwise, use the different primary`, e.WitnessIndex), "wit
// respond or couldn't find the block, then we ignore it and move on to
// the next witness.
if _, ok := e.Reason.(provider.ErrBadLightBlock); ok {
c.logger.Info("Witness sent us invalid header / vals -> removing it", "witness", c.witnesses[e.WitnessIndex])
c.logger.Info("Witness sent us invalid header / vals -> removing it",
"witness", c.witnesses[e.WitnessIndex], "err", err)
witnessesToRemove = append(witnessesToRemove, e.WitnessIndex)
}
}
}
// we need to make sure that we remove witnesses by index in the reverse


+ 24
- 0
light/proxy/proxy.go View File

@ -8,7 +8,9 @@ import (
"github.com/tendermint/tendermint/libs/log"
tmpubsub "github.com/tendermint/tendermint/libs/pubsub"
"github.com/tendermint/tendermint/light"
lrpc "github.com/tendermint/tendermint/light/rpc"
rpchttp "github.com/tendermint/tendermint/rpc/client/http"
rpcserver "github.com/tendermint/tendermint/rpc/jsonrpc/server"
)
@ -21,6 +23,28 @@ type Proxy struct {
Listener net.Listener
}
// NewProxy creates the struct used to run an HTTP server for serving light
// client rpc requests.
func NewProxy(
lightClient *light.Client,
listenAddr, providerAddr string,
config *rpcserver.Config,
logger log.Logger,
opts ...lrpc.Option,
) (*Proxy, error) {
rpcClient, err := rpchttp.NewWithTimeout(providerAddr, "/websocket", uint(config.WriteTimeout.Seconds()))
if err != nil {
return nil, fmt.Errorf("failed to create http client for %s: %w", providerAddr, err)
}
return &Proxy{
Addr: listenAddr,
Config: config,
Client: lrpc.NewClient(rpcClient, lightClient, opts...),
Logger: logger,
}, nil
}
// ListenAndServe configures the rpcserver.WebsocketManager, sets up the RPC
// routes to proxy via Client, and starts up an HTTP server on the TCP network
// address p.Addr.


+ 22
- 0
light/rpc/client.go View File

@ -5,6 +5,7 @@ import (
"context"
"errors"
"fmt"
"regexp"
"time"
"github.com/gogo/protobuf/proto"
@ -61,6 +62,27 @@ func KeyPathFn(fn KeyPathFunc) Option {
}
}
// DefaultMerkleKeyPathFn creates a function used to generate merkle key paths
// from a path string and a key. This is the default used by the cosmos SDK.
// This merkle key paths are required when verifying /abci_query calls
func DefaultMerkleKeyPathFn() KeyPathFunc {
// regexp for extracting store name from /abci_query path
storeNameRegexp := regexp.MustCompile(`\/store\/(.+)\/key`)
return func(path string, key []byte) (merkle.KeyPath, error) {
matches := storeNameRegexp.FindStringSubmatch(path)
if len(matches) != 2 {
return nil, fmt.Errorf("can't find store name in %s using %s", path, storeNameRegexp)
}
storeName := matches[1]
kp := merkle.KeyPath{}
kp = kp.AppendKey([]byte(storeName), merkle.KeyEncodingURL)
kp = kp.AppendKey(key, merkle.KeyEncodingURL)
return kp, nil
}
}
// NewClient returns a new client.
func NewClient(next rpcclient.Client, lc LightClient, opts ...Option) *Client {
c := &Client{


+ 1
- 1
test/e2e/app/app.go View File

@ -117,7 +117,7 @@ func (app *Application) Commit() abci.ResponseCommit {
if err != nil {
panic(err)
}
logger.Info("Created state sync snapshot", "height", snapshot.Height)
app.logger.Info("Created state sync snapshot", "height", snapshot.Height)
}
retainHeight := int64(0)
if app.cfg.RetainBlocks > 0 {


+ 1
- 0
test/e2e/app/config.go View File

@ -14,6 +14,7 @@ type Config struct {
Listen string
Protocol string
Dir string
Mode string `toml:"mode"`
PersistInterval uint64 `toml:"persist_interval"`
SnapshotInterval uint64 `toml:"snapshot_interval"`
RetainBlocks uint64 `toml:"retain_blocks"`


+ 88
- 1
test/e2e/app/main.go View File

@ -1,11 +1,14 @@
package main
import (
"context"
"errors"
"fmt"
"net/http"
"os"
"path/filepath"
"strconv"
"strings"
"time"
"github.com/spf13/viper"
@ -16,10 +19,16 @@ import (
tmflags "github.com/tendermint/tendermint/libs/cli/flags"
"github.com/tendermint/tendermint/libs/log"
tmnet "github.com/tendermint/tendermint/libs/net"
"github.com/tendermint/tendermint/light"
lproxy "github.com/tendermint/tendermint/light/proxy"
lrpc "github.com/tendermint/tendermint/light/rpc"
dbs "github.com/tendermint/tendermint/light/store/db"
"github.com/tendermint/tendermint/node"
"github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/privval"
"github.com/tendermint/tendermint/proxy"
rpcserver "github.com/tendermint/tendermint/rpc/jsonrpc/server"
e2e "github.com/tendermint/tendermint/test/e2e/pkg"
mcs "github.com/tendermint/tendermint/test/maverick/consensus"
maverick "github.com/tendermint/tendermint/test/maverick/node"
)
@ -66,7 +75,11 @@ func run(configFile string) error {
err = startApp(cfg)
case "builtin":
if len(cfg.Misbehaviors) == 0 {
err = startNode(cfg)
if cfg.Mode == string(e2e.ModeLight) {
err = startLightClient(cfg)
} else {
err = startNode(cfg)
}
} else {
err = startMaverick(cfg)
}
@ -131,6 +144,64 @@ func startNode(cfg *Config) error {
return n.Start()
}
func startLightClient(cfg *Config) error {
tmcfg, nodeLogger, _, err := setupNode()
if err != nil {
return err
}
dbContext := &node.DBContext{ID: "light", Config: tmcfg}
lightDB, err := node.DefaultDBProvider(dbContext)
if err != nil {
return err
}
providers := rpcEndpoints(tmcfg.P2P.PersistentPeers)
c, err := light.NewHTTPClient(
context.Background(),
cfg.ChainID,
light.TrustOptions{
Period: tmcfg.StateSync.TrustPeriod,
Height: tmcfg.StateSync.TrustHeight,
Hash: tmcfg.StateSync.TrustHashBytes(),
},
providers[0],
providers[1:],
dbs.New(lightDB, "light"),
light.Logger(nodeLogger),
)
if err != nil {
return err
}
rpccfg := rpcserver.DefaultConfig()
rpccfg.MaxBodyBytes = tmcfg.RPC.MaxBodyBytes
rpccfg.MaxHeaderBytes = tmcfg.RPC.MaxHeaderBytes
rpccfg.MaxOpenConnections = tmcfg.RPC.MaxOpenConnections
// If necessary adjust global WriteTimeout to ensure it's greater than
// TimeoutBroadcastTxCommit.
// See https://github.com/tendermint/tendermint/issues/3435
if rpccfg.WriteTimeout <= tmcfg.RPC.TimeoutBroadcastTxCommit {
rpccfg.WriteTimeout = tmcfg.RPC.TimeoutBroadcastTxCommit + 1*time.Second
}
p, err := lproxy.NewProxy(c, tmcfg.RPC.ListenAddress, providers[0], rpccfg, nodeLogger,
lrpc.KeyPathFn(lrpc.DefaultMerkleKeyPathFn()))
if err != nil {
return err
}
logger.Info("Starting proxy...", "laddr", tmcfg.RPC.ListenAddress)
if err := p.ListenAndServe(); err != http.ErrServerClosed {
// Error starting or closing listener:
logger.Error("proxy ListenAndServe", "err", err)
}
return nil
}
// FIXME: Temporarily disconnected maverick until it is redesigned
// startMaverick starts a Maverick node that runs the application directly. It assumes the Tendermint
// configuration is in $TMHOME/config/tendermint.toml.
func startMaverick(cfg *Config) error {
@ -238,3 +309,19 @@ func setupNode() (*config.Config, log.Logger, *p2p.NodeKey, error) {
return tmcfg, nodeLogger, nodeKey, nil
}
// rpcEndpoints takes a list of persistent peers and splits them into a list of rpc endpoints
// using 26657 as the port number
func rpcEndpoints(peers string) []string {
arr := strings.Split(peers, ",")
endpoints := make([]string, len(arr))
for i, v := range arr {
urlString := strings.SplitAfter(v, "@")[1]
hostName := strings.Split(urlString, ":26656")[0]
// use RPC port instead
port := 26657
rpcEndpoint := "http://" + hostName + ":" + fmt.Sprint(port)
endpoints[i] = rpcEndpoint
}
return endpoints
}

+ 29
- 4
test/e2e/generator/generate.go View File

@ -74,7 +74,7 @@ func generateTestnet(r *rand.Rand, opt map[string]interface{}) (e2e.Manifest, er
Nodes: map[string]*e2e.ManifestNode{},
}
var numSeeds, numValidators, numFulls int
var numSeeds, numValidators, numFulls, numLightClients int
switch opt["topology"].(string) {
case "single":
numValidators = 1
@ -82,7 +82,8 @@ func generateTestnet(r *rand.Rand, opt map[string]interface{}) (e2e.Manifest, er
numValidators = 4
case "large":
// FIXME Networks are kept small since large ones use too much CPU.
numSeeds = r.Intn(4)
numSeeds = r.Intn(3)
numLightClients = r.Intn(3)
numValidators = 4 + r.Intn(7)
numFulls = r.Intn(5)
default:
@ -143,11 +144,16 @@ func generateTestnet(r *rand.Rand, opt map[string]interface{}) (e2e.Manifest, er
// We now set up peer discovery for nodes. Seed nodes are fully meshed with
// each other, while non-seed nodes either use a set of random seeds or a
// set of random peers that start before themselves.
var seedNames, peerNames []string
var seedNames, peerNames, lightProviders []string
for name, node := range manifest.Nodes {
if node.Mode == string(e2e.ModeSeed) {
seedNames = append(seedNames, name)
} else {
// if the full node or validator is an ideal candidate, it is added as a light provider.
// There are at least two archive nodes so there should be at least two ideal candidates
if (node.StartAt == 0 || node.StartAt == manifest.InitialHeight) && node.RetainBlocks == 0 {
lightProviders = append(lightProviders, name)
}
peerNames = append(peerNames, name)
}
}
@ -179,6 +185,14 @@ func generateTestnet(r *rand.Rand, opt map[string]interface{}) (e2e.Manifest, er
}
}
// lastly, set up the light clients
for i := 1; i <= numLightClients; i++ {
startAt := manifest.InitialHeight + 5
manifest.Nodes[fmt.Sprintf("light%02d", i)] = generateLightNode(
r, startAt+(5*int64(i)), lightProviders,
)
}
return manifest, nil
}
@ -210,7 +224,7 @@ func generateNode(
node.SnapshotInterval = 3
}
if node.Mode == "validator" {
if node.Mode == string(e2e.ModeValidator) {
misbehaveAt := startAt + 5 + int64(r.Intn(10))
if startAt == 0 {
misbehaveAt += initialHeight - 1
@ -245,6 +259,17 @@ func generateNode(
return &node
}
func generateLightNode(r *rand.Rand, startAt int64, providers []string) *e2e.ManifestNode {
return &e2e.ManifestNode{
Mode: string(e2e.ModeLight),
StartAt: startAt,
Database: nodeDatabases.Choose(r).(string),
ABCIProtocol: "builtin",
PersistInterval: ptrUint64(0),
PersistentPeers: providers,
}
}
func ptrUint64(i uint64) *uint64 {
return &i
}


+ 11
- 0
test/e2e/networks/ci.toml View File

@ -1,6 +1,7 @@
# This testnet is run by CI, and attempts to cover a broad range of
# functionality with a single network.
ipv6 = true
initial_height = 1000
initial_state = { initial01 = "a", initial02 = "b", initial03 = "c" }
@ -89,3 +90,13 @@ fast_sync = "v0"
state_sync = true
seeds = ["seed01"]
perturb = ["restart"]
[node.light01]
mode= "light"
start_at= 1005
persistent_peers = ["validator01", "validator02", "validator03"]
[node.light02]
mode= "light"
start_at= 1015
persistent_peers = ["validator04", "full01", "validator05"]

+ 1
- 1
test/e2e/networks/simple.toml View File

@ -2,4 +2,4 @@
[node.validator02]
[node.validator03]
[node.validator04]

+ 5
- 4
test/e2e/pkg/manifest.go View File

@ -54,9 +54,9 @@ type Manifest struct {
// ManifestNode represents a node in a testnet manifest.
type ManifestNode struct {
// Mode specifies the type of node: "validator", "full", or "seed". Defaults to
// "validator". Full nodes do not get a signing key (a dummy key is generated),
// and seed nodes run in seed mode with the PEX reactor enabled.
// Mode specifies the type of node: "validator", "full", "light" or "seed".
// Defaults to "validator". Full nodes do not get a signing key (a dummy key
// is generated), and seed nodes run in seed mode with the PEX reactor enabled.
Mode string `toml:"mode"`
// Seeds is the list of node names to use as P2P seed nodes. Defaults to none.
@ -64,7 +64,8 @@ type ManifestNode struct {
// PersistentPeers is a list of node names to maintain persistent P2P
// connections to. If neither seeds nor persistent peers are specified,
// this defaults to all other nodes in the network.
// this defaults to all other nodes in the network. For light clients,
// this relates to the providers the light client is connected to.
PersistentPeers []string `toml:"persistent_peers"`
// Database specifies the database backend: "goleveldb", "cleveldb",


+ 11
- 2
test/e2e/pkg/testnet.go View File

@ -33,6 +33,7 @@ type Perturbation string
const (
ModeValidator Mode = "validator"
ModeFull Mode = "full"
ModeLight Mode = "light"
ModeSeed Mode = "seed"
ProtocolBuiltin Protocol = "builtin"
@ -144,7 +145,7 @@ func LoadTestnet(file string) (*Testnet, error) {
ProxyPort: proxyPortGen.Next(),
Mode: ModeValidator,
Database: "goleveldb",
ABCIProtocol: ProtocolUNIX,
ABCIProtocol: ProtocolBuiltin,
PrivvalProtocol: ProtocolFile,
StartAt: nodeManifest.StartAt,
FastSync: nodeManifest.FastSync,
@ -311,6 +312,9 @@ func (n Node) Validate(testnet Testnet) error {
default:
return fmt.Errorf("invalid ABCI protocol setting %q", n.ABCIProtocol)
}
if n.Mode == ModeLight && n.ABCIProtocol != ProtocolBuiltin {
return errors.New("light client must use builtin protocol")
}
switch n.PrivvalProtocol {
case ProtocolFile, ProtocolUNIX, ProtocolTCP:
default:
@ -385,7 +389,7 @@ func (t Testnet) LookupNode(name string) *Node {
func (t Testnet) ArchiveNodes() []*Node {
nodes := []*Node{}
for _, node := range t.Nodes {
if node.Mode != ModeSeed && node.StartAt == 0 && node.RetainBlocks == 0 {
if !node.Stateless() && node.StartAt == 0 && node.RetainBlocks == 0 {
nodes = append(nodes, node)
}
}
@ -459,6 +463,11 @@ func (n Node) Client() (*rpchttp.HTTP, error) {
return rpchttp.New(fmt.Sprintf("http://127.0.0.1:%v", n.ProxyPort), "/websocket")
}
// Stateless returns true if the node is either a seed node or a light node
func (n Node) Stateless() bool {
return n.Mode == ModeLight || n.Mode == ModeSeed
}
// keyGenerator generates pseudorandom Ed25519 keys based on a seed.
type keyGenerator struct {
random *rand.Rand


+ 18
- 6
test/e2e/runner/setup.go View File

@ -64,23 +64,23 @@ func Setup(testnet *e2e.Testnet) error {
for _, node := range testnet.Nodes {
nodeDir := filepath.Join(testnet.Dir, node.Name)
dirs := []string{
filepath.Join(nodeDir, "config"),
filepath.Join(nodeDir, "data"),
filepath.Join(nodeDir, "data", "app"),
}
for _, dir := range dirs {
// light clients don't need an app directory
if node.Mode == e2e.ModeLight && strings.Contains(dir, "app") {
continue
}
err := os.MkdirAll(dir, 0755)
if err != nil {
return err
}
}
err = genesis.SaveAs(filepath.Join(nodeDir, "config", "genesis.json"))
if err != nil {
return err
}
cfg, err := MakeConfig(node)
if err != nil {
return err
@ -96,6 +96,16 @@ func Setup(testnet *e2e.Testnet) error {
return err
}
if node.Mode == e2e.ModeLight {
// stop early if a light client
continue
}
err = genesis.SaveAs(filepath.Join(nodeDir, "config", "genesis.json"))
if err != nil {
return err
}
err = (&p2p.NodeKey{PrivKey: node.NodeKey}).SaveAs(filepath.Join(nodeDir, "config", "node_key.json"))
if err != nil {
return err
@ -266,7 +276,7 @@ func MakeConfig(node *e2e.Node) (*config.Config, error) {
case e2e.ModeSeed:
cfg.P2P.SeedMode = true
cfg.P2P.PexReactor = true
case e2e.ModeFull:
case e2e.ModeFull, e2e.ModeLight:
// Don't need to do anything, since we're using a dummy privval key by default.
default:
return nil, fmt.Errorf("unexpected mode %q", node.Mode)
@ -315,6 +325,8 @@ func MakeAppConfig(node *e2e.Node) ([]byte, error) {
"chain_id": node.Testnet.Name,
"dir": "data/app",
"listen": AppAddressUNIX,
"mode": node.Mode,
"proxy_port": node.ProxyPort,
"protocol": "socket",
"persist_interval": node.PersistInterval,
"snapshot_interval": node.SnapshotInterval,


+ 1
- 1
test/e2e/runner/start.go View File

@ -58,7 +58,7 @@ func Start(testnet *e2e.Testnet) error {
// Update any state sync nodes with a trusted height and hash
for _, node := range nodeQueue {
if node.StateSync {
if node.StateSync || node.Mode == e2e.ModeLight {
err = UpdateConfigStateSync(node, block.Height, blockID.Hash.Bytes())
if err != nil {
return err


+ 20
- 5
test/e2e/tests/app_test.go View File

@ -16,7 +16,7 @@ import (
// Tests that any initial state given in genesis has made it into the app.
func TestApp_InitialState(t *testing.T) {
testNode(t, func(t *testing.T, node e2e.Node) {
if node.Mode == e2e.ModeSeed {
if node.Stateless() {
return
}
if len(node.Testnet.InitialState) == 0 {
@ -81,12 +81,27 @@ func TestApp_Tx(t *testing.T) {
value := fmt.Sprintf("%x", bz)
tx := types.Tx(fmt.Sprintf("%v=%v", key, value))
_, err = client.BroadcastTxCommit(ctx, tx)
resp, err := client.BroadcastTxCommit(ctx, tx)
require.NoError(t, err)
// wait for the tx to be persisted in the tx indexer
time.Sleep(500 * time.Millisecond)
hash := tx.Hash()
txResp, err := client.Tx(ctx, hash, false)
require.NoError(t, err)
assert.Equal(t, txResp.Tx, tx)
assert.Equal(t, txResp.Height, resp.Height)
resp, err := client.ABCIQuery(ctx, "", []byte(key))
// NOTE: we don't test abci query of the light client
if node.Mode == e2e.ModeLight {
return
}
abciResp, err := client.ABCIQuery(ctx, "", []byte(key))
require.NoError(t, err)
assert.Equal(t, key, string(resp.Response.Key))
assert.Equal(t, value, string(resp.Response.Value))
assert.Equal(t, key, string(abciResp.Response.Key))
assert.Equal(t, value, string(abciResp.Response.Value))
})
}

Loading…
Cancel
Save