Combine local and http into client package, unify tests with table-driven tests

This commit is contained in:
Ethan Frey 2017-02-23 14:16:59 +01:00
parent 202146e4ce
commit 931af6a072
9 changed files with 314 additions and 538 deletions

View File

@ -1,211 +0,0 @@
package http_test
import (
"encoding/json"
"strings"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
merkle "github.com/tendermint/go-merkle"
merktest "github.com/tendermint/merkleeyes/testutil"
"github.com/tendermint/tendermint/rpc/client/http"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpctest "github.com/tendermint/tendermint/rpc/test"
"github.com/tendermint/tendermint/types"
)
// GetClient gets a rpc client pointing to the test tendermint rpc
func GetClient() *http.Client {
rpcAddr := rpctest.GetConfig().GetString("rpc_laddr")
return http.New(rpcAddr, "/websocket")
}
// Make sure status is correct (we connect properly)
func TestStatus(t *testing.T) {
c := GetClient()
chainID := rpctest.GetConfig().GetString("chain_id")
status, err := c.Status()
require.Nil(t, err, "%+v", err)
assert.Equal(t, chainID, status.NodeInfo.Network)
}
// Make sure info is correct (we connect properly)
func TestInfo(t *testing.T) {
c := GetClient()
status, err := c.Status()
require.Nil(t, err, "%+v", err)
info, err := c.ABCIInfo()
require.Nil(t, err, "%+v", err)
assert.EqualValues(t, status.LatestBlockHeight, info.Response.LastBlockHeight)
assert.True(t, strings.HasPrefix(info.Response.Data, "size"))
}
func TestNetInfo(t *testing.T) {
c := GetClient()
netinfo, err := c.NetInfo()
require.Nil(t, err, "%+v", err)
assert.True(t, netinfo.Listening)
assert.Equal(t, 0, len(netinfo.Peers))
}
func TestDialSeeds(t *testing.T) {
c := GetClient()
// FIXME: fix server so it doesn't panic on invalid input
_, err := c.DialSeeds([]string{"12.34.56.78:12345"})
require.Nil(t, err, "%+v", err)
}
func TestGenesisAndValidators(t *testing.T) {
c := GetClient()
chainID := rpctest.GetConfig().GetString("chain_id")
// make sure this is the right genesis file
gen, err := c.Genesis()
require.Nil(t, err, "%+v", err)
assert.Equal(t, chainID, gen.Genesis.ChainID)
// get the genesis validator
require.Equal(t, 1, len(gen.Genesis.Validators))
gval := gen.Genesis.Validators[0]
// get the current validators
vals, err := c.Validators()
require.Nil(t, err, "%+v", err)
require.Equal(t, 1, len(vals.Validators))
val := vals.Validators[0]
// make sure the current set is also the genesis set
assert.Equal(t, gval.Amount, val.VotingPower)
assert.Equal(t, gval.PubKey, val.PubKey)
}
// Make some app checks
func TestAppCalls(t *testing.T) {
assert, require := assert.New(t), require.New(t)
c := GetClient()
// get an offset of height to avoid racing and guessing
s, err := c.Status()
require.Nil(err)
// sh is start height or status height
sh := s.LatestBlockHeight
// look for the future
_, err = c.Block(sh + 2)
assert.NotNil(err) // no block yet
// write something
k, v, tx := merktest.MakeTxKV()
_, err = c.BroadcastTxCommit(tx)
require.Nil(err, "%+v", err)
// wait before querying
time.Sleep(time.Second * 1)
qres, err := c.ABCIQuery("/key", k, false)
if assert.Nil(err) && assert.True(qres.Response.Code.IsOK()) {
data := qres.Response
// assert.Equal(k, data.GetKey()) // only returned for proofs
assert.Equal(v, data.GetValue())
}
// +/- 1 making my head hurt
h := int(qres.Response.Height) - 1
// and we can even check the block is added
block, err := c.Block(h)
require.Nil(err, "%+v", err)
appHash := block.BlockMeta.Header.AppHash
assert.True(len(appHash) > 0)
assert.EqualValues(h, block.BlockMeta.Header.Height)
// check blockchain info, now that we know there is info
// TODO: is this commented somewhere that they are returned
// in order of descending height???
info, err := c.BlockchainInfo(h-2, h)
require.Nil(err, "%+v", err)
assert.True(info.LastHeight > 2)
if assert.Equal(3, len(info.BlockMetas)) {
lastMeta := info.BlockMetas[0]
assert.EqualValues(h, lastMeta.Header.Height)
bMeta := block.BlockMeta
assert.Equal(bMeta.Header.AppHash, lastMeta.Header.AppHash)
assert.Equal(bMeta.BlockID, lastMeta.BlockID)
}
// and get the corresponding commit with the same apphash
commit, err := c.Commit(h)
require.Nil(err, "%+v", err)
cappHash := commit.Header.AppHash
assert.Equal(appHash, cappHash)
assert.NotNil(commit.Commit)
// compare the commits (note Commit(2) has commit from Block(3))
commit2, err := c.Commit(h - 1)
require.Nil(err, "%+v", err)
assert.Equal(block.Block.LastCommit, commit2.Commit)
// and we got a proof that works!
pres, err := c.ABCIQuery("/key", k, true)
if assert.Nil(err) && assert.True(pres.Response.Code.IsOK()) {
proof, err := merkle.ReadProof(pres.Response.GetProof())
if assert.Nil(err) {
key := pres.Response.GetKey()
value := pres.Response.GetValue()
assert.Equal(appHash, proof.RootHash)
valid := proof.Verify(key, value, appHash)
assert.True(valid)
}
}
}
func TestSubscriptions(t *testing.T) {
require := require.New(t)
c := GetClient()
err := c.StartWebsocket()
require.Nil(err)
defer c.StopWebsocket()
// subscribe to a transaction event
_, _, tx := merktest.MakeTxKV()
eventType := types.EventStringTx(types.Tx(tx))
c.Subscribe(eventType)
// set up a listener
r, e := c.GetEventChannels()
go func() {
// send a tx and wait for it to propogate
_, err = c.BroadcastTxCommit(tx)
require.Nil(err, string(tx))
}()
checkData := func(data []byte, kind byte) {
x := []interface{}{}
err := json.Unmarshal(data, &x)
require.Nil(err)
// gotta love wire's json format
require.EqualValues(kind, x[0])
}
res := <-r
checkData(res, ctypes.ResultTypeSubscribe)
// read one event, must be success
select {
case res := <-r:
checkData(res, ctypes.ResultTypeEvent)
// this is good.. let's get the data... ugh...
// result := new(ctypes.TMResult)
// wire.ReadJSON(result, res, &err)
// require.Nil(err, "%+v", err)
// event, ok := (*result).(*ctypes.ResultEvent)
// require.True(ok)
// assert.Equal("foo", event.Name)
// data, ok := event.Data.(types.EventDataTx)
// require.True(ok)
// assert.EqualValues(0, data.Code)
// assert.EqualValues(tx, data.Tx)
case err := <-e:
// this is a failure
require.Nil(err)
}
}

