tendermint/node/node.go

438 lines
13 KiB
Go
Raw Normal View History

package node
2014-07-08 00:02:04 -07:00
import (
2015-06-14 15:18:17 -07:00
"bytes"
"io/ioutil"
"net"
2015-04-10 02:12:17 -07:00
"net/http"
"strings"
2016-01-12 16:30:31 -08:00
"sync"
2015-05-06 10:50:57 -07:00
"time"
2014-07-08 15:33:26 -07:00
. "github.com/tendermint/go-common"
2016-05-08 15:00:58 -07:00
cfg "github.com/tendermint/go-config"
2015-11-01 11:34:08 -08:00
"github.com/tendermint/go-crypto"
dbm "github.com/tendermint/go-db"
2016-01-12 13:54:27 -08:00
"github.com/tendermint/go-events"
2015-11-01 11:34:08 -08:00
"github.com/tendermint/go-p2p"
2016-01-12 13:54:27 -08:00
"github.com/tendermint/go-rpc"
"github.com/tendermint/go-rpc/server"
2015-11-01 11:34:08 -08:00
"github.com/tendermint/go-wire"
bc "github.com/tendermint/tendermint/blockchain"
"github.com/tendermint/tendermint/consensus"
2015-04-01 17:30:16 -07:00
mempl "github.com/tendermint/tendermint/mempool"
2015-12-01 20:12:01 -08:00
"github.com/tendermint/tendermint/proxy"
rpccore "github.com/tendermint/tendermint/rpc/core"
2015-04-01 17:30:16 -07:00
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tendermint/version"
2016-03-24 10:42:05 -07:00
tmspcli "github.com/tendermint/tmsp/client"
2016-02-26 18:29:46 -08:00
"github.com/tendermint/tmsp/example/dummy"
2016-02-26 17:07:04 -08:00
"github.com/tendermint/tmsp/example/nil"
2014-07-08 00:02:04 -07:00
)
2015-04-27 07:25:11 -07:00
import _ "net/http/pprof"
2014-07-09 18:33:44 -07:00
type Node struct {
2016-05-08 15:00:58 -07:00
config cfg.Config
2014-10-22 17:20:44 -07:00
sw *p2p.Switch
evsw *events.EventSwitch
blockStore *bc.BlockStore
bcReactor *bc.BlockchainReactor
mempoolReactor *mempl.MempoolReactor
2015-01-11 14:27:46 -08:00
consensusState *consensus.ConsensusState
2014-10-22 17:20:44 -07:00
consensusReactor *consensus.ConsensusReactor
privValidator *types.PrivValidator
2015-12-01 20:12:01 -08:00
genesisDoc *types.GenesisDoc
2015-11-01 11:34:08 -08:00
privKey crypto.PrivKeyEd25519
2014-07-09 18:33:44 -07:00
}
2014-07-08 00:02:04 -07:00
2016-05-08 15:00:58 -07:00
func NewNode(config cfg.Config, privValidator *types.PrivValidator, getProxyApp func(proxyAddr string, appHash []byte) proxy.AppConn) *Node {
2016-04-13 15:23:25 -07:00
EnsureDir(config.GetString("db_dir"), 0700) // incase we use memdb, cswal still gets written here
2014-10-22 17:20:44 -07:00
// Get BlockStore
2016-05-08 15:00:58 -07:00
blockStoreDB := dbm.NewDB("blockstore", config.GetString("db_backend"), config.GetString("db_dir"))
blockStore := bc.NewBlockStore(blockStoreDB)
2014-10-22 17:20:44 -07:00
2016-05-08 15:00:58 -07:00
// Get State db
stateDB := dbm.NewDB("state", config.GetString("db_backend"), config.GetString("db_dir"))
2014-10-22 17:20:44 -07:00
// Get State
2016-05-08 15:00:58 -07:00
state := getState(config, stateDB)
2015-12-01 20:12:01 -08:00
// Create two proxyAppConn connections,
2015-12-01 20:12:01 -08:00
// one for the consensus and one for the mempool.
proxyAddr := config.GetString("proxy_app")
proxyAppConnMempool := getProxyApp(proxyAddr, state.AppHash)
proxyAppConnConsensus := getProxyApp(proxyAddr, state.AppHash)
2015-12-01 20:12:01 -08:00
// add the chainid and number of validators to the global config
config.Set("chain_id", state.ChainID)
config.Set("num_vals", state.Validators.Size())
2014-10-22 17:20:44 -07:00
// Generate node PrivKey
2015-11-01 11:34:08 -08:00
privKey := crypto.GenPrivKeyEd25519()
// Make event switch
eventSwitch := events.NewEventSwitch()
_, err := eventSwitch.Start()
if err != nil {
Exit(Fmt("Failed to start switch: %v", err))
}
// Decide whether to fast-sync or not
// We don't fast-sync when the only validator is us.
fastSync := config.GetBool("fast_sync")
if state.Validators.Size() == 1 {
addr, _ := state.Validators.GetByIndex(0)
if bytes.Equal(privValidator.Address, addr) {
fastSync = false
}
}
// Make BlockchainReactor
bcReactor := bc.NewBlockchainReactor(state.Copy(), proxyAppConnConsensus, blockStore, fastSync)
// Make MempoolReactor
2016-05-08 15:00:58 -07:00
mempool := mempl.NewMempool(config, proxyAppConnMempool)
mempoolReactor := mempl.NewMempoolReactor(config, mempool)
2014-10-22 17:20:44 -07:00
// Make ConsensusReactor
2016-05-08 15:00:58 -07:00
consensusState := consensus.NewConsensusState(config, state.Copy(), proxyAppConnConsensus, blockStore, mempool)
consensusReactor := consensus.NewConsensusReactor(consensusState, blockStore, fastSync)
if privValidator != nil {
consensusReactor.SetPrivValidator(privValidator)
}
2014-10-22 17:20:44 -07:00
2015-12-22 12:23:22 -08:00
// deterministic accountability
2016-01-18 11:10:05 -08:00
err = consensusState.OpenWAL(config.GetString("cswal"))
if err != nil {
2016-01-18 11:10:05 -08:00
log.Error("Failed to open cswal", "error", err.Error())
}
2015-12-22 12:23:22 -08:00
// Make p2p network switch
2016-05-08 15:00:58 -07:00
sw := p2p.NewSwitch(config)
sw.AddReactor("MEMPOOL", mempoolReactor)
sw.AddReactor("BLOCKCHAIN", bcReactor)
sw.AddReactor("CONSENSUS", consensusReactor)
2014-07-08 00:02:04 -07:00
// add the event switch to all services
// they should all satisfy events.Eventable
SetEventSwitch(eventSwitch, bcReactor, mempoolReactor, consensusReactor)
2015-09-24 12:48:44 -07:00
// run the profile server
profileHost := config.GetString("prof_laddr")
if profileHost != "" {
go func() {
log.Warn("Profile server", "error", http.ListenAndServe(profileHost, nil))
}()
}
2014-07-10 22:14:23 -07:00
return &Node{
2016-05-08 15:00:58 -07:00
config: config,
2014-10-22 17:20:44 -07:00
sw: sw,
evsw: eventSwitch,
2015-01-06 15:51:41 -08:00
blockStore: blockStore,
bcReactor: bcReactor,
2014-10-22 17:20:44 -07:00
mempoolReactor: mempoolReactor,
2015-01-11 14:27:46 -08:00
consensusState: consensusState,
2014-10-22 17:20:44 -07:00
consensusReactor: consensusReactor,
privValidator: privValidator,
2015-12-01 20:12:01 -08:00
genesisDoc: state.GenesisDoc,
privKey: privKey,
2014-07-09 18:33:44 -07:00
}
}
// Call Start() after adding the listeners.
func (n *Node) Start() error {
2016-05-08 15:00:58 -07:00
n.sw.SetNodeInfo(makeNodeInfo(n.config, n.sw, n.privKey))
n.sw.SetNodePrivKey(n.privKey)
_, err := n.sw.Start()
return err
2014-07-09 18:33:44 -07:00
}
func (n *Node) Stop() {
2015-07-19 14:49:13 -07:00
log.Notice("Stopping Node")
// TODO: gracefully disconnect from peers.
n.sw.Stop()
}
// Add the event switch to reactors, mempool, etc.
func SetEventSwitch(evsw *events.EventSwitch, eventables ...events.Eventable) {
for _, e := range eventables {
e.SetEventSwitch(evsw)
}
}
// Add a Listener to accept inbound peer connections.
// Add listeners before starting the Node.
// The first listener is the primary listener (in NodeInfo)
2014-07-10 22:14:23 -07:00
func (n *Node) AddListener(l p2p.Listener) {
2015-07-19 14:49:13 -07:00
log.Notice(Fmt("Added %v", l))
2015-04-01 14:52:25 -07:00
n.sw.AddListener(l)
2014-07-09 18:33:44 -07:00
}
2014-07-08 00:02:04 -07:00
2016-02-18 16:21:02 -08:00
func (n *Node) StartRPC() ([]net.Listener, error) {
rpccore.SetBlockStore(n.blockStore)
rpccore.SetConsensusState(n.consensusState)
rpccore.SetConsensusReactor(n.consensusReactor)
rpccore.SetMempoolReactor(n.mempoolReactor)
rpccore.SetSwitch(n.sw)
rpccore.SetPrivValidator(n.privValidator)
rpccore.SetGenesisDoc(n.genesisDoc)
2015-04-10 02:12:17 -07:00
2016-05-08 15:00:58 -07:00
listenAddrs := strings.Split(n.config.GetString("rpc_laddr"), ",")
2016-02-18 16:21:02 -08:00
// we may expose the rpc over both a unix and tcp socket
listeners := make([]net.Listener, len(listenAddrs))
for i, listenAddr := range listenAddrs {
mux := http.NewServeMux()
wm := rpcserver.NewWebsocketManager(rpccore.Routes, n.evsw)
mux.HandleFunc("/websocket", wm.WebsocketHandler)
rpcserver.RegisterRPCFuncs(mux, rpccore.Routes)
listener, err := rpcserver.StartHTTPServer(listenAddr, mux)
if err != nil {
return nil, err
}
listeners[i] = listener
}
return listeners, nil
}
func (n *Node) Switch() *p2p.Switch {
return n.sw
}
2015-05-15 07:27:22 -07:00
func (n *Node) BlockStore() *bc.BlockStore {
return n.blockStore
}
func (n *Node) ConsensusState() *consensus.ConsensusState {
return n.consensusState
}
func (n *Node) MempoolReactor() *mempl.MempoolReactor {
return n.mempoolReactor
}
2015-04-17 13:18:50 -07:00
func (n *Node) EventSwitch() *events.EventSwitch {
return n.evsw
}
2016-05-08 15:00:58 -07:00
func makeNodeInfo(config cfg.Config, sw *p2p.Switch, privKey crypto.PrivKeyEd25519) *p2p.NodeInfo {
2015-11-01 11:34:08 -08:00
nodeInfo := &p2p.NodeInfo{
PubKey: privKey.PubKey().(crypto.PubKeyEd25519),
Moniker: config.GetString("moniker"),
2015-11-01 11:34:08 -08:00
Network: config.GetString("chain_id"),
Version: version.Version,
2015-11-01 11:34:08 -08:00
Other: []string{
Fmt("wire_version=%v", wire.Version),
Fmt("p2p_version=%v", p2p.Version),
Fmt("consensus_version=%v", consensus.Version),
Fmt("rpc_version=%v/%v", rpc.Version, rpccore.Version),
2015-09-15 16:11:45 -07:00
},
}
2015-07-10 08:50:58 -07:00
// include git hash in the nodeInfo if available
if rev, err := ReadFile(config.GetString("revision_file")); err == nil {
2015-11-01 11:34:08 -08:00
nodeInfo.Other = append(nodeInfo.Other, Fmt("revision=%v", string(rev)))
2015-07-10 08:50:58 -07:00
}
if !sw.IsListening() {
return nodeInfo
}
2015-07-10 08:50:58 -07:00
p2pListener := sw.Listeners()[0]
p2pHost := p2pListener.ExternalAddress().IP.String()
p2pPort := p2pListener.ExternalAddress().Port
rpcListenAddr := config.GetString("rpc_laddr")
// We assume that the rpcListener has the same ExternalAddress.
2015-07-10 08:39:49 -07:00
// This is probably true because both P2P and RPC listeners use UPnP,
// except of course if the rpc is only bound to localhost
nodeInfo.ListenAddr = Fmt("%v:%v", p2pHost, p2pPort)
nodeInfo.Other = append(nodeInfo.Other, Fmt("rpc_addr=%v", rpcListenAddr))
return nodeInfo
}
2016-01-18 11:10:05 -08:00
// Get a connection to the proxyAppConn addr.
// Check the current hash, and panic if it doesn't match.
2016-04-13 15:23:25 -07:00
func GetProxyApp(addr string, hash []byte) (proxyAppConn proxy.AppConn) {
2016-01-18 11:10:05 -08:00
// use local app (for testing)
2016-02-26 18:29:46 -08:00
switch addr {
case "nilapp":
2016-02-26 17:07:04 -08:00
app := nilapp.NewNilApplication()
2016-01-18 11:10:05 -08:00
mtx := new(sync.Mutex)
2016-03-24 10:42:05 -07:00
proxyAppConn = tmspcli.NewLocalClient(mtx, app)
2016-02-26 18:29:46 -08:00
case "dummy":
app := dummy.NewDummyApplication()
mtx := new(sync.Mutex)
2016-03-24 10:42:05 -07:00
proxyAppConn = tmspcli.NewLocalClient(mtx, app)
2016-02-26 18:29:46 -08:00
default:
2016-03-23 20:22:45 -07:00
// Run forever in a loop
2016-02-08 00:48:58 -08:00
remoteApp, err := proxy.NewRemoteAppConn(addr)
2016-01-18 11:10:05 -08:00
if err != nil {
Exit(Fmt("Failed to connect to proxy for mempool: %v", err))
}
proxyAppConn = remoteApp
}
// Check the hash
2016-03-23 02:49:30 -07:00
res := proxyAppConn.CommitSync()
if res.IsErr() {
PanicCrisis(Fmt("Error in getting proxyAppConn hash: %v", res))
2016-01-18 11:10:05 -08:00
}
2016-03-23 02:49:30 -07:00
if !bytes.Equal(hash, res.Data) {
2016-03-23 20:22:45 -07:00
log.Warn(Fmt("ProxyApp hash does not match. Expected %X, got %X", hash, res.Data))
2016-01-18 11:10:05 -08:00
}
return proxyAppConn
}
// Load the most recent state from "state" db,
// or create a new one (and save) from genesis.
2016-05-08 15:00:58 -07:00
func getState(config cfg.Config, stateDB dbm.DB) *sm.State {
2016-01-18 11:10:05 -08:00
state := sm.LoadState(stateDB)
if state == nil {
state = sm.MakeGenesisStateFromFile(stateDB, config.GetString("genesis_file"))
state.Save()
}
return state
}
2015-04-01 14:52:25 -07:00
//------------------------------------------------------------------------------
2015-11-04 16:09:43 -08:00
// Users wishing to use an external signer for their validators
// should fork tendermint/tendermint and implement RunNode to
2016-04-13 15:23:25 -07:00
// load their custom priv validator and call NewNode(privVal, getProxyFunc)
2016-05-08 15:00:58 -07:00
func RunNode(config cfg.Config) {
// Wait until the genesis doc becomes available
genDocFile := config.GetString("genesis_file")
if !FileExists(genDocFile) {
log.Notice(Fmt("Waiting for genesis file %v...", genDocFile))
for {
time.Sleep(time.Second)
2015-12-01 20:12:01 -08:00
if !FileExists(genDocFile) {
continue
}
jsonBlob, err := ioutil.ReadFile(genDocFile)
if err != nil {
Exit(Fmt("Couldn't read GenesisDoc file: %v", err))
}
2015-11-01 11:34:08 -08:00
genDoc := types.GenesisDocFromJSON(jsonBlob)
if genDoc.ChainID == "" {
PanicSanity(Fmt("Genesis doc %v must include non-empty chain_id", genDocFile))
}
config.Set("chain_id", genDoc.ChainID)
config.Set("genesis_doc", genDoc)
}
}
2016-01-12 16:04:34 -08:00
// Get PrivValidator
privValidatorFile := config.GetString("priv_validator_file")
privValidator := types.LoadOrGenPrivValidator(privValidatorFile)
2014-07-12 14:52:31 -07:00
// Create & start node
2016-05-08 15:00:58 -07:00
n := NewNode(config, privValidator, GetProxyApp)
2015-11-01 11:34:08 -08:00
l := p2p.NewDefaultListener("tcp", config.GetString("node_laddr"), config.GetBool("skip_upnp"))
2014-07-10 22:14:23 -07:00
n.AddListener(l)
err := n.Start()
if err != nil {
Exit(Fmt("Failed to start node: %v", err))
}
2014-07-09 18:33:44 -07:00
log.Notice("Started node", "nodeInfo", n.sw.NodeInfo())
2014-10-22 17:20:44 -07:00
// If seedNode is provided by config, dial out.
2015-07-10 08:39:49 -07:00
if config.GetString("seeds") != "" {
seeds := strings.Split(config.GetString("seeds"), ",")
n.sw.DialSeeds(seeds)
2014-07-09 18:33:44 -07:00
}
2014-07-08 15:33:26 -07:00
2014-11-27 04:04:07 -08:00
// Run the RPC server.
if config.GetString("rpc_laddr") != "" {
2015-07-19 16:42:52 -07:00
_, err := n.StartRPC()
if err != nil {
PanicCrisis(err)
}
2014-11-27 04:04:07 -08:00
}
// Sleep forever and then...
2015-04-08 11:35:17 -07:00
TrapSignal(func() {
n.Stop()
})
2014-07-08 15:33:26 -07:00
}
2015-12-01 20:12:01 -08:00
2016-04-13 15:23:25 -07:00
func (n *Node) NodeInfo() *p2p.NodeInfo {
return n.sw.NodeInfo()
}
func (n *Node) DialSeeds(seeds []string) {
n.sw.DialSeeds(seeds)
}
2016-01-18 11:10:05 -08:00
//------------------------------------------------------------------------------
// replay
// convenience for replay mode
2016-05-08 15:00:58 -07:00
func newConsensusState(config cfg.Config) *consensus.ConsensusState {
2015-12-22 12:23:22 -08:00
// Get BlockStore
2016-05-08 15:00:58 -07:00
blockStoreDB := dbm.NewDB("blockstore", config.GetString("db_backend"), config.GetString("db_dir"))
2015-12-22 12:23:22 -08:00
blockStore := bc.NewBlockStore(blockStoreDB)
// Get State
2016-05-08 15:00:58 -07:00
stateDB := dbm.NewDB("state", config.GetString("db_backend"), config.GetString("db_dir"))
2015-12-22 12:23:22 -08:00
state := sm.MakeGenesisStateFromFile(stateDB, config.GetString("genesis_file"))
2016-01-14 16:04:01 -08:00
// Create two proxyAppConn connections,
2015-12-22 12:23:22 -08:00
// one for the consensus and one for the mempool.
proxyAddr := config.GetString("proxy_app")
2016-04-13 15:23:25 -07:00
proxyAppConnMempool := GetProxyApp(proxyAddr, state.AppHash)
proxyAppConnConsensus := GetProxyApp(proxyAddr, state.AppHash)
2015-12-22 12:23:22 -08:00
// add the chainid to the global config
config.Set("chain_id", state.ChainID)
// Make event switch
eventSwitch := events.NewEventSwitch()
_, err := eventSwitch.Start()
if err != nil {
Exit(Fmt("Failed to start event switch: %v", err))
2015-12-01 20:12:01 -08:00
}
2016-05-08 15:00:58 -07:00
mempool := mempl.NewMempool(config, proxyAppConnMempool)
2015-12-01 20:12:01 -08:00
2016-05-08 15:00:58 -07:00
consensusState := consensus.NewConsensusState(config, state.Copy(), proxyAppConnConsensus, blockStore, mempool)
2015-12-22 12:23:22 -08:00
consensusState.SetEventSwitch(eventSwitch)
2015-12-22 22:27:40 -08:00
return consensusState
}
2016-05-08 15:00:58 -07:00
func RunReplayConsole(config cfg.Config) {
2016-01-18 11:10:05 -08:00
walFile := config.GetString("cswal")
if walFile == "" {
Exit("cswal file name not set in tendermint config")
2016-01-10 13:33:52 -08:00
}
2015-12-01 20:12:01 -08:00
2016-05-08 15:00:58 -07:00
consensusState := newConsensusState(config)
2015-12-22 22:27:40 -08:00
2016-01-18 11:10:05 -08:00
if err := consensusState.ReplayConsole(walFile); err != nil {
2015-12-22 22:27:40 -08:00
Exit(Fmt("Error during consensus replay: %v", err))
2015-12-01 20:12:01 -08:00
}
2015-12-22 22:27:40 -08:00
}
2016-05-08 15:00:58 -07:00
func RunReplay(config cfg.Config) {
2016-01-18 11:10:05 -08:00
walFile := config.GetString("cswal")
if walFile == "" {
Exit("cswal file name not set in tendermint config")
2015-12-01 20:12:01 -08:00
}
2016-05-08 15:00:58 -07:00
consensusState := newConsensusState(config)
2015-12-22 12:23:22 -08:00
2016-01-18 11:10:05 -08:00
if err := consensusState.ReplayMessages(walFile); err != nil {
2015-12-22 12:23:22 -08:00
Exit(Fmt("Error during consensus replay: %v", err))
}
log.Notice("Replay run successfully")
2015-12-01 20:12:01 -08:00
}