Browse Source

No global config

pull/213/merge
Jae Kwon 8 years ago
parent
commit
ad17090a0f
30 changed files with 122 additions and 331 deletions
  1. +0
    -13
      cmd/tendermint/config.go
  2. +6
    -5
      cmd/tendermint/main.go
  3. +2
    -6
      config/tendermint_test/config.go
  4. +6
    -12
      consensus/common_test.go
  5. +0
    -13
      consensus/config.go
  6. +7
    -5
      consensus/height_vote_set.go
  7. +2
    -2
      consensus/height_vote_set_test.go
  8. +1
    -1
      consensus/mempool_test.go
  9. +4
    -5
      consensus/replay.go
  10. +11
    -8
      consensus/state.go
  11. +3
    -3
      consensus/state_test.go
  12. +0
    -14
      mempool/config.go
  13. +7
    -3
      mempool/mempool.go
  14. +2
    -5
      mempool/mempool_test.go
  15. +6
    -3
      mempool/reactor.go
  16. +0
    -14
      node/config.go
  17. +28
    -23
      node/node.go
  18. +2
    -5
      node/node_test.go
  19. +0
    -13
      rpc/core/config.go
  20. +0
    -27
      rpc/core/dev.go
  21. +0
    -9
      rpc/core/routes.go
  22. +0
    -71
      rpc/test/client_test.go
  23. +0
    -13
      rpc/test/config.go
  24. +6
    -2
      rpc/test/helpers.go
  25. +1
    -6
      types/block.go
  26. +0
    -13
      types/config.go
  27. +2
    -10
      types/proposal_test.go
  28. +0
    -14
      types/vote.go
  29. +24
    -6
      types/vote_set.go
  30. +2
    -7
      types/vote_set_test.go

+ 0
- 13
cmd/tendermint/config.go View File

@ -1,13 +0,0 @@
package main
import (
cfg "github.com/tendermint/go-config"
)
var config cfg.Config = nil
func init() {
cfg.OnConfig(func(newConfig cfg.Config) {
config = newConfig
})
}

+ 6
- 5
cmd/tendermint/main.go View File