View File

@ -1,5 +1,16 @@
package client
import (
"encoding/json"
"github.com/pkg/errors"
"github.com/tendermint/go-rpc/client"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
"github.com/tendermint/tendermint/types"
)
/*
package http returns a Client implementation that communicates
HTTP is a Client implementation that communicates
with a tendermint node over json rpc and websockets.
This is the main implementation you probably want to use in
@ -7,21 +18,7 @@ production code. There are other implementations when calling
the tendermint node in-process (local), or when you want to mock
out the server for test code (mock).
*/
package http
import (
"encoding/json"
"github.com/pkg/errors"
"github.com/tendermint/go-rpc/client"
"github.com/tendermint/tendermint/rpc/client"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
"github.com/tendermint/tendermint/types"
)
// Client is a Client implementation that communicates over
// JSONRPC
type Client struct {
type HTTP struct {
remote string
endpoint string
rpc *rpcclient.ClientJSONRPC
@ -30,19 +27,19 @@ type Client struct {
// New takes a remote endpoint in the form tcp://<host>:<port>
// and the websocket path (which always seems to be "/websocket")
func New(remote, wsEndpoint string) *Client {
return &Client{
func NewHTTP(remote, wsEndpoint string) *HTTP {
return &HTTP{
rpc: rpcclient.NewClientJSONRPC(remote),
remote: remote,
endpoint: wsEndpoint,
}
}
func (c *Client) _assertIsClient() client.Client {
func (c *HTTP) _assertIsClient() Client {
return c
}
func (c *Client) Status() (*ctypes.ResultStatus, error) {
func (c *HTTP) Status() (*ctypes.ResultStatus, error) {
tmResult := new(ctypes.TMResult)
_, err := c.rpc.Call("status", []interface{}{}, tmResult)
if err != nil {
@ -52,7 +49,7 @@ func (c *Client) Status() (*ctypes.ResultStatus, error) {
return (*tmResult).(*ctypes.ResultStatus), nil
}
func (c *Client) ABCIInfo() (*ctypes.ResultABCIInfo, error) {
func (c *HTTP) ABCIInfo() (*ctypes.ResultABCIInfo, error) {
tmResult := new(ctypes.TMResult)
_, err := c.rpc.Call("abci_info", []interface{}{}, tmResult)
if err != nil {
@ -61,7 +58,7 @@ func (c *Client) ABCIInfo() (*ctypes.ResultABCIInfo, error) {
return (*tmResult).(*ctypes.ResultABCIInfo), nil
}
func (c *Client) ABCIQuery(path string, data []byte, prove bool) (*ctypes.ResultABCIQuery, error) {
func (c *HTTP) ABCIQuery(path string, data []byte, prove bool) (*ctypes.ResultABCIQuery, error) {
tmResult := new(ctypes.TMResult)
_, err := c.rpc.Call("abci_query", []interface{}{path, data, prove}, tmResult)
if err != nil {
@ -70,7 +67,7 @@ func (c *Client) ABCIQuery(path string, data []byte, prove bool) (*ctypes.Result
return (*tmResult).(*ctypes.ResultABCIQuery), nil
}
func (c *Client) BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
func (c *HTTP) BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
tmResult := new(ctypes.TMResult)
_, err := c.rpc.Call("broadcast_tx_commit", []interface{}{tx}, tmResult)
if err != nil {
@ -79,15 +76,15 @@ func (c *Client) BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTxCommit
return (*tmResult).(*ctypes.ResultBroadcastTxCommit), nil
}
func (c *Client) BroadcastTxAsync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
func (c *HTTP) BroadcastTxAsync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
return c.broadcastTX("broadcast_tx_async", tx)
}
func (c *Client) BroadcastTxSync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
func (c *HTTP) BroadcastTxSync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
return c.broadcastTX("broadcast_tx_sync", tx)
}
func (c *Client) broadcastTX(route string, tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
func (c *HTTP) broadcastTX(route string, tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
tmResult := new(ctypes.TMResult)
_, err := c.rpc.Call(route, []interface{}{tx}, tmResult)
if err != nil {
@ -96,7 +93,7 @@ func (c *Client) broadcastTX(route string, tx types.Tx) (*ctypes.ResultBroadcast
return (*tmResult).(*ctypes.ResultBroadcastTx), nil
}
func (c *Client) NetInfo() (*ctypes.ResultNetInfo, error) {
func (c *HTTP) NetInfo() (*ctypes.ResultNetInfo, error) {
tmResult := new(ctypes.TMResult)
_, err := c.rpc.Call("net_info", nil, tmResult)
if err != nil {
@ -105,7 +102,7 @@ func (c *Client) NetInfo() (*ctypes.ResultNetInfo, error) {
return (*tmResult).(*ctypes.ResultNetInfo), nil
}
func (c *Client) DialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) {
func (c *HTTP) DialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) {
tmResult := new(ctypes.TMResult)
// TODO: is this the correct way to marshall seeds?
_, err := c.rpc.Call("dial_seeds", []interface{}{seeds}, tmResult)
@ -115,7 +112,7 @@ func (c *Client) DialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) {
return (*tmResult).(*ctypes.ResultDialSeeds), nil
}
func (c *Client) BlockchainInfo(minHeight, maxHeight int) (*ctypes.ResultBlockchainInfo, error) {
func (c *HTTP) BlockchainInfo(minHeight, maxHeight int) (*ctypes.ResultBlockchainInfo, error) {
tmResult := new(ctypes.TMResult)
_, err := c.rpc.Call("blockchain", []interface{}{minHeight, maxHeight}, tmResult)
if err != nil {
@ -124,7 +121,7 @@ func (c *Client) BlockchainInfo(minHeight, maxHeight int) (*ctypes.ResultBlockch
return (*tmResult).(*ctypes.ResultBlockchainInfo), nil
}
func (c *Client) Genesis() (*ctypes.ResultGenesis, error) {
func (c *HTTP) Genesis() (*ctypes.ResultGenesis, error) {
tmResult := new(ctypes.TMResult)
_, err := c.rpc.Call("genesis", nil, tmResult)
if err != nil {
@ -133,7 +130,7 @@ func (c *Client) Genesis() (*ctypes.ResultGenesis, error) {
return (*tmResult).(*ctypes.ResultGenesis), nil
}
func (c *Client) Block(height int) (*ctypes.ResultBlock, error) {
func (c *HTTP) Block(height int) (*ctypes.ResultBlock, error) {
tmResult := new(ctypes.TMResult)
_, err := c.rpc.Call("block", []interface{}{height}, tmResult)
if err != nil {
@ -142,7 +139,7 @@ func (c *Client) Block(height int) (*ctypes.ResultBlock, error) {
return (*tmResult).(*ctypes.ResultBlock), nil
}
func (c *Client) Commit(height int) (*ctypes.ResultCommit, error) {
func (c *HTTP) Commit(height int) (*ctypes.ResultCommit, error) {
tmResult := new(ctypes.TMResult)
_, err := c.rpc.Call("commit", []interface{}{height}, tmResult)
if err != nil {
@ -151,7 +148,7 @@ func (c *Client) Commit(height int) (*ctypes.ResultCommit, error) {
return (*tmResult).(*ctypes.ResultCommit), nil
}
func (c *Client) Validators() (*ctypes.ResultValidators, error) {
func (c *HTTP) Validators() (*ctypes.ResultValidators, error) {
tmResult := new(ctypes.TMResult)
_, err := c.rpc.Call("validators", nil, tmResult)
if err != nil {
@ -164,7 +161,7 @@ func (c *Client) Validators() (*ctypes.ResultValidators, error) {
// StartWebsocket starts up a websocket and a listener goroutine
// if already started, do nothing
func (c *Client) StartWebsocket() error {
func (c *HTTP) StartWebsocket() error {
var err error
if c.ws == nil {
ws := rpcclient.NewWSClient(c.remote, c.endpoint)
@ -177,7 +174,7 @@ func (c *Client) StartWebsocket() error {
}
// StopWebsocket stops the websocket connection
func (c *Client) StopWebsocket() {
func (c *HTTP) StopWebsocket() {
if c.ws != nil {
c.ws.Stop()
c.ws = nil
@ -185,17 +182,17 @@ func (c *Client) StopWebsocket() {
}
// GetEventChannels returns the results and error channel from the websocket
func (c *Client) GetEventChannels() (chan json.RawMessage, chan error) {
func (c *HTTP) GetEventChannels() (chan json.RawMessage, chan error) {
if c.ws == nil {
return nil, nil
}
return c.ws.ResultsCh, c.ws.ErrorsCh
}
func (c *Client) Subscribe(event string) error {
func (c *HTTP) Subscribe(event string) error {
return errors.Wrap(c.ws.Subscribe(event), "Subscribe")
}
func (c *Client) Unsubscribe(event string) error {
func (c *HTTP) Unsubscribe(event string) error {
return errors.Wrap(c.ws.Unsubscribe(event), "Unsubscribe")
}

View File

@ -1,21 +1,21 @@
/*
package client provides a general purpose interface for connecting
package client provides a general purpose interface (Client) for connecting
to a tendermint node, as well as higher-level functionality.
The main implementation for production code is http, which connects
via http to the jsonrpc interface of the tendermint node.
The main implementation for production code is client.HTTP, which
connects via http to the jsonrpc interface of the tendermint node.
For connecting to a node running in the same process (eg. when
compiling the abci app in the same process), you can use the local
compiling the abci app in the same process), you can use the client.Local
implementation.
For mocking out server responses during testing to see behavior for
arbitrary return values, use the mock package.
In addition to the Client interface, which should be used externally
for maximum flexibility and testability, this package also provides
a wrapper that accepts any Client implementation and adds some
higher-level functionality.
for maximum flexibility and testability, and two implementations,
this package also provides helper functions that work on any Client
implementation.
*/
package client

View File

@ -1,45 +0,0 @@
package local_test
import (
"math/rand"
meapp "github.com/tendermint/merkleeyes/app"
wire "github.com/tendermint/go-wire"
)
// MakeTxKV returns a text transaction, allong with expected key, value pair
func MakeTxKV() ([]byte, []byte, []byte) {
k := RandAsciiBytes(8)
v := RandAsciiBytes(8)
return k, v, makeSet(k, v)
}
// blatently copied from merkleeyes/app/app_test.go
// constructs a "set" transaction
func makeSet(key, value []byte) []byte {
tx := make([]byte, 1+wire.ByteSliceSize(key)+wire.ByteSliceSize(value))
buf := tx
buf[0] = meapp.WriteSet // Set TypeByte
buf = buf[1:]
n, err := wire.PutByteSlice(buf, key)
if err != nil {
panic(err)
}
buf = buf[n:]
n, err = wire.PutByteSlice(buf, value)
if err != nil {
panic(err)
}
return tx
}
const letterBytes = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ"
func RandAsciiBytes(n int) []byte {
b := make([]byte, n)
for i := range b {
b[i] = letterBytes[rand.Intn(len(letterBytes))]
}
return b
}

View File

@ -1,22 +0,0 @@
package local_test
import (
"os"
"testing"
meapp "github.com/tendermint/merkleeyes/app"
nm "github.com/tendermint/tendermint/node"
rpctest "github.com/tendermint/tendermint/rpc/test"
)
var node *nm.Node
func TestMain(m *testing.M) {
// configure a node, but don't start the server
app := meapp.NewMerkleEyesApp("", 100)
node = rpctest.StartTendermint(app)
code := m.Run()
// and shut down proper at the end
os.Exit(code)
}

View File

@ -1,181 +0,0 @@
package local_test
import (
"strings"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
merkle "github.com/tendermint/go-merkle"
"github.com/tendermint/tendermint/rpc/client/local"
rpctest "github.com/tendermint/tendermint/rpc/test"
)
// GetClient gets a rpc client pointing to the test tendermint rpc
func GetClient() local.Client {
return local.New(node)
}
// Make sure status is correct (we connect properly)
func TestStatus(t *testing.T) {
c := GetClient()
status, err := c.Status()
require.Nil(t, err, "%+v", err)
require.NotNil(t, status.PubKey)
}
// Make sure info is correct (we connect properly)
func TestInfo(t *testing.T) {
c := GetClient()
status, err := c.Status()
require.Nil(t, err, "%+v", err)
info, err := c.ABCIInfo()
require.Nil(t, err, "%+v", err)
assert.EqualValues(t, status.LatestBlockHeight, info.Response.LastBlockHeight)
assert.True(t, strings.HasPrefix(info.Response.Data, "size:"))
}
func TestNetInfo(t *testing.T) {
c := GetClient()
netinfo, err := c.NetInfo()
require.Nil(t, err, "%+v", err)
assert.True(t, netinfo.Listening)
assert.Equal(t, 0, len(netinfo.Peers))
}
func TestDialSeeds(t *testing.T) {
c := GetClient()
// FIXME: fix server so it doesn't panic on invalid input
_, err := c.DialSeeds([]string{"12.34.56.78:12345"})
require.Nil(t, err, "%+v", err)
}
func TestGenesisAndValidators(t *testing.T) {
c := GetClient()
chainID := rpctest.GetConfig().GetString("chain_id")
// make sure this is the right genesis file
gen, err := c.Genesis()
require.Nil(t, err, "%+v", err)
assert.Equal(t, chainID, gen.Genesis.ChainID)
// get the genesis validator
require.Equal(t, 1, len(gen.Genesis.Validators))
gval := gen.Genesis.Validators[0]
// get the current validators
vals, err := c.Validators()
require.Nil(t, err, "%+v", err)
require.Equal(t, 1, len(vals.Validators))
val := vals.Validators[0]
// make sure the current set is also the genesis set
assert.Equal(t, gval.Amount, val.VotingPower)
assert.Equal(t, gval.PubKey, val.PubKey)
}
// Make some app checks
func TestAppCalls(t *testing.T) {
assert, require := assert.New(t), require.New(t)
c := GetClient()
_, err := c.Block(1)
assert.NotNil(err) // no block yet
k, v, tx := MakeTxKV()
_, err = c.BroadcastTxCommit(tx)
require.Nil(err, "%+v", err)
// wait before querying
time.Sleep(time.Second * 1)
qres, err := c.ABCIQuery("/key", k, false)
if assert.Nil(err) && assert.True(qres.Response.Code.IsOK()) {
data := qres.Response
// assert.Equal(k, data.GetKey()) // only returned for proofs
assert.Equal(v, data.GetValue())
}
// and we can even check the block is added
block, err := c.Block(3)
require.Nil(err, "%+v", err)
appHash := block.BlockMeta.Header.AppHash
assert.True(len(appHash) > 0)
assert.EqualValues(3, block.BlockMeta.Header.Height)
// check blockchain info, now that we know there is info
// TODO: is this commented somewhere that they are returned
// in order of descending height???
info, err := c.BlockchainInfo(1, 3)
require.Nil(err, "%+v", err)
assert.True(info.LastHeight > 2)
if assert.Equal(3, len(info.BlockMetas)) {
lastMeta := info.BlockMetas[0]
assert.EqualValues(3, lastMeta.Header.Height)
bMeta := block.BlockMeta
assert.Equal(bMeta.Header.AppHash, lastMeta.Header.AppHash)
assert.Equal(bMeta.BlockID, lastMeta.BlockID)
}
// and get the corresponding commit with the same apphash
commit, err := c.Commit(3)
require.Nil(err, "%+v", err)
cappHash := commit.Header.AppHash
assert.Equal(appHash, cappHash)
assert.NotNil(commit.Commit)
// compare the commits (note Commit(2) has commit from Block(3))
commit2, err := c.Commit(2)
require.Nil(err, "%+v", err)
assert.Equal(block.Block.LastCommit, commit2.Commit)
// and we got a proof that works!
pres, err := c.ABCIQuery("/key", k, true)
if assert.Nil(err) && assert.True(pres.Response.Code.IsOK()) {
proof, err := merkle.ReadProof(pres.Response.GetProof())
if assert.Nil(err) {
key := pres.Response.GetKey()
value := pres.Response.GetValue()
assert.Equal(appHash, proof.RootHash)
valid := proof.Verify(key, value, appHash)
assert.True(valid)
}
}
}
/*
func TestSubscriptions(t *testing.T) {
assert, require := assert.New(t), require.New(t)
c := GetClient()
err := c.StartWebsocket()
require.Nil(err)
defer c.StopWebsocket()
// subscribe to a transaction event
_, _, tx := MakeTxKV()
// this causes a panic in tendermint core!!!
eventType := types.EventStringTx(types.Tx(tx))
c.Subscribe(eventType)
read := 0
// set up a listener
r, e := c.GetEventChannels()
go func() {
// read one event in the background
select {
case <-r:
// TODO: actually parse this or something
read += 1
case err := <-e:
panic(err)
}
}()
// make sure nothing has happened yet.
assert.Equal(0, read)
// send a tx and wait for it to propogate
_, err = c.BroadcastTxCommit(tx)
assert.Nil(err, string(tx))
// wait before querying
time.Sleep(time.Second)
// now make sure the event arrived
assert.Equal(1, read)
}
*/

View File

@ -1,6 +1,15 @@
package client
import (
nm "github.com/tendermint/tendermint/node"
"github.com/tendermint/tendermint/rpc/core"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
"github.com/tendermint/tendermint/types"
)
/*
package local returns a Client implementation that
directly executes the rpc functions on a given node.
Local is a Client implementation that directly executes the rpc
functions on a given node, without going through HTTP or GRPC
This implementation is useful for:
@ -9,89 +18,79 @@ of going through an http server
* Communication between an ABCI app and tendermin core when they
are compiled in process.
For real clients, you probably want the "http" package. For more
powerful control during testing, you probably want the "mock" package.
For real clients, you probably want to use client.HTTP. For more
powerful control during testing, you probably want the "client/mock" package.
*/
package local
import (
nm "github.com/tendermint/tendermint/node"
"github.com/tendermint/tendermint/rpc/client"
"github.com/tendermint/tendermint/rpc/core"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
"github.com/tendermint/tendermint/types"
)
type Client struct {
type Local struct {
node *nm.Node
}
// New configures this to call the Node directly.
// NewLocal configures a client that calls the Node directly.
//
// Note that given how rpc/core works with package singletons, that
// you can only have one node per process. So make sure test cases
// don't run in parallel, or try to simulate an entire network in
// one process...
func New(node *nm.Node) Client {
func NewLocal(node *nm.Node) Local {
node.ConfigureRPC()
return Client{
return Local{
node: node,
}
}
func (c Client) _assertIsClient() client.Client {
func (c Local) _assertIsClient() Client {
return c
}
func (c Client) Status() (*ctypes.ResultStatus, error) {
func (c Local) Status() (*ctypes.ResultStatus, error) {
return core.Status()
}
func (c Client) ABCIInfo() (*ctypes.ResultABCIInfo, error) {
func (c Local) ABCIInfo() (*ctypes.ResultABCIInfo, error) {
return core.ABCIInfo()
}
func (c Client) ABCIQuery(path string, data []byte, prove bool) (*ctypes.ResultABCIQuery, error) {
func (c Local) ABCIQuery(path string, data []byte, prove bool) (*ctypes.ResultABCIQuery, error) {
return core.ABCIQuery(path, data, prove)
}
func (c Client) BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
func (c Local) BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
return core.BroadcastTxCommit(tx)
}
func (c Client) BroadcastTxAsync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
func (c Local) BroadcastTxAsync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
return core.BroadcastTxAsync(tx)
}
func (c Client) BroadcastTxSync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
func (c Local) BroadcastTxSync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) {
return core.BroadcastTxSync(tx)
}
func (c Client) NetInfo() (*ctypes.ResultNetInfo, error) {
func (c Local) NetInfo() (*ctypes.ResultNetInfo, error) {
return core.NetInfo()
}
func (c Client) DialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) {
func (c Local) DialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) {
return core.UnsafeDialSeeds(seeds)
}
func (c Client) BlockchainInfo(minHeight, maxHeight int) (*ctypes.ResultBlockchainInfo, error) {
func (c Local) BlockchainInfo(minHeight, maxHeight int) (*ctypes.ResultBlockchainInfo, error) {
return core.BlockchainInfo(minHeight, maxHeight)
}
func (c Client) Genesis() (*ctypes.ResultGenesis, error) {
func (c Local) Genesis() (*ctypes.ResultGenesis, error) {
return core.Genesis()
}
func (c Client) Block(height int) (*ctypes.ResultBlock, error) {
func (c Local) Block(height int) (*ctypes.ResultBlock, error) {
return core.Block(height)
}
func (c Client) Commit(height int) (*ctypes.ResultCommit, error) {
func (c Local) Commit(height int) (*ctypes.ResultCommit, error) {
return core.Commit(height)
}
func (c Client) Validators() (*ctypes.ResultValidators, error) {
func (c Local) Validators() (*ctypes.ResultValidators, error) {
return core.Validators()
}

View File

@ -1,17 +1,20 @@
package http_test
package client_test
import (
"os"
"testing"
meapp "github.com/tendermint/merkleeyes/app"
nm "github.com/tendermint/tendermint/node"
rpctest "github.com/tendermint/tendermint/rpc/test"
)
var node *nm.Node
func TestMain(m *testing.M) {
// start a tendermint node (and merkleeyes) in the background to test against
app := meapp.NewMerkleEyesApp("", 100)
node := rpctest.StartTendermint(app)
node = rpctest.StartTendermint(app)
code := m.Run()
// and shut down proper at the end

236
rpc/client/rpc_test.go Normal file
View File

@ -0,0 +1,236 @@
package client_test
import (
"encoding/json"
"strings"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
merkle "github.com/tendermint/go-merkle"
merktest "github.com/tendermint/merkleeyes/testutil"
"github.com/tendermint/tendermint/rpc/client"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpctest "github.com/tendermint/tendermint/rpc/test"
"github.com/tendermint/tendermint/types"
)
func getHTTPClient() *client.HTTP {
rpcAddr := rpctest.GetConfig().GetString("rpc_laddr")
return client.NewHTTP(rpcAddr, "/websocket")
}
func getLocalClient() client.Local {
return client.NewLocal(node)
}
// GetClients returns a slice of clients for table-driven tests
func GetClients() []client.Client {
return []client.Client{
getHTTPClient(),
getLocalClient(),
}
}
// Make sure status is correct (we connect properly)
func TestStatus(t *testing.T) {
for i, c := range GetClients() {
chainID := rpctest.GetConfig().GetString("chain_id")
status, err := c.Status()
require.Nil(t, err, "%d: %+v", i, err)
assert.Equal(t, chainID, status.NodeInfo.Network)
}
}
// Make sure info is correct (we connect properly)
func TestInfo(t *testing.T) {
for i, c := range GetClients() {
// status, err := c.Status()
// require.Nil(t, err, "%+v", err)
info, err := c.ABCIInfo()
require.Nil(t, err, "%d: %+v", i, err)
// TODO: this is not correct - fix merkleeyes!
// assert.EqualValues(t, status.LatestBlockHeight, info.Response.LastBlockHeight)
assert.True(t, strings.HasPrefix(info.Response.Data, "size"))
}
}
func TestNetInfo(t *testing.T) {
for i, c := range GetClients() {
nc, ok := c.(client.NetworkClient)
require.True(t, ok, "%d", i)
netinfo, err := nc.NetInfo()
require.Nil(t, err, "%d: %+v", i, err)
assert.True(t, netinfo.Listening)
assert.Equal(t, 0, len(netinfo.Peers))
}
}
func TestDialSeeds(t *testing.T) {
for i, c := range GetClients() {
// FIXME: fix server so it doesn't panic on invalid input
nc, ok := c.(client.NetworkClient)
require.True(t, ok, "%d", i)
_, err := nc.DialSeeds([]string{"12.34.56.78:12345"})
require.Nil(t, err, "%d: %+v", i, err)
}
}
func TestGenesisAndValidators(t *testing.T) {
for i, c := range GetClients() {
chainID := rpctest.GetConfig().GetString("chain_id")
// make sure this is the right genesis file
gen, err := c.Genesis()
require.Nil(t, err, "%d: %+v", i, err)
assert.Equal(t, chainID, gen.Genesis.ChainID)
// get the genesis validator
require.Equal(t, 1, len(gen.Genesis.Validators))
gval := gen.Genesis.Validators[0]
// get the current validators
vals, err := c.Validators()
require.Nil(t, err, "%d: %+v", i, err)
require.Equal(t, 1, len(vals.Validators))
val := vals.Validators[0]
// make sure the current set is also the genesis set
assert.Equal(t, gval.Amount, val.VotingPower)
assert.Equal(t, gval.PubKey, val.PubKey)
}
}
// Make some app checks
func TestAppCalls(t *testing.T) {
assert, require := assert.New(t), require.New(t)
for i, c := range GetClients() {
// get an offset of height to avoid racing and guessing
s, err := c.Status()
require.Nil(err, "%d: %+v", i, err)
// sh is start height or status height
sh := s.LatestBlockHeight
// look for the future
_, err = c.Block(sh + 2)
assert.NotNil(err) // no block yet
// write something
k, v, tx := merktest.MakeTxKV()
_, err = c.BroadcastTxCommit(tx)
require.Nil(err, "%d: %+v", i, err)
// wait before querying
time.Sleep(time.Second * 1)
qres, err := c.ABCIQuery("/key", k, false)
if assert.Nil(err) && assert.True(qres.Response.Code.IsOK()) {
data := qres.Response
// assert.Equal(k, data.GetKey()) // only returned for proofs
assert.Equal(v, data.GetValue())
}
// +/- 1 making my head hurt
h := int(qres.Response.Height) - 1
// and we can even check the block is added
block, err := c.Block(h)
require.Nil(err, "%d: %+v", i, err)
appHash := block.BlockMeta.Header.AppHash
assert.True(len(appHash) > 0)
assert.EqualValues(h, block.BlockMeta.Header.Height)
// check blockchain info, now that we know there is info
// TODO: is this commented somewhere that they are returned
// in order of descending height???
info, err := c.BlockchainInfo(h-2, h)
require.Nil(err, "%d: %+v", i, err)
assert.True(info.LastHeight > 2)
if assert.Equal(3, len(info.BlockMetas)) {
lastMeta := info.BlockMetas[0]
assert.EqualValues(h, lastMeta.Header.Height)
bMeta := block.BlockMeta
assert.Equal(bMeta.Header.AppHash, lastMeta.Header.AppHash)
assert.Equal(bMeta.BlockID, lastMeta.BlockID)
}
// and get the corresponding commit with the same apphash
commit, err := c.Commit(h)
require.Nil(err, "%d: %+v", i, err)
cappHash := commit.Header.AppHash
assert.Equal(appHash, cappHash)
assert.NotNil(commit.Commit)
// compare the commits (note Commit(2) has commit from Block(3))
commit2, err := c.Commit(h - 1)
require.Nil(err, "%d: %+v", i, err)
assert.Equal(block.Block.LastCommit, commit2.Commit)
// and we got a proof that works!
pres, err := c.ABCIQuery("/key", k, true)
if assert.Nil(err) && assert.True(pres.Response.Code.IsOK()) {
proof, err := merkle.ReadProof(pres.Response.GetProof())
if assert.Nil(err) {
key := pres.Response.GetKey()
value := pres.Response.GetValue()
assert.Equal(appHash, proof.RootHash)
valid := proof.Verify(key, value, appHash)
assert.True(valid)
}
}
}
}
// TestSubscriptions only works for HTTPClient
//
// TODO: generalize this functionality -> Local and Client
func TestSubscriptions(t *testing.T) {
require := require.New(t)
c := getHTTPClient()
err := c.StartWebsocket()
require.Nil(err)
defer c.StopWebsocket()
// subscribe to a transaction event
_, _, tx := merktest.MakeTxKV()
eventType := types.EventStringTx(types.Tx(tx))
c.Subscribe(eventType)
// set up a listener
r, e := c.GetEventChannels()
go func() {
// send a tx and wait for it to propogate
_, err = c.BroadcastTxCommit(tx)
require.Nil(err, string(tx))
}()
checkData := func(data []byte, kind byte) {
x := []interface{}{}
err := json.Unmarshal(data, &x)
require.Nil(err)
// gotta love wire's json format
require.EqualValues(kind, x[0])
}
res := <-r
checkData(res, ctypes.ResultTypeSubscribe)
// read one event, must be success
select {
case res := <-r:
checkData(res, ctypes.ResultTypeEvent)
// this is good.. let's get the data... ugh...
// result := new(ctypes.TMResult)
// wire.ReadJSON(result, res, &err)
// require.Nil(err, "%+v", err)
// event, ok := (*result).(*ctypes.ResultEvent)
// require.True(ok)
// assert.Equal("foo", event.Name)
// data, ok := event.Data.(types.EventDataTx)
// require.True(ok)
// assert.EqualValues(0, data.Code)
// assert.EqualValues(tx, data.Tx)
case err := <-e:
// this is a failure
require.Nil(err)
}
}