You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

845 lines
25 KiB

max-bytes PR follow-up (#2318) * ReapMaxTxs: return all txs if max is negative this mirrors ReapMaxBytes behavior See https://github.com/tendermint/tendermint/pull/2184#discussion_r214439950 * increase MaxAminoOverheadForBlock tested with: ``` func TestMaxAminoOverheadForBlock(t *testing.T) { maxChainID := "" for i := 0; i < MaxChainIDLen; i++ { maxChainID += "𠜎" } h := Header{ ChainID: maxChainID, Height: 10, Time: time.Now().UTC(), NumTxs: 100, TotalTxs: 200, LastBlockID: makeBlockID(make([]byte, 20), 300, make([]byte, 20)), LastCommitHash: tmhash.Sum([]byte("last_commit_hash")), DataHash: tmhash.Sum([]byte("data_hash")), ValidatorsHash: tmhash.Sum([]byte("validators_hash")), NextValidatorsHash: tmhash.Sum([]byte("next_validators_hash")), ConsensusHash: tmhash.Sum([]byte("consensus_hash")), AppHash: tmhash.Sum([]byte("app_hash")), LastResultsHash: tmhash.Sum([]byte("last_results_hash")), EvidenceHash: tmhash.Sum([]byte("evidence_hash")), ProposerAddress: tmhash.Sum([]byte("proposer_address")), } b := Block{ Header: h, Data: Data{Txs: makeTxs(10000, 100)}, Evidence: EvidenceData{}, LastCommit: &Commit{}, } bz, err := cdc.MarshalBinary(b) require.NoError(t, err) assert.Equal(t, MaxHeaderBytes+MaxAminoOverheadForBlock-2, len(bz)-1000000-20000-1) } ``` * fix MaxYYY constants calculation by using math.MaxInt64 See https://github.com/tendermint/tendermint/pull/2184#discussion_r214444244 * pass mempool filter as an option See https://github.com/tendermint/tendermint/pull/2184#discussion_r214445869 * fixes after Dev's comments
6 years ago
  1. package node
  2. import (
  3. "bytes"
  4. "context"
  5. "errors"
  6. "fmt"
  7. "net"
  8. "net/http"
  9. "time"
  10. "github.com/prometheus/client_golang/prometheus"
  11. "github.com/prometheus/client_golang/prometheus/promhttp"
  12. amino "github.com/tendermint/go-amino"
  13. abci "github.com/tendermint/tendermint/abci/types"
  14. "github.com/tendermint/tendermint/crypto/ed25519"
  15. cmn "github.com/tendermint/tendermint/libs/common"
  16. dbm "github.com/tendermint/tendermint/libs/db"
  17. "github.com/tendermint/tendermint/libs/log"
  18. bc "github.com/tendermint/tendermint/blockchain"
  19. cfg "github.com/tendermint/tendermint/config"
  20. cs "github.com/tendermint/tendermint/consensus"
  21. "github.com/tendermint/tendermint/evidence"
  22. mempl "github.com/tendermint/tendermint/mempool"
  23. "github.com/tendermint/tendermint/p2p"
  24. "github.com/tendermint/tendermint/p2p/pex"
  25. "github.com/tendermint/tendermint/privval"
  26. "github.com/tendermint/tendermint/proxy"
  27. rpccore "github.com/tendermint/tendermint/rpc/core"
  28. ctypes "github.com/tendermint/tendermint/rpc/core/types"
  29. grpccore "github.com/tendermint/tendermint/rpc/grpc"
  30. rpc "github.com/tendermint/tendermint/rpc/lib"
  31. rpcserver "github.com/tendermint/tendermint/rpc/lib/server"
  32. sm "github.com/tendermint/tendermint/state"
  33. "github.com/tendermint/tendermint/state/txindex"
  34. "github.com/tendermint/tendermint/state/txindex/kv"
  35. "github.com/tendermint/tendermint/state/txindex/null"
  36. "github.com/tendermint/tendermint/types"
  37. tmtime "github.com/tendermint/tendermint/types/time"
  38. "github.com/tendermint/tendermint/version"
  39. _ "net/http/pprof"
  40. "strings"
  41. )
  42. //------------------------------------------------------------------------------
  43. // DBContext specifies config information for loading a new DB.
  44. type DBContext struct {
  45. ID string
  46. Config *cfg.Config
  47. }
  48. // DBProvider takes a DBContext and returns an instantiated DB.
  49. type DBProvider func(*DBContext) (dbm.DB, error)
  50. // DefaultDBProvider returns a database using the DBBackend and DBDir
  51. // specified in the ctx.Config.
  52. func DefaultDBProvider(ctx *DBContext) (dbm.DB, error) {
  53. dbType := dbm.DBBackendType(ctx.Config.DBBackend)
  54. return dbm.NewDB(ctx.ID, dbType, ctx.Config.DBDir()), nil
  55. }
  56. // GenesisDocProvider returns a GenesisDoc.
  57. // It allows the GenesisDoc to be pulled from sources other than the
  58. // filesystem, for instance from a distributed key-value store cluster.
  59. type GenesisDocProvider func() (*types.GenesisDoc, error)
  60. // DefaultGenesisDocProviderFunc returns a GenesisDocProvider that loads
  61. // the GenesisDoc from the config.GenesisFile() on the filesystem.
  62. func DefaultGenesisDocProviderFunc(config *cfg.Config) GenesisDocProvider {
  63. return func() (*types.GenesisDoc, error) {
  64. return types.GenesisDocFromFile(config.GenesisFile())
  65. }
  66. }
  67. // NodeProvider takes a config and a logger and returns a ready to go Node.
  68. type NodeProvider func(*cfg.Config, log.Logger) (*Node, error)
  69. // DefaultNewNode returns a Tendermint node with default settings for the
  70. // PrivValidator, ClientCreator, GenesisDoc, and DBProvider.
  71. // It implements NodeProvider.
  72. func DefaultNewNode(config *cfg.Config, logger log.Logger) (*Node, error) {
  73. // Generate node PrivKey
  74. nodeKey, err := p2p.LoadOrGenNodeKey(config.NodeKeyFile())
  75. if err != nil {
  76. return nil, err
  77. }
  78. return NewNode(config,
  79. privval.LoadOrGenFilePV(config.PrivValidatorFile()),
  80. nodeKey,
  81. proxy.DefaultClientCreator(config.ProxyApp, config.ABCI, config.DBDir()),
  82. DefaultGenesisDocProviderFunc(config),
  83. DefaultDBProvider,
  84. DefaultMetricsProvider(config.Instrumentation),
  85. logger,
  86. )
  87. }
  88. // MetricsProvider returns a consensus, p2p and mempool Metrics.
  89. type MetricsProvider func() (*cs.Metrics, *p2p.Metrics, *mempl.Metrics)
  90. // DefaultMetricsProvider returns Metrics build using Prometheus client library
  91. // if Prometheus is enabled. Otherwise, it returns no-op Metrics.
  92. func DefaultMetricsProvider(config *cfg.InstrumentationConfig) MetricsProvider {
  93. return func() (*cs.Metrics, *p2p.Metrics, *mempl.Metrics) {
  94. if config.Prometheus {
  95. return cs.PrometheusMetrics(), p2p.PrometheusMetrics(), mempl.PrometheusMetrics()
  96. }
  97. return cs.NopMetrics(), p2p.NopMetrics(), mempl.NopMetrics()
  98. }
  99. }
  100. //------------------------------------------------------------------------------
  101. // Node is the highest level interface to a full Tendermint node.
  102. // It includes all configuration information and running services.
  103. type Node struct {
  104. cmn.BaseService
  105. // config
  106. config *cfg.Config
  107. genesisDoc *types.GenesisDoc // initial validator set
  108. privValidator types.PrivValidator // local node's validator key
  109. // network
  110. transport *p2p.MultiplexTransport
  111. sw *p2p.Switch // p2p connections
  112. addrBook pex.AddrBook // known peers
  113. nodeInfo p2p.NodeInfo
  114. nodeKey *p2p.NodeKey // our node privkey
  115. isListening bool
  116. // services
  117. eventBus *types.EventBus // pub/sub for services
  118. stateDB dbm.DB
  119. blockStore *bc.BlockStore // store the blockchain to disk
  120. bcReactor *bc.BlockchainReactor // for fast-syncing
  121. mempoolReactor *mempl.MempoolReactor // for gossipping transactions
  122. consensusState *cs.ConsensusState // latest consensus state
  123. consensusReactor *cs.ConsensusReactor // for participating in the consensus
  124. evidencePool *evidence.EvidencePool // tracking evidence
  125. proxyApp proxy.AppConns // connection to the application
  126. rpcListeners []net.Listener // rpc servers
  127. txIndexer txindex.TxIndexer
  128. indexerService *txindex.IndexerService
  129. prometheusSrv *http.Server
  130. }
  131. // NewNode returns a new, ready to go, Tendermint Node.
  132. func NewNode(config *cfg.Config,
  133. privValidator types.PrivValidator,
  134. nodeKey *p2p.NodeKey,
  135. clientCreator proxy.ClientCreator,
  136. genesisDocProvider GenesisDocProvider,
  137. dbProvider DBProvider,
  138. metricsProvider MetricsProvider,
  139. logger log.Logger) (*Node, error) {
  140. // Get BlockStore
  141. blockStoreDB, err := dbProvider(&DBContext{"blockstore", config})
  142. if err != nil {
  143. return nil, err
  144. }
  145. blockStore := bc.NewBlockStore(blockStoreDB)
  146. // Get State
  147. stateDB, err := dbProvider(&DBContext{"state", config})
  148. if err != nil {
  149. return nil, err
  150. }
  151. // Get genesis doc
  152. // TODO: move to state package?
  153. genDoc, err := loadGenesisDoc(stateDB)
  154. if err != nil {
  155. genDoc, err = genesisDocProvider()
  156. if err != nil {
  157. return nil, err
  158. }
  159. // save genesis doc to prevent a certain class of user errors (e.g. when it
  160. // was changed, accidentally or not). Also good for audit trail.
  161. saveGenesisDoc(stateDB, genDoc)
  162. }
  163. state, err := sm.LoadStateFromDBOrGenesisDoc(stateDB, genDoc)
  164. if err != nil {
  165. return nil, err
  166. }
  167. // Create the proxyApp and establish connections to the ABCI app (consensus, mempool, query).
  168. proxyApp := proxy.NewAppConns(clientCreator)
  169. proxyApp.SetLogger(logger.With("module", "proxy"))
  170. if err := proxyApp.Start(); err != nil {
  171. return nil, fmt.Errorf("Error starting proxy app connections: %v", err)
  172. }
  173. // Create the handshaker, which calls RequestInfo and replays any blocks
  174. // as necessary to sync tendermint with the app.
  175. consensusLogger := logger.With("module", "consensus")
  176. handshaker := cs.NewHandshaker(stateDB, state, blockStore, genDoc)
  177. handshaker.SetLogger(consensusLogger)
  178. if err := handshaker.Handshake(proxyApp); err != nil {
  179. return nil, fmt.Errorf("Error during handshake: %v", err)
  180. }
  181. // reload the state (it may have been updated by the handshake)
  182. state = sm.LoadState(stateDB)
  183. // If an address is provided, listen on the socket for a
  184. // connection from an external signing process.
  185. if config.PrivValidatorListenAddr != "" {
  186. var (
  187. // TODO: persist this key so external signer
  188. // can actually authenticate us
  189. privKey = ed25519.GenPrivKey()
  190. pvsc = privval.NewSocketPV(
  191. logger.With("module", "privval"),
  192. config.PrivValidatorListenAddr,
  193. privKey,
  194. )
  195. )
  196. if err := pvsc.Start(); err != nil {
  197. return nil, fmt.Errorf("Error starting private validator client: %v", err)
  198. }
  199. privValidator = pvsc
  200. }
  201. // Decide whether to fast-sync or not
  202. // We don't fast-sync when the only validator is us.
  203. fastSync := config.FastSync
  204. if state.Validators.Size() == 1 {
  205. addr, _ := state.Validators.GetByIndex(0)
  206. if bytes.Equal(privValidator.GetAddress(), addr) {
  207. fastSync = false
  208. }
  209. }
  210. // Log whether this node is a validator or an observer
  211. if state.Validators.HasAddress(privValidator.GetAddress()) {
  212. consensusLogger.Info("This node is a validator", "addr", privValidator.GetAddress(), "pubKey", privValidator.GetPubKey())
  213. } else {
  214. consensusLogger.Info("This node is not a validator", "addr", privValidator.GetAddress(), "pubKey", privValidator.GetPubKey())
  215. }
  216. csMetrics, p2pMetrics, memplMetrics := metricsProvider()
  217. // Make MempoolReactor
  218. maxDataBytes := types.MaxDataBytesUnknownEvidence(
  219. state.ConsensusParams.BlockSize.MaxBytes,
  220. state.Validators.Size(),
  221. )
  222. mempool := mempl.NewMempool(
  223. config.Mempool,
  224. proxyApp.Mempool(),
  225. state.LastBlockHeight,
  226. mempl.WithMetrics(memplMetrics),
  227. mempl.WithFilter(func(tx types.Tx) bool { return len(tx) <= maxDataBytes }),
  228. )
  229. mempoolLogger := logger.With("module", "mempool")
  230. mempool.SetLogger(mempoolLogger)
  231. mempool.InitWAL() // no need to have the mempool wal during tests
  232. mempoolReactor := mempl.NewMempoolReactor(config.Mempool, mempool)
  233. mempoolReactor.SetLogger(mempoolLogger)
  234. if config.Consensus.WaitForTxs() {
  235. mempool.EnableTxsAvailable()
  236. }
  237. // Make Evidence Reactor
  238. evidenceDB, err := dbProvider(&DBContext{"evidence", config})
  239. if err != nil {
  240. return nil, err
  241. }
  242. evidenceLogger := logger.With("module", "evidence")
  243. evidenceStore := evidence.NewEvidenceStore(evidenceDB)
  244. evidencePool := evidence.NewEvidencePool(stateDB, evidenceStore)
  245. evidencePool.SetLogger(evidenceLogger)
  246. evidenceReactor := evidence.NewEvidenceReactor(evidencePool)
  247. evidenceReactor.SetLogger(evidenceLogger)
  248. blockExecLogger := logger.With("module", "state")
  249. // make block executor for consensus and blockchain reactors to execute blocks
  250. blockExec := sm.NewBlockExecutor(stateDB, blockExecLogger, proxyApp.Consensus(), mempool, evidencePool)
  251. // Make BlockchainReactor
  252. bcReactor := bc.NewBlockchainReactor(state.Copy(), blockExec, blockStore, fastSync)
  253. bcReactor.SetLogger(logger.With("module", "blockchain"))
  254. // Make ConsensusReactor
  255. consensusState := cs.NewConsensusState(
  256. config.Consensus,
  257. state.Copy(),
  258. blockExec,
  259. blockStore,
  260. mempool,
  261. evidencePool,
  262. cs.WithMetrics(csMetrics),
  263. )
  264. consensusState.SetLogger(consensusLogger)
  265. if privValidator != nil {
  266. consensusState.SetPrivValidator(privValidator)
  267. }
  268. consensusReactor := cs.NewConsensusReactor(consensusState, fastSync)
  269. consensusReactor.SetLogger(consensusLogger)
  270. eventBus := types.NewEventBus()
  271. eventBus.SetLogger(logger.With("module", "events"))
  272. // services which will be publishing and/or subscribing for messages (events)
  273. // consensusReactor will set it on consensusState and blockExecutor
  274. consensusReactor.SetEventBus(eventBus)
  275. // Transaction indexing
  276. var txIndexer txindex.TxIndexer
  277. switch config.TxIndex.Indexer {
  278. case "kv":
  279. store, err := dbProvider(&DBContext{"tx_index", config})
  280. if err != nil {
  281. return nil, err
  282. }
  283. if config.TxIndex.IndexTags != "" {
  284. txIndexer = kv.NewTxIndex(store, kv.IndexTags(splitAndTrimEmpty(config.TxIndex.IndexTags, ",", " ")))
  285. } else if config.TxIndex.IndexAllTags {
  286. txIndexer = kv.NewTxIndex(store, kv.IndexAllTags())
  287. } else {
  288. txIndexer = kv.NewTxIndex(store)
  289. }
  290. default:
  291. txIndexer = &null.TxIndex{}
  292. }
  293. indexerService := txindex.NewIndexerService(txIndexer, eventBus)
  294. indexerService.SetLogger(logger.With("module", "txindex"))
  295. var (
  296. p2pLogger = logger.With("module", "p2p")
  297. nodeInfo = makeNodeInfo(config, nodeKey.ID(), txIndexer, genDoc.ChainID)
  298. )
  299. // Setup Transport.
  300. var (
  301. transport = p2p.NewMultiplexTransport(nodeInfo, *nodeKey)
  302. connFilters = []p2p.ConnFilterFunc{}
  303. peerFilters = []p2p.PeerFilterFunc{}
  304. )
  305. if !config.P2P.AllowDuplicateIP {
  306. connFilters = append(connFilters, p2p.ConnDuplicateIPFilter())
  307. }
  308. // Filter peers by addr or pubkey with an ABCI query.
  309. // If the query return code is OK, add peer.
  310. // XXX: Query format subject to change
  311. if config.FilterPeers {
  312. connFilters = append(
  313. connFilters,
  314. // ABCI query for address filtering.
  315. func(_ p2p.ConnSet, c net.Conn, _ []net.IP) error {
  316. res, err := proxyApp.Query().QuerySync(abci.RequestQuery{
  317. Path: fmt.Sprintf("/p2p/filter/addr/%s", c.RemoteAddr().String()),
  318. })
  319. if err != nil {
  320. return err
  321. }
  322. if res.IsErr() {
  323. return fmt.Errorf("Error querying abci app: %v", res)
  324. }
  325. return nil
  326. },
  327. )
  328. peerFilters = append(
  329. peerFilters,
  330. // ABCI query for ID filtering.
  331. func(_ p2p.IPeerSet, p p2p.Peer) error {
  332. res, err := proxyApp.Query().QuerySync(abci.RequestQuery{
  333. Path: fmt.Sprintf("/p2p/filter/id/%s", p.ID()),
  334. })
  335. if err != nil {
  336. return err
  337. }
  338. if res.IsErr() {
  339. return fmt.Errorf("Error querying abci app: %v", res)
  340. }
  341. return nil
  342. },
  343. )
  344. }
  345. p2p.MultiplexTransportConnFilters(connFilters...)(transport)
  346. // Setup Switch.
  347. sw := p2p.NewSwitch(
  348. config.P2P,
  349. transport,
  350. p2p.WithMetrics(p2pMetrics),
  351. p2p.SwitchPeerFilters(peerFilters...),
  352. )
  353. sw.SetLogger(p2pLogger)
  354. sw.AddReactor("MEMPOOL", mempoolReactor)
  355. sw.AddReactor("BLOCKCHAIN", bcReactor)
  356. sw.AddReactor("CONSENSUS", consensusReactor)
  357. sw.AddReactor("EVIDENCE", evidenceReactor)
  358. sw.SetNodeInfo(nodeInfo)
  359. sw.SetNodeKey(nodeKey)
  360. p2pLogger.Info("P2P Node ID", "ID", nodeKey.ID(), "file", config.NodeKeyFile())
  361. // Optionally, start the pex reactor
  362. //
  363. // TODO:
  364. //
  365. // We need to set Seeds and PersistentPeers on the switch,
  366. // since it needs to be able to use these (and their DNS names)
  367. // even if the PEX is off. We can include the DNS name in the NetAddress,
  368. // but it would still be nice to have a clear list of the current "PersistentPeers"
  369. // somewhere that we can return with net_info.
  370. //
  371. // If PEX is on, it should handle dialing the seeds. Otherwise the switch does it.
  372. // Note we currently use the addrBook regardless at least for AddOurAddress
  373. addrBook := pex.NewAddrBook(config.P2P.AddrBookFile(), config.P2P.AddrBookStrict)
  374. // Add ourselves to addrbook to prevent dialing ourselves
  375. addrBook.AddOurAddress(nodeInfo.NetAddress())
  376. addrBook.SetLogger(p2pLogger.With("book", config.P2P.AddrBookFile()))
  377. if config.P2P.PexReactor {
  378. // TODO persistent peers ? so we can have their DNS addrs saved
  379. pexReactor := pex.NewPEXReactor(addrBook,
  380. &pex.PEXReactorConfig{
  381. Seeds: splitAndTrimEmpty(config.P2P.Seeds, ",", " "),
  382. SeedMode: config.P2P.SeedMode,
  383. })
  384. pexReactor.SetLogger(p2pLogger)
  385. sw.AddReactor("PEX", pexReactor)
  386. }
  387. sw.SetAddrBook(addrBook)
  388. // run the profile server
  389. profileHost := config.ProfListenAddress
  390. if profileHost != "" {
  391. go func() {
  392. logger.Error("Profile server", "err", http.ListenAndServe(profileHost, nil))
  393. }()
  394. }
  395. node := &Node{
  396. config: config,
  397. genesisDoc: genDoc,
  398. privValidator: privValidator,
  399. transport: transport,
  400. sw: sw,
  401. addrBook: addrBook,
  402. nodeInfo: nodeInfo,
  403. nodeKey: nodeKey,
  404. stateDB: stateDB,
  405. blockStore: blockStore,
  406. bcReactor: bcReactor,
  407. mempoolReactor: mempoolReactor,
  408. consensusState: consensusState,
  409. consensusReactor: consensusReactor,
  410. evidencePool: evidencePool,
  411. proxyApp: proxyApp,
  412. txIndexer: txIndexer,
  413. indexerService: indexerService,
  414. eventBus: eventBus,
  415. }
  416. node.BaseService = *cmn.NewBaseService(logger, "Node", node)
  417. return node, nil
  418. }
  419. // OnStart starts the Node. It implements cmn.Service.
  420. func (n *Node) OnStart() error {
  421. now := tmtime.Now()
  422. genTime := n.genesisDoc.GenesisTime
  423. if genTime.After(now) {
  424. n.Logger.Info("Genesis time is in the future. Sleeping until then...", "genTime", genTime)
  425. time.Sleep(genTime.Sub(now))
  426. }
  427. err := n.eventBus.Start()
  428. if err != nil {
  429. return err
  430. }
  431. // Add private IDs to addrbook to block those peers being added
  432. n.addrBook.AddPrivateIDs(splitAndTrimEmpty(n.config.P2P.PrivatePeerIDs, ",", " "))
  433. // Start the RPC server before the P2P server
  434. // so we can eg. receive txs for the first block
  435. if n.config.RPC.ListenAddress != "" {
  436. listeners, err := n.startRPC()
  437. if err != nil {
  438. return err
  439. }
  440. n.rpcListeners = listeners
  441. }
  442. if n.config.Instrumentation.Prometheus &&
  443. n.config.Instrumentation.PrometheusListenAddr != "" {
  444. n.prometheusSrv = n.startPrometheusServer(n.config.Instrumentation.PrometheusListenAddr)
  445. }
  446. // Start the transport.
  447. addr, err := p2p.NewNetAddressStringWithOptionalID(n.config.P2P.ListenAddress)
  448. if err != nil {
  449. return err
  450. }
  451. if err := n.transport.Listen(*addr); err != nil {
  452. return err
  453. }
  454. n.isListening = true
  455. // Start the switch (the P2P server).
  456. err = n.sw.Start()
  457. if err != nil {
  458. return err
  459. }
  460. // Always connect to persistent peers
  461. if n.config.P2P.PersistentPeers != "" {
  462. err = n.sw.DialPeersAsync(n.addrBook, splitAndTrimEmpty(n.config.P2P.PersistentPeers, ",", " "), true)
  463. if err != nil {
  464. return err
  465. }
  466. }
  467. // start tx indexer
  468. return n.indexerService.Start()
  469. }
  470. // OnStop stops the Node. It implements cmn.Service.
  471. func (n *Node) OnStop() {
  472. n.BaseService.OnStop()
  473. n.Logger.Info("Stopping Node")
  474. // first stop the non-reactor services
  475. n.eventBus.Stop()
  476. n.indexerService.Stop()
  477. // now stop the reactors
  478. // TODO: gracefully disconnect from peers.
  479. n.sw.Stop()
  480. if err := n.transport.Close(); err != nil {
  481. n.Logger.Error("Error closing transport", "err", err)
  482. }
  483. n.isListening = false
  484. // finally stop the listeners / external services
  485. for _, l := range n.rpcListeners {
  486. n.Logger.Info("Closing rpc listener", "listener", l)
  487. if err := l.Close(); err != nil {
  488. n.Logger.Error("Error closing listener", "listener", l, "err", err)
  489. }
  490. }
  491. if pvsc, ok := n.privValidator.(*privval.SocketPV); ok {
  492. if err := pvsc.Stop(); err != nil {
  493. n.Logger.Error("Error stopping priv validator socket client", "err", err)
  494. }
  495. }
  496. if n.prometheusSrv != nil {
  497. if err := n.prometheusSrv.Shutdown(context.Background()); err != nil {
  498. // Error from closing listeners, or context timeout:
  499. n.Logger.Error("Prometheus HTTP server Shutdown", "err", err)
  500. }
  501. }
  502. }
  503. // RunForever waits for an interrupt signal and stops the node.
  504. func (n *Node) RunForever() {
  505. // Sleep forever and then...
  506. cmn.TrapSignal(func() {
  507. n.Stop()
  508. })
  509. }
  510. // ConfigureRPC sets all variables in rpccore so they will serve
  511. // rpc calls from this node
  512. func (n *Node) ConfigureRPC() {
  513. rpccore.SetStateDB(n.stateDB)
  514. rpccore.SetBlockStore(n.blockStore)
  515. rpccore.SetConsensusState(n.consensusState)
  516. rpccore.SetMempool(n.mempoolReactor.Mempool)
  517. rpccore.SetEvidencePool(n.evidencePool)
  518. rpccore.SetP2PPeers(n.sw)
  519. rpccore.SetP2PTransport(n)
  520. rpccore.SetPubKey(n.privValidator.GetPubKey())
  521. rpccore.SetGenesisDoc(n.genesisDoc)
  522. rpccore.SetAddrBook(n.addrBook)
  523. rpccore.SetProxyAppQuery(n.proxyApp.Query())
  524. rpccore.SetTxIndexer(n.txIndexer)
  525. rpccore.SetConsensusReactor(n.consensusReactor)
  526. rpccore.SetEventBus(n.eventBus)
  527. rpccore.SetLogger(n.Logger.With("module", "rpc"))
  528. }
  529. func (n *Node) startRPC() ([]net.Listener, error) {
  530. n.ConfigureRPC()
  531. listenAddrs := splitAndTrimEmpty(n.config.RPC.ListenAddress, ",", " ")
  532. coreCodec := amino.NewCodec()
  533. ctypes.RegisterAmino(coreCodec)
  534. if n.config.RPC.Unsafe {
  535. rpccore.AddUnsafeRoutes()
  536. }
  537. // we may expose the rpc over both a unix and tcp socket
  538. listeners := make([]net.Listener, len(listenAddrs))
  539. for i, listenAddr := range listenAddrs {
  540. mux := http.NewServeMux()
  541. rpcLogger := n.Logger.With("module", "rpc-server")
  542. wm := rpcserver.NewWebsocketManager(rpccore.Routes, coreCodec, rpcserver.EventSubscriber(n.eventBus))
  543. wm.SetLogger(rpcLogger.With("protocol", "websocket"))
  544. mux.HandleFunc("/websocket", wm.WebsocketHandler)
  545. rpcserver.RegisterRPCFuncs(mux, rpccore.Routes, coreCodec, rpcLogger)
  546. listener, err := rpcserver.StartHTTPServer(
  547. listenAddr,
  548. mux,
  549. rpcLogger,
  550. rpcserver.Config{MaxOpenConnections: n.config.RPC.MaxOpenConnections},
  551. )
  552. if err != nil {
  553. return nil, err
  554. }
  555. listeners[i] = listener
  556. }
  557. // we expose a simplified api over grpc for convenience to app devs
  558. grpcListenAddr := n.config.RPC.GRPCListenAddress
  559. if grpcListenAddr != "" {
  560. listener, err := grpccore.StartGRPCServer(
  561. grpcListenAddr,
  562. grpccore.Config{
  563. MaxOpenConnections: n.config.RPC.GRPCMaxOpenConnections,
  564. },
  565. )
  566. if err != nil {
  567. return nil, err
  568. }
  569. listeners = append(listeners, listener)
  570. }
  571. return listeners, nil
  572. }
  573. // startPrometheusServer starts a Prometheus HTTP server, listening for metrics
  574. // collectors on addr.
  575. func (n *Node) startPrometheusServer(addr string) *http.Server {
  576. srv := &http.Server{
  577. Addr: addr,
  578. Handler: promhttp.InstrumentMetricHandler(
  579. prometheus.DefaultRegisterer, promhttp.HandlerFor(
  580. prometheus.DefaultGatherer,
  581. promhttp.HandlerOpts{MaxRequestsInFlight: n.config.Instrumentation.MaxOpenConnections},
  582. ),
  583. ),
  584. }
  585. go func() {
  586. if err := srv.ListenAndServe(); err != http.ErrServerClosed {
  587. // Error starting or closing listener:
  588. n.Logger.Error("Prometheus HTTP server ListenAndServe", "err", err)
  589. }
  590. }()
  591. return srv
  592. }
  593. // Switch returns the Node's Switch.
  594. func (n *Node) Switch() *p2p.Switch {
  595. return n.sw
  596. }
  597. // BlockStore returns the Node's BlockStore.
  598. func (n *Node) BlockStore() *bc.BlockStore {
  599. return n.blockStore
  600. }
  601. // ConsensusState returns the Node's ConsensusState.
  602. func (n *Node) ConsensusState() *cs.ConsensusState {
  603. return n.consensusState
  604. }
  605. // ConsensusReactor returns the Node's ConsensusReactor.
  606. func (n *Node) ConsensusReactor() *cs.ConsensusReactor {
  607. return n.consensusReactor
  608. }
  609. // MempoolReactor returns the Node's MempoolReactor.
  610. func (n *Node) MempoolReactor() *mempl.MempoolReactor {
  611. return n.mempoolReactor
  612. }
  613. // EvidencePool returns the Node's EvidencePool.
  614. func (n *Node) EvidencePool() *evidence.EvidencePool {
  615. return n.evidencePool
  616. }
  617. // EventBus returns the Node's EventBus.
  618. func (n *Node) EventBus() *types.EventBus {
  619. return n.eventBus
  620. }
  621. // PrivValidator returns the Node's PrivValidator.
  622. // XXX: for convenience only!
  623. func (n *Node) PrivValidator() types.PrivValidator {
  624. return n.privValidator
  625. }
  626. // GenesisDoc returns the Node's GenesisDoc.
  627. func (n *Node) GenesisDoc() *types.GenesisDoc {
  628. return n.genesisDoc
  629. }
  630. // ProxyApp returns the Node's AppConns, representing its connections to the ABCI application.
  631. func (n *Node) ProxyApp() proxy.AppConns {
  632. return n.proxyApp
  633. }
  634. //------------------------------------------------------------------------------
  635. func (n *Node) Listeners() []string {
  636. return []string{
  637. fmt.Sprintf("Listener(@%v)", n.config.P2P.ExternalAddress),
  638. }
  639. }
  640. func (n *Node) IsListening() bool {
  641. return n.isListening
  642. }
  643. // NodeInfo returns the Node's Info from the Switch.
  644. func (n *Node) NodeInfo() p2p.NodeInfo {
  645. return n.nodeInfo
  646. }
  647. func makeNodeInfo(
  648. config *cfg.Config,
  649. nodeID p2p.ID,
  650. txIndexer txindex.TxIndexer,
  651. chainID string,
  652. ) p2p.NodeInfo {
  653. txIndexerStatus := "on"
  654. if _, ok := txIndexer.(*null.TxIndex); ok {
  655. txIndexerStatus = "off"
  656. }
  657. nodeInfo := p2p.NodeInfo{
  658. ID: nodeID,
  659. Network: chainID,
  660. Version: version.Version,
  661. Channels: []byte{
  662. bc.BlockchainChannel,
  663. cs.StateChannel, cs.DataChannel, cs.VoteChannel, cs.VoteSetBitsChannel,
  664. mempl.MempoolChannel,
  665. evidence.EvidenceChannel,
  666. },
  667. Moniker: config.Moniker,
  668. Other: p2p.NodeInfoOther{
  669. AminoVersion: amino.Version,
  670. P2PVersion: p2p.Version,
  671. ConsensusVersion: cs.Version,
  672. RPCVersion: fmt.Sprintf("%v/%v", rpc.Version, rpccore.Version),
  673. TxIndex: txIndexerStatus,
  674. RPCAddress: config.RPC.ListenAddress,
  675. },
  676. }
  677. if config.P2P.PexReactor {
  678. nodeInfo.Channels = append(nodeInfo.Channels, pex.PexChannel)
  679. }
  680. lAddr := config.P2P.ExternalAddress
  681. if lAddr == "" {
  682. lAddr = config.P2P.ListenAddress
  683. }
  684. nodeInfo.ListenAddr = lAddr
  685. return nodeInfo
  686. }
  687. //------------------------------------------------------------------------------
  688. var (
  689. genesisDocKey = []byte("genesisDoc")
  690. )
  691. // panics if failed to unmarshal bytes
  692. func loadGenesisDoc(db dbm.DB) (*types.GenesisDoc, error) {
  693. bytes := db.Get(genesisDocKey)
  694. if len(bytes) == 0 {
  695. return nil, errors.New("Genesis doc not found")
  696. }
  697. var genDoc *types.GenesisDoc
  698. err := cdc.UnmarshalJSON(bytes, &genDoc)
  699. if err != nil {
  700. cmn.PanicCrisis(fmt.Sprintf("Failed to load genesis doc due to unmarshaling error: %v (bytes: %X)", err, bytes))
  701. }
  702. return genDoc, nil
  703. }
  704. // panics if failed to marshal the given genesis document
  705. func saveGenesisDoc(db dbm.DB, genDoc *types.GenesisDoc) {
  706. bytes, err := cdc.MarshalJSON(genDoc)
  707. if err != nil {
  708. cmn.PanicCrisis(fmt.Sprintf("Failed to save genesis doc due to marshaling error: %v", err))
  709. }
  710. db.SetSync(genesisDocKey, bytes)
  711. }
  712. // splitAndTrimEmpty slices s into all subslices separated by sep and returns a
  713. // slice of the string s with all leading and trailing Unicode code points
  714. // contained in cutset removed. If sep is empty, SplitAndTrim splits after each
  715. // UTF-8 sequence. First part is equivalent to strings.SplitN with a count of
  716. // -1. also filter out empty strings, only return non-empty strings.
  717. func splitAndTrimEmpty(s, sep, cutset string) []string {
  718. if s == "" {
  719. return []string{}
  720. }
  721. spl := strings.Split(s, sep)
  722. nonEmptyStrings := make([]string, 0, len(spl))
  723. for i := 0; i < len(spl); i++ {
  724. element := strings.Trim(spl[i], cutset)
  725. if element != "" {
  726. nonEmptyStrings = append(nonEmptyStrings, element)
  727. }
  728. }
  729. return nonEmptyStrings
  730. }