@ -11,6 +11,8 @@ import (
"github.com/tendermint/tendermint/version"
)
var config cfg.Config
func main() {
args := os.Args[1:]
@ -28,18 +30,17 @@ Commands:
}
// Get configuration
config := tmcfg.GetConfig("")
config = tmcfg.GetConfig("")
parseFlags(config, args[1:]) // Command line overrides
cfg.ApplyConfig(config) // Notify modules of new config
switch args[0] {
case "node":
node.RunNode()
node.RunNode(config)
case "replay":
if len(args) > 1 && args[1] == "console" {
node.RunReplayConsole()
node.RunReplayConsole(config)
} else {
node.RunReplay()
node.RunReplay(config)
}
case "init":
init_files()


+ 2
- 6
config/tendermint_test/config.go View File

@ -16,11 +16,6 @@ func init() {
EnsureDir(os.Getenv("HOME")+"/.tendermint_test", 0700)
}
func ResetConfig(path string) {
rootDir := os.Getenv("HOME") + "/.tendermint_test/" + path
cfg.ApplyConfig(GetConfig(rootDir))
}
func initTMRoot(rootDir string) {
// Remove ~/.tendermint_test_bak
if FileExists(rootDir + "_bak") {
@ -56,7 +51,8 @@ func initTMRoot(rootDir string) {
MustWriteFile(privFilePath, []byte(defaultPrivValidator), 0644)
}
func GetConfig(rootDir string) cfg.Config {
func ResetConfig(localPath string) cfg.Config {
rootDir := os.Getenv("HOME") + "/.tendermint_test/" + localPath
initTMRoot(rootDir)
configFilePath := path.Join(rootDir, "config.toml")


+ 6
- 12
consensus/common_test.go View File

@ -8,10 +8,10 @@ import (
"testing"
"time"
cfg "github.com/tendermint/go-config"
dbm "github.com/tendermint/go-db"
"github.com/tendermint/go-events"
bc "github.com/tendermint/tendermint/blockchain"
"github.com/tendermint/tendermint/config/tendermint_test"
mempl "github.com/tendermint/tendermint/mempool"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
@ -21,15 +21,9 @@ import (
"github.com/tendermint/tmsp/example/counter"
)
var chainID string
var config cfg.Config // NOTE: must be reset for each _test.go file
var ensureTimeout = time.Duration(2)
func init() {
tendermint_test.ResetConfig("consensus_common_test")
chainID = config.GetString("chain_id")
}
type validatorStub struct {
Height int
Round int
@ -50,7 +44,7 @@ func (vs *validatorStub) signVote(voteType byte, hash []byte, header types.PartS
BlockHash: hash,
BlockPartsHeader: header,
}
err := vs.PrivValidator.SignVote(chainID, vote)
err := vs.PrivValidator.SignVote(config.GetString("chain_id"), vote)
return vote, err
}
@ -72,7 +66,7 @@ func decideProposal(cs1 *ConsensusState, cs2 *validatorStub, height, round int)
// Make proposal
proposal = types.NewProposal(height, round, blockParts.Header(), cs1.Votes.POLRound())
if err := cs2.SignProposal(chainID, proposal); err != nil {
if err := cs2.SignProposal(config.GetString("chain_id"), proposal); err != nil {
panic(err)
}
return
@ -318,10 +312,10 @@ func newConsensusState(state *sm.State, pv *types.PrivValidator, app tmsp.Applic
proxyAppConnCon := tmspcli.NewLocalClient(mtx, app)
// Make Mempool
mempool := mempl.NewMempool(proxyAppConnMem)
mempool := mempl.NewMempool(config, proxyAppConnMem)
// Make ConsensusReactor
cs := NewConsensusState(state, proxyAppConnCon, blockStore, mempool)
cs := NewConsensusState(config, state, proxyAppConnCon, blockStore, mempool)
cs.SetPrivValidator(pv)
evsw := events.NewEventSwitch()


+ 0
- 13
consensus/config.go View File

@ -1,13 +0,0 @@
package consensus
import (
cfg "github.com/tendermint/go-config"
)
var config cfg.Config = nil
func init() {
cfg.OnConfig(func(newConfig cfg.Config) {
config = newConfig
})
}

+ 7
- 5
consensus/height_vote_set.go View File

@ -26,8 +26,9 @@ we create a new entry in roundVoteSets but also remember the
peer to prevent abuse.
*/
type HeightVoteSet struct {
height int
valSet *types.ValidatorSet
chainID string
height int
valSet *types.ValidatorSet
mtx sync.Mutex
round int // max tracked round
@ -35,8 +36,9 @@ type HeightVoteSet struct {
peerCatchupRounds map[string]int // keys: peer.Key; values: round
}
func NewHeightVoteSet(height int, valSet *types.ValidatorSet) *HeightVoteSet {
func NewHeightVoteSet(chainID string, height int, valSet *types.ValidatorSet) *HeightVoteSet {
hvs := &HeightVoteSet{
chainID: chainID,
height: height,
valSet: valSet,
roundVoteSets: make(map[int]RoundVoteSet),
@ -78,8 +80,8 @@ func (hvs *HeightVoteSet) addRound(round int) {
PanicSanity("addRound() for an existing round")
}
log.Debug("addRound(round)", "round", round)
prevotes := types.NewVoteSet(hvs.height, round, types.VoteTypePrevote, hvs.valSet)
precommits := types.NewVoteSet(hvs.height, round, types.VoteTypePrecommit, hvs.valSet)
prevotes := types.NewVoteSet(hvs.chainID, hvs.height, round, types.VoteTypePrevote, hvs.valSet)
precommits := types.NewVoteSet(hvs.chainID, hvs.height, round, types.VoteTypePrecommit, hvs.valSet)
hvs.roundVoteSets[round] = RoundVoteSet{
Prevotes: prevotes,
Precommits: precommits,


+ 2
- 2
consensus/height_vote_set_test.go View File

@ -8,13 +8,13 @@ import (
)
func init() {
tendermint_test.ResetConfig("consensus_height_vote_set_test")
config = tendermint_test.ResetConfig("consensus_height_vote_set_test")
}
func TestPeerCatchupRounds(t *testing.T) {
valSet, privVals := types.RandValidatorSet(10, 1)
hvs := NewHeightVoteSet(1, valSet)
hvs := NewHeightVoteSet(config.GetString("chain_id"), 1, valSet)
vote999_0 := makeVoteHR(t, 1, 999, privVals[0])
added, _, err := hvs.AddByIndex(0, vote999_0, "peer1")


+ 1
- 1
consensus/mempool_test.go View File

@ -14,7 +14,7 @@ import (
)
func init() {
tendermint_test.ResetConfig("consensus_mempool_test")
config = tendermint_test.ResetConfig("consensus_mempool_test")
}
func TestTxConcurrentWithCommit(t *testing.T) {


+ 4
- 5
consensus/replay.go View File

@ -230,10 +230,9 @@ func (pb *playback) replayReset(count int, newStepCh chan interface{}) error {
pb.cs.Stop()
newCs := NewConsensusState(pb.genesisState.Copy(), pb.cs.proxyAppConn, pb.cs.blockStore, pb.cs.mempool)
newCs.SetEventSwitch(pb.cs.evsw)
newCs.startForReplay()
newCS := NewConsensusState(pb.cs.config, pb.genesisState.Copy(), pb.cs.proxyAppConn, pb.cs.blockStore, pb.cs.mempool)
newCS.SetEventSwitch(pb.cs.evsw)
newCS.startForReplay()
pb.fp.Close()
fp, err := os.OpenFile(pb.fileName, os.O_RDONLY, 0666)
@ -245,7 +244,7 @@ func (pb *playback) replayReset(count int, newStepCh chan interface{}) error {
count = pb.count - count
log.Notice(Fmt("Reseting from %d to %d", pb.count, count))
pb.count = 0
pb.cs = newCs
pb.cs = newCS
for i := 0; pb.scanner.Scan() && i < count; i++ {
if err := pb.cs.readReplayMessage(pb.scanner.Bytes(), newStepCh); err != nil {
return err


+ 11
- 8
consensus/state.go View File

@ -9,6 +9,7 @@ import (
"time"
. "github.com/tendermint/go-common"
cfg "github.com/tendermint/go-config"
"github.com/tendermint/go-events"
"github.com/tendermint/go-wire"
bc "github.com/tendermint/tendermint/blockchain"
@ -52,8 +53,8 @@ func (tp *TimeoutParams) Commit(t time.Time) time.Time {
return t.Add(time.Duration(tp.Commit0) * time.Millisecond)
}
// Initialize parameters from config file
func InitTimeoutParamsFromConfig() *TimeoutParams {
// Initialize parameters from config
func InitTimeoutParamsFromConfig(config cfg.Config) *TimeoutParams {
return &TimeoutParams{
Propose0: config.GetInt("timeout_propose"),
ProposeDelta: config.GetInt("timeout_propose_delta"),
@ -213,6 +214,7 @@ func (ti *timeoutInfo) String() string {
type ConsensusState struct {
QuitService
config cfg.Config
proxyAppConn proxy.AppConn
blockStore *bc.BlockStore
mempool *mempl.Mempool
@ -236,8 +238,9 @@ type ConsensusState struct {
nSteps int // used for testing to limit the number of transitions the state makes
}
func NewConsensusState(state *sm.State, proxyAppConn proxy.AppConn, blockStore *bc.BlockStore, mempool *mempl.Mempool) *ConsensusState {
func NewConsensusState(config cfg.Config, state *sm.State, proxyAppConn proxy.AppConn, blockStore *bc.BlockStore, mempool *mempl.Mempool) *ConsensusState {
cs := &ConsensusState{
config: config,
proxyAppConn: proxyAppConn,
blockStore: blockStore,
mempool: mempool,
@ -246,7 +249,7 @@ func NewConsensusState(state *sm.State, proxyAppConn proxy.AppConn, blockStore *
timeoutTicker: new(time.Ticker),
tickChan: make(chan timeoutInfo, tickTockBufferSize),
tockChan: make(chan timeoutInfo, tickTockBufferSize),
timeoutParams: InitTimeoutParamsFromConfig(),
timeoutParams: InitTimeoutParamsFromConfig(config),
}
cs.updateToState(state)
// Don't call scheduleRound0 yet.
@ -328,7 +331,7 @@ func (cs *ConsensusState) OnStop() {
func (cs *ConsensusState) OpenWAL(file string) (err error) {
cs.mtx.Lock()
defer cs.mtx.Unlock()
wal, err := NewWAL(file, config.GetBool("cswal_light"))
wal, err := NewWAL(file, cs.config.GetBool("cswal_light"))
if err != nil {
return err
}
@ -439,7 +442,7 @@ func (cs *ConsensusState) reconstructLastCommit(state *sm.State) {
return
}
seenCommit := cs.blockStore.LoadSeenCommit(state.LastBlockHeight)
lastPrecommits := types.NewVoteSet(state.LastBlockHeight, seenCommit.Round(), types.VoteTypePrecommit, state.LastValidators)
lastPrecommits := types.NewVoteSet(cs.config.GetString("chain_id"), state.LastBlockHeight, seenCommit.Round(), types.VoteTypePrecommit, state.LastValidators)
for idx, precommit := range seenCommit.Precommits {
if precommit == nil {
continue
@ -509,7 +512,7 @@ func (cs *ConsensusState) updateToState(state *sm.State) {
cs.LockedRound = 0
cs.LockedBlock = nil
cs.LockedBlockParts = nil
cs.Votes = NewHeightVoteSet(height, validators)
cs.Votes = NewHeightVoteSet(cs.config.GetString("chain_id"), height, validators)
cs.CommitRound = -1
cs.LastCommit = lastPrecommits
cs.LastValidators = state.LastValidators
@ -878,7 +881,7 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
}
// Mempool validated transactions
txs := cs.mempool.Reap(config.GetInt("block_size"))
txs := cs.mempool.Reap(cs.config.GetInt("block_size"))
block = &types.Block{
Header: &types.Header{


+ 3
- 3
consensus/state_test.go View File

@ -12,7 +12,7 @@ import (
)
func init() {
tendermint_test.ResetConfig("consensus_state_test")
config = tendermint_test.ResetConfig("consensus_state_test")
}
func (tp *TimeoutParams) ensureProposeTimeout() time.Duration {
@ -199,7 +199,7 @@ func TestBadProposal(t *testing.T) {
propBlock.AppHash = stateHash
propBlockParts := propBlock.MakePartSet()
proposal := types.NewProposal(cs2.Height, round, propBlockParts.Header(), -1)
if err := cs2.SignProposal(chainID, proposal); err != nil {
if err := cs2.SignProposal(config.GetString("chain_id"), proposal); err != nil {
t.Fatal("failed to sign bad proposal", err)
}
@ -857,7 +857,7 @@ func TestLockPOLSafety2(t *testing.T) {
// in round 2 we see the polkad block from round 0
newProp := types.NewProposal(height, 2, propBlockParts0.Header(), 0)
if err := cs3.SignProposal(chainID, newProp); err != nil {
if err := cs3.SignProposal(config.GetString("chain_id"), newProp); err != nil {
t.Fatal(err)
}
cs1.SetProposalAndBlock(newProp, propBlock0, propBlockParts0, "some peer")


+ 0
- 14
mempool/config.go View File

@ -1,14 +0,0 @@
package mempool
import (
cfg "github.com/tendermint/go-config"
)
var config cfg.Config = nil
func init() {
cfg.OnConfig(func(newConfig cfg.Config) {
config = newConfig
})
}

+ 7
- 3
mempool/mempool.go View File

@ -9,6 +9,7 @@ import (
"github.com/tendermint/go-clist"
. "github.com/tendermint/go-common"
cfg "github.com/tendermint/go-config"
"github.com/tendermint/tendermint/proxy"
"github.com/tendermint/tendermint/types"
tmsp "github.com/tendermint/tmsp/types"
@ -45,6 +46,8 @@ TODO: Better handle tmsp client errors. (make it automatically handle connection
const cacheSize = 100000
type Mempool struct {
config cfg.Config
proxyMtx sync.Mutex
proxyAppConn proxy.AppConn
txs *clist.CList // concurrent linked-list of good txs
@ -60,8 +63,9 @@ type Mempool struct {
cacheList *list.List
}
func NewMempool(proxyAppConn proxy.AppConn) *Mempool {
func NewMempool(config cfg.Config, proxyAppConn proxy.AppConn) *Mempool {
mempool := &Mempool{
config: config,
proxyAppConn: proxyAppConn,
txs: clist.New(),
counter: 0,
@ -248,8 +252,8 @@ func (mem *Mempool) Update(height int, txs []types.Tx) {
// Recheck mempool txs if any txs were committed in the block
// NOTE/XXX: in some apps a tx could be invalidated due to EndBlock,
// so we really still do need to recheck, but this is for debugging
if config.GetBool("mempool_recheck") &&
(config.GetBool("mempool_recheck_empty") || len(txs) > 0) {
if mem.config.GetBool("mempool_recheck") &&
(mem.config.GetBool("mempool_recheck_empty") || len(txs) > 0) {
log.Info("Recheck txs", "numtxs", len(goodTxs))
mem.recheckTxs(goodTxs)
// At this point, mem.txs are being rechecked.


+ 2
- 5
mempool/mempool_test.go View File

@ -11,18 +11,15 @@ import (
"github.com/tendermint/tmsp/example/counter"
)
func init() {
tendermint_test.ResetConfig("mempool_mempool_test")
}
func TestSerialReap(t *testing.T) {
config := tendermint_test.ResetConfig("mempool_mempool_test")
app := counter.NewCounterApplication(true)
app.SetOption("serial", "on")
mtx := new(sync.Mutex)
appConnMem := tmspcli.NewLocalClient(mtx, app)
appConnCon := tmspcli.NewLocalClient(mtx, app)
mempool := NewMempool(appConnMem)
mempool := NewMempool(config, appConnMem)
appendTxsRange := func(start, end int) {
// Append some txs.


+ 6
- 3
mempool/reactor.go View File

@ -8,6 +8,7 @@ import (
"github.com/tendermint/go-clist"
. "github.com/tendermint/go-common"
cfg "github.com/tendermint/go-config"
"github.com/tendermint/go-events"
"github.com/tendermint/go-p2p"
"github.com/tendermint/go-wire"
@ -25,12 +26,14 @@ const (
// MempoolReactor handles mempool tx broadcasting amongst peers.
type MempoolReactor struct {
p2p.BaseReactor
Mempool *Mempool // TODO: un-expose
config cfg.Config
Mempool *Mempool
evsw *events.EventSwitch
}
func NewMempoolReactor(mempool *Mempool) *MempoolReactor {
func NewMempoolReactor(config cfg.Config, mempool *Mempool) *MempoolReactor {
memR := &MempoolReactor{
config: config,
Mempool: mempool,
}
memR.BaseReactor = *p2p.NewBaseReactor(log, "MempoolReactor", memR)
@ -101,7 +104,7 @@ type Peer interface {
// TODO: Handle mempool or reactor shutdown?
// As is this routine may block forever if no new txs come in.
func (memR *MempoolReactor) broadcastTxRoutine(peer Peer) {
if !config.GetBool("mempool_broadcast") {
if !memR.config.GetBool("mempool_broadcast") {
return
}


+ 0
- 14
node/config.go View File

@ -1,14 +0,0 @@
package node
import (
cfg "github.com/tendermint/go-config"
)
var config cfg.Config = nil
func init() {
cfg.OnConfig(func(newConfig cfg.Config) {
config = newConfig
})
}

+ 28
- 23
node/node.go View File

@ -10,6 +10,7 @@ import (
"time"
. "github.com/tendermint/go-common"
cfg "github.com/tendermint/go-config"
"github.com/tendermint/go-crypto"
dbm "github.com/tendermint/go-db"
"github.com/tendermint/go-events"
@ -33,6 +34,7 @@ import (
import _ "net/http/pprof"
type Node struct {
config cfg.Config
sw *p2p.Switch
evsw *events.EventSwitch
blockStore *bc.BlockStore
@ -45,16 +47,19 @@ type Node struct {
privKey crypto.PrivKeyEd25519
}
func NewNode(privValidator *types.PrivValidator, getProxyApp func(proxyAddr string, appHash []byte) proxy.AppConn) *Node {
func NewNode(config cfg.Config, privValidator *types.PrivValidator, getProxyApp func(proxyAddr string, appHash []byte) proxy.AppConn) *Node {
EnsureDir(config.GetString("db_dir"), 0700) // incase we use memdb, cswal still gets written here
// Get BlockStore
blockStoreDB := dbm.GetDB("blockstore")
blockStoreDB := dbm.NewDB("blockstore", config.GetString("db_backend"), config.GetString("db_dir"))
blockStore := bc.NewBlockStore(blockStoreDB)
// Get State db
stateDB := dbm.NewDB("state", config.GetString("db_backend"), config.GetString("db_dir"))
// Get State
state := getState()
state := getState(config, stateDB)
// Create two proxyAppConn connections,
// one for the consensus and one for the mempool.
@ -90,11 +95,11 @@ func NewNode(privValidator *types.PrivValidator, getProxyApp func(proxyAddr stri
bcReactor := bc.NewBlockchainReactor(state.Copy(), proxyAppConnConsensus, blockStore, fastSync)
// Make MempoolReactor
mempool := mempl.NewMempool(proxyAppConnMempool)
mempoolReactor := mempl.NewMempoolReactor(mempool)
mempool := mempl.NewMempool(config, proxyAppConnMempool)
mempoolReactor := mempl.NewMempoolReactor(config, mempool)
// Make ConsensusReactor
consensusState := consensus.NewConsensusState(state.Copy(), proxyAppConnConsensus, blockStore, mempool)
consensusState := consensus.NewConsensusState(config, state.Copy(), proxyAppConnConsensus, blockStore, mempool)
consensusReactor := consensus.NewConsensusReactor(consensusState, blockStore, fastSync)
if privValidator != nil {
consensusReactor.SetPrivValidator(privValidator)
@ -107,7 +112,7 @@ func NewNode(privValidator *types.PrivValidator, getProxyApp func(proxyAddr stri
}
// Make p2p network switch
sw := p2p.NewSwitch()
sw := p2p.NewSwitch(config)
sw.AddReactor("MEMPOOL", mempoolReactor)
sw.AddReactor("BLOCKCHAIN", bcReactor)
sw.AddReactor("CONSENSUS", consensusReactor)
@ -125,6 +130,7 @@ func NewNode(privValidator *types.PrivValidator, getProxyApp func(proxyAddr stri
}
return &Node{
config: config,
sw: sw,
evsw: eventSwitch,
blockStore: blockStore,
@ -140,7 +146,7 @@ func NewNode(privValidator *types.PrivValidator, getProxyApp func(proxyAddr stri
// Call Start() after adding the listeners.
func (n *Node) Start() error {
n.sw.SetNodeInfo(makeNodeInfo(n.sw, n.privKey))
n.sw.SetNodeInfo(makeNodeInfo(n.config, n.sw, n.privKey))
n.sw.SetNodePrivKey(n.privKey)
_, err := n.sw.Start()
return err
@ -176,7 +182,7 @@ func (n *Node) StartRPC() ([]net.Listener, error) {
rpccore.SetPrivValidator(n.privValidator)
rpccore.SetGenesisDoc(n.genesisDoc)
listenAddrs := strings.Split(config.GetString("rpc_laddr"), ",")
listenAddrs := strings.Split(n.config.GetString("rpc_laddr"), ",")
// we may expose the rpc over both a unix and tcp socket
listeners := make([]net.Listener, len(listenAddrs))
@ -214,7 +220,7 @@ func (n *Node) EventSwitch() *events.EventSwitch {
return n.evsw
}
func makeNodeInfo(sw *p2p.Switch, privKey crypto.PrivKeyEd25519) *p2p.NodeInfo {
func makeNodeInfo(config cfg.Config, sw *p2p.Switch, privKey crypto.PrivKeyEd25519) *p2p.NodeInfo {
nodeInfo := &p2p.NodeInfo{
PubKey: privKey.PubKey().(crypto.PubKeyEd25519),
@ -287,8 +293,7 @@ func GetProxyApp(addr string, hash []byte) (proxyAppConn proxy.AppConn) {
// Load the most recent state from "state" db,
// or create a new one (and save) from genesis.
func getState() *sm.State {
stateDB := dbm.GetDB("state")
func getState(config cfg.Config, stateDB dbm.DB) *sm.State {
state := sm.LoadState(stateDB)
if state == nil {
state = sm.MakeGenesisStateFromFile(stateDB, config.GetString("genesis_file"))
@ -302,7 +307,7 @@ func getState() *sm.State {
// Users wishing to use an external signer for their validators
// should fork tendermint/tendermint and implement RunNode to
// load their custom priv validator and call NewNode(privVal, getProxyFunc)
func RunNode() {
func RunNode(config cfg.Config) {
// Wait until the genesis doc becomes available
genDocFile := config.GetString("genesis_file")
if !FileExists(genDocFile) {
@ -330,7 +335,7 @@ func RunNode() {
privValidator := types.LoadOrGenPrivValidator(privValidatorFile)
// Create & start node
n := NewNode(privValidator, GetProxyApp)
n := NewNode(config, privValidator, GetProxyApp)
l := p2p.NewDefaultListener("tcp", config.GetString("node_laddr"), config.GetBool("skip_upnp"))
n.AddListener(l)
err := n.Start()
@ -372,13 +377,13 @@ func (n *Node) DialSeeds(seeds []string) {
// replay
// convenience for replay mode
func newConsensusState() *consensus.ConsensusState {
func newConsensusState(config cfg.Config) *consensus.ConsensusState {
// Get BlockStore
blockStoreDB := dbm.GetDB("blockstore")
blockStoreDB := dbm.NewDB("blockstore", config.GetString("db_backend"), config.GetString("db_dir"))
blockStore := bc.NewBlockStore(blockStoreDB)
// Get State
stateDB := dbm.GetDB("state")
stateDB := dbm.NewDB("state", config.GetString("db_backend"), config.GetString("db_dir"))
state := sm.MakeGenesisStateFromFile(stateDB, config.GetString("genesis_file"))
// Create two proxyAppConn connections,
@ -397,33 +402,33 @@ func newConsensusState() *consensus.ConsensusState {
Exit(Fmt("Failed to start event switch: %v", err))
}
mempool := mempl.NewMempool(proxyAppConnMempool)
mempool := mempl.NewMempool(config, proxyAppConnMempool)
consensusState := consensus.NewConsensusState(state.Copy(), proxyAppConnConsensus, blockStore, mempool)
consensusState := consensus.NewConsensusState(config, state.Copy(), proxyAppConnConsensus, blockStore, mempool)
consensusState.SetEventSwitch(eventSwitch)
return consensusState
}
func RunReplayConsole() {
func RunReplayConsole(config cfg.Config) {
walFile := config.GetString("cswal")
if walFile == "" {
Exit("cswal file name not set in tendermint config")
}
consensusState := newConsensusState()
consensusState := newConsensusState(config)
if err := consensusState.ReplayConsole(walFile); err != nil {
Exit(Fmt("Error during consensus replay: %v", err))
}
}
func RunReplay() {
func RunReplay(config cfg.Config) {
walFile := config.GetString("cswal")
if walFile == "" {
Exit("cswal file name not set in tendermint config")
}
consensusState := newConsensusState()
consensusState := newConsensusState(config)
if err := consensusState.ReplayMessages(walFile); err != nil {
Exit(Fmt("Error during consensus replay: %v", err))


+ 2
- 5
node/node_test.go View File

@ -9,18 +9,15 @@ import (
"github.com/tendermint/tendermint/types"
)
func init() {
tendermint_test.ResetConfig("node_node_test")
}
func TestNodeStartStop(t *testing.T) {
config := tendermint_test.ResetConfig("node_node_test")
// Get PrivValidator
privValidatorFile := config.GetString("priv_validator_file")
privValidator := types.LoadOrGenPrivValidator(privValidatorFile)
// Create & start node
n := NewNode(privValidator, GetProxyApp)
n := NewNode(config, privValidator, GetProxyApp)
l := p2p.NewDefaultListener("tcp", config.GetString("node_laddr"), config.GetBool("skip_upnp"))
n.AddListener(l)
n.Start()


+ 0
- 13
rpc/core/config.go View File

@ -1,13 +0,0 @@
package core
import (
cfg "github.com/tendermint/go-config"
)
var config cfg.Config = nil
func init() {
cfg.OnConfig(func(newConfig cfg.Config) {
config = newConfig
})
}

+ 0
- 27
rpc/core/dev.go View File

@ -1,39 +1,12 @@
package core
import (
"fmt"
"os"
"runtime/pprof"
"strconv"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
)
func UnsafeSetConfig(typ, key, value string) (*ctypes.ResultUnsafeSetConfig, error) {
switch typ {
case "string":
config.Set(key, value)
case "int":
val, err := strconv.Atoi(value)
if err != nil {
return nil, fmt.Errorf("non-integer value found. key:%s; value:%s; err:%v", key, value, err)
}
config.Set(key, val)
case "bool":
switch value {
case "true":
config.Set(key, true)
case "false":
config.Set(key, false)
default:
return nil, fmt.Errorf("bool value must be true or false. got %s", value)
}
default:
return nil, fmt.Errorf("Unknown type %s", typ)
}
return &ctypes.ResultUnsafeSetConfig{}, nil
}
var profFile *os.File
func UnsafeStartCPUProfiler(filename string) (*ctypes.ResultUnsafeProfile, error) {


+ 0
- 9
rpc/core/routes.go View File

@ -24,7 +24,6 @@ var Routes = map[string]*rpc.RPCFunc{
"unconfirmed_txs": rpc.NewRPCFunc(UnconfirmedTxsResult, ""),
"num_unconfirmed_txs": rpc.NewRPCFunc(NumUnconfirmedTxsResult, ""),
"unsafe_set_config": rpc.NewRPCFunc(UnsafeSetConfigResult, "type,key,value"),
"unsafe_start_cpu_profiler": rpc.NewRPCFunc(UnsafeStartCPUProfilerResult, "filename"),
"unsafe_stop_cpu_profiler": rpc.NewRPCFunc(UnsafeStopCPUProfilerResult, ""),
"unsafe_write_heap_profile": rpc.NewRPCFunc(UnsafeWriteHeapProfileResult, "filename"),
@ -142,14 +141,6 @@ func BroadcastTxAsyncResult(tx []byte) (ctypes.TMResult, error) {
}
}
func UnsafeSetConfigResult(typ, key, value string) (ctypes.TMResult, error) {
if r, err := UnsafeSetConfig(typ, key, value); err != nil {
return nil, err
} else {
return r, nil
}
}
func UnsafeStartCPUProfilerResult(filename string) (ctypes.TMResult, error) {
if r, err := UnsafeStartCPUProfiler(filename); err != nil {
return nil, err


+ 0
- 71
rpc/test/client_test.go View File

@ -4,16 +4,10 @@ import (
"fmt"
"testing"
"github.com/tendermint/tendermint/config/tendermint_test"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
"github.com/tendermint/tendermint/types"
)
func init() {
tendermint_test.ResetConfig("rpc_test_client_test")
initGlobalVariables()
}
//--------------------------------------------------------------------------------
// Test the HTTP client
//--------------------------------------------------------------------------------
@ -48,71 +42,6 @@ func testStatus(t *testing.T, statusI interface{}) {
}
}
//--------------------------------------------------------------------------------
// unsafe_set_config
var stringVal = "my string"
var intVal = 987654321
var boolVal = true
// don't change these
var testCasesUnsafeSetConfig = [][]string{
[]string{"string", "key1", stringVal},
[]string{"int", "key2", fmt.Sprintf("%v", intVal)},
[]string{"bool", "key3", fmt.Sprintf("%v", boolVal)},
}
func TestURIUnsafeSetConfig(t *testing.T) {
for _, testCase := range testCasesUnsafeSetConfig {
tmResult := new(ctypes.TMResult)
_, err := clientURI.Call("unsafe_set_config", map[string]interface{}{
"type": testCase[0],
"key": testCase[1],
"value": testCase[2],
}, tmResult)
if err != nil {
t.Fatal(err)
}
}
testUnsafeSetConfig(t)
}
func TestJSONUnsafeSetConfig(t *testing.T) {
for _, testCase := range testCasesUnsafeSetConfig {
tmResult := new(ctypes.TMResult)
_, err := clientJSON.Call("unsafe_set_config", []interface{}{testCase[0], testCase[1], testCase[2]}, tmResult)
if err != nil {
t.Fatal(err)
}
}
testUnsafeSetConfig(t)
}
func testUnsafeSetConfig(t *testing.T) {
s := config.GetString("key1")
if s != stringVal {
t.Fatalf("got %v, expected %v", s, stringVal)
}
i := config.GetInt("key2")
if i != intVal {
t.Fatalf("got %v, expected %v", i, intVal)
}
b := config.GetBool("key3")
if b != boolVal {
t.Fatalf("got %v, expected %v", b, boolVal)
}
}
/*func TestURIBroadcastTx(t *testing.T) {
testBroadcastTx(t, "HTTP")
}*/
/*func TestJSONBroadcastTx(t *testing.T) {
testBroadcastTx(t, "JSONRPC")
}*/
// TODO
/*
func testBroadcastTx(t *testing.T, typ string) {


+ 0
- 13
rpc/test/config.go View File

@ -1,13 +0,0 @@
package rpctest
import (
cfg "github.com/tendermint/go-config"
)
var config cfg.Config = nil
func init() {
cfg.OnConfig(func(newConfig cfg.Config) {
config = newConfig
})
}

+ 6
- 2
rpc/test/helpers.go View File

@ -5,10 +5,12 @@ import (
"time"
. "github.com/tendermint/go-common"
cfg "github.com/tendermint/go-config"
"github.com/tendermint/go-p2p"
"github.com/tendermint/go-wire"
client "github.com/tendermint/go-rpc/client"
"github.com/tendermint/tendermint/config/tendermint_test"
nm "github.com/tendermint/tendermint/node"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
"github.com/tendermint/tendermint/types"
@ -16,6 +18,7 @@ import (
// global variables for use across all tests
var (
config cfg.Config
node *nm.Node
mempoolCount = 0
chainID string
@ -28,7 +31,8 @@ var (
)
// initialize config and create new node
func initGlobalVariables() {
func init() {
config = tendermint_test.ResetConfig("rpc_test_client_test")
chainID = config.GetString("chain_id")
rpcAddr = config.GetString("rpc_laddr")
requestAddr = rpcAddr
@ -51,7 +55,7 @@ func newNode(ready chan struct{}) {
// Create & start node
privValidatorFile := config.GetString("priv_validator_file")
privValidator := types.LoadOrGenPrivValidator(privValidatorFile)
node = nm.NewNode(privValidator, nm.GetProxyApp)
node = nm.NewNode(config, privValidator, nm.GetProxyApp)
l := p2p.NewDefaultListener("tcp", config.GetString("node_laddr"), true)
node.AddListener(l)
node.Start()


+ 1
- 6
types/block.go View File

@ -75,7 +75,7 @@ func (b *Block) FillHeader() {
// Computes and returns the block hash.
// If the block is incomplete, block hash is nil for safety.
func (b *Block) Hash() []byte {
fmt.Println(">>", b.Data)
// fmt.Println(">>", b.Data)
if b.Header == nil || b.Data == nil || b.LastCommit == nil {
return nil
}
@ -336,11 +336,6 @@ type Data struct {
}
func (data *Data) Hash() []byte {
if config.GetBool("disable_data_hash") {
// we could use the part_set hash instead
data.hash = []byte{}
return data.hash
}
if data.hash == nil {
data.hash = data.Txs.Hash() // NOTE: leaves of merkle tree are TxIDs
}


+ 0
- 13
types/config.go View File

@ -1,13 +0,0 @@
package types
import (
cfg "github.com/tendermint/go-config"
)
var config cfg.Config = nil
func init() {
cfg.OnConfig(func(newConfig cfg.Config) {
config = newConfig
})
}

+ 2
- 10
types/proposal_test.go View File

@ -2,15 +2,8 @@ package types
import (
"testing"
. "github.com/tendermint/go-common"
"github.com/tendermint/tendermint/config/tendermint_test"
)
func init() {
tendermint_test.ResetConfig("types_proposal_test")
}
func TestProposalSignable(t *testing.T) {
proposal := &Proposal{
Height: 12345,
@ -18,11 +11,10 @@ func TestProposalSignable(t *testing.T) {
BlockPartsHeader: PartSetHeader{111, []byte("blockparts")},
POLRound: -1,
}
signBytes := SignBytes(config.GetString("chain_id"), proposal)
signBytes := SignBytes("test_chain_id", proposal)
signStr := string(signBytes)
expected := Fmt(`{"chain_id":"%s","proposal":{"block_parts_header":{"hash":"626C6F636B7061727473","total":111},"height":12345,"pol_round":-1,"round":23456}}`,
config.GetString("chain_id"))
expected := `{"chain_id":"test_chain_id","proposal":{"block_parts_header":{"hash":"626C6F636B7061727473","total":111},"height":12345,"pol_round":-1,"round":23456}}`
if signStr != expected {
t.Errorf("Got unexpected sign string for SendTx. Expected:\n%v\nGot:\n%v", expected, signStr)
}


+ 0
- 14
types/vote.go View File

@ -69,17 +69,3 @@ func (vote *Vote) String() string {
return fmt.Sprintf("Vote{%v/%02d/%v(%v) %X#%v %v}", vote.Height, vote.Round, vote.Type, typeString, Fingerprint(vote.BlockHash), vote.BlockPartsHeader, vote.Signature)
}
//--------------------------------------------------------------------------------
// TODO: Move blocks/Commit to here?
// Common interface between *consensus.VoteSet and types.Commit
type VoteSetReader interface {
Height() int
Round() int
Type() byte
Size() int
BitArray() *BitArray
GetByIndex(int) *Vote
IsCommit() bool
}

+ 24
- 6
types/vote_set.go View File

@ -16,9 +16,10 @@ import (
// A commit of prior rounds can be added added in lieu of votes/precommits.
// NOTE: Assumes that the sum total of voting power does not exceed MaxUInt64.
type VoteSet struct {
height int
round int
type_ byte
chainID string
height int
round int
type_ byte
mtx sync.Mutex
valSet *ValidatorSet
@ -32,11 +33,12 @@ type VoteSet struct {
}
// Constructs a new VoteSet struct used to accumulate votes for given height/round.
func NewVoteSet(height int, round int, type_ byte, valSet *ValidatorSet) *VoteSet {
func NewVoteSet(chainID string, height int, round int, type_ byte, valSet *ValidatorSet) *VoteSet {
if height == 0 {
PanicSanity("Cannot make VoteSet for height == 0, doesn't make sense.")
}
return &VoteSet{
chainID: chainID,
height: height,
round: round,
type_: type_,
@ -48,6 +50,10 @@ func NewVoteSet(height int, round int, type_ byte, valSet *ValidatorSet) *VoteSe
}
}
func (voteSet *VoteSet) ChainID() string {
return voteSet.chainID
}
func (voteSet *VoteSet) Height() int {
if voteSet == nil {
return 0
@ -134,7 +140,7 @@ func (voteSet *VoteSet) addVote(val *Validator, valIndex int, vote *Vote) (bool,
return false, valIndex, nil
} else {
// Check signature.
if !val.PubKey.VerifyBytes(SignBytes(config.GetString("chain_id"), vote), vote.Signature) {
if !val.PubKey.VerifyBytes(SignBytes(voteSet.chainID, vote), vote.Signature) {
// Bad signature.
return false, 0, ErrVoteInvalidSignature
}
@ -146,7 +152,7 @@ func (voteSet *VoteSet) addVote(val *Validator, valIndex int, vote *Vote) (bool,
}
// Check signature.
if !val.PubKey.VerifyBytes(SignBytes(config.GetString("chain_id"), vote), vote.Signature) {
if !val.PubKey.VerifyBytes(SignBytes(voteSet.chainID, vote), vote.Signature) {
// Bad signature.
return false, 0, ErrVoteInvalidSignature
}
@ -305,3 +311,15 @@ func (voteSet *VoteSet) MakeCommit() *Commit {
Precommits: precommits,
}
}
//----------------------------------------
// Common interface between *consensus.VoteSet and types.Commit
type VoteSetReader interface {
Height() int
Round() int
Type() byte
Size() int
BitArray() *BitArray
GetByIndex(int) *Vote
IsCommit() bool
}

+ 2
- 7
types/vote_set_test.go View File

@ -6,19 +6,14 @@ import (
. "github.com/tendermint/go-common"
. "github.com/tendermint/go-common/test"
"github.com/tendermint/go-crypto"
"github.com/tendermint/tendermint/config/tendermint_test"
"testing"
)
func init() {
tendermint_test.ResetConfig("types_vote_set_test")
}
// Move it out?
func randVoteSet(height int, round int, type_ byte, numValidators int, votingPower int64) (*VoteSet, *ValidatorSet, []*PrivValidator) {
valSet, privValidators := RandValidatorSet(numValidators, votingPower)
return NewVoteSet(height, round, type_, valSet), valSet, privValidators
return NewVoteSet("test_chain_id", height, round, type_, valSet), valSet, privValidators
}
// Convenience: Return new vote with different height
@ -57,7 +52,7 @@ func withBlockPartsHeader(vote *Vote, blockPartsHeader PartSetHeader) *Vote {
}
func signAddVote(privVal *PrivValidator, vote *Vote, voteSet *VoteSet) (bool, error) {
vote.Signature = privVal.Sign(SignBytes(config.GetString("chain_id"), vote)).(crypto.SignatureEd25519)
vote.Signature = privVal.Sign(SignBytes(voteSet.ChainID(), vote)).(crypto.SignatureEd25519)
added, _, err := voteSet.AddByAddress(privVal.Address, vote)
return added, err
}


Loading…
Cancel
Save