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.

528 lines
16 KiB

Fix eth events query and sync inconsistent state - kvdb - Fix path in Last when doing `setNew` - Only close if db != nil, and after closing, always set db to nil - This will avoid a panic in the case where the db is closed but there's an error soon after, and a future call tries to close again. This is because pebble.Close() will panic if the db is already closed. - Avoid calling pebble methods when a the Storage interface already implements that method (like Close). - statedb - In test, avoid calling KVDB method if the same method is available for the StateDB (like MakeCheckpoint, CurrentBatch). - eth - In *EventByBlock methods, take blockHash as input argument and use it when querying the event logs. Previously the blockHash was only taken from the logs results *only if* there was any log. This caused the following issue: if there was no logs, it was not possible to know if the result was from the expected block or an uncle block! By querying logs by blockHash we make sure that even if there are no logs, they are from the right block. - Note that now the function can either be called with a blockNum or blockHash, but not both at the same time. - sync - If there's an error during call to Sync call resetState, which internally resets the stateDB to avoid stale checkpoints (and a corresponding invalid increase in the StateDB batchNum). - During a Sync, after very batch processed, make sure that the StateDB currentBatch corresponds to the batchNum in the smart contract log/event.
3 years ago
Update coordinator to work better under real net - cli / node - Update handler of SIGINT so that after 3 SIGINTs, the process terminates unconditionally - coordinator - Store stats without pointer - In all functions that send a variable via channel, check for context done to avoid deadlock (due to no process reading from the channel, which has no queue) when the node is stopped. - Abstract `canForge` so that it can be used outside of the `Coordinator` - In `canForge` check the blockNumber in current and next slot. - Update tests due to smart contract changes in slot handling, and minimum bid defaults - TxManager - Add consts, vars and stats to allow evaluating `canForge` - Add `canForge` method (not used yet) - Store batch and nonces status (last success and last pending) - Track nonces internally instead of relying on the ethereum node (this is required to work with ganache when there are pending txs) - Handle the (common) case of the receipt not being found after the tx is sent. - Don't start the main loop until we get an initial messae fo the stats and vars (so that in the loop the stats and vars are set to synchronizer values) - When a tx fails, check and discard all the failed transactions before sending the message to stop the pipeline. This will avoid sending consecutive messages of stop the pipeline when multiple txs are detected to be failed consecutively. Also, future txs of the same pipeline after a discarded txs are discarded, and their nonces reused. - Robust handling of nonces: - If geth returns nonce is too low, increase it - If geth returns nonce too hight, decrease it - If geth returns underpriced, increase gas price - If geth returns replace underpriced, increase gas price - Add support for resending transactions after a timeout - Store `BatchInfos` in a queue - Pipeline - When an error is found, stop forging batches and send a message to the coordinator to stop the pipeline with information of the failed batch number so that in a restart, non-failed batches are not repated. - When doing a reset of the stateDB, if possible reset from the local checkpoint instead of resetting from the synchronizer. This allows resetting from a batch that is valid but not yet sent / synced. - Every time a pipeline is started, assign it a number from a counter. This allows the TxManager to ignore batches from stopped pipelines, via a message sent by the coordinator. - Avoid forging when we haven't reached the rollup genesis block number. - Add config parameter `StartSlotBlocksDelay`: StartSlotBlocksDelay is the number of blocks of delay to wait before starting the pipeline when we reach a slot in which we can forge. - When detecting a reorg, only reset the pipeline if the batch from which the pipeline started changed and wasn't sent by us. - Add config parameter `ScheduleBatchBlocksAheadCheck`: ScheduleBatchBlocksAheadCheck is the number of blocks ahead in which the forger address is checked to be allowed to forge (apart from checking the next block), used to decide when to stop scheduling new batches (by stopping the pipeline). For example, if we are at block 10 and ScheduleBatchBlocksAheadCheck is 5, eventhough at block 11 we canForge, the pipeline will be stopped if we can't forge at block 15. This value should be the expected number of blocks it takes between scheduling a batch and having it mined. - Add config parameter `SendBatchBlocksMarginCheck`: SendBatchBlocksMarginCheck is the number of margin blocks ahead in which the coordinator is also checked to be allowed to forge, apart from the next block; used to decide when to stop sending batches to the smart contract. For example, if we are at block 10 and SendBatchBlocksMarginCheck is 5, eventhough at block 11 we canForge, the batch will be discarded if we can't forge at block 15. - Add config parameter `TxResendTimeout`: TxResendTimeout is the timeout after which a non-mined ethereum transaction will be resent (reusing the nonce) with a newly calculated gas price - Add config parameter `MaxGasPrice`: MaxGasPrice is the maximum gas price allowed for ethereum transactions - Add config parameter `NoReuseNonce`: NoReuseNonce disables reusing nonces of pending transactions for new replacement transactions. This is useful for testing with Ganache. - Extend BatchInfo with more useful information for debugging - eth / ethereum client - Add necessary methods to create the auth object for transactions manually so that we can set the nonce, gas price, gas limit, etc manually - Update `RollupForgeBatch` to take an auth object as input (so that the coordinator can set parameters manually) - synchronizer - In stats, add `NextSlot` - In stats, store full last batch instead of just last batch number - Instead of calculating a nextSlot from scratch every time, update the current struct (only updating the forger info if we are Synced) - Afer every processed batch, check that the calculated StateDB MTRoot matches the StateRoot found in the forgeBatch event.
3 years ago
Update coordinator to work better under real net - cli / node - Update handler of SIGINT so that after 3 SIGINTs, the process terminates unconditionally - coordinator - Store stats without pointer - In all functions that send a variable via channel, check for context done to avoid deadlock (due to no process reading from the channel, which has no queue) when the node is stopped. - Abstract `canForge` so that it can be used outside of the `Coordinator` - In `canForge` check the blockNumber in current and next slot. - Update tests due to smart contract changes in slot handling, and minimum bid defaults - TxManager - Add consts, vars and stats to allow evaluating `canForge` - Add `canForge` method (not used yet) - Store batch and nonces status (last success and last pending) - Track nonces internally instead of relying on the ethereum node (this is required to work with ganache when there are pending txs) - Handle the (common) case of the receipt not being found after the tx is sent. - Don't start the main loop until we get an initial messae fo the stats and vars (so that in the loop the stats and vars are set to synchronizer values) - When a tx fails, check and discard all the failed transactions before sending the message to stop the pipeline. This will avoid sending consecutive messages of stop the pipeline when multiple txs are detected to be failed consecutively. Also, future txs of the same pipeline after a discarded txs are discarded, and their nonces reused. - Robust handling of nonces: - If geth returns nonce is too low, increase it - If geth returns nonce too hight, decrease it - If geth returns underpriced, increase gas price - If geth returns replace underpriced, increase gas price - Add support for resending transactions after a timeout - Store `BatchInfos` in a queue - Pipeline - When an error is found, stop forging batches and send a message to the coordinator to stop the pipeline with information of the failed batch number so that in a restart, non-failed batches are not repated. - When doing a reset of the stateDB, if possible reset from the local checkpoint instead of resetting from the synchronizer. This allows resetting from a batch that is valid but not yet sent / synced. - Every time a pipeline is started, assign it a number from a counter. This allows the TxManager to ignore batches from stopped pipelines, via a message sent by the coordinator. - Avoid forging when we haven't reached the rollup genesis block number. - Add config parameter `StartSlotBlocksDelay`: StartSlotBlocksDelay is the number of blocks of delay to wait before starting the pipeline when we reach a slot in which we can forge. - When detecting a reorg, only reset the pipeline if the batch from which the pipeline started changed and wasn't sent by us. - Add config parameter `ScheduleBatchBlocksAheadCheck`: ScheduleBatchBlocksAheadCheck is the number of blocks ahead in which the forger address is checked to be allowed to forge (apart from checking the next block), used to decide when to stop scheduling new batches (by stopping the pipeline). For example, if we are at block 10 and ScheduleBatchBlocksAheadCheck is 5, eventhough at block 11 we canForge, the pipeline will be stopped if we can't forge at block 15. This value should be the expected number of blocks it takes between scheduling a batch and having it mined. - Add config parameter `SendBatchBlocksMarginCheck`: SendBatchBlocksMarginCheck is the number of margin blocks ahead in which the coordinator is also checked to be allowed to forge, apart from the next block; used to decide when to stop sending batches to the smart contract. For example, if we are at block 10 and SendBatchBlocksMarginCheck is 5, eventhough at block 11 we canForge, the batch will be discarded if we can't forge at block 15. - Add config parameter `TxResendTimeout`: TxResendTimeout is the timeout after which a non-mined ethereum transaction will be resent (reusing the nonce) with a newly calculated gas price - Add config parameter `MaxGasPrice`: MaxGasPrice is the maximum gas price allowed for ethereum transactions - Add config parameter `NoReuseNonce`: NoReuseNonce disables reusing nonces of pending transactions for new replacement transactions. This is useful for testing with Ganache. - Extend BatchInfo with more useful information for debugging - eth / ethereum client - Add necessary methods to create the auth object for transactions manually so that we can set the nonce, gas price, gas limit, etc manually - Update `RollupForgeBatch` to take an auth object as input (so that the coordinator can set parameters manually) - synchronizer - In stats, add `NextSlot` - In stats, store full last batch instead of just last batch number - Instead of calculating a nextSlot from scratch every time, update the current struct (only updating the forger info if we are Synced) - Afer every processed batch, check that the calculated StateDB MTRoot matches the StateRoot found in the forgeBatch event.
3 years ago
Fix eth events query and sync inconsistent state - kvdb - Fix path in Last when doing `setNew` - Only close if db != nil, and after closing, always set db to nil - This will avoid a panic in the case where the db is closed but there's an error soon after, and a future call tries to close again. This is because pebble.Close() will panic if the db is already closed. - Avoid calling pebble methods when a the Storage interface already implements that method (like Close). - statedb - In test, avoid calling KVDB method if the same method is available for the StateDB (like MakeCheckpoint, CurrentBatch). - eth - In *EventByBlock methods, take blockHash as input argument and use it when querying the event logs. Previously the blockHash was only taken from the logs results *only if* there was any log. This caused the following issue: if there was no logs, it was not possible to know if the result was from the expected block or an uncle block! By querying logs by blockHash we make sure that even if there are no logs, they are from the right block. - Note that now the function can either be called with a blockNum or blockHash, but not both at the same time. - sync - If there's an error during call to Sync call resetState, which internally resets the stateDB to avoid stale checkpoints (and a corresponding invalid increase in the StateDB batchNum). - During a Sync, after very batch processed, make sure that the StateDB currentBatch corresponds to the batchNum in the smart contract log/event.
3 years ago
Fix eth events query and sync inconsistent state - kvdb - Fix path in Last when doing `setNew` - Only close if db != nil, and after closing, always set db to nil - This will avoid a panic in the case where the db is closed but there's an error soon after, and a future call tries to close again. This is because pebble.Close() will panic if the db is already closed. - Avoid calling pebble methods when a the Storage interface already implements that method (like Close). - statedb - In test, avoid calling KVDB method if the same method is available for the StateDB (like MakeCheckpoint, CurrentBatch). - eth - In *EventByBlock methods, take blockHash as input argument and use it when querying the event logs. Previously the blockHash was only taken from the logs results *only if* there was any log. This caused the following issue: if there was no logs, it was not possible to know if the result was from the expected block or an uncle block! By querying logs by blockHash we make sure that even if there are no logs, they are from the right block. - Note that now the function can either be called with a blockNum or blockHash, but not both at the same time. - sync - If there's an error during call to Sync call resetState, which internally resets the stateDB to avoid stale checkpoints (and a corresponding invalid increase in the StateDB batchNum). - During a Sync, after very batch processed, make sure that the StateDB currentBatch corresponds to the batchNum in the smart contract log/event.
3 years ago
  1. package statedb
  2. import (
  3. "errors"
  4. "fmt"
  5. "math/big"
  6. "github.com/hermeznetwork/hermez-node/common"
  7. "github.com/hermeznetwork/hermez-node/db/kvdb"
  8. "github.com/hermeznetwork/hermez-node/log"
  9. "github.com/hermeznetwork/tracerr"
  10. "github.com/iden3/go-merkletree"
  11. "github.com/iden3/go-merkletree/db"
  12. "github.com/iden3/go-merkletree/db/pebble"
  13. )
  14. var (
  15. // ErrStateDBWithoutMT is used when a method that requires a MerkleTree
  16. // is called in a StateDB that does not have a MerkleTree defined
  17. ErrStateDBWithoutMT = errors.New("Can not call method to use MerkleTree in a StateDB without MerkleTree")
  18. // ErrAccountAlreadyExists is used when CreateAccount is called and the
  19. // Account already exists
  20. ErrAccountAlreadyExists = errors.New("Can not CreateAccount because Account already exists")
  21. // ErrIdxNotFound is used when trying to get the Idx from EthAddr or
  22. // EthAddr&ToBJJ
  23. ErrIdxNotFound = errors.New("Idx can not be found")
  24. // ErrGetIdxNoCase is used when trying to get the Idx from EthAddr &
  25. // BJJ with not compatible combination
  26. ErrGetIdxNoCase = errors.New("Can not get Idx due unexpected combination of ethereum Address & BabyJubJub PublicKey")
  27. // PrefixKeyIdx is the key prefix for idx in the db
  28. PrefixKeyIdx = []byte("i:")
  29. // PrefixKeyAccHash is the key prefix for account hash in the db
  30. PrefixKeyAccHash = []byte("h:")
  31. // PrefixKeyMT is the key prefix for merkle tree in the db
  32. PrefixKeyMT = []byte("m:")
  33. // PrefixKeyAddr is the key prefix for address in the db
  34. PrefixKeyAddr = []byte("a:")
  35. // PrefixKeyAddrBJJ is the key prefix for address-babyjubjub in the db
  36. PrefixKeyAddrBJJ = []byte("ab:")
  37. )
  38. const (
  39. // TypeSynchronizer defines a StateDB used by the Synchronizer, that
  40. // generates the ExitTree when processing the txs
  41. TypeSynchronizer = "synchronizer"
  42. // TypeTxSelector defines a StateDB used by the TxSelector, without
  43. // computing ExitTree neither the ZKInputs
  44. TypeTxSelector = "txselector"
  45. // TypeBatchBuilder defines a StateDB used by the BatchBuilder, that
  46. // generates the ExitTree and the ZKInput when processing the txs
  47. TypeBatchBuilder = "batchbuilder"
  48. // MaxNLevels is the maximum value of NLevels for the merkle tree,
  49. // which comes from the fact that AccountIdx has 48 bits.
  50. MaxNLevels = 48
  51. )
  52. // TypeStateDB determines the type of StateDB
  53. type TypeStateDB string
  54. // Config of the StateDB
  55. type Config struct {
  56. // Path where the checkpoints will be stored
  57. Path string
  58. // Keep is the number of old checkpoints to keep. If 0, all
  59. // checkpoints are kept.
  60. Keep int
  61. // NoLast skips having an opened DB with a checkpoint to the last
  62. // batchNum for thread-safe reads.
  63. NoLast bool
  64. // Type of StateDB (
  65. Type TypeStateDB
  66. // NLevels is the number of merkle tree levels in case the Type uses a
  67. // merkle tree. If the Type doesn't use a merkle tree, NLevels should
  68. // be 0.
  69. NLevels int
  70. // At every checkpoint, check that there are no gaps between the
  71. // checkpoints
  72. noGapsCheck bool
  73. }
  74. // StateDB represents the StateDB object
  75. type StateDB struct {
  76. cfg Config
  77. db *kvdb.KVDB
  78. MT *merkletree.MerkleTree
  79. }
  80. // Last offers a subset of view methods of the StateDB that can be
  81. // called via the LastRead method of StateDB in a thread-safe manner to obtain
  82. // a consistent view to the last batch of the StateDB.
  83. type Last struct {
  84. db db.Storage
  85. }
  86. // GetAccount returns the account for the given Idx
  87. func (s *Last) GetAccount(idx common.Idx) (*common.Account, error) {
  88. return GetAccountInTreeDB(s.db, idx)
  89. }
  90. // GetCurrentBatch returns the current BatchNum stored in Last.db
  91. func (s *Last) GetCurrentBatch() (common.BatchNum, error) {
  92. cbBytes, err := s.db.Get(kvdb.KeyCurrentBatch)
  93. if tracerr.Unwrap(err) == db.ErrNotFound {
  94. return 0, nil
  95. } else if err != nil {
  96. return 0, tracerr.Wrap(err)
  97. }
  98. return common.BatchNumFromBytes(cbBytes)
  99. }
  100. // DB returns the underlying storage of Last
  101. func (s *Last) DB() db.Storage {
  102. return s.db
  103. }
  104. // GetAccounts returns all the accounts in the db. Use for debugging pruposes
  105. // only.
  106. func (s *Last) GetAccounts() ([]common.Account, error) {
  107. return getAccounts(s.db)
  108. }
  109. // NewStateDB creates a new StateDB, allowing to use an in-memory or in-disk
  110. // storage. Checkpoints older than the value defined by `keep` will be
  111. // deleted.
  112. // func NewStateDB(pathDB string, keep int, typ TypeStateDB, nLevels int) (*StateDB, error) {
  113. func NewStateDB(cfg Config) (*StateDB, error) {
  114. var kv *kvdb.KVDB
  115. var err error
  116. kv, err = kvdb.NewKVDB(kvdb.Config{Path: cfg.Path, Keep: cfg.Keep,
  117. NoGapsCheck: cfg.noGapsCheck, NoLast: cfg.NoLast})
  118. if err != nil {
  119. return nil, tracerr.Wrap(err)
  120. }
  121. var mt *merkletree.MerkleTree = nil
  122. if cfg.Type == TypeSynchronizer || cfg.Type == TypeBatchBuilder {
  123. mt, err = merkletree.NewMerkleTree(kv.StorageWithPrefix(PrefixKeyMT), cfg.NLevels)
  124. if err != nil {
  125. return nil, tracerr.Wrap(err)
  126. }
  127. }
  128. if cfg.Type == TypeTxSelector && cfg.NLevels != 0 {
  129. return nil, tracerr.Wrap(fmt.Errorf("invalid StateDB parameters: StateDB type==TypeStateDB can not have nLevels!=0"))
  130. }
  131. return &StateDB{
  132. cfg: cfg,
  133. db: kv,
  134. MT: mt,
  135. }, nil
  136. }
  137. // Type returns the StateDB configured Type
  138. func (s *StateDB) Type() TypeStateDB {
  139. return s.cfg.Type
  140. }
  141. // LastRead is a thread-safe method to query the last checkpoint of the StateDB
  142. // via the Last type methods
  143. func (s *StateDB) LastRead(fn func(sdbLast *Last) error) error {
  144. return s.db.LastRead(
  145. func(db *pebble.Storage) error {
  146. return fn(&Last{
  147. db: db,
  148. })
  149. },
  150. )
  151. }
  152. // LastGetAccount is a thread-safe method to query an account in the last
  153. // checkpoint of the StateDB.
  154. func (s *StateDB) LastGetAccount(idx common.Idx) (*common.Account, error) {
  155. var account *common.Account
  156. if err := s.LastRead(func(sdb *Last) error {
  157. var err error
  158. account, err = sdb.GetAccount(idx)
  159. return err
  160. }); err != nil {
  161. return nil, tracerr.Wrap(err)
  162. }
  163. return account, nil
  164. }
  165. // LastGetCurrentBatch is a thread-safe method to get the current BatchNum in
  166. // the last checkpoint of the StateDB.
  167. func (s *StateDB) LastGetCurrentBatch() (common.BatchNum, error) {
  168. var batchNum common.BatchNum
  169. if err := s.LastRead(func(sdb *Last) error {
  170. var err error
  171. batchNum, err = sdb.GetCurrentBatch()
  172. return err
  173. }); err != nil {
  174. return 0, tracerr.Wrap(err)
  175. }
  176. return batchNum, nil
  177. }
  178. // LastMTGetRoot returns the root of the underlying Merkle Tree in the last
  179. // checkpoint of the StateDB.
  180. func (s *StateDB) LastMTGetRoot() (*big.Int, error) {
  181. var root *big.Int
  182. if err := s.LastRead(func(sdb *Last) error {
  183. mt, err := merkletree.NewMerkleTree(sdb.DB().WithPrefix(PrefixKeyMT), s.cfg.NLevels)
  184. if err != nil {
  185. return tracerr.Wrap(err)
  186. }
  187. root = mt.Root().BigInt()
  188. return nil
  189. }); err != nil {
  190. return nil, tracerr.Wrap(err)
  191. }
  192. return root, nil
  193. }
  194. // MakeCheckpoint does a checkpoint at the given batchNum in the defined path.
  195. // Internally this advances & stores the current BatchNum, and then stores a
  196. // Checkpoint of the current state of the StateDB.
  197. func (s *StateDB) MakeCheckpoint() error {
  198. log.Debugw("Making StateDB checkpoint", "batch", s.CurrentBatch()+1, "type", s.cfg.Type)
  199. return s.db.MakeCheckpoint()
  200. }
  201. // CurrentBatch returns the current in-memory CurrentBatch of the StateDB.db
  202. func (s *StateDB) CurrentBatch() common.BatchNum {
  203. return s.db.CurrentBatch
  204. }
  205. // CurrentIdx returns the current in-memory CurrentIdx of the StateDB.db
  206. func (s *StateDB) CurrentIdx() common.Idx {
  207. return s.db.CurrentIdx
  208. }
  209. // getCurrentBatch returns the current BatchNum stored in the StateDB.db
  210. func (s *StateDB) getCurrentBatch() (common.BatchNum, error) {
  211. return s.db.GetCurrentBatch()
  212. }
  213. // GetCurrentIdx returns the stored Idx from the localStateDB, which is the
  214. // last Idx used for an Account in the localStateDB.
  215. func (s *StateDB) GetCurrentIdx() (common.Idx, error) {
  216. return s.db.GetCurrentIdx()
  217. }
  218. // SetCurrentIdx stores Idx in the StateDB
  219. func (s *StateDB) SetCurrentIdx(idx common.Idx) error {
  220. return s.db.SetCurrentIdx(idx)
  221. }
  222. // Reset resets the StateDB to the checkpoint at the given batchNum. Reset
  223. // does not delete the checkpoints between old current and the new current,
  224. // those checkpoints will remain in the storage, and eventually will be
  225. // deleted when MakeCheckpoint overwrites them.
  226. func (s *StateDB) Reset(batchNum common.BatchNum) error {
  227. log.Debugw("Making StateDB Reset", "batch", batchNum, "type", s.cfg.Type)
  228. if err := s.db.Reset(batchNum); err != nil {
  229. return tracerr.Wrap(err)
  230. }
  231. if s.MT != nil {
  232. // open the MT for the current s.db
  233. mt, err := merkletree.NewMerkleTree(s.db.StorageWithPrefix(PrefixKeyMT), s.MT.MaxLevels())
  234. if err != nil {
  235. return tracerr.Wrap(err)
  236. }
  237. s.MT = mt
  238. }
  239. return nil
  240. }
  241. // GetAccount returns the account for the given Idx
  242. func (s *StateDB) GetAccount(idx common.Idx) (*common.Account, error) {
  243. return GetAccountInTreeDB(s.db.DB(), idx)
  244. }
  245. func accountsIter(db db.Storage, fn func(a *common.Account) (bool, error)) error {
  246. idxDB := db.WithPrefix(PrefixKeyIdx)
  247. if err := idxDB.Iterate(func(k []byte, v []byte) (bool, error) {
  248. idx, err := common.IdxFromBytes(k)
  249. if err != nil {
  250. return false, tracerr.Wrap(err)
  251. }
  252. acc, err := GetAccountInTreeDB(db, idx)
  253. if err != nil {
  254. return false, tracerr.Wrap(err)
  255. }
  256. ok, err := fn(acc)
  257. if err != nil {
  258. return false, tracerr.Wrap(err)
  259. }
  260. return ok, nil
  261. }); err != nil {
  262. return tracerr.Wrap(err)
  263. }
  264. return nil
  265. }
  266. func getAccounts(db db.Storage) ([]common.Account, error) {
  267. accs := []common.Account{}
  268. if err := accountsIter(
  269. db,
  270. func(a *common.Account) (bool, error) {
  271. accs = append(accs, *a)
  272. return true, nil
  273. },
  274. ); err != nil {
  275. return nil, tracerr.Wrap(err)
  276. }
  277. return accs, nil
  278. }
  279. // TestGetAccounts returns all the accounts in the db. Use only in tests.
  280. // Outside tests getting all the accounts is discouraged because it's an
  281. // expensive operation, but if you must do it, use `LastRead()` method to get a
  282. // thread-safe and consistent view of the stateDB.
  283. func (s *StateDB) TestGetAccounts() ([]common.Account, error) {
  284. return getAccounts(s.db.DB())
  285. }
  286. // GetAccountInTreeDB is abstracted from StateDB to be used from StateDB and
  287. // from ExitTree. GetAccount returns the account for the given Idx
  288. func GetAccountInTreeDB(sto db.Storage, idx common.Idx) (*common.Account, error) {
  289. idxBytes, err := idx.Bytes()
  290. if err != nil {
  291. return nil, tracerr.Wrap(err)
  292. }
  293. vBytes, err := sto.Get(append(PrefixKeyIdx, idxBytes[:]...))
  294. if err != nil {
  295. return nil, tracerr.Wrap(err)
  296. }
  297. accBytes, err := sto.Get(append(PrefixKeyAccHash, vBytes...))
  298. if err != nil {
  299. return nil, tracerr.Wrap(err)
  300. }
  301. var b [32 * common.NLeafElems]byte
  302. copy(b[:], accBytes)
  303. account, err := common.AccountFromBytes(b)
  304. if err != nil {
  305. return nil, tracerr.Wrap(err)
  306. }
  307. account.Idx = idx
  308. return account, nil
  309. }
  310. // CreateAccount creates a new Account in the StateDB for the given Idx. If
  311. // StateDB.MT==nil, MerkleTree is not affected, otherwise updates the
  312. // MerkleTree, returning a CircomProcessorProof.
  313. func (s *StateDB) CreateAccount(idx common.Idx, account *common.Account) (*merkletree.CircomProcessorProof, error) {
  314. cpp, err := CreateAccountInTreeDB(s.db.DB(), s.MT, idx, account)
  315. if err != nil {
  316. return cpp, tracerr.Wrap(err)
  317. }
  318. // store idx by EthAddr & BJJ
  319. err = s.setIdxByEthAddrBJJ(idx, account.EthAddr, account.BJJ, account.TokenID)
  320. return cpp, tracerr.Wrap(err)
  321. }
  322. // CreateAccountInTreeDB is abstracted from StateDB to be used from StateDB and
  323. // from ExitTree. Creates a new Account in the StateDB for the given Idx. If
  324. // StateDB.MT==nil, MerkleTree is not affected, otherwise updates the
  325. // MerkleTree, returning a CircomProcessorProof.
  326. func CreateAccountInTreeDB(sto db.Storage, mt *merkletree.MerkleTree, idx common.Idx, account *common.Account) (*merkletree.CircomProcessorProof, error) {
  327. // store at the DB the key: v, and value: leaf.Bytes()
  328. v, err := account.HashValue()
  329. if err != nil {
  330. return nil, tracerr.Wrap(err)
  331. }
  332. accountBytes, err := account.Bytes()
  333. if err != nil {
  334. return nil, tracerr.Wrap(err)
  335. }
  336. // store the Leaf value
  337. tx, err := sto.NewTx()
  338. if err != nil {
  339. return nil, tracerr.Wrap(err)
  340. }
  341. idxBytes, err := idx.Bytes()
  342. if err != nil {
  343. return nil, tracerr.Wrap(err)
  344. }
  345. _, err = tx.Get(append(PrefixKeyIdx, idxBytes[:]...))
  346. if tracerr.Unwrap(err) != db.ErrNotFound {
  347. return nil, tracerr.Wrap(ErrAccountAlreadyExists)
  348. }
  349. err = tx.Put(append(PrefixKeyAccHash, v.Bytes()...), accountBytes[:])
  350. if err != nil {
  351. return nil, tracerr.Wrap(err)
  352. }
  353. err = tx.Put(append(PrefixKeyIdx, idxBytes[:]...), v.Bytes())
  354. if err != nil {
  355. return nil, tracerr.Wrap(err)
  356. }
  357. if err := tx.Commit(); err != nil {
  358. return nil, tracerr.Wrap(err)
  359. }
  360. if mt != nil {
  361. return mt.AddAndGetCircomProof(idx.BigInt(), v)
  362. }
  363. return nil, nil
  364. }
  365. // UpdateAccount updates the Account in the StateDB for the given Idx. If
  366. // StateDB.mt==nil, MerkleTree is not affected, otherwise updates the
  367. // MerkleTree, returning a CircomProcessorProof.
  368. func (s *StateDB) UpdateAccount(idx common.Idx, account *common.Account) (*merkletree.CircomProcessorProof, error) {
  369. return UpdateAccountInTreeDB(s.db.DB(), s.MT, idx, account)
  370. }
  371. // UpdateAccountInTreeDB is abstracted from StateDB to be used from StateDB and
  372. // from ExitTree. Updates the Account in the StateDB for the given Idx. If
  373. // StateDB.mt==nil, MerkleTree is not affected, otherwise updates the
  374. // MerkleTree, returning a CircomProcessorProof.
  375. func UpdateAccountInTreeDB(sto db.Storage, mt *merkletree.MerkleTree, idx common.Idx, account *common.Account) (*merkletree.CircomProcessorProof, error) {
  376. // store at the DB the key: v, and value: account.Bytes()
  377. v, err := account.HashValue()
  378. if err != nil {
  379. return nil, tracerr.Wrap(err)
  380. }
  381. accountBytes, err := account.Bytes()
  382. if err != nil {
  383. return nil, tracerr.Wrap(err)
  384. }
  385. tx, err := sto.NewTx()
  386. if err != nil {
  387. return nil, tracerr.Wrap(err)
  388. }
  389. err = tx.Put(append(PrefixKeyAccHash, v.Bytes()...), accountBytes[:])
  390. if err != nil {
  391. return nil, tracerr.Wrap(err)
  392. }
  393. idxBytes, err := idx.Bytes()
  394. if err != nil {
  395. return nil, tracerr.Wrap(err)
  396. }
  397. err = tx.Put(append(PrefixKeyIdx, idxBytes[:]...), v.Bytes())
  398. if err != nil {
  399. return nil, tracerr.Wrap(err)
  400. }
  401. if err := tx.Commit(); err != nil {
  402. return nil, tracerr.Wrap(err)
  403. }
  404. if mt != nil {
  405. proof, err := mt.Update(idx.BigInt(), v)
  406. return proof, tracerr.Wrap(err)
  407. }
  408. return nil, nil
  409. }
  410. // MTGetProof returns the CircomVerifierProof for a given Idx
  411. func (s *StateDB) MTGetProof(idx common.Idx) (*merkletree.CircomVerifierProof, error) {
  412. if s.MT == nil {
  413. return nil, tracerr.Wrap(ErrStateDBWithoutMT)
  414. }
  415. p, err := s.MT.GenerateSCVerifierProof(idx.BigInt(), s.MT.Root())
  416. if err != nil {
  417. return nil, tracerr.Wrap(err)
  418. }
  419. return p, nil
  420. }
  421. // Close the StateDB
  422. func (s *StateDB) Close() {
  423. s.db.Close()
  424. }
  425. // LocalStateDB represents the local StateDB which allows to make copies from
  426. // the synchronizer StateDB, and is used by the tx-selector and the
  427. // batch-builder. LocalStateDB is an in-memory storage.
  428. type LocalStateDB struct {
  429. *StateDB
  430. synchronizerStateDB *StateDB
  431. }
  432. // NewLocalStateDB returns a new LocalStateDB connected to the given
  433. // synchronizerDB. Checkpoints older than the value defined by `keep` will be
  434. // deleted.
  435. func NewLocalStateDB(cfg Config, synchronizerDB *StateDB) (*LocalStateDB, error) {
  436. cfg.noGapsCheck = true
  437. cfg.NoLast = true
  438. s, err := NewStateDB(cfg)
  439. if err != nil {
  440. return nil, tracerr.Wrap(err)
  441. }
  442. return &LocalStateDB{
  443. s,
  444. synchronizerDB,
  445. }, nil
  446. }
  447. // CheckpointExists returns true if the checkpoint exists
  448. func (l *LocalStateDB) CheckpointExists(batchNum common.BatchNum) (bool, error) {
  449. return l.db.CheckpointExists(batchNum)
  450. }
  451. // Reset performs a reset in the LocaStateDB. If fromSynchronizer is true, it
  452. // gets the state from LocalStateDB.synchronizerStateDB for the given batchNum.
  453. // If fromSynchronizer is false, get the state from LocalStateDB checkpoints.
  454. func (l *LocalStateDB) Reset(batchNum common.BatchNum, fromSynchronizer bool) error {
  455. if fromSynchronizer {
  456. log.Debugw("Making StateDB ResetFromSynchronizer", "batch", batchNum, "type", l.cfg.Type)
  457. if err := l.db.ResetFromSynchronizer(batchNum, l.synchronizerStateDB.db); err != nil {
  458. return tracerr.Wrap(err)
  459. }
  460. // open the MT for the current s.db
  461. if l.MT != nil {
  462. mt, err := merkletree.NewMerkleTree(l.db.StorageWithPrefix(PrefixKeyMT),
  463. l.MT.MaxLevels())
  464. if err != nil {
  465. return tracerr.Wrap(err)
  466. }
  467. l.MT = mt
  468. }
  469. return nil
  470. }
  471. // use checkpoint from LocalStateDB
  472. return l.StateDB.Reset(batchNum)
  473. }