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.

434 lines
14 KiB

  1. package statesync
  2. import (
  3. "bytes"
  4. "context"
  5. "errors"
  6. "fmt"
  7. "math/rand"
  8. "strings"
  9. "sync"
  10. "time"
  11. dbm "github.com/tendermint/tm-db"
  12. "github.com/tendermint/tendermint/internal/p2p"
  13. sm "github.com/tendermint/tendermint/internal/state"
  14. "github.com/tendermint/tendermint/libs/log"
  15. "github.com/tendermint/tendermint/light"
  16. lightprovider "github.com/tendermint/tendermint/light/provider"
  17. lighthttp "github.com/tendermint/tendermint/light/provider/http"
  18. lightrpc "github.com/tendermint/tendermint/light/rpc"
  19. lightdb "github.com/tendermint/tendermint/light/store/db"
  20. ssproto "github.com/tendermint/tendermint/proto/tendermint/statesync"
  21. rpchttp "github.com/tendermint/tendermint/rpc/client/http"
  22. "github.com/tendermint/tendermint/types"
  23. "github.com/tendermint/tendermint/version"
  24. )
  25. //go:generate ../../scripts/mockery_generate.sh StateProvider
  26. // StateProvider is a provider of trusted state data for bootstrapping a node. This refers
  27. // to the state.State object, not the state machine. There are two implementations. One
  28. // uses the P2P layer and the other uses the RPC layer. Both use light client verification.
  29. type StateProvider interface {
  30. // AppHash returns the app hash after the given height has been committed.
  31. AppHash(ctx context.Context, height uint64) ([]byte, error)
  32. // Commit returns the commit at the given height.
  33. Commit(ctx context.Context, height uint64) (*types.Commit, error)
  34. // State returns a state object at the given height.
  35. State(ctx context.Context, height uint64) (sm.State, error)
  36. }
  37. type stateProviderRPC struct {
  38. sync.Mutex // light.Client is not concurrency-safe
  39. lc *light.Client
  40. initialHeight int64
  41. providers map[lightprovider.Provider]string
  42. logger log.Logger
  43. }
  44. // NewRPCStateProvider creates a new StateProvider using a light client and RPC clients.
  45. func NewRPCStateProvider(
  46. ctx context.Context,
  47. chainID string,
  48. initialHeight int64,
  49. servers []string,
  50. trustOptions light.TrustOptions,
  51. logger log.Logger,
  52. ) (StateProvider, error) {
  53. if len(servers) < 2 {
  54. return nil, fmt.Errorf("at least 2 RPC servers are required, got %d", len(servers))
  55. }
  56. providers := make([]lightprovider.Provider, 0, len(servers))
  57. providerRemotes := make(map[lightprovider.Provider]string)
  58. for _, server := range servers {
  59. client, err := rpcClient(server)
  60. if err != nil {
  61. return nil, fmt.Errorf("failed to set up RPC client: %w", err)
  62. }
  63. provider := lighthttp.NewWithClient(chainID, client)
  64. providers = append(providers, provider)
  65. // We store the RPC addresses keyed by provider, so we can find the address of the primary
  66. // provider used by the light client and use it to fetch consensus parameters.
  67. providerRemotes[provider] = server
  68. }
  69. lc, err := light.NewClient(ctx, chainID, trustOptions, providers[0], providers[1:],
  70. lightdb.New(dbm.NewMemDB()), light.Logger(logger))
  71. if err != nil {
  72. return nil, err
  73. }
  74. return &stateProviderRPC{
  75. logger: logger,
  76. lc: lc,
  77. initialHeight: initialHeight,
  78. providers: providerRemotes,
  79. }, nil
  80. }
  81. func (s *stateProviderRPC) verifyLightBlockAtHeight(ctx context.Context, height uint64, ts time.Time) (*types.LightBlock, error) {
  82. ctx, cancel := context.WithTimeout(ctx, 20*time.Second)
  83. defer cancel()
  84. return s.lc.VerifyLightBlockAtHeight(ctx, int64(height), ts)
  85. }
  86. // AppHash implements part of StateProvider. It calls the application to verify the
  87. // light blocks at heights h+1 and h+2 and, if verification succeeds, reports the app
  88. // hash for the block at height h+1 which correlates to the state at height h.
  89. func (s *stateProviderRPC) AppHash(ctx context.Context, height uint64) ([]byte, error) {
  90. s.Lock()
  91. defer s.Unlock()
  92. // We have to fetch the next height, which contains the app hash for the previous height.
  93. header, err := s.verifyLightBlockAtHeight(ctx, height+1, time.Now())
  94. if err != nil {
  95. return nil, err
  96. }
  97. // We also try to fetch the blocks at H+2, since we need these
  98. // when building the state while restoring the snapshot. This avoids the race
  99. // condition where we try to restore a snapshot before H+2 exists.
  100. _, err = s.verifyLightBlockAtHeight(ctx, height+2, time.Now())
  101. if err != nil {
  102. return nil, err
  103. }
  104. return header.AppHash, nil
  105. }
  106. // Commit implements StateProvider.
  107. func (s *stateProviderRPC) Commit(ctx context.Context, height uint64) (*types.Commit, error) {
  108. s.Lock()
  109. defer s.Unlock()
  110. header, err := s.verifyLightBlockAtHeight(ctx, height, time.Now())
  111. if err != nil {
  112. return nil, err
  113. }
  114. return header.Commit, nil
  115. }
  116. // State implements StateProvider.
  117. func (s *stateProviderRPC) State(ctx context.Context, height uint64) (sm.State, error) {
  118. s.Lock()
  119. defer s.Unlock()
  120. state := sm.State{
  121. ChainID: s.lc.ChainID(),
  122. InitialHeight: s.initialHeight,
  123. }
  124. if state.InitialHeight == 0 {
  125. state.InitialHeight = 1
  126. }
  127. // The snapshot height maps onto the state heights as follows:
  128. //
  129. // height: last block, i.e. the snapshotted height
  130. // height+1: current block, i.e. the first block we'll process after the snapshot
  131. // height+2: next block, i.e. the second block after the snapshot
  132. //
  133. // We need to fetch the NextValidators from height+2 because if the application changed
  134. // the validator set at the snapshot height then this only takes effect at height+2.
  135. lastLightBlock, err := s.verifyLightBlockAtHeight(ctx, height, time.Now())
  136. if err != nil {
  137. return sm.State{}, err
  138. }
  139. currentLightBlock, err := s.verifyLightBlockAtHeight(ctx, height+1, time.Now())
  140. if err != nil {
  141. return sm.State{}, err
  142. }
  143. nextLightBlock, err := s.verifyLightBlockAtHeight(ctx, height+2, time.Now())
  144. if err != nil {
  145. return sm.State{}, err
  146. }
  147. state.Version = sm.Version{
  148. Consensus: currentLightBlock.Version,
  149. Software: version.TMVersion,
  150. }
  151. state.LastBlockHeight = lastLightBlock.Height
  152. state.LastBlockTime = lastLightBlock.Time
  153. state.LastBlockID = lastLightBlock.Commit.BlockID
  154. state.AppHash = currentLightBlock.AppHash
  155. state.LastResultsHash = currentLightBlock.LastResultsHash
  156. state.LastValidators = lastLightBlock.ValidatorSet
  157. state.Validators = currentLightBlock.ValidatorSet
  158. state.NextValidators = nextLightBlock.ValidatorSet
  159. state.LastHeightValidatorsChanged = nextLightBlock.Height
  160. // We'll also need to fetch consensus params via RPC, using light client verification.
  161. primaryURL, ok := s.providers[s.lc.Primary()]
  162. if !ok || primaryURL == "" {
  163. return sm.State{}, fmt.Errorf("could not find address for primary light client provider")
  164. }
  165. primaryRPC, err := rpcClient(primaryURL)
  166. if err != nil {
  167. return sm.State{}, fmt.Errorf("unable to create RPC client: %w", err)
  168. }
  169. rpcclient := lightrpc.NewClient(s.logger, primaryRPC, s.lc)
  170. result, err := rpcclient.ConsensusParams(ctx, &currentLightBlock.Height)
  171. if err != nil {
  172. return sm.State{}, fmt.Errorf("unable to fetch consensus parameters for height %v: %w",
  173. nextLightBlock.Height, err)
  174. }
  175. state.ConsensusParams = result.ConsensusParams
  176. state.LastHeightConsensusParamsChanged = currentLightBlock.Height
  177. return state, nil
  178. }
  179. // rpcClient sets up a new RPC client
  180. func rpcClient(server string) (*rpchttp.HTTP, error) {
  181. if !strings.Contains(server, "://") {
  182. server = "http://" + server
  183. }
  184. return rpchttp.New(server)
  185. }
  186. type stateProviderP2P struct {
  187. sync.Mutex // light.Client is not concurrency-safe
  188. lc *light.Client
  189. initialHeight int64
  190. paramsSendCh *p2p.Channel
  191. paramsRecvCh chan types.ConsensusParams
  192. }
  193. // NewP2PStateProvider creates a light client state
  194. // provider but uses a dispatcher connected to the P2P layer
  195. func NewP2PStateProvider(
  196. ctx context.Context,
  197. chainID string,
  198. initialHeight int64,
  199. providers []lightprovider.Provider,
  200. trustOptions light.TrustOptions,
  201. paramsSendCh *p2p.Channel,
  202. logger log.Logger,
  203. ) (StateProvider, error) {
  204. if len(providers) < 2 {
  205. return nil, fmt.Errorf("at least 2 peers are required, got %d", len(providers))
  206. }
  207. lc, err := light.NewClient(ctx, chainID, trustOptions, providers[0], providers[1:],
  208. lightdb.New(dbm.NewMemDB()), light.Logger(logger))
  209. if err != nil {
  210. return nil, err
  211. }
  212. return &stateProviderP2P{
  213. lc: lc,
  214. initialHeight: initialHeight,
  215. paramsSendCh: paramsSendCh,
  216. paramsRecvCh: make(chan types.ConsensusParams),
  217. }, nil
  218. }
  219. func (s *stateProviderP2P) verifyLightBlockAtHeight(ctx context.Context, height uint64, ts time.Time) (*types.LightBlock, error) {
  220. ctx, cancel := context.WithTimeout(ctx, 20*time.Second)
  221. defer cancel()
  222. return s.lc.VerifyLightBlockAtHeight(ctx, int64(height), ts)
  223. }
  224. // AppHash implements StateProvider.
  225. func (s *stateProviderP2P) AppHash(ctx context.Context, height uint64) ([]byte, error) {
  226. s.Lock()
  227. defer s.Unlock()
  228. // We have to fetch the next height, which contains the app hash for the previous height.
  229. header, err := s.verifyLightBlockAtHeight(ctx, height+1, time.Now())
  230. if err != nil {
  231. return nil, err
  232. }
  233. // We also try to fetch the blocks at H+2, since we need these
  234. // when building the state while restoring the snapshot. This avoids the race
  235. // condition where we try to restore a snapshot before H+2 exists.
  236. _, err = s.verifyLightBlockAtHeight(ctx, height+2, time.Now())
  237. if err != nil {
  238. return nil, err
  239. }
  240. return header.AppHash, nil
  241. }
  242. // Commit implements StateProvider.
  243. func (s *stateProviderP2P) Commit(ctx context.Context, height uint64) (*types.Commit, error) {
  244. s.Lock()
  245. defer s.Unlock()
  246. header, err := s.verifyLightBlockAtHeight(ctx, height, time.Now())
  247. if err != nil {
  248. return nil, err
  249. }
  250. return header.Commit, nil
  251. }
  252. // State implements StateProvider.
  253. func (s *stateProviderP2P) State(ctx context.Context, height uint64) (sm.State, error) {
  254. s.Lock()
  255. defer s.Unlock()
  256. state := sm.State{
  257. ChainID: s.lc.ChainID(),
  258. InitialHeight: s.initialHeight,
  259. }
  260. if state.InitialHeight == 0 {
  261. state.InitialHeight = 1
  262. }
  263. // The snapshot height maps onto the state heights as follows:
  264. //
  265. // height: last block, i.e. the snapshotted height
  266. // height+1: current block, i.e. the first block we'll process after the snapshot
  267. // height+2: next block, i.e. the second block after the snapshot
  268. //
  269. // We need to fetch the NextValidators from height+2 because if the application changed
  270. // the validator set at the snapshot height then this only takes effect at height+2.
  271. lastLightBlock, err := s.verifyLightBlockAtHeight(ctx, height, time.Now())
  272. if err != nil {
  273. return sm.State{}, err
  274. }
  275. currentLightBlock, err := s.verifyLightBlockAtHeight(ctx, height+1, time.Now())
  276. if err != nil {
  277. return sm.State{}, err
  278. }
  279. nextLightBlock, err := s.verifyLightBlockAtHeight(ctx, height+2, time.Now())
  280. if err != nil {
  281. return sm.State{}, err
  282. }
  283. state.Version = sm.Version{
  284. Consensus: currentLightBlock.Version,
  285. Software: version.TMVersion,
  286. }
  287. state.LastBlockHeight = lastLightBlock.Height
  288. state.LastBlockTime = lastLightBlock.Time
  289. state.LastBlockID = lastLightBlock.Commit.BlockID
  290. state.AppHash = currentLightBlock.AppHash
  291. state.LastResultsHash = currentLightBlock.LastResultsHash
  292. state.LastValidators = lastLightBlock.ValidatorSet
  293. state.Validators = currentLightBlock.ValidatorSet
  294. state.NextValidators = nextLightBlock.ValidatorSet
  295. state.LastHeightValidatorsChanged = nextLightBlock.Height
  296. // We'll also need to fetch consensus params via P2P.
  297. state.ConsensusParams, err = s.consensusParams(ctx, currentLightBlock.Height)
  298. if err != nil {
  299. return sm.State{}, err
  300. }
  301. // validate the consensus params
  302. if !bytes.Equal(nextLightBlock.ConsensusHash, state.ConsensusParams.HashConsensusParams()) {
  303. return sm.State{}, fmt.Errorf("consensus params hash mismatch at height %d. Expected %v, got %v",
  304. currentLightBlock.Height, nextLightBlock.ConsensusHash, state.ConsensusParams.HashConsensusParams())
  305. }
  306. // set the last height changed to the current height
  307. state.LastHeightConsensusParamsChanged = currentLightBlock.Height
  308. return state, nil
  309. }
  310. // addProvider dynamically adds a peer as a new witness. A limit of 6 providers is kept as a
  311. // heuristic. Too many overburdens the network and too little compromises the second layer of security.
  312. func (s *stateProviderP2P) addProvider(p lightprovider.Provider) {
  313. if len(s.lc.Witnesses()) < 6 {
  314. s.lc.AddProvider(p)
  315. }
  316. }
  317. // consensusParams sends out a request for consensus params blocking
  318. // until one is returned.
  319. //
  320. // If it fails to get a valid set of consensus params from any of the
  321. // providers it returns an error; however, it will retry indefinitely
  322. // (with backoff) until the context is canceled.
  323. func (s *stateProviderP2P) consensusParams(ctx context.Context, height int64) (types.ConsensusParams, error) {
  324. var iterCount int64
  325. timer := time.NewTimer(0)
  326. defer timer.Stop()
  327. for {
  328. params, err := s.tryGetConsensusParamsFromWitnesses(ctx, height)
  329. if err != nil {
  330. return types.ConsensusParams{}, err
  331. }
  332. if params != nil {
  333. return *params, nil
  334. }
  335. iterCount++
  336. // jitter+backoff the retry loop
  337. timer.Reset(time.Duration(iterCount)*consensusParamsResponseTimeout +
  338. time.Duration(100*rand.Int63n(iterCount))*time.Millisecond) // nolint:gosec
  339. select {
  340. case <-ctx.Done():
  341. return types.ConsensusParams{}, ctx.Err()
  342. case <-timer.C:
  343. }
  344. }
  345. }
  346. // tryGetConsensusParamsFromWitnesses attempts to get consensus
  347. // parameters from the light clients available witnesses. If both
  348. // return parameters are nil, then it can be retried.
  349. func (s *stateProviderP2P) tryGetConsensusParamsFromWitnesses(
  350. ctx context.Context,
  351. height int64,
  352. ) (*types.ConsensusParams, error) {
  353. timer := time.NewTimer(0)
  354. defer timer.Stop()
  355. for _, provider := range s.lc.Witnesses() {
  356. p, ok := provider.(*BlockProvider)
  357. if !ok {
  358. panic("expected p2p state provider to use p2p block providers")
  359. }
  360. // extract the nodeID of the provider
  361. peer, err := types.NewNodeID(p.String())
  362. if err != nil {
  363. return nil, fmt.Errorf("invalid provider (%s) node id: %w", p.String(), err)
  364. }
  365. if err := s.paramsSendCh.Send(ctx, p2p.Envelope{
  366. To: peer,
  367. Message: &ssproto.ParamsRequest{
  368. Height: uint64(height),
  369. },
  370. }); err != nil {
  371. return nil, err
  372. }
  373. timer.Reset(consensusParamsResponseTimeout)
  374. select {
  375. // if we get no response from this provider we move on to the next one
  376. case <-timer.C:
  377. continue
  378. case <-ctx.Done():
  379. return nil, ctx.Err()
  380. case params, ok := <-s.paramsRecvCh:
  381. if !ok {
  382. return nil, errors.New("params channel closed")
  383. }
  384. return &params, nil
  385. }
  386. }
  387. // signal to caller to retry.
  388. return nil, nil
  389. }