Merge pull request #625 from cosmos/bucky/lcd-test-cleanup

lcd test cleanup
This commit is contained in:
Ethan Buchman 2018-03-18 02:13:27 +01:00 committed by GitHub
commit 10f50a31fc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 402 additions and 182 deletions

View File

@ -106,6 +106,7 @@ func SignAndBuild(name, passphrase string, msg sdk.Msg, cdc *wire.Codec) ([]byte
// sign and build
bz := signMsg.Bytes()
sig, pubkey, err := keybase.Sign(name, passphrase, bz)
if err != nil {
return nil, err

View File

@ -6,8 +6,8 @@ import (
dbm "github.com/tendermint/tmlibs/db"
)
// TODO explain what the keybase is used for
// GetKeyBase initializes a keybase based on the configuration
// GetKeyBase initializes a keybase based on the given db.
// The KeyBase manages all activity requiring access to a key.
func GetKeyBase(db dbm.DB) keys.Keybase {
keybase := keys.New(
db,

69
client/lcd/helpers.go Normal file
View File

@ -0,0 +1,69 @@
package lcd
// NOTE: COPIED VERBATIM FROM tendermint/tendermint/rpc/test/helpers.go
import (
"fmt"
"os"
"path/filepath"
"strings"
cmn "github.com/tendermint/tmlibs/common"
cfg "github.com/tendermint/tendermint/config"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpcclient "github.com/tendermint/tendermint/rpc/lib/client"
)
var globalConfig *cfg.Config
func waitForRPC() {
laddr := GetConfig().RPC.ListenAddress
fmt.Println("LADDR", laddr)
client := rpcclient.NewJSONRPCClient(laddr)
result := new(ctypes.ResultStatus)
for {
_, err := client.Call("status", map[string]interface{}{}, result)
if err == nil {
return
}
}
}
// f**ing long, but unique for each test
func makePathname() string {
// get path
p, err := os.Getwd()
if err != nil {
panic(err)
}
// fmt.Println(p)
sep := string(filepath.Separator)
return strings.Replace(p, sep, "_", -1)
}
func randPort() int {
return int(cmn.RandUint16()/2 + 10000)
}
func makeAddrs() (string, string, string) {
start := randPort()
return fmt.Sprintf("tcp://0.0.0.0:%d", start),
fmt.Sprintf("tcp://0.0.0.0:%d", start+1),
fmt.Sprintf("tcp://0.0.0.0:%d", start+2)
}
// GetConfig returns a config for the test cases as a singleton
func GetConfig() *cfg.Config {
if globalConfig == nil {
pathname := makePathname()
globalConfig = cfg.ResetTestRoot(pathname)
// and we use random ports to run in parallel
tm, rpc, _ := makeAddrs()
globalConfig.P2P.ListenAddress = tm
globalConfig.RPC.ListenAddress = rpc
globalConfig.TxIndex.IndexTags = "app.creator" // see kvstore application
}
return globalConfig
}

View File

@ -5,52 +5,79 @@ import (
"encoding/json"
"fmt"
"io/ioutil"
"net"
"net/http"
"os"
"path/filepath"
"regexp"
"testing"
"time"
"github.com/spf13/viper"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
abci "github.com/tendermint/abci/types"
cryptoKeys "github.com/tendermint/go-crypto/keys"
"github.com/tendermint/tendermint/p2p"
tmcfg "github.com/tendermint/tendermint/config"
nm "github.com/tendermint/tendermint/node"
p2p "github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/proxy"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
tmrpc "github.com/tendermint/tendermint/rpc/lib/server"
tmtypes "github.com/tendermint/tendermint/types"
dbm "github.com/tendermint/tmlibs/db"
"github.com/tendermint/tmlibs/log"
"github.com/cosmos/cosmos-sdk/client"
client "github.com/cosmos/cosmos-sdk/client"
keys "github.com/cosmos/cosmos-sdk/client/keys"
"github.com/cosmos/cosmos-sdk/tests"
bapp "github.com/cosmos/cosmos-sdk/examples/basecoin/app"
btypes "github.com/cosmos/cosmos-sdk/examples/basecoin/types"
sdk "github.com/cosmos/cosmos-sdk/types"
"github.com/cosmos/cosmos-sdk/wire"
"github.com/cosmos/cosmos-sdk/x/auth"
)
var (
coinDenom = "mycoin"
coinAmount = int64(10000000)
// XXX bad globals
port string // XXX: but it's the int ...
name string = "test"
password string = "0123456789"
seed string
sendAddr string
)
func TestKeys(t *testing.T) {
kill, port, _ := setupEnvironment(t)
defer kill()
// empty keys
res, body := request(t, port, "GET", "/keys", nil)
require.Equal(t, http.StatusOK, res.StatusCode, body)
assert.Equal(t, "[]", body, "Expected an empty array")
// XXX: the test comes with a key setup
/*
res, body := request(t, port, "GET", "/keys", nil)
require.Equal(t, http.StatusOK, res.StatusCode, body)
assert.Equal(t, "[]", body, "Expected an empty array")
*/
// get seed
res, body = request(t, port, "GET", "/keys/seed", nil)
res, body := request(t, port, "GET", "/keys/seed", nil)
require.Equal(t, http.StatusOK, res.StatusCode, body)
seed := body
newSeed := body
reg, err := regexp.Compile(`([a-z]+ ){12}`)
require.Nil(t, err)
match := reg.MatchString(seed)
assert.True(t, match, "Returned seed has wrong foramt", seed)
newName := "test_newname"
newPassword := "0987654321"
// add key
var jsonStr = []byte(`{"name":"test_fail", "password":"1234567890"}`)
var jsonStr = []byte(fmt.Sprintf(`{"name":"test_fail", "password":"%s"}`, password))
res, body = request(t, port, "POST", "/keys", jsonStr)
assert.Equal(t, http.StatusBadRequest, res.StatusCode, "Account creation should require a seed")
jsonStr = []byte(fmt.Sprintf(`{"name":"test", "password":"1234567890", "seed": "%s"}`, seed))
jsonStr = []byte(fmt.Sprintf(`{"name":"%s", "password":"%s", "seed": "%s"}`, newName, newPassword, newSeed))
res, body = request(t, port, "POST", "/keys", jsonStr)
assert.Equal(t, http.StatusOK, res.StatusCode, body)
@ -60,41 +87,42 @@ func TestKeys(t *testing.T) {
// existing keys
res, body = request(t, port, "GET", "/keys", nil)
require.Equal(t, http.StatusOK, res.StatusCode, body)
var m [1]keys.KeyOutput
var m [2]keys.KeyOutput
err = json.Unmarshal([]byte(body), &m)
require.Nil(t, err)
assert.Equal(t, m[0].Name, "test", "Did not serve keys name correctly")
assert.Equal(t, m[0].Address, addr, "Did not serve keys Address correctly")
assert.Equal(t, m[0].Name, name, "Did not serve keys name correctly")
assert.Equal(t, m[0].Address, sendAddr, "Did not serve keys Address correctly")
assert.Equal(t, m[1].Name, newName, "Did not serve keys name correctly")
assert.Equal(t, m[1].Address, addr, "Did not serve keys Address correctly")
// select key
res, body = request(t, port, "GET", "/keys/test", nil)
keyEndpoint := fmt.Sprintf("/keys/%s", newName)
res, body = request(t, port, "GET", keyEndpoint, nil)
require.Equal(t, http.StatusOK, res.StatusCode, body)
var m2 keys.KeyOutput
err = json.Unmarshal([]byte(body), &m2)
require.Nil(t, err)
assert.Equal(t, "test", m2.Name, "Did not serve keys name correctly")
assert.Equal(t, newName, m2.Name, "Did not serve keys name correctly")
assert.Equal(t, addr, m2.Address, "Did not serve keys Address correctly")
// update key
jsonStr = []byte(`{"old_password":"1234567890", "new_password":"12345678901"}`)
res, body = request(t, port, "PUT", "/keys/test", jsonStr)
jsonStr = []byte(fmt.Sprintf(`{"old_password":"%s", "new_password":"12345678901"}`, newPassword))
res, body = request(t, port, "PUT", keyEndpoint, jsonStr)
require.Equal(t, http.StatusOK, res.StatusCode, body)
// here it should say unauthorized as we changed the password before
res, body = request(t, port, "PUT", "/keys/test", jsonStr)
res, body = request(t, port, "PUT", keyEndpoint, jsonStr)
require.Equal(t, http.StatusUnauthorized, res.StatusCode, body)
// delete key
jsonStr = []byte(`{"password":"12345678901"}`)
res, body = request(t, port, "DELETE", "/keys/test", jsonStr)
res, body = request(t, port, "DELETE", keyEndpoint, jsonStr)
require.Equal(t, http.StatusOK, res.StatusCode, body)
}
func TestVersion(t *testing.T) {
kill, port, _ := setupEnvironment(t)
defer kill()
// node info
res, body := request(t, port, "GET", "/version", nil)
@ -107,8 +135,6 @@ func TestVersion(t *testing.T) {
}
func TestNodeStatus(t *testing.T) {
kill, port, _ := setupEnvironment(t)
defer kill()
// node info
res, body := request(t, port, "GET", "/node_info", nil)
@ -130,8 +156,6 @@ func TestNodeStatus(t *testing.T) {
}
func TestBlock(t *testing.T) {
kill, port, _ := setupEnvironment(t)
defer kill()
time.Sleep(time.Second * 2) // TODO: LOL -> wait for blocks
@ -162,10 +186,6 @@ func TestBlock(t *testing.T) {
}
func TestValidators(t *testing.T) {
kill, port, _ := setupEnvironment(t)
defer kill()
time.Sleep(time.Second * 2) // TODO: LOL -> wait for blocks
var resultVals ctypes.ResultValidators
@ -194,17 +214,13 @@ func TestValidators(t *testing.T) {
}
func TestCoinSend(t *testing.T) {
kill, port, seed := setupEnvironment(t)
defer kill()
time.Sleep(time.Second * 2) // TO
// query empty
res, body := request(t, port, "GET", "/accounts/8FA6AB57AD6870F6B5B2E57735F38F2F30E73CB6", nil)
require.Equal(t, http.StatusNoContent, res.StatusCode, body)
// create TX
addr, receiveAddr, resultTx := doSend(t, port, seed)
receiveAddr, resultTx := doSend(t, port, seed)
time.Sleep(time.Second * 2) // T
@ -213,7 +229,7 @@ func TestCoinSend(t *testing.T) {
assert.Equal(t, uint32(0), resultTx.DeliverTx.Code)
// query sender
res, body = request(t, port, "GET", "/accounts/"+addr, nil)
res, body = request(t, port, "GET", "/accounts/"+sendAddr, nil)
require.Equal(t, http.StatusOK, res.StatusCode, body)
var m auth.BaseAccount
@ -221,8 +237,8 @@ func TestCoinSend(t *testing.T) {
require.Nil(t, err)
coins := m.Coins
mycoins := coins[0]
assert.Equal(t, "mycoin", mycoins.Denom)
assert.Equal(t, int64(9007199254740991), mycoins.Amount)
assert.Equal(t, coinDenom, mycoins.Denom)
assert.Equal(t, coinAmount-1, mycoins.Amount)
// query receiver
res, body = request(t, port, "GET", "/accounts/"+receiveAddr, nil)
@ -232,13 +248,11 @@ func TestCoinSend(t *testing.T) {
require.Nil(t, err)
coins = m.Coins
mycoins = coins[0]
assert.Equal(t, "mycoin", mycoins.Denom)
assert.Equal(t, coinDenom, mycoins.Denom)
assert.Equal(t, int64(1), mycoins.Amount)
}
func TestTxs(t *testing.T) {
kill, port, seed := setupEnvironment(t)
defer kill()
// TODO: re-enable once we can get txs by tag
@ -253,7 +267,7 @@ func TestTxs(t *testing.T) {
// assert.Equal(t, "[]", body)
// create TX
_, _, resultTx := doSend(t, port, seed)
_, resultTx := doSend(t, port, seed)
time.Sleep(time.Second * 2) // TO
@ -277,29 +291,110 @@ func TestTxs(t *testing.T) {
//__________________________________________________________
// helpers
// TODO/XXX: We should be spawning what we need in process, not shelling out
func setupEnvironment(t *testing.T) (kill func(), port string, seed string) {
dir, err := ioutil.TempDir("", "tmp-basecoin-")
require.Nil(t, err)
// strt TM and the LCD in process, listening on their respective sockets
func startTMAndLCD() (*nm.Node, net.Listener, error) {
seed = tests.TestInitBasecoin(t, dir)
// get chain ID
bz, err := ioutil.ReadFile(filepath.Join(dir, "config", "genesis.json"))
require.Nil(t, err)
var gen tmtypes.GenesisDoc
err = json.Unmarshal(bz, &gen)
require.Nil(t, err)
cmdNode := tests.StartNodeServerForTest(t, dir)
cmdLCD, port := tests.StartLCDServerForTest(t, dir, gen.ChainID)
kill = func() {
cmdLCD.Process.Kill()
cmdLCD.Process.Wait()
cmdNode.Process.Kill()
cmdNode.Process.Wait()
os.Remove(dir)
kb, err := keys.GetKeyBase() // dbm.NewMemDB()) // :(
if err != nil {
return nil, nil, err
}
return kill, port, seed
var info cryptoKeys.Info
info, seed, err = kb.Create(name, password, cryptoKeys.AlgoEd25519) // XXX global seed
if err != nil {
return nil, nil, err
}
pubKey := info.PubKey
sendAddr = pubKey.Address().String() // XXX global
config := GetConfig()
config.Consensus.TimeoutCommit = 1000
config.Consensus.SkipTimeoutCommit = false
logger := log.NewTMLogger(log.NewSyncWriter(os.Stdout))
logger = log.NewFilter(logger, log.AllowError())
privValidatorFile := config.PrivValidatorFile()
privVal := tmtypes.LoadOrGenPrivValidatorFS(privValidatorFile)
app := bapp.NewBasecoinApp(logger, dbm.NewMemDB())
genesisFile := config.GenesisFile()
genDoc, err := tmtypes.GenesisDocFromFile(genesisFile)
if err != nil {
return nil, nil, err
}
coins := sdk.Coins{{coinDenom, coinAmount}}
appState := btypes.GenesisState{
Accounts: []*btypes.GenesisAccount{
{
Name: "tester",
Address: pubKey.Address(),
Coins: coins,
},
},
}
stateBytes, err := json.Marshal(appState)
if err != nil {
return nil, nil, err
}
genDoc.AppState = stateBytes
cdc := wire.NewCodec()
// LCD listen address
port = fmt.Sprintf("%d", 17377) // XXX
listenAddr := fmt.Sprintf("tcp://localhost:%s", port) // XXX
// XXX: need to set this so LCD knows the tendermint node address!
viper.Set(client.FlagNode, config.RPC.ListenAddress)
viper.Set(client.FlagChainID, genDoc.ChainID)
node, err := startTM(config, logger, genDoc, privVal, app)
if err != nil {
return nil, nil, err
}
lcd, err := startLCD(cdc, logger, listenAddr)
if err != nil {
return nil, nil, err
}
time.Sleep(time.Second * 2)
return node, lcd, nil
}
// Create & start in-process tendermint node with memdb
// and in-process abci application.
// TODO: need to clean up the WAL dir or enable it to be not persistent
func startTM(cfg *tmcfg.Config, logger log.Logger, genDoc *tmtypes.GenesisDoc, privVal tmtypes.PrivValidator, app abci.Application) (*nm.Node, error) {
genDocProvider := func() (*tmtypes.GenesisDoc, error) { return genDoc, nil }
dbProvider := func(*nm.DBContext) (dbm.DB, error) { return dbm.NewMemDB(), nil }
n, err := nm.NewNode(cfg,
privVal,
proxy.NewLocalClientCreator(app),
genDocProvider,
dbProvider,
logger.With("module", "node"))
if err != nil {
return nil, err
}
err = n.Start()
if err != nil {
return nil, err
}
// wait for rpc
waitForRPC()
logger.Info("Tendermint running!")
return n, err
}
// start the LCD. note this blocks!
func startLCD(cdc *wire.Codec, logger log.Logger, listenAddr string) (net.Listener, error) {
handler := createHandler(cdc)
return tmrpc.StartHTTPServer(listenAddr, handler, logger)
}
func request(t *testing.T, port, method, path string, payload []byte) (*http.Response, string) {
@ -318,13 +413,7 @@ func request(t *testing.T, port, method, path string, payload []byte) (*http.Res
return res, string(output)
}
func doSend(t *testing.T, port, seed string) (sendAddr string, receiveAddr string, resultTx ctypes.ResultBroadcastTxCommit) {
// create account from seed who has keys
var jsonStr = []byte(fmt.Sprintf(`{"name":"test", "password":"1234567890", "seed": "%s"}`, seed))
res, body := request(t, port, "POST", "/keys", jsonStr)
assert.Equal(t, http.StatusOK, res.StatusCode, body)
sendAddr = body
func doSend(t *testing.T, port, seed string) (receiveAddr string, resultTx ctypes.ResultBroadcastTxCommit) {
// create receive address
kb := client.MockKeyBase()
@ -332,13 +421,23 @@ func doSend(t *testing.T, port, seed string) (sendAddr string, receiveAddr strin
require.Nil(t, err)
receiveAddr = receiveInfo.PubKey.Address().String()
// get the account to get the sequence
res, body := request(t, port, "GET", "/accounts/"+sendAddr, nil)
// require.Equal(t, http.StatusOK, res.StatusCode, body)
acc := auth.BaseAccount{}
err = json.Unmarshal([]byte(body), &acc)
require.Nil(t, err)
fmt.Println("BODY", body)
fmt.Println("ACC", acc)
sequence := acc.Sequence
// send
jsonStr = []byte(`{ "name":"test", "password":"1234567890", "amount":[{ "denom": "mycoin", "amount": 1 }] }`)
jsonStr := []byte(fmt.Sprintf(`{ "name":"%s", "password":"%s", "sequence":%d, "amount":[{ "denom": "%s", "amount": 1 }] }`, name, password, sequence, coinDenom))
res, body = request(t, port, "POST", "/accounts/"+receiveAddr+"/send", jsonStr)
require.Equal(t, http.StatusOK, res.StatusCode, body)
err = json.Unmarshal([]byte(body), &resultTx)
require.Nil(t, err)
return sendAddr, receiveAddr, resultTx
return receiveAddr, resultTx
}

38
client/lcd/main_test.go Normal file
View File

@ -0,0 +1,38 @@
package lcd
import (
"fmt"
"os"
"testing"
nm "github.com/tendermint/tendermint/node"
)
var node *nm.Node
// See https://golang.org/pkg/testing/#hdr-Main
// for more details
func TestMain(m *testing.M) {
// start a basecoind node and LCD server in the background to test against
// run all the tests against a single server instance
node, lcd, err := startTMAndLCD()
if err != nil {
fmt.Println(err)
os.Exit(1)
}
code := m.Run()
// tear down
// TODO: cleanup
// TODO: it would be great if TM could run without
// persiting anything in the first place
node.Stop()
node.Wait()
// just a listener ...
lcd.Close()
os.Exit(code)
}

View File

@ -2,10 +2,15 @@ package lcd
import (
"net/http"
"os"
"github.com/gorilla/mux"
"github.com/spf13/cobra"
"github.com/spf13/viper"
"github.com/tendermint/tmlibs/log"
tmserver "github.com/tendermint/tendermint/rpc/lib/server"
cmn "github.com/tendermint/tmlibs/common"
client "github.com/cosmos/cosmos-sdk/client"
keys "github.com/cosmos/cosmos-sdk/client/keys"
@ -18,8 +23,8 @@ import (
)
const (
flagBind = "bind"
flagCORS = "cors"
flagListenAddr = "laddr"
flagCORS = "cors"
)
// ServeCommand will generate a long-running rest server
@ -29,33 +34,49 @@ func ServeCommand(cdc *wire.Codec) *cobra.Command {
cmd := &cobra.Command{
Use: "rest-server",
Short: "Start LCD (light-client daemon), a local REST server",
RunE: startRESTServer(cdc),
RunE: startRESTServerFn(cdc),
}
// TODO: handle unix sockets also?
cmd.Flags().StringP(flagBind, "b", "localhost:1317", "Interface and port that server binds to")
cmd.Flags().StringP(flagListenAddr, "a", "tcp://localhost:1317", "Address for server to listen on")
cmd.Flags().String(flagCORS, "", "Set to domains that can make CORS requests (* for all)")
cmd.Flags().StringP(client.FlagChainID, "c", "", "ID of chain we connect to")
cmd.Flags().StringP(client.FlagNode, "n", "tcp://localhost:46657", "Node to connect to")
return cmd
}
func startRESTServer(cdc *wire.Codec) func(cmd *cobra.Command, args []string) error {
func startRESTServerFn(cdc *wire.Codec) func(cmd *cobra.Command, args []string) error {
return func(cmd *cobra.Command, args []string) error {
bind := viper.GetString(flagBind)
r := initRouter(cdc)
return http.ListenAndServe(bind, r)
listenAddr := viper.GetString(flagListenAddr)
handler := createHandler(cdc)
logger := log.NewTMLogger(log.NewSyncWriter(os.Stdout)).
With("module", "rest-server")
listener, err := tmserver.StartHTTPServer(listenAddr, handler, logger)
if err != nil {
return err
}
// Wait forever and cleanup
cmn.TrapSignal(func() {
err := listener.Close()
logger.Error("Error closing listener", "err", err)
})
return nil
}
}
func initRouter(cdc *wire.Codec) http.Handler {
func createHandler(cdc *wire.Codec) http.Handler {
r := mux.NewRouter()
r.HandleFunc("/version", version.VersionRequestHandler).Methods("GET")
kb, err := keys.GetKeyBase() //XXX
if err != nil {
panic(err)
}
// TODO make more functional? aka r = keys.RegisterRoutes(r)
keys.RegisterRoutes(r)
rpc.RegisterRoutes(r)
tx.RegisterRoutes(r, cdc)
auth.RegisterRoutes(r, cdc, "main") // TODO should use a variable not just a string
bank.RegisterRoutes(r, cdc)
auth.RegisterRoutes(r, cdc, "main")
bank.RegisterRoutes(r, cdc, kb)
return r
}

View File

@ -13,13 +13,14 @@ import (
"github.com/cosmos/cosmos-sdk/client/lcd"
"github.com/cosmos/cosmos-sdk/client/rpc"
"github.com/cosmos/cosmos-sdk/client/tx"
coolcmd "github.com/cosmos/cosmos-sdk/examples/basecoin/x/cool/commands"
"github.com/cosmos/cosmos-sdk/version"
authcmd "github.com/cosmos/cosmos-sdk/x/auth/commands"
bankcmd "github.com/cosmos/cosmos-sdk/x/bank/commands"
"github.com/cosmos/cosmos-sdk/examples/basecoin/app"
"github.com/cosmos/cosmos-sdk/examples/basecoin/types"
coolcmd "github.com/cosmos/cosmos-sdk/examples/basecoin/x/cool/commands"
)
// gaiacliCmd is the entry point for this binary
@ -41,6 +42,10 @@ func main() {
// get the codec
cdc := app.MakeCodec()
// TODO: setup keybase, viper object, etc. to be passed into
// the below functions and eliminate global vars, like we do
// with the cdc
// add standard rpc, and tx commands
rpc.AddCommands(basecliCmd)
basecliCmd.AddCommand(client.LineBreak)

View File

@ -59,6 +59,7 @@ func generateApp(rootDir string, logger log.Logger) (abci.Application, error) {
}
func main() {
// TODO: set logger through CLI
logger := log.NewTMLogger(log.NewSyncWriter(os.Stdout)).
With("module", "main")

View File

@ -2,33 +2,27 @@ package server
import (
"encoding/json"
"fmt"
"io/ioutil"
"github.com/spf13/cobra"
"github.com/tendermint/go-crypto/keys"
"github.com/tendermint/go-crypto/keys/words"
cmn "github.com/tendermint/tmlibs/common"
dbm "github.com/tendermint/tmlibs/db"
"github.com/tendermint/tmlibs/log"
tcmd "github.com/tendermint/tendermint/cmd/tendermint/commands"
cfg "github.com/tendermint/tendermint/config"
tmtypes "github.com/tendermint/tendermint/types"
sdk "github.com/cosmos/cosmos-sdk/types"
)
// InitCmd will initialize all files for tendermint,
// along with proper app_options.
// along with proper app_state.
// The application can pass in a function to generate
// proper options. And may want to use GenerateCoinKey
// proper state. And may want to use GenerateCoinKey
// to create default account(s).
func InitCmd(gen GenOptions, logger log.Logger) *cobra.Command {
func InitCmd(gen GenAppState, logger log.Logger) *cobra.Command {
cmd := initCmd{
gen: gen,
logger: logger,
genAppState: gen,
logger: logger,
}
return &cobra.Command{
Use: "init",
@ -37,39 +31,14 @@ func InitCmd(gen GenOptions, logger log.Logger) *cobra.Command {
}
}
// GenOptions can parse command-line and flag to
// generate default app_options for the genesis file.
// GenAppState can parse command-line and flag to
// generate default app_state for the genesis file.
// This is application-specific
type GenOptions func(args []string) (json.RawMessage, error)
// GenerateCoinKey returns the address of a public key,
// along with the secret phrase to recover the private key.
// You can give coins to this address and return the recovery
// phrase to the user to access them.
func GenerateCoinKey() (sdk.Address, string, error) {
// construct an in-memory key store
codec, err := words.LoadCodec("english")
if err != nil {
return nil, "", err
}
keybase := keys.New(
dbm.NewMemDB(),
codec,
)
// generate a private key, with recovery phrase
info, secret, err := keybase.Create("name", "pass", keys.AlgoEd25519)
if err != nil {
return nil, "", err
}
addr := info.PubKey.Address()
return addr, secret, nil
}
type GenAppState func(args []string) (json.RawMessage, error)
type initCmd struct {
gen GenOptions
logger log.Logger
genAppState GenAppState
logger log.Logger
}
func (c initCmd) run(cmd *cobra.Command, args []string) error {
@ -85,19 +54,19 @@ func (c initCmd) run(cmd *cobra.Command, args []string) error {
}
// no app_options, leave like tendermint
if c.gen == nil {
if c.genAppState == nil {
return nil
}
// Now, we want to add the custom app_options
options, err := c.gen(args)
// Now, we want to add the custom app_state
appState, err := c.genAppState(args)
if err != nil {
return err
}
// And add them to the genesis file
genFile := config.GenesisFile()
return addGenesisOptions(genFile, options)
return addGenesisState(genFile, appState)
}
// This was copied from tendermint/cmd/tendermint/commands/init.go
@ -141,7 +110,7 @@ func (c initCmd) initTendermintFiles(config *cfg.Config) error {
// so we can add one line.
type GenesisDoc map[string]json.RawMessage
func addGenesisOptions(filename string, options json.RawMessage) error {
func addGenesisState(filename string, appState json.RawMessage) error {
bz, err := ioutil.ReadFile(filename)
if err != nil {
return err
@ -153,7 +122,7 @@ func addGenesisOptions(filename string, options json.RawMessage) error {
return err
}
doc["app_state"] = options
doc["app_state"] = appState
out, err := json.MarshalIndent(doc, "", " ")
if err != nil {
return err
@ -161,23 +130,3 @@ func addGenesisOptions(filename string, options json.RawMessage) error {
return ioutil.WriteFile(filename, out, 0600)
}
// GetGenesisJSON returns a new tendermint genesis with Basecoin app_options
// that grant a large amount of "mycoin" to a single address
// TODO: A better UX for generating genesis files
func GetGenesisJSON(pubkey, chainID, denom, addr string, options string) string {
return fmt.Sprintf(`{
"accounts": [{
"address": "%s",
"coins": [
{
"denom": "%s",
"amount": 9007199254740992
}
]
}],
"plugin_options": [
"coin/issuer", {"app": "sigs", "addr": "%s"}%s
]
}`, addr, denom, addr, options)
}

34
server/key.go Normal file
View File

@ -0,0 +1,34 @@
package server
import (
"github.com/tendermint/go-crypto/keys"
"github.com/tendermint/go-crypto/keys/words"
dbm "github.com/tendermint/tmlibs/db"
sdk "github.com/cosmos/cosmos-sdk/types"
)
// GenerateCoinKey returns the address of a public key,
// along with the secret phrase to recover the private key.
// You can give coins to this address and return the recovery
// phrase to the user to access them.
func GenerateCoinKey() (sdk.Address, string, error) {
// construct an in-memory key store
codec, err := words.LoadCodec("english")
if err != nil {
return nil, "", err
}
keybase := keys.New(
dbm.NewMemDB(),
codec,
)
// generate a private key, with recovery phrase
info, secret, err := keybase.Create("name", "pass", keys.AlgoEd25519)
if err != nil {
return nil, "", err
}
addr := info.PubKey.Address()
return addr, secret, nil
}

View File

@ -23,14 +23,14 @@ const (
// appGenerator lets us lazily initialize app, using home dir
// and other flags (?) to start
type appGenerator func(string, log.Logger) (abci.Application, error)
type appCreator func(string, log.Logger) (abci.Application, error)
// StartCmd runs the service passed in, either
// stand-alone, or in-process with tendermint
func StartCmd(app appGenerator, logger log.Logger) *cobra.Command {
func StartCmd(app appCreator, logger log.Logger) *cobra.Command {
start := startCmd{
app: app,
logger: logger,
appCreator: app,
logger: logger,
}
cmd := &cobra.Command{
Use: "start",
@ -48,8 +48,8 @@ func StartCmd(app appGenerator, logger log.Logger) *cobra.Command {
}
type startCmd struct {
app appGenerator
logger log.Logger
appCreator appCreator
logger log.Logger
}
func (s startCmd) run(cmd *cobra.Command, args []string) error {
@ -65,7 +65,7 @@ func (s startCmd) startStandAlone() error {
// Generate the app in the proper dir
addr := viper.GetString(flagAddress)
home := viper.GetString("home")
app, err := s.app(home, s.logger)
app, err := s.appCreator(home, s.logger)
if err != nil {
return err
}
@ -92,7 +92,7 @@ func (s startCmd) startInProcess() error {
}
home := cfg.RootDir
app, err := s.app(home, s.logger)
app, err := s.appCreator(home, s.logger)
if err != nil {
return err
}

View File

@ -27,8 +27,7 @@ func TestStartStandAlone(t *testing.T) {
startCmd.Flags().Set(flagAddress, FreeTCPAddr(t)) // set to a new free address
timeout := time.Duration(3) * time.Second
ch := RunOrTimeout(startCmd, timeout, t)
close(ch)
RunOrTimeout(startCmd, timeout, t)
}
func TestStartWithTendermint(t *testing.T) {
@ -50,6 +49,5 @@ func TestStartWithTendermint(t *testing.T) {
//a, _ := startCmd.Flags().GetString(flagAddress)
//panic(a)
ch := RunOrTimeout(startCmd, timeout, t)
close(ch)
RunOrTimeout(startCmd, timeout, t)
}

View File

@ -2,7 +2,6 @@ package types
import (
"encoding/json"
"fmt"
)
// Transactions messages must fulfill the Msg
@ -125,7 +124,6 @@ type StdSignDoc struct {
// StdSignBytes returns the bytes to sign for a transaction.
// TODO: change the API to just take a chainID and StdTx ?
func StdSignBytes(chainID string, sequences []int64, fee StdFee, msg Msg) []byte {
fmt.Println("FEE BYTES BABY", fee, string(fee.Bytes()))
bz, err := json.Marshal(StdSignDoc{
ChainID: chainID,
Sequences: sequences,

View File

@ -5,6 +5,7 @@ import (
"fmt"
sdk "github.com/cosmos/cosmos-sdk/types"
"github.com/spf13/viper"
)
// NewAnteHandler returns an AnteHandler that checks
@ -46,6 +47,12 @@ func NewAnteHandler(accountMapper sdk.AccountMapper) sdk.AnteHandler {
sequences[i] = sigs[i].Sequence
}
fee := stdTx.Fee
chainID := ctx.ChainID()
// XXX: major hack; need to get ChainID
// into the app right away (#565)
if chainID == "" {
chainID = viper.GetString("chain-id")
}
signBytes := sdk.StdSignBytes(ctx.ChainID(), sequences, fee, msg)
// Check sig and nonce and collect signer accounts.

View File

@ -1,11 +1,14 @@
package rest
import (
"github.com/cosmos/cosmos-sdk/wire"
"github.com/gorilla/mux"
keys "github.com/tendermint/go-crypto/keys"
"github.com/cosmos/cosmos-sdk/wire"
)
// RegisterRoutes - Central function to define routes that get registered by the main application
func RegisterRoutes(r *mux.Router, cdc *wire.Codec) {
r.HandleFunc("/accounts/{address}/send", SendRequestHandler(cdc)).Methods("POST")
func RegisterRoutes(r *mux.Router, cdc *wire.Codec, kb keys.Keybase) {
r.HandleFunc("/accounts/{address}/send", SendRequestHandler(cdc, kb)).Methods("POST")
}

View File

@ -7,9 +7,11 @@ import (
"net/http"
"github.com/gorilla/mux"
"github.com/spf13/viper"
"github.com/tendermint/go-crypto/keys"
"github.com/cosmos/cosmos-sdk/client"
"github.com/cosmos/cosmos-sdk/client/builder"
"github.com/cosmos/cosmos-sdk/client/keys"
sdk "github.com/cosmos/cosmos-sdk/types"
"github.com/cosmos/cosmos-sdk/wire"
"github.com/cosmos/cosmos-sdk/x/bank/commands"
@ -26,7 +28,7 @@ type sendBody struct {
}
// SendRequestHandler - http request handler to send coins to a address
func SendRequestHandler(cdc *wire.Codec) func(http.ResponseWriter, *http.Request) {
func SendRequestHandler(cdc *wire.Codec, kb keys.Keybase) func(http.ResponseWriter, *http.Request) {
c := commands.Commander{cdc}
return func(w http.ResponseWriter, r *http.Request) {
// collect data
@ -47,13 +49,6 @@ func SendRequestHandler(cdc *wire.Codec) func(http.ResponseWriter, *http.Request
return
}
kb, err := keys.GetKeyBase()
if err != nil {
w.WriteHeader(http.StatusInternalServerError)
w.Write([]byte(err.Error()))
return
}
info, err := kb.Get(m.LocalAccountName)
if err != nil {
w.WriteHeader(http.StatusUnauthorized)
@ -78,6 +73,8 @@ func SendRequestHandler(cdc *wire.Codec) func(http.ResponseWriter, *http.Request
}
// sign
// XXX: OMG
viper.Set(client.FlagSequence, m.Sequence)
txBytes, err := builder.SignAndBuild(m.LocalAccountName, m.Password, msg, c.Cdc)
if err != nil {
w.WriteHeader(http.StatusUnauthorized)