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.

412 lines
13 KiB

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