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.

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