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.

646 lines
17 KiB

  1. package historydb
  2. import (
  3. "database/sql"
  4. "errors"
  5. "fmt"
  6. ethCommon "github.com/ethereum/go-ethereum/common"
  7. "github.com/hermeznetwork/hermez-node/common"
  8. "github.com/hermeznetwork/hermez-node/db"
  9. "github.com/hermeznetwork/hermez-node/log"
  10. "github.com/iden3/go-iden3-crypto/babyjub"
  11. "github.com/jmoiron/sqlx"
  12. //nolint:errcheck // driver for postgres DB
  13. _ "github.com/lib/pq"
  14. "github.com/russross/meddler"
  15. )
  16. // TODO(Edu): Document here how HistoryDB is kept consistent
  17. // HistoryDB persist the historic of the rollup
  18. type HistoryDB struct {
  19. db *sqlx.DB
  20. }
  21. // BlockData contains the information of a Block
  22. type BlockData struct {
  23. Block *common.Block
  24. // Rollup
  25. // L1UserTxs that were submitted in the block
  26. L1UserTxs []common.L1Tx
  27. Batches []BatchData
  28. RegisteredTokens []common.Token
  29. RollupVars *common.RollupVars
  30. // Auction
  31. Bids []common.Bid
  32. Coordinators []common.Coordinator
  33. AuctionVars *common.AuctionVars
  34. WithdrawDelayerVars *common.WithdrawDelayerVars
  35. // TODO: enable when common.WithdrawalDelayerVars is Merged from Synchronizer PR
  36. // WithdrawalDelayerVars *common.WithdrawalDelayerVars
  37. }
  38. // BatchData contains the information of a Batch
  39. type BatchData struct {
  40. // L1UserTxs that were forged in the batch
  41. L1Batch bool // TODO: Remove once Batch.ForgeL1TxsNum is a pointer
  42. L1UserTxs []common.L1Tx
  43. L1CoordinatorTxs []common.L1Tx
  44. L2Txs []common.L2Tx
  45. CreatedAccounts []common.Account
  46. ExitTree []common.ExitInfo
  47. Batch *common.Batch
  48. }
  49. // NewBatchData creates an empty BatchData with the slices initialized.
  50. func NewBatchData() *BatchData {
  51. return &BatchData{
  52. L1Batch: false,
  53. L1UserTxs: make([]common.L1Tx, 0),
  54. L1CoordinatorTxs: make([]common.L1Tx, 0),
  55. L2Txs: make([]common.L2Tx, 0),
  56. CreatedAccounts: make([]common.Account, 0),
  57. ExitTree: make([]common.ExitInfo, 0),
  58. Batch: &common.Batch{},
  59. }
  60. }
  61. // NewHistoryDB initialize the DB
  62. func NewHistoryDB(db *sqlx.DB) *HistoryDB {
  63. return &HistoryDB{db: db}
  64. }
  65. // AddBlock insert a block into the DB
  66. func (hdb *HistoryDB) AddBlock(block *common.Block) error { return hdb.addBlock(hdb.db, block) }
  67. func (hdb *HistoryDB) addBlock(d meddler.DB, block *common.Block) error {
  68. return meddler.Insert(d, "block", block)
  69. }
  70. // AddBlocks inserts blocks into the DB
  71. func (hdb *HistoryDB) AddBlocks(blocks []common.Block) error {
  72. return hdb.addBlocks(hdb.db, blocks)
  73. }
  74. func (hdb *HistoryDB) addBlocks(d meddler.DB, blocks []common.Block) error {
  75. return db.BulkInsert(
  76. d,
  77. `INSERT INTO block (
  78. eth_block_num,
  79. timestamp,
  80. hash
  81. ) VALUES %s;`,
  82. blocks[:],
  83. )
  84. }
  85. // GetBlock retrieve a block from the DB, given a block number
  86. func (hdb *HistoryDB) GetBlock(blockNum int64) (*common.Block, error) {
  87. block := &common.Block{}
  88. err := meddler.QueryRow(
  89. hdb.db, block,
  90. "SELECT * FROM block WHERE eth_block_num = $1;", blockNum,
  91. )
  92. return block, err
  93. }
  94. // GetBlocks retrieve blocks from the DB, given a range of block numbers defined by from and to
  95. func (hdb *HistoryDB) GetBlocks(from, to int64) ([]common.Block, error) {
  96. var blocks []*common.Block
  97. err := meddler.QueryAll(
  98. hdb.db, &blocks,
  99. "SELECT * FROM block WHERE $1 <= eth_block_num AND eth_block_num < $2;",
  100. from, to,
  101. )
  102. return db.SlicePtrsToSlice(blocks).([]common.Block), err
  103. }
  104. // GetLastBlock retrieve the block with the highest block number from the DB
  105. func (hdb *HistoryDB) GetLastBlock() (*common.Block, error) {
  106. block := &common.Block{}
  107. err := meddler.QueryRow(
  108. hdb.db, block, "SELECT * FROM block ORDER BY eth_block_num DESC LIMIT 1;",
  109. )
  110. return block, err
  111. }
  112. // AddBatch insert a Batch into the DB
  113. func (hdb *HistoryDB) AddBatch(batch *common.Batch) error { return hdb.addBatch(hdb.db, batch) }
  114. func (hdb *HistoryDB) addBatch(d meddler.DB, batch *common.Batch) error {
  115. return meddler.Insert(d, "batch", batch)
  116. }
  117. // AddBatches insert Bids into the DB
  118. func (hdb *HistoryDB) AddBatches(batches []common.Batch) error {
  119. return hdb.addBatches(hdb.db, batches)
  120. }
  121. func (hdb *HistoryDB) addBatches(d meddler.DB, batches []common.Batch) error {
  122. return db.BulkInsert(
  123. d,
  124. `INSERT INTO batch (
  125. batch_num,
  126. eth_block_num,
  127. forger_addr,
  128. fees_collected,
  129. state_root,
  130. num_accounts,
  131. exit_root,
  132. forge_l1_txs_num,
  133. slot_num,
  134. total_fees_usd
  135. ) VALUES %s;`,
  136. batches[:],
  137. )
  138. }
  139. // GetBatches retrieve batches from the DB, given a range of batch numbers defined by from and to
  140. func (hdb *HistoryDB) GetBatches(from, to common.BatchNum) ([]common.Batch, error) {
  141. var batches []*common.Batch
  142. err := meddler.QueryAll(
  143. hdb.db, &batches,
  144. "SELECT * FROM batch WHERE $1 <= batch_num AND batch_num < $2;",
  145. from, to,
  146. )
  147. return db.SlicePtrsToSlice(batches).([]common.Batch), err
  148. }
  149. // GetLastBatchNum returns the BatchNum of the latest forged batch
  150. func (hdb *HistoryDB) GetLastBatchNum() (common.BatchNum, error) {
  151. row := hdb.db.QueryRow("SELECT batch_num FROM batch ORDER BY batch_num DESC LIMIT 1;")
  152. var batchNum common.BatchNum
  153. return batchNum, row.Scan(&batchNum)
  154. }
  155. // GetLastL1TxsNum returns the greatest ForgeL1TxsNum in the DB. If there's no
  156. // batch in the DB (nil, nil) is returned.
  157. func (hdb *HistoryDB) GetLastL1TxsNum() (*int64, error) {
  158. row := hdb.db.QueryRow("SELECT MAX(forge_l1_txs_num) FROM batch;")
  159. lastL1TxsNum := new(int64)
  160. return lastL1TxsNum, row.Scan(&lastL1TxsNum)
  161. }
  162. // Reorg deletes all the information that was added into the DB after the
  163. // lastValidBlock. If lastValidBlock is negative, all block information is
  164. // deleted.
  165. func (hdb *HistoryDB) Reorg(lastValidBlock int64) error {
  166. var err error
  167. if lastValidBlock < 0 {
  168. _, err = hdb.db.Exec("DELETE FROM block;")
  169. } else {
  170. _, err = hdb.db.Exec("DELETE FROM block WHERE eth_block_num > $1;", lastValidBlock)
  171. }
  172. return err
  173. }
  174. // SyncPoD stores all the data that can be changed / added on a block in the PoD SC
  175. func (hdb *HistoryDB) SyncPoD(
  176. blockNum uint64,
  177. bids []common.Bid,
  178. coordinators []common.Coordinator,
  179. vars *common.AuctionVars,
  180. ) error {
  181. return nil
  182. }
  183. // AddBids insert Bids into the DB
  184. func (hdb *HistoryDB) AddBids(bids []common.Bid) error { return hdb.addBids(hdb.db, bids) }
  185. func (hdb *HistoryDB) addBids(d meddler.DB, bids []common.Bid) error {
  186. // TODO: check the coordinator info
  187. return db.BulkInsert(
  188. d,
  189. "INSERT INTO bid (slot_num, bid_value, eth_block_num, bidder_addr) VALUES %s;",
  190. bids[:],
  191. )
  192. }
  193. // GetBids return the bids
  194. func (hdb *HistoryDB) GetBids() ([]common.Bid, error) {
  195. var bids []*common.Bid
  196. err := meddler.QueryAll(
  197. hdb.db, &bids,
  198. "SELECT * FROM bid;",
  199. )
  200. return db.SlicePtrsToSlice(bids).([]common.Bid), err
  201. }
  202. // AddCoordinators insert Coordinators into the DB
  203. func (hdb *HistoryDB) AddCoordinators(coordinators []common.Coordinator) error {
  204. return hdb.addCoordinators(hdb.db, coordinators)
  205. }
  206. func (hdb *HistoryDB) addCoordinators(d meddler.DB, coordinators []common.Coordinator) error {
  207. return db.BulkInsert(
  208. d,
  209. "INSERT INTO coordinator (bidder_addr, forger_addr, eth_block_num, url) VALUES %s;",
  210. coordinators[:],
  211. )
  212. }
  213. // AddExitTree insert Exit tree into the DB
  214. func (hdb *HistoryDB) AddExitTree(exitTree []common.ExitInfo) error {
  215. return hdb.addExitTree(hdb.db, exitTree)
  216. }
  217. func (hdb *HistoryDB) addExitTree(d meddler.DB, exitTree []common.ExitInfo) error {
  218. return db.BulkInsert(
  219. d,
  220. "INSERT INTO exit_tree (batch_num, account_idx, merkle_proof, balance, "+
  221. "instant_withdrawn, delayed_withdraw_request, delayed_withdrawn) VALUES %s;",
  222. exitTree[:],
  223. )
  224. }
  225. // AddToken insert a token into the DB
  226. func (hdb *HistoryDB) AddToken(token *common.Token) error {
  227. return meddler.Insert(hdb.db, "token", token)
  228. }
  229. // AddTokens insert tokens into the DB
  230. func (hdb *HistoryDB) AddTokens(tokens []common.Token) error { return hdb.addTokens(hdb.db, tokens) }
  231. func (hdb *HistoryDB) addTokens(d meddler.DB, tokens []common.Token) error {
  232. return db.BulkInsert(
  233. d,
  234. `INSERT INTO token (
  235. token_id,
  236. eth_block_num,
  237. eth_addr,
  238. name,
  239. symbol,
  240. decimals,
  241. usd,
  242. usd_update
  243. ) VALUES %s;`,
  244. tokens[:],
  245. )
  246. }
  247. // UpdateTokenValue updates the USD value of a token
  248. func (hdb *HistoryDB) UpdateTokenValue(tokenSymbol string, value float64) error {
  249. _, err := hdb.db.Exec(
  250. "UPDATE token SET usd = $1 WHERE symbol = $2;",
  251. value, tokenSymbol,
  252. )
  253. return err
  254. }
  255. // GetTokens returns a list of tokens from the DB
  256. func (hdb *HistoryDB) GetTokens() ([]common.Token, error) {
  257. var tokens []*common.Token
  258. err := meddler.QueryAll(
  259. hdb.db, &tokens,
  260. "SELECT * FROM token ORDER BY token_id;",
  261. )
  262. return db.SlicePtrsToSlice(tokens).([]common.Token), err
  263. }
  264. // GetTokenSymbols returns all the token symbols from the DB
  265. func (hdb *HistoryDB) GetTokenSymbols() ([]string, error) {
  266. var tokenSymbols []string
  267. rows, err := hdb.db.Query("SELECT symbol FROM token;")
  268. if err != nil {
  269. return nil, err
  270. }
  271. sym := new(string)
  272. for rows.Next() {
  273. err = rows.Scan(sym)
  274. if err != nil {
  275. return nil, err
  276. }
  277. tokenSymbols = append(tokenSymbols, *sym)
  278. }
  279. return tokenSymbols, nil
  280. }
  281. // AddAccounts insert accounts into the DB
  282. func (hdb *HistoryDB) AddAccounts(accounts []common.Account) error {
  283. return hdb.addAccounts(hdb.db, accounts)
  284. }
  285. func (hdb *HistoryDB) addAccounts(d meddler.DB, accounts []common.Account) error {
  286. return db.BulkInsert(
  287. d,
  288. `INSERT INTO account (
  289. idx,
  290. token_id,
  291. batch_num,
  292. bjj,
  293. eth_addr
  294. ) VALUES %s;`,
  295. accounts[:],
  296. )
  297. }
  298. // GetAccounts returns a list of accounts from the DB
  299. func (hdb *HistoryDB) GetAccounts() ([]common.Account, error) {
  300. var accs []*common.Account
  301. err := meddler.QueryAll(
  302. hdb.db, &accs,
  303. "SELECT * FROM account ORDER BY idx;",
  304. )
  305. return db.SlicePtrsToSlice(accs).([]common.Account), err
  306. }
  307. // AddL1Txs inserts L1 txs to the DB. USD and LoadAmountUSD will be set automatically before storing the tx.
  308. // If the tx is originated by a coordinator, BatchNum must be provided. If it's originated by a user,
  309. // BatchNum should be null, and the value will be setted by a trigger when a batch forges the tx.
  310. func (hdb *HistoryDB) AddL1Txs(l1txs []common.L1Tx) error { return hdb.addL1Txs(hdb.db, l1txs) }
  311. // addL1Txs inserts L1 txs to the DB. USD and LoadAmountUSD will be set automatically before storing the tx.
  312. // If the tx is originated by a coordinator, BatchNum must be provided. If it's originated by a user,
  313. // BatchNum should be null, and the value will be setted by a trigger when a batch forges the tx.
  314. func (hdb *HistoryDB) addL1Txs(d meddler.DB, l1txs []common.L1Tx) error {
  315. txs := []common.Tx{}
  316. for _, tx := range l1txs {
  317. txs = append(txs, *(tx.Tx()))
  318. }
  319. return hdb.addTxs(d, txs)
  320. }
  321. // AddL2Txs inserts L2 txs to the DB. USD and FeeUSD will be set automatically before storing the tx.
  322. func (hdb *HistoryDB) AddL2Txs(l2txs []common.L2Tx) error { return hdb.addL2Txs(hdb.db, l2txs) }
  323. // addL2Txs inserts L2 txs to the DB. USD and FeeUSD will be set automatically before storing the tx.
  324. func (hdb *HistoryDB) addL2Txs(d meddler.DB, l2txs []common.L2Tx) error {
  325. txs := []common.Tx{}
  326. for _, tx := range l2txs {
  327. txs = append(txs, *(tx.Tx()))
  328. }
  329. return hdb.addTxs(d, txs)
  330. }
  331. func (hdb *HistoryDB) addTxs(d meddler.DB, txs []common.Tx) error {
  332. return db.BulkInsert(
  333. d,
  334. `INSERT INTO tx (
  335. is_l1,
  336. id,
  337. type,
  338. position,
  339. from_idx,
  340. to_idx,
  341. amount,
  342. amount_f,
  343. token_id,
  344. amount_usd,
  345. batch_num,
  346. eth_block_num,
  347. to_forge_l1_txs_num,
  348. user_origin,
  349. from_eth_addr,
  350. from_bjj,
  351. load_amount,
  352. load_amount_f,
  353. load_amount_usd,
  354. fee,
  355. fee_usd,
  356. nonce
  357. ) VALUES %s;`,
  358. txs[:],
  359. )
  360. }
  361. // GetTxs returns a list of txs from the DB
  362. func (hdb *HistoryDB) GetTxs() ([]common.Tx, error) {
  363. var txs []*common.Tx
  364. err := meddler.QueryAll(
  365. hdb.db, &txs,
  366. `SELECT * FROM tx
  367. ORDER BY (batch_num, position) ASC`,
  368. )
  369. return db.SlicePtrsToSlice(txs).([]common.Tx), err
  370. }
  371. // GetHistoryTxs returns a list of txs from the DB using the HistoryTx struct
  372. func (hdb *HistoryDB) GetHistoryTxs(
  373. ethAddr *ethCommon.Address, bjj *babyjub.PublicKey,
  374. tokenID, idx, batchNum *uint, txType *common.TxType,
  375. offset, limit *uint, last bool,
  376. ) ([]HistoryTx, int, error) {
  377. if ethAddr != nil && bjj != nil {
  378. return nil, 0, errors.New("ethAddr and bjj are incompatible")
  379. }
  380. var query string
  381. var args []interface{}
  382. queryStr := `SELECT tx.*, token.token_id, token.eth_block_num AS token_block,
  383. token.eth_addr, token.name, token.symbol, token.decimals, token.usd,
  384. token.usd_update, block.timestamp, count(*) OVER() AS total_items
  385. FROM tx
  386. INNER JOIN token ON tx.token_id = token.token_id
  387. INNER JOIN block ON tx.eth_block_num = block.eth_block_num `
  388. // Apply filters
  389. nextIsAnd := false
  390. // ethAddr filter
  391. if ethAddr != nil {
  392. queryStr = `WITH acc AS
  393. (select idx from account where eth_addr = ?) ` + queryStr
  394. queryStr += ", acc WHERE (tx.from_idx IN(acc.idx) OR tx.to_idx IN(acc.idx)) "
  395. nextIsAnd = true
  396. args = append(args, ethAddr)
  397. } else if bjj != nil { // bjj filter
  398. queryStr = `WITH acc AS
  399. (select idx from account where bjj = ?) ` + queryStr
  400. queryStr += ", acc WHERE (tx.from_idx IN(acc.idx) OR tx.to_idx IN(acc.idx)) "
  401. nextIsAnd = true
  402. args = append(args, bjj)
  403. }
  404. // tokenID filter
  405. if tokenID != nil {
  406. if nextIsAnd {
  407. queryStr += "AND "
  408. } else {
  409. queryStr += "WHERE "
  410. }
  411. queryStr += "tx.token_id = ? "
  412. args = append(args, tokenID)
  413. nextIsAnd = true
  414. }
  415. // idx filter
  416. if idx != nil {
  417. if nextIsAnd {
  418. queryStr += "AND "
  419. } else {
  420. queryStr += "WHERE "
  421. }
  422. queryStr += "(tx.from_idx = ? OR tx.to_idx = ?) "
  423. args = append(args, idx, idx)
  424. nextIsAnd = true
  425. }
  426. // batchNum filter
  427. if batchNum != nil {
  428. if nextIsAnd {
  429. queryStr += "AND "
  430. } else {
  431. queryStr += "WHERE "
  432. }
  433. queryStr += "tx.batch_num = ? "
  434. args = append(args, batchNum)
  435. nextIsAnd = true
  436. }
  437. // txType filter
  438. if txType != nil {
  439. if nextIsAnd {
  440. queryStr += "AND "
  441. } else {
  442. queryStr += "WHERE "
  443. }
  444. queryStr += "tx.type = ? "
  445. args = append(args, txType)
  446. // nextIsAnd = true
  447. }
  448. // pagination
  449. if last {
  450. queryStr += "ORDER BY (batch_num, position) DESC NULLS FIRST "
  451. } else {
  452. queryStr += "ORDER BY (batch_num, position) ASC NULLS LAST "
  453. queryStr += fmt.Sprintf("OFFSET %d ", *offset)
  454. }
  455. queryStr += fmt.Sprintf("LIMIT %d;", *limit)
  456. query = hdb.db.Rebind(queryStr)
  457. // log.Debug(query)
  458. txsPtrs := []*HistoryTx{}
  459. if err := meddler.QueryAll(hdb.db, &txsPtrs, query, args...); err != nil {
  460. return nil, 0, err
  461. }
  462. txs := db.SlicePtrsToSlice(txsPtrs).([]HistoryTx)
  463. if len(txs) == 0 {
  464. return nil, 0, sql.ErrNoRows
  465. } else if last {
  466. tmp := []HistoryTx{}
  467. for i := len(txs) - 1; i >= 0; i-- {
  468. tmp = append(tmp, txs[i])
  469. }
  470. txs = tmp
  471. }
  472. return txs, txs[0].TotalItems, nil
  473. }
  474. // GetTx returns a tx from the DB
  475. func (hdb *HistoryDB) GetTx(txID common.TxID) (*common.Tx, error) {
  476. tx := new(common.Tx)
  477. return tx, meddler.QueryRow(
  478. hdb.db, tx,
  479. "SELECT * FROM tx WHERE id = $1;",
  480. txID,
  481. )
  482. }
  483. // // GetL1UserTxs gets L1 User Txs to be forged in a batch that will create an account
  484. // // TODO: This is currently not used. Figure out if it should be used somewhere or removed.
  485. // func (hdb *HistoryDB) GetL1UserTxs(toForgeL1TxsNum int64) ([]*common.Tx, error) {
  486. // var txs []*common.Tx
  487. // err := meddler.QueryAll(
  488. // hdb.db, &txs,
  489. // "SELECT * FROM tx WHERE to_forge_l1_txs_num = $1 AND is_l1 = TRUE AND user_origin = TRUE;",
  490. // toForgeL1TxsNum,
  491. // )
  492. // return txs, err
  493. // }
  494. // TODO: Think about chaning all the queries that return a last value, to queries that return the next valid value.
  495. // GetLastTxsPosition for a given to_forge_l1_txs_num
  496. func (hdb *HistoryDB) GetLastTxsPosition(toForgeL1TxsNum int64) (int, error) {
  497. row := hdb.db.QueryRow("SELECT MAX(position) FROM tx WHERE to_forge_l1_txs_num = $1;", toForgeL1TxsNum)
  498. var lastL1TxsPosition int
  499. return lastL1TxsPosition, row.Scan(&lastL1TxsPosition)
  500. }
  501. // AddBlockSCData stores all the information of a block retrieved by the Synchronizer
  502. func (hdb *HistoryDB) AddBlockSCData(blockData *BlockData) (err error) {
  503. txn, err := hdb.db.Begin()
  504. if err != nil {
  505. return err
  506. }
  507. defer func() {
  508. if err != nil {
  509. errRollback := txn.Rollback()
  510. if errRollback != nil {
  511. log.Errorw("Rollback", "err", errRollback)
  512. }
  513. }
  514. }()
  515. // Add block
  516. err = hdb.addBlock(txn, blockData.Block)
  517. if err != nil {
  518. return err
  519. }
  520. // Add Coordinators
  521. if len(blockData.Coordinators) > 0 {
  522. err = hdb.addCoordinators(txn, blockData.Coordinators)
  523. if err != nil {
  524. return err
  525. }
  526. }
  527. // Add Bids
  528. if len(blockData.Bids) > 0 {
  529. err = hdb.addBids(txn, blockData.Bids)
  530. if err != nil {
  531. return err
  532. }
  533. }
  534. // Add Tokens
  535. if len(blockData.RegisteredTokens) > 0 {
  536. err = hdb.addTokens(txn, blockData.RegisteredTokens)
  537. if err != nil {
  538. return err
  539. }
  540. }
  541. // Add l1 Txs
  542. if len(blockData.L1UserTxs) > 0 {
  543. err = hdb.addL1Txs(txn, blockData.L1UserTxs)
  544. if err != nil {
  545. return err
  546. }
  547. }
  548. // Add Batches
  549. for _, batch := range blockData.Batches {
  550. // Add Batch: this will trigger an update on the DB
  551. // that will set the batch num of forged L1 txs in this batch
  552. err = hdb.addBatch(txn, batch.Batch)
  553. if err != nil {
  554. return err
  555. }
  556. // Add unforged l1 Txs
  557. if batch.L1Batch {
  558. if len(batch.L1CoordinatorTxs) > 0 {
  559. err = hdb.addL1Txs(txn, batch.L1CoordinatorTxs)
  560. if err != nil {
  561. return err
  562. }
  563. }
  564. }
  565. // Add l2 Txs
  566. if len(batch.L2Txs) > 0 {
  567. err = hdb.addL2Txs(txn, batch.L2Txs)
  568. if err != nil {
  569. return err
  570. }
  571. }
  572. // Add accounts
  573. if len(batch.CreatedAccounts) > 0 {
  574. err = hdb.addAccounts(txn, batch.CreatedAccounts)
  575. if err != nil {
  576. return err
  577. }
  578. }
  579. // Add exit tree
  580. if len(batch.ExitTree) > 0 {
  581. err = hdb.addExitTree(txn, batch.ExitTree)
  582. if err != nil {
  583. return err
  584. }
  585. }
  586. // TODO: INSERT CONTRACTS VARS
  587. }
  588. return txn.Commit()
  589. }