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.

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