Browse Source

Update synchronizer and DB with last contracts updates

- API
	- When updating network info, handle cases where no batches exists and
	  where no forgers exists
- cli/node
	- Update `cfg.buidler.toml` config file to a working version
- common
	- Add new smart contract structs and extend some existing ones to
	  reflect updates regarding events from the smart contracts
- SQL
	- Add new tables and extend existing ones to reflect updates regarding
	  events from the smart contracts
- db/historydb
	- Add functions to insert new smart contract events data
	- Fix unclosed rows that led to inconsistent sql driver state (replace
	  NamedQuery by NamedExec).  This fixes the error:
	  `pq: unexpected Parse response 'C'`
- db/l2db
	- Close rows after usage
- eth
	- In Rollup event, introduce a new UpdateBucketsParameter when there's a
	  SafeMode event, with `SafeMode = true`
- synchronizer
	- synchronize new events
	- avoid calling `auction.CanForge` before the genesisBlock to avoid
	  getting a revert.
feature/sql-semaphore1
Eduard S 3 years ago
parent
commit
20b8d0561f
16 changed files with 543 additions and 198 deletions
  1. +17
    -8
      api/state.go
  2. +37
    -40
      cli/node/cfg.buidler.toml
  3. +10
    -7
      common/block.go
  4. +30
    -12
      common/ethrollup.go
  5. +15
    -1
      common/ethwdelayer.go
  6. +2
    -2
      common/l1tx.go
  7. +4
    -4
      coordinator/coordinator.go
  8. +5
    -5
      coordinator/purger.go
  9. +186
    -74
      db/historydb/historydb.go
  10. +83
    -1
      db/historydb/historydb_test.go
  11. +12
    -11
      db/l2db/l2db.go
  12. +29
    -1
      db/migrations/0001.sql
  13. +9
    -2
      db/utils.go
  14. +15
    -0
      eth/rollup.go
  15. +78
    -23
      synchronizer/synchronizer.go
  16. +11
    -7
      test/ethclient.go

+ 17
- 8
api/state.go

@ -1,6 +1,7 @@
package api
import (
"database/sql"
"net/http"
"time"
@ -13,11 +14,11 @@ import (
// Network define status of the network
type Network struct {
LastEthBlock int64 `json:"lastEthereumBlock"`
LastSyncBlock int64 `json:"lastSynchedBlock"`
LastBatch historydb.BatchAPI `json:"lastBatch"`
CurrentSlot int64 `json:"currentSlot"`
NextForgers []NextForger `json:"nextForgers"`
LastEthBlock int64 `json:"lastEthereumBlock"`
LastSyncBlock int64 `json:"lastSynchedBlock"`
LastBatch *historydb.BatchAPI `json:"lastBatch"`
CurrentSlot int64 `json:"currentSlot"`
NextForgers []NextForger `json:"nextForgers"`
}
// NextForger is a representation of the information of a coordinator and the period will forge
@ -89,18 +90,22 @@ func (a *API) UpdateNetworkInfo(
lastBatchNum common.BatchNum, currentSlot int64,
) error {
lastBatch, err := a.h.GetBatchAPI(lastBatchNum)
if err != nil {
if tracerr.Unwrap(err) == sql.ErrNoRows {
lastBatch = nil
} else if err != nil {
return tracerr.Wrap(err)
}
lastClosedSlot := currentSlot + int64(a.status.Auction.ClosedAuctionSlots)
nextForgers, err := a.getNextForgers(lastSyncBlock, currentSlot, lastClosedSlot)
if err != nil {
if tracerr.Unwrap(err) == sql.ErrNoRows {
nextForgers = nil
} else if err != nil {
return tracerr.Wrap(err)
}
a.status.Lock()
a.status.Network.LastSyncBlock = lastSyncBlock.Num
a.status.Network.LastEthBlock = lastEthBlock.Num
a.status.Network.LastBatch = *lastBatch
a.status.Network.LastBatch = lastBatch
a.status.Network.CurrentSlot = currentSlot
a.status.Network.NextForgers = nextForgers
a.status.Unlock()
@ -157,6 +162,10 @@ func (a *API) getNextForgers(lastBlock common.Block, currentSlot, lastClosedSlot
// UpdateMetrics update Status.Metrics information
func (a *API) UpdateMetrics() error {
a.status.RLock()
if a.status.Network.LastBatch == nil {
a.status.RUnlock()
return nil
}
batchNum := a.status.Network.LastBatch.BatchNum
a.status.RUnlock()
metrics, err := a.h.GetMetrics(batchNum)

+ 37
- 40
cli/node/cfg.buidler.toml

@ -25,20 +25,21 @@ SyncLoopInterval = "1s"
StatsRefreshPeriod = "1s"
[Synchronizer.StartBlockNum]
Rollup = 1
Auction = 1
WDelayer = 1
Rollup = 6
Auction = 3
WDelayer = 7
[SmartContracts]
Rollup = "0x6F4e99522F4eB37e0B73D0C0373147893EF12fD5"
Auction = "0x5E0816F0f8bC560cB2B9e9C87187BeCac8c2021F"
WDelayer = "0x5D94e3e7aeC542aB0F9129B9a7BAdeb5B3Ca0f77"
TokenHEZ = "0x2b7dEe2CF60484325716A1c6A193519c8c3b19F3"
Rollup = "0x8EEaea23686c319133a7cC110b840d1591d9AeE0"
Auction = "0x317113D2593e3efF1FfAE0ba2fF7A61861Df7ae5"
WDelayer = "0x5E0816F0f8bC560cB2B9e9C87187BeCac8c2021F"
TokenHEZ = "0x5D94e3e7aeC542aB0F9129B9a7BAdeb5B3Ca0f77"
TokenHEZName = "Hermez Network Token"
[Synchronizer.InitialVariables.Auction]
DonationAddress = "0x0000000000000000000000000000000000000001"
BootCoordinator = "0xb4124cEB3451635DAcedd11767f004d8a28c6eE7"
BootCoordinatorURL = "https://boot.coordinator.io"
DefaultSlotSetBid = [
"10000000000000000000",
"10000000000000000000",
@ -47,6 +48,7 @@ TokenHEZName = "Hermez Network Token"
"10000000000000000000",
"10000000000000000000",
]
DefaultSlotSetBidSlotNum = 0
ClosedAuctionSlots = 2
OpenAuctionSlots = 4320
AllocationRatio = [4000, 4000, 2000]
@ -55,9 +57,8 @@ TokenHEZName = "Hermez Network Token"
[Synchronizer.InitialVariables.WDelayer]
# HermezRollupAddress =
HermezGovernanceDAOAddress = "0x0000000000000000000000000000000000000001"
WhiteHackGroupAddress = "0x0000000000000000000000000000000000000001"
HermezKeeperAddress = "0x0000000000000000000000000000000000000001"
HermezGovernanceAddress = "0x0000000000000000000000000000000000000001"
EmergencyCouncilAddress = "0x0000000000000000000000000000000000000001"
WithdrawalDelay = 60
EmergencyModeStartingTime = 0
EmergencyMode = false
@ -66,33 +67,29 @@ TokenHEZName = "Hermez Network Token"
FeeAddToken = "10"
ForgeL1L2BatchTimeout = 10
WithdrawalDelay = 1209600 # 60 * 60 * 24 * 7 * 2
# [[Synchronizer.InitialVariables.Rollup.Buckets]]
# CeilUSD = 0
# BlockStamp = 0
# Withdrawals = 0
# BlockWithdrawalRate = 0
# MaxWithdrawals = 0
# [[Synchronizer.InitialVariables.Rollup.Buckets]]
# CeilUSD = 0
# BlockStamp = 0
# Withdrawals = 0
# BlockWithdrawalRate = 0
# MaxWithdrawals = 0
# [[Synchronizer.InitialVariables.Rollup.Buckets]]
# CeilUSD = 0
# BlockStamp = 0
# Withdrawals = 0
# BlockWithdrawalRate = 0
# MaxWithdrawals = 0
# [[Synchronizer.InitialVariables.Rollup.Buckets]]
# CeilUSD = 0
# BlockStamp = 0
# Withdrawals = 0
# BlockWithdrawalRate = 0
# MaxWithdrawals = 0
# [[Synchronizer.InitialVariables.Rollup.Buckets]]
# CeilUSD = 0
# BlockStamp = 0
# Withdrawals = 0
# BlockWithdrawalRate = 0
# MaxWithdrawals = 0
SafeMode = false
[[Synchronizer.InitialVariables.Rollup.Buckets]]
CeilUSD = 0
Withdrawals = 0
BlockWithdrawalRate = 0
MaxWithdrawals = 0
[[Synchronizer.InitialVariables.Rollup.Buckets]]
CeilUSD = 0
Withdrawals = 0
BlockWithdrawalRate = 0
MaxWithdrawals = 0
[[Synchronizer.InitialVariables.Rollup.Buckets]]
CeilUSD = 0
Withdrawals = 0
BlockWithdrawalRate = 0
MaxWithdrawals = 0
[[Synchronizer.InitialVariables.Rollup.Buckets]]
CeilUSD = 0
Withdrawals = 0
BlockWithdrawalRate = 0
MaxWithdrawals = 0
[[Synchronizer.InitialVariables.Rollup.Buckets]]
CeilUSD = 0
Withdrawals = 0
BlockWithdrawalRate = 0
MaxWithdrawals = 0

+ 10
- 7
common/block.go

@ -18,11 +18,13 @@ type Block struct {
// RollupData contains information returned by the Rollup smart contract
type RollupData struct {
// L1UserTxs that were submitted in the block
L1UserTxs []L1Tx
Batches []BatchData
AddedTokens []Token
Withdrawals []WithdrawInfo
Vars *RollupVariables
L1UserTxs []L1Tx
Batches []BatchData
AddedTokens []Token
Withdrawals []WithdrawInfo
UpdateBucketWithdraw []BucketUpdate
TokenExchanges []TokenExchange
Vars *RollupVariables
}
// NewRollupData creates an empty RollupData with the slices initialized.
@ -66,8 +68,9 @@ type WDelayerData struct {
Vars *WDelayerVariables
Deposits []WDelayerTransfer
// We use an array because there can be multiple deposits in a single eth transaction
DepositsByTxHash map[ethCommon.Hash][]*WDelayerTransfer
Withdrawals []WDelayerTransfer
DepositsByTxHash map[ethCommon.Hash][]*WDelayerTransfer
Withdrawals []WDelayerTransfer
EscapeHatchWithdrawals []WDelayerEscapeHatchWithdrawal
}
// NewWDelayerData creates an empty WDelayerData.

+ 30
- 12
common/ethrollup.go

@ -146,22 +146,40 @@ type RollupConstants struct {
WithdrawDelayerContract ethCommon.Address `json:"withdrawDelayerContract"`
}
// Bucket are the variables of each Bucket of Rollup Smart Contract
type Bucket struct {
CeilUSD uint64 `json:"ceilUSD"`
BlockStamp uint64 `json:"blockStamp"`
Withdrawals uint64 `json:"withdrawals"`
BlockWithdrawalRate uint64 `json:"blockWithdrawalRate"`
MaxWithdrawals uint64 `json:"maxWithdrawals"`
// BucketParams are the parameter variables of each Bucket of Rollup Smart
// Contract
type BucketParams struct {
CeilUSD *big.Int `json:"ceilUSD"`
Withdrawals *big.Int `json:"withdrawals"`
BlockWithdrawalRate *big.Int `json:"blockWithdrawalRate"`
MaxWithdrawals *big.Int `json:"maxWithdrawals"`
}
// BucketUpdate are the bucket updates (tracking the withdrawals value changes)
// in Rollup Smart Contract
type BucketUpdate struct {
EthBlockNum int64 `json:"ethereumBlockNum" meddler:"eth_block_num"`
NumBucket int `json:"numBucket" meddler:"num_bucket"`
BlockStamp int64 `json:"blockStamp" meddler:"block_stamp"`
Withdrawals *big.Int `json:"withdrawals" meddler:"withdrawals,bigint"`
}
// TokenExchange are the exchange value for tokens registered in the Rollup
// Smart Contract
type TokenExchange struct {
EthBlockNum int64 `json:"ethereumBlockNum" meddler:"eth_block_num"`
Address ethCommon.Address `json:"address" meddler:"eth_addr"`
ValueUSD int64 `json:"valueUSD" meddler:"value_usd"`
}
// RollupVariables are the variables of the Rollup Smart Contract
type RollupVariables struct {
EthBlockNum int64 `json:"ethereumBlockNum" meddler:"eth_block_num"`
FeeAddToken *big.Int `json:"feeAddToken" meddler:"fee_add_token,bigint" validate:"required"`
ForgeL1L2BatchTimeout int64 `json:"forgeL1L2BatchTimeout" meddler:"forge_l1_timeout" validate:"required"`
WithdrawalDelay uint64 `json:"withdrawalDelay" meddler:"withdrawal_delay" validate:"required"`
Buckets [RollupConstNumBuckets]Bucket `json:"buckets" meddler:"buckets,json"`
EthBlockNum int64 `json:"ethereumBlockNum" meddler:"eth_block_num"`
FeeAddToken *big.Int `json:"feeAddToken" meddler:"fee_add_token,bigint" validate:"required"`
ForgeL1L2BatchTimeout int64 `json:"forgeL1L2BatchTimeout" meddler:"forge_l1_timeout" validate:"required"`
WithdrawalDelay uint64 `json:"withdrawalDelay" meddler:"withdrawal_delay" validate:"required"`
Buckets [RollupConstNumBuckets]BucketParams `json:"buckets" meddler:"buckets,json"`
SafeMode bool `json:"safeMode" meddler:"safe_mode"`
}
// Copy returns a deep copy of the Variables

+ 15
- 1
common/ethwdelayer.go

@ -1,6 +1,10 @@
package common
import ethCommon "github.com/ethereum/go-ethereum/common"
import (
"math/big"
ethCommon "github.com/ethereum/go-ethereum/common"
)
// WDelayerConstants are the constants of the Withdrawal Delayer Smart Contract
type WDelayerConstants struct {
@ -12,6 +16,16 @@ type WDelayerConstants struct {
HermezRollup ethCommon.Address `json:"hermezRollup"`
}
// WDelayerEscapeHatchWithdrawal is an escape hatch withdrawal of the
// Withdrawal Delayer Smart Contract
type WDelayerEscapeHatchWithdrawal struct {
EthBlockNum int64 `json:"ethereumBlockNum" meddler:"eth_block_num"`
Who ethCommon.Address `json:"who" meddler:"who_addr"`
To ethCommon.Address `json:"to" meddler:"to_addr"`
TokenAddr ethCommon.Address `json:"tokenAddr" meddler:"token_addr"`
Amount *big.Int `json:"amount" meddler:"amount,bigint"`
}
// WDelayerVariables are the variables of the Withdrawal Delayer Smart Contract
type WDelayerVariables struct {
EthBlockNum int64 `json:"ethereumBlockNum" meddler:"eth_block_num"`

+ 2
- 2
common/l1tx.go

@ -228,12 +228,12 @@ func L1TxFromDataAvailability(b []byte, nLevels uint32) (*L1Tx, error) {
l1tx := L1Tx{}
fromIdx, err := IdxFromBytes(ethCommon.LeftPadBytes(fromIdxBytes, 6))
if err != nil {
return nil, err
return nil, tracerr.Wrap(err)
}
l1tx.FromIdx = fromIdx
toIdx, err := IdxFromBytes(ethCommon.LeftPadBytes(toIdxBytes, 6))
if err != nil {
return nil, err
return nil, tracerr.Wrap(err)
}
l1tx.ToIdx = toIdx
l1tx.EffectiveAmount = Float16FromBytes(amountBytes).BigInt()

+ 4
- 4
coordinator/coordinator.go

@ -244,17 +244,17 @@ func (c *Coordinator) handleMsgSyncBlock(ctx context.Context, msg *MsgSyncBlock)
// }
if c.purger.CanInvalidate(stats.Sync.LastBlock.Num, stats.Sync.LastBatch) {
if err := c.txSelector.Reset(common.BatchNum(stats.Sync.LastBatch)); err != nil {
return err
return tracerr.Wrap(err)
}
}
_, err := c.purger.InvalidateMaybe(c.l2DB, c.txSelector.LocalAccountsDB(),
stats.Sync.LastBlock.Num, stats.Sync.LastBatch)
if err != nil {
return err
return tracerr.Wrap(err)
}
_, err = c.purger.PurgeMaybe(c.l2DB, stats.Sync.LastBlock.Num, stats.Sync.LastBatch)
if err != nil {
return err
return tracerr.Wrap(err)
}
}
return nil
@ -723,7 +723,7 @@ func (p *Pipeline) forgeSendServerProof(ctx context.Context, batchNum common.Bat
_, err := p.purger.InvalidateMaybe(p.l2DB, p.txSelector.LocalAccountsDB(),
p.stats.Sync.LastBlock.Num, int64(batchNum))
if err != nil {
return nil, err
return nil, tracerr.Wrap(err)
}
_, err = p.purger.PurgeMaybe(p.l2DB, p.stats.Sync.LastBlock.Num, int64(batchNum))
if err != nil {

+ 5
- 5
coordinator/purger.go

@ -78,7 +78,7 @@ func (p *Purger) InvalidateMaybe(l2DB *l2db.L2DB, stateDB *statedb.LocalStateDB,
p.lastInvalidateBatch = batchNum
log.Debugw("Purger: invalidating l2txs in pool", "block", blockNum, "batch", batchNum)
err := poolMarkInvalidOldNonces(l2DB, stateDB, common.BatchNum(batchNum))
return true, err
return true, tracerr.Wrap(err)
}
//nolint:unused,deadcode
@ -129,20 +129,20 @@ func poolMarkInvalidOldNonces(l2DB *l2db.L2DB, stateDB *statedb.LocalStateDB,
batchNum common.BatchNum) error {
idxs, err := l2DB.GetPendingUniqueFromIdxs()
if err != nil {
return err
return tracerr.Wrap(err)
}
idxsNonce := make([]common.IdxNonce, len(idxs))
lastIdx, err := stateDB.GetIdx()
if err != nil {
return err
return tracerr.Wrap(err)
}
for i, idx := range idxs {
acc, err := stateDB.GetAccount(idx)
if err != nil {
if tracerr.Unwrap(err) != db.ErrNotFound {
return err
return tracerr.Wrap(err)
} else if idx <= lastIdx {
return fmt.Errorf("account with idx %v not found: %w", idx, err)
return tracerr.Wrap(fmt.Errorf("account with idx %v not found: %w", idx, err))
}
}
idxsNonce[i].Idx = idx

+ 186
- 74
db/historydb/historydb.go

@ -47,16 +47,16 @@ func (hdb *HistoryDB) DB() *sqlx.DB {
// AddBlock insert a block into the DB
func (hdb *HistoryDB) AddBlock(block *common.Block) error { return hdb.addBlock(hdb.db, block) }
func (hdb *HistoryDB) addBlock(d meddler.DB, block *common.Block) error {
return meddler.Insert(d, "block", block)
return tracerr.Wrap(meddler.Insert(d, "block", block))
}
// AddBlocks inserts blocks into the DB
func (hdb *HistoryDB) AddBlocks(blocks []common.Block) error {
return hdb.addBlocks(hdb.db, blocks)
return tracerr.Wrap(hdb.addBlocks(hdb.db, blocks))
}
func (hdb *HistoryDB) addBlocks(d meddler.DB, blocks []common.Block) error {
return db.BulkInsert(
return tracerr.Wrap(db.BulkInsert(
d,
`INSERT INTO block (
eth_block_num,
@ -64,7 +64,7 @@ func (hdb *HistoryDB) addBlocks(d meddler.DB, blocks []common.Block) error {
hash
) VALUES %s;`,
blocks[:],
)
))
}
// GetBlock retrieve a block from the DB, given a block number
@ -125,7 +125,7 @@ func (hdb *HistoryDB) addBatch(d meddler.DB, batch *common.Batch) error {
var tokenPrices []*tokenPrice
if len(tokenIDs) > 0 {
query, args, err := sqlx.In(
"SELECT token_id, usd, decimals FROM token WHERE token_id IN (?)",
"SELECT token_id, usd, decimals FROM token WHERE token_id IN (?);",
tokenIDs,
)
if err != nil {
@ -150,12 +150,12 @@ func (hdb *HistoryDB) addBatch(d meddler.DB, batch *common.Batch) error {
}
batch.TotalFeesUSD = &total
// Insert to DB
return meddler.Insert(d, "batch", batch)
return tracerr.Wrap(meddler.Insert(d, "batch", batch))
}
// AddBatches insert Bids into the DB
func (hdb *HistoryDB) AddBatches(batches []common.Batch) error {
return hdb.addBatches(hdb.db, batches)
return tracerr.Wrap(hdb.addBatches(hdb.db, batches))
}
func (hdb *HistoryDB) addBatches(d meddler.DB, batches []common.Batch) error {
for i := 0; i < len(batches); i++ {
@ -169,12 +169,12 @@ func (hdb *HistoryDB) addBatches(d meddler.DB, batches []common.Batch) error {
// GetBatchAPI return the batch with the given batchNum
func (hdb *HistoryDB) GetBatchAPI(batchNum common.BatchNum) (*BatchAPI, error) {
batch := &BatchAPI{}
return batch, meddler.QueryRow(
return batch, tracerr.Wrap(meddler.QueryRow(
hdb.db, batch,
`SELECT batch.*, block.timestamp, block.hash
FROM batch INNER JOIN block ON batch.eth_block_num = block.eth_block_num
WHERE batch_num = $1;`, batchNum,
)
))
}
// GetBatchesAPI return the batches applying the given filters
@ -295,14 +295,14 @@ func (hdb *HistoryDB) GetBatches(from, to common.BatchNum) ([]common.Batch, erro
func (hdb *HistoryDB) GetBatchesLen(slotNum int64) (int, error) {
row := hdb.db.QueryRow("SELECT COUNT(*) FROM batch WHERE slot_num = $1;", slotNum)
var batchesLen int
return batchesLen, row.Scan(&batchesLen)
return batchesLen, tracerr.Wrap(row.Scan(&batchesLen))
}
// GetLastBatchNum returns the BatchNum of the latest forged batch
func (hdb *HistoryDB) GetLastBatchNum() (common.BatchNum, error) {
row := hdb.db.QueryRow("SELECT batch_num FROM batch ORDER BY batch_num DESC LIMIT 1;")
var batchNum common.BatchNum
return batchNum, row.Scan(&batchNum)
return batchNum, tracerr.Wrap(row.Scan(&batchNum))
}
// GetLastL1BatchBlockNum returns the blockNum of the latest forged l1Batch
@ -311,7 +311,7 @@ func (hdb *HistoryDB) GetLastL1BatchBlockNum() (int64, error) {
WHERE forge_l1_txs_num IS NOT NULL
ORDER BY batch_num DESC LIMIT 1;`)
var blockNum int64
return blockNum, row.Scan(&blockNum)
return blockNum, tracerr.Wrap(row.Scan(&blockNum))
}
// GetLastL1TxsNum returns the greatest ForgeL1TxsNum in the DB from forged
@ -319,7 +319,7 @@ func (hdb *HistoryDB) GetLastL1BatchBlockNum() (int64, error) {
func (hdb *HistoryDB) GetLastL1TxsNum() (*int64, error) {
row := hdb.db.QueryRow("SELECT MAX(forge_l1_txs_num) FROM batch;")
lastL1TxsNum := new(int64)
return lastL1TxsNum, row.Scan(&lastL1TxsNum)
return lastL1TxsNum, tracerr.Wrap(row.Scan(&lastL1TxsNum))
}
// Reorg deletes all the information that was added into the DB after the
@ -338,12 +338,15 @@ func (hdb *HistoryDB) Reorg(lastValidBlock int64) error {
// AddBids insert Bids into the DB
func (hdb *HistoryDB) AddBids(bids []common.Bid) error { return hdb.addBids(hdb.db, bids) }
func (hdb *HistoryDB) addBids(d meddler.DB, bids []common.Bid) error {
if len(bids) == 0 {
return nil
}
// TODO: check the coordinator info
return db.BulkInsert(
return tracerr.Wrap(db.BulkInsert(
d,
"INSERT INTO bid (slot_num, bid_value, eth_block_num, bidder_addr) VALUES %s;",
bids[:],
)
))
}
// GetAllBids retrieve all bids from the DB
@ -508,31 +511,40 @@ func (hdb *HistoryDB) GetBidsAPI(
// AddCoordinators insert Coordinators into the DB
func (hdb *HistoryDB) AddCoordinators(coordinators []common.Coordinator) error {
return hdb.addCoordinators(hdb.db, coordinators)
return tracerr.Wrap(hdb.addCoordinators(hdb.db, coordinators))
}
func (hdb *HistoryDB) addCoordinators(d meddler.DB, coordinators []common.Coordinator) error {
return db.BulkInsert(
if len(coordinators) == 0 {
return nil
}
return tracerr.Wrap(db.BulkInsert(
d,
"INSERT INTO coordinator (bidder_addr, forger_addr, eth_block_num, url) VALUES %s;",
coordinators[:],
)
))
}
// AddExitTree insert Exit tree into the DB
func (hdb *HistoryDB) AddExitTree(exitTree []common.ExitInfo) error {
return hdb.addExitTree(hdb.db, exitTree)
return tracerr.Wrap(hdb.addExitTree(hdb.db, exitTree))
}
func (hdb *HistoryDB) addExitTree(d meddler.DB, exitTree []common.ExitInfo) error {
return db.BulkInsert(
if len(exitTree) == 0 {
return nil
}
return tracerr.Wrap(db.BulkInsert(
d,
"INSERT INTO exit_tree (batch_num, account_idx, merkle_proof, balance, "+
"instant_withdrawn, delayed_withdraw_request, delayed_withdrawn) VALUES %s;",
exitTree[:],
)
))
}
func (hdb *HistoryDB) updateExitTree(d sqlx.Ext, blockNum int64,
rollupWithdrawals []common.WithdrawInfo, wDelayerWithdrawals []common.WDelayerTransfer) error {
if len(rollupWithdrawals) == 0 && len(wDelayerWithdrawals) == 0 {
return nil
}
type withdrawal struct {
BatchNum int64 `db:"batch_num"`
AccountIdx int64 `db:"account_idx"`
@ -589,10 +601,10 @@ func (hdb *HistoryDB) updateExitTree(d sqlx.Ext, blockNum int64,
) as d (batch_num, account_idx, instant_withdrawn, delayed_withdraw_request, delayed_withdrawn, owner, token)
WHERE
(d.batch_num IS NOT NULL AND e.batch_num = d.batch_num AND e.account_idx = d.account_idx) OR
(d.delayed_withdrawn IS NOT NULL AND e.delayed_withdrawn IS NULL AND e.owner = d.owner AND e.token = d.token)
(d.delayed_withdrawn IS NOT NULL AND e.delayed_withdrawn IS NULL AND e.owner = d.owner AND e.token = d.token);
`
if len(withdrawals) > 0 {
if _, err := sqlx.NamedQuery(d, query, withdrawals); err != nil {
if _, err := sqlx.NamedExec(d, query, withdrawals); err != nil {
return tracerr.Wrap(err)
}
}
@ -602,13 +614,16 @@ func (hdb *HistoryDB) updateExitTree(d sqlx.Ext, blockNum int64,
// AddToken insert a token into the DB
func (hdb *HistoryDB) AddToken(token *common.Token) error {
return meddler.Insert(hdb.db, "token", token)
return tracerr.Wrap(meddler.Insert(hdb.db, "token", token))
}
// AddTokens insert tokens into the DB
func (hdb *HistoryDB) AddTokens(tokens []common.Token) error { return hdb.addTokens(hdb.db, tokens) }
func (hdb *HistoryDB) addTokens(d meddler.DB, tokens []common.Token) error {
return db.BulkInsert(
if len(tokens) == 0 {
return nil
}
return tracerr.Wrap(db.BulkInsert(
d,
`INSERT INTO token (
token_id,
@ -619,7 +634,7 @@ func (hdb *HistoryDB) addTokens(d meddler.DB, tokens []common.Token) error {
decimals
) VALUES %s;`,
tokens[:],
)
))
}
// UpdateTokenValue updates the USD value of a token
@ -728,6 +743,7 @@ func (hdb *HistoryDB) GetTokenSymbols() ([]string, error) {
if err != nil {
return nil, tracerr.Wrap(err)
}
defer db.RowsClose(rows)
sym := new(string)
for rows.Next() {
err = rows.Scan(sym)
@ -741,10 +757,13 @@ func (hdb *HistoryDB) GetTokenSymbols() ([]string, error) {
// AddAccounts insert accounts into the DB
func (hdb *HistoryDB) AddAccounts(accounts []common.Account) error {
return hdb.addAccounts(hdb.db, accounts)
return tracerr.Wrap(hdb.addAccounts(hdb.db, accounts))
}
func (hdb *HistoryDB) addAccounts(d meddler.DB, accounts []common.Account) error {
return db.BulkInsert(
if len(accounts) == 0 {
return nil
}
return tracerr.Wrap(db.BulkInsert(
d,
`INSERT INTO account (
idx,
@ -754,7 +773,7 @@ func (hdb *HistoryDB) addAccounts(d meddler.DB, accounts []common.Account) error
eth_addr
) VALUES %s;`,
accounts[:],
)
))
}
// GetAllAccounts returns a list of accounts from the DB
@ -771,7 +790,9 @@ func (hdb *HistoryDB) GetAllAccounts() ([]common.Account, error) {
// If the tx is originated by a coordinator, BatchNum must be provided. If it's originated by a user,
// BatchNum should be null, and the value will be setted by a trigger when a batch forges the tx.
// EffectiveAmount and EffectiveDepositAmount are seted with default values by the DB.
func (hdb *HistoryDB) AddL1Txs(l1txs []common.L1Tx) error { return hdb.addL1Txs(hdb.db, l1txs) }
func (hdb *HistoryDB) AddL1Txs(l1txs []common.L1Tx) error {
return tracerr.Wrap(hdb.addL1Txs(hdb.db, l1txs))
}
// addL1Txs inserts L1 txs to the DB. USD and DepositAmountUSD will be set automatically before storing the tx.
// If the tx is originated by a coordinator, BatchNum must be provided. If it's originated by a user,
@ -806,11 +827,13 @@ func (hdb *HistoryDB) addL1Txs(d meddler.DB, l1txs []common.L1Tx) error {
DepositAmountFloat: &depositAmountFloat,
})
}
return hdb.addTxs(d, txs)
return tracerr.Wrap(hdb.addTxs(d, txs))
}
// AddL2Txs inserts L2 txs to the DB. TokenID, USD and FeeUSD will be set automatically before storing the tx.
func (hdb *HistoryDB) AddL2Txs(l2txs []common.L2Tx) error { return hdb.addL2Txs(hdb.db, l2txs) }
func (hdb *HistoryDB) AddL2Txs(l2txs []common.L2Tx) error {
return tracerr.Wrap(hdb.addL2Txs(hdb.db, l2txs))
}
// addL2Txs inserts L2 txs to the DB. TokenID, USD and FeeUSD will be set automatically before storing the tx.
func (hdb *HistoryDB) addL2Txs(d meddler.DB, l2txs []common.L2Tx) error {
@ -835,11 +858,14 @@ func (hdb *HistoryDB) addL2Txs(d meddler.DB, l2txs []common.L2Tx) error {
Nonce: &l2txs[i].Nonce,
})
}
return hdb.addTxs(d, txs)
return tracerr.Wrap(hdb.addTxs(d, txs))
}
func (hdb *HistoryDB) addTxs(d meddler.DB, txs []txWrite) error {
return db.BulkInsert(
if len(txs) == 0 {
return nil
}
return tracerr.Wrap(db.BulkInsert(
d,
`INSERT INTO tx (
is_l1,
@ -863,7 +889,7 @@ func (hdb *HistoryDB) addTxs(d meddler.DB, txs []txWrite) error {
nonce
) VALUES %s;`,
txs[:],
)
))
}
// // GetTxs returns a list of txs from the DB
@ -1232,7 +1258,7 @@ func (hdb *HistoryDB) GetUnforgedL1UserTxs(toForgeL1TxsNum int64) ([]common.L1Tx
func (hdb *HistoryDB) GetLastTxsPosition(toForgeL1TxsNum int64) (int, error) {
row := hdb.db.QueryRow("SELECT MAX(position) FROM tx WHERE to_forge_l1_txs_num = $1;", toForgeL1TxsNum)
var lastL1TxsPosition int
return lastL1TxsPosition, row.Scan(&lastL1TxsPosition)
return lastL1TxsPosition, tracerr.Wrap(row.Scan(&lastL1TxsPosition))
}
// GetSCVars returns the rollup, auction and wdelayer smart contracts variables at their last update.
@ -1257,15 +1283,95 @@ func (hdb *HistoryDB) GetSCVars() (*common.RollupVariables, *common.AuctionVaria
}
func (hdb *HistoryDB) setRollupVars(d meddler.DB, rollup *common.RollupVariables) error {
return meddler.Insert(d, "rollup_vars", rollup)
return tracerr.Wrap(meddler.Insert(d, "rollup_vars", rollup))
}
func (hdb *HistoryDB) setAuctionVars(d meddler.DB, auction *common.AuctionVariables) error {
return meddler.Insert(d, "auction_vars", auction)
return tracerr.Wrap(meddler.Insert(d, "auction_vars", auction))
}
func (hdb *HistoryDB) setWDelayerVars(d meddler.DB, wDelayer *common.WDelayerVariables) error {
return meddler.Insert(d, "wdelayer_vars", wDelayer)
return tracerr.Wrap(meddler.Insert(d, "wdelayer_vars", wDelayer))
}
func (hdb *HistoryDB) addBucketUpdates(d meddler.DB, bucketUpdates []common.BucketUpdate) error {
if len(bucketUpdates) == 0 {
return nil
}
return tracerr.Wrap(db.BulkInsert(
d,
`INSERT INTO bucket_update (
eth_block_num,
num_bucket,
block_stamp,
withdrawals
) VALUES %s;`,
bucketUpdates[:],
))
}
// GetAllBucketUpdates retrieves all the bucket updates
func (hdb *HistoryDB) GetAllBucketUpdates() ([]common.BucketUpdate, error) {
var bucketUpdates []*common.BucketUpdate
err := meddler.QueryAll(
hdb.db, &bucketUpdates,
"SELECT * FROM bucket_update;",
)
return db.SlicePtrsToSlice(bucketUpdates).([]common.BucketUpdate), tracerr.Wrap(err)
}
func (hdb *HistoryDB) addTokenExchanges(d meddler.DB, tokenExchanges []common.TokenExchange) error {
if len(tokenExchanges) == 0 {
return nil
}
return tracerr.Wrap(db.BulkInsert(
d,
`INSERT INTO token_exchange (
eth_block_num,
eth_addr,
value_usd
) VALUES %s;`,
tokenExchanges[:],
))
}
// GetAllTokenExchanges retrieves all the token exchanges
func (hdb *HistoryDB) GetAllTokenExchanges() ([]common.TokenExchange, error) {
var tokenExchanges []*common.TokenExchange
err := meddler.QueryAll(
hdb.db, &tokenExchanges,
"SELECT * FROM token_exchange;",
)
return db.SlicePtrsToSlice(tokenExchanges).([]common.TokenExchange), tracerr.Wrap(err)
}
func (hdb *HistoryDB) addEscapeHatchWithdrawals(d meddler.DB,
escapeHatchWithdrawals []common.WDelayerEscapeHatchWithdrawal) error {
if len(escapeHatchWithdrawals) == 0 {
return nil
}
return tracerr.Wrap(db.BulkInsert(
d,
`INSERT INTO escape_hatch_withdrawal (
eth_block_num,
who_addr,
to_addr,
token_addr,
amount
) VALUES %s;`,
escapeHatchWithdrawals[:],
))
}
// GetAllEscapeHatchWithdrawals retrieves all the escape hatch withdrawals
func (hdb *HistoryDB) GetAllEscapeHatchWithdrawals() ([]common.WDelayerEscapeHatchWithdrawal, error) {
var escapeHatchWithdrawals []*common.WDelayerEscapeHatchWithdrawal
err := meddler.QueryAll(
hdb.db, &escapeHatchWithdrawals,
"SELECT * FROM escape_hatch_withdrawal;",
)
return db.SlicePtrsToSlice(escapeHatchWithdrawals).([]common.WDelayerEscapeHatchWithdrawal),
tracerr.Wrap(err)
}
// SetInitialSCVars sets the initial state of rollup, auction, wdelayer smart
@ -1305,6 +1411,9 @@ func (hdb *HistoryDB) SetInitialSCVars(rollup *common.RollupVariables,
// setL1UserTxEffectiveAmounts sets the EffectiveAmount and EffectiveDepositAmount
// of the given l1UserTxs (with an UPDATE)
func (hdb *HistoryDB) setL1UserTxEffectiveAmounts(d sqlx.Ext, txs []common.L1Tx) error {
if len(txs) == 0 {
return nil
}
// Effective amounts are stored as success flags in the DB, with true value by default
// to reduce the amount of updates. Therefore, only amounts that became uneffective should be
// updated to become false
@ -1336,10 +1445,10 @@ func (hdb *HistoryDB) setL1UserTxEffectiveAmounts(d sqlx.Ext, txs []common.L1Tx)
(NULL::::BYTEA, NULL::::BOOL, NULL::::BOOL),
(:id, :amount_success, :deposit_amount_success)
) as tx_update (id, amount_success, deposit_amount_success)
WHERE tx.id = tx_update.id
WHERE tx.id = tx_update.id;
`
if len(txUpdates) > 0 {
if _, err := sqlx.NamedQuery(d, query, txUpdates); err != nil {
if _, err := sqlx.NamedExec(d, query, txUpdates); err != nil {
return tracerr.Wrap(err)
}
}
@ -1367,24 +1476,18 @@ func (hdb *HistoryDB) AddBlockSCData(blockData *common.BlockData) (err error) {
}
// Add Coordinators
if len(blockData.Auction.Coordinators) > 0 {
if err := hdb.addCoordinators(txn, blockData.Auction.Coordinators); err != nil {
return tracerr.Wrap(err)
}
if err := hdb.addCoordinators(txn, blockData.Auction.Coordinators); err != nil {
return tracerr.Wrap(err)
}
// Add Bids
if len(blockData.Auction.Bids) > 0 {
if err := hdb.addBids(txn, blockData.Auction.Bids); err != nil {
return tracerr.Wrap(err)
}
if err := hdb.addBids(txn, blockData.Auction.Bids); err != nil {
return tracerr.Wrap(err)
}
// Add Tokens
if len(blockData.Rollup.AddedTokens) > 0 {
if err := hdb.addTokens(txn, blockData.Rollup.AddedTokens); err != nil {
return tracerr.Wrap(err)
}
if err := hdb.addTokens(txn, blockData.Rollup.AddedTokens); err != nil {
return tracerr.Wrap(err)
}
// Prepare user L1 txs to be added.
@ -1420,10 +1523,8 @@ func (hdb *HistoryDB) AddBlockSCData(blockData *common.BlockData) (err error) {
// Set the EffectiveAmount and EffectiveDepositAmount of all the
// L1UserTxs that have been forged in this batch
if len(batch.L1UserTxs) > 0 {
if err = hdb.setL1UserTxEffectiveAmounts(txn, batch.L1UserTxs); err != nil {
return tracerr.Wrap(err)
}
if err = hdb.setL1UserTxEffectiveAmounts(txn, batch.L1UserTxs); err != nil {
return tracerr.Wrap(err)
}
// Add Batch: this will trigger an update on the DB
@ -1433,24 +1534,18 @@ func (hdb *HistoryDB) AddBlockSCData(blockData *common.BlockData) (err error) {
}
// Add accounts
if len(batch.CreatedAccounts) > 0 {
if err := hdb.addAccounts(txn, batch.CreatedAccounts); err != nil {
return tracerr.Wrap(err)
}
if err := hdb.addAccounts(txn, batch.CreatedAccounts); err != nil {
return tracerr.Wrap(err)
}
// Add forged l1 coordinator Txs
if len(batch.L1CoordinatorTxs) > 0 {
if err := hdb.addL1Txs(txn, batch.L1CoordinatorTxs); err != nil {
return tracerr.Wrap(err)
}
if err := hdb.addL1Txs(txn, batch.L1CoordinatorTxs); err != nil {
return tracerr.Wrap(err)
}
// Add l2 Txs
if len(batch.L2Txs) > 0 {
if err := hdb.addL2Txs(txn, batch.L2Txs); err != nil {
return tracerr.Wrap(err)
}
if err := hdb.addL2Txs(txn, batch.L2Txs); err != nil {
return tracerr.Wrap(err)
}
// Add user L1 txs that will be forged in next batch
@ -1461,10 +1556,8 @@ func (hdb *HistoryDB) AddBlockSCData(blockData *common.BlockData) (err error) {
}
// Add exit tree
if len(batch.ExitTree) > 0 {
if err := hdb.addExitTree(txn, batch.ExitTree); err != nil {
return tracerr.Wrap(err)
}
if err := hdb.addExitTree(txn, batch.ExitTree); err != nil {
return tracerr.Wrap(err)
}
}
// Add user L1 txs that won't be forged in this block
@ -1473,6 +1566,8 @@ func (hdb *HistoryDB) AddBlockSCData(blockData *common.BlockData) (err error) {
return tracerr.Wrap(err)
}
}
// Set SC Vars if there was an update
if blockData.Rollup.Vars != nil {
if err := hdb.setRollupVars(txn, blockData.Rollup.Vars); err != nil {
return tracerr.Wrap(err)
@ -1489,11 +1584,28 @@ func (hdb *HistoryDB) AddBlockSCData(blockData *common.BlockData) (err error) {
}
}
// Update withdrawals in exit tree table
if err := hdb.updateExitTree(txn, blockData.Block.Num,
blockData.Rollup.Withdrawals, blockData.WDelayer.Withdrawals); err != nil {
return tracerr.Wrap(err)
}
// Add Escape Hatch Withdrawals
if err := hdb.addEscapeHatchWithdrawals(txn,
blockData.WDelayer.EscapeHatchWithdrawals); err != nil {
return tracerr.Wrap(err)
}
// Add Buckets withdrawals updates
if err := hdb.addBucketUpdates(txn, blockData.Rollup.UpdateBucketWithdraw); err != nil {
return tracerr.Wrap(err)
}
// Add Token exchange updates
if err := hdb.addTokenExchanges(txn, blockData.Rollup.TokenExchanges); err != nil {
return tracerr.Wrap(err)
}
return tracerr.Wrap(txn.Commit())
}
@ -1544,7 +1656,7 @@ func (hdb *HistoryDB) GetCoordinatorsAPI(fromItem, limit *uint, order string) ([
// AddAuctionVars insert auction vars into the DB
func (hdb *HistoryDB) AddAuctionVars(auctionVars *common.AuctionVariables) error {
return meddler.Insert(hdb.db, "auction_vars", auctionVars)
return tracerr.Wrap(meddler.Insert(hdb.db, "auction_vars", auctionVars))
}
// GetAuctionVars returns auction variables

+ 83
- 1
db/historydb/historydb_test.go

@ -647,7 +647,8 @@ func exampleInitSCVars() (*common.RollupVariables, *common.AuctionVariables, *co
big.NewInt(10),
12,
13,
[5]common.Bucket{},
[5]common.BucketParams{},
false,
}
//nolint:govet
auction := &common.AuctionVariables{
@ -937,6 +938,87 @@ func TestGetBestBidCoordinator(t *testing.T) {
require.Equal(t, sql.ErrNoRows, tracerr.Unwrap(err))
}
func TestAddBucketUpdates(t *testing.T) {
test.WipeDB(historyDB.DB())
const fromBlock int64 = 1
const toBlock int64 = 5 + 1
setTestBlocks(fromBlock, toBlock)
bucketUpdates := []common.BucketUpdate{
{
EthBlockNum: 4,
NumBucket: 0,
BlockStamp: 4,
Withdrawals: big.NewInt(123),
},
{
EthBlockNum: 5,
NumBucket: 2,
BlockStamp: 5,
Withdrawals: big.NewInt(42),
},
}
err := historyDB.addBucketUpdates(historyDB.db, bucketUpdates)
require.NoError(t, err)
dbBucketUpdates, err := historyDB.GetAllBucketUpdates()
require.NoError(t, err)
assert.Equal(t, bucketUpdates, dbBucketUpdates)
}
func TestAddTokenExchanges(t *testing.T) {
test.WipeDB(historyDB.DB())
const fromBlock int64 = 1
const toBlock int64 = 5 + 1
setTestBlocks(fromBlock, toBlock)
tokenExchanges := []common.TokenExchange{
{
EthBlockNum: 4,
Address: ethCommon.BigToAddress(big.NewInt(111)),
ValueUSD: 12345,
},
{
EthBlockNum: 5,
Address: ethCommon.BigToAddress(big.NewInt(222)),
ValueUSD: 67890,
},
}
err := historyDB.addTokenExchanges(historyDB.db, tokenExchanges)
require.NoError(t, err)
dbTokenExchanges, err := historyDB.GetAllTokenExchanges()
require.NoError(t, err)
assert.Equal(t, tokenExchanges, dbTokenExchanges)
}
func TestAddEscapeHatchWithdrawals(t *testing.T) {
test.WipeDB(historyDB.DB())
const fromBlock int64 = 1
const toBlock int64 = 5 + 1
setTestBlocks(fromBlock, toBlock)
escapeHatchWithdrawals := []common.WDelayerEscapeHatchWithdrawal{
{
EthBlockNum: 4,
Who: ethCommon.BigToAddress(big.NewInt(111)),
To: ethCommon.BigToAddress(big.NewInt(222)),
TokenAddr: ethCommon.BigToAddress(big.NewInt(333)),
Amount: big.NewInt(10002),
},
{
EthBlockNum: 5,
Who: ethCommon.BigToAddress(big.NewInt(444)),
To: ethCommon.BigToAddress(big.NewInt(555)),
TokenAddr: ethCommon.BigToAddress(big.NewInt(666)),
Amount: big.NewInt(20003),
},
}
err := historyDB.addEscapeHatchWithdrawals(historyDB.db, escapeHatchWithdrawals)
require.NoError(t, err)
dbEscapeHatchWithdrawals, err := historyDB.GetAllEscapeHatchWithdrawals()
require.NoError(t, err)
assert.Equal(t, escapeHatchWithdrawals, dbEscapeHatchWithdrawals)
}
// setTestBlocks WARNING: this will delete the blocks and recreate them
func setTestBlocks(from, to int64) []common.Block {
test.WipeDB(historyDB.DB())

+ 12
- 11
db/l2db/l2db.go

@ -59,26 +59,26 @@ func (l2db *L2DB) AddAccountCreationAuth(auth *common.AccountCreationAuth) error
// GetAccountCreationAuth returns an account creation authorization from the DB
func (l2db *L2DB) GetAccountCreationAuth(addr ethCommon.Address) (*common.AccountCreationAuth, error) {
auth := new(common.AccountCreationAuth)
return auth, meddler.QueryRow(
return auth, tracerr.Wrap(meddler.QueryRow(
l2db.db, auth,
"SELECT * FROM account_creation_auth WHERE eth_addr = $1;",
addr,
)
))
}
// GetAccountCreationAuthAPI returns an account creation authorization from the DB
func (l2db *L2DB) GetAccountCreationAuthAPI(addr ethCommon.Address) (*AccountCreationAuthAPI, error) {
auth := new(AccountCreationAuthAPI)
return auth, meddler.QueryRow(
return auth, tracerr.Wrap(meddler.QueryRow(
l2db.db, auth,
"SELECT * FROM account_creation_auth WHERE eth_addr = $1;",
addr,
)
))
}
// AddTx inserts a tx to the pool
func (l2db *L2DB) AddTx(tx *PoolL2TxWrite) error {
return meddler.Insert(l2db.db, "tx_pool", tx)
return tracerr.Wrap(meddler.Insert(l2db.db, "tx_pool", tx))
}
// AddTxTest inserts a tx into the L2DB. This is useful for test purposes,
@ -122,7 +122,7 @@ func (l2db *L2DB) AddTxTest(tx *common.PoolL2Tx) error {
amountF, _ := f.Float64()
insertTx.AmountFloat = amountF
// insert tx
return meddler.Insert(l2db.db, "tx_pool", insertTx)
return tracerr.Wrap(meddler.Insert(l2db.db, "tx_pool", insertTx))
}
// selectPoolTxAPI select part of queries to get PoolL2TxRead
@ -147,21 +147,21 @@ FROM tx_pool INNER JOIN token ON tx_pool.token_id = token.token_id `
// GetTx return the specified Tx in common.PoolL2Tx format
func (l2db *L2DB) GetTx(txID common.TxID) (*common.PoolL2Tx, error) {
tx := new(common.PoolL2Tx)
return tx, meddler.QueryRow(
return tx, tracerr.Wrap(meddler.QueryRow(
l2db.db, tx,
selectPoolTxCommon+"WHERE tx_id = $1;",
txID,
)
))
}
// GetTxAPI return the specified Tx in PoolTxAPI format
func (l2db *L2DB) GetTxAPI(txID common.TxID) (*PoolTxAPI, error) {
tx := new(PoolTxAPI)
return tx, meddler.QueryRow(
return tx, tracerr.Wrap(meddler.QueryRow(
l2db.db, tx,
selectPoolTxAPI+"WHERE tx_id = $1;",
txID,
)
))
}
// GetPendingTxs return all the pending txs of the L2DB, that have a non NULL AbsoluteFee
@ -252,6 +252,7 @@ func (l2db *L2DB) GetPendingUniqueFromIdxs() ([]common.Idx, error) {
if err != nil {
return nil, tracerr.Wrap(err)
}
defer db.RowsClose(rows)
var idx common.Idx
for rows.Next() {
err = rows.Scan(&idx)
@ -285,7 +286,7 @@ func (l2db *L2DB) CheckNonces(updatedAccounts []common.IdxNonce, batchNum common
// named query which works with slices, and doens't handle an extra
// individual argument.
query := fmt.Sprintf(checkNoncesQuery, batchNum)
if _, err := sqlx.NamedQuery(l2db.db, query, updatedAccounts); err != nil {
if _, err := sqlx.NamedExec(l2db.db, query, updatedAccounts); err != nil {
return tracerr.Wrap(err)
}
return nil

+ 29
- 1
db/migrations/0001.sql

@ -536,7 +536,32 @@ CREATE TABLE rollup_vars (
fee_add_token BYTEA NOT NULL,
forge_l1_timeout BIGINT NOT NULL,
withdrawal_delay BIGINT NOT NULL,
buckets BYTEA
buckets BYTEA NOT NULL,
safe_mode BOOLEAN NOT NULL
);
CREATE TABLE bucket_update (
item_id SERIAL PRIMARY KEY,
eth_block_num BIGINT NOT NULL REFERENCES block (eth_block_num) ON DELETE CASCADE,
num_bucket BIGINT NOT NULL,
block_stamp BIGINT NOT NULL,
withdrawals BYTEA NOT NULL
);
CREATE TABLE token_exchange (
item_id SERIAL PRIMARY KEY,
eth_block_num BIGINT NOT NULL REFERENCES block (eth_block_num) ON DELETE CASCADE,
eth_addr BYTEA NOT NULL,
value_usd BIGINT NOT NULL
);
CREATE TABLE escape_hatch_withdrawal (
item_id SERIAL PRIMARY KEY,
eth_block_num BIGINT NOT NULL REFERENCES block (eth_block_num) ON DELETE CASCADE,
who_addr BYTEA NOT NULL,
to_addr BYTEA NOT NULL,
token_addr BYTEA NOT NULL,
amount BYTEA NOT NULL
);
CREATE TABLE auction_vars (
@ -635,6 +660,9 @@ DROP TABLE account_creation_auth;
DROP TABLE tx_pool;
DROP TABLE auction_vars;
DROP TABLE rollup_vars;
DROP TABLE escape_hatch_withdrawal;
DROP TABLE bucket_update;
DROP TABLE token_exchange;
DROP TABLE wdelayer_vars;
DROP TABLE tx;
DROP TABLE exit_tree;

+ 9
- 2
db/utils.go

@ -1,6 +1,7 @@
package db
import (
"database/sql"
"fmt"
"math/big"
"reflect"
@ -168,8 +169,14 @@ func SlicePtrsToSlice(slice interface{}) interface{} {
// Rollback an sql transaction, and log the error if it's not nil
func Rollback(txn *sqlx.Tx) {
err := txn.Rollback()
if err != nil {
if err := txn.Rollback(); err != nil {
log.Errorw("Rollback", "err", err)
}
}
// RowsClose close the rows of an sql query, and log the errir if it's not nil
func RowsClose(rows *sql.Rows) {
if err := rows.Close(); err != nil {
log.Errorw("rows.Close", "err", err)
}
}

+ 15
- 0
eth/rollup.go

@ -131,6 +131,7 @@ type rollupEventUpdateBucketsParametersAux struct {
type RollupEventUpdateBucketsParameters struct {
// ArrayBuckets [common.RollupConstNumBuckets][4]*big.Int
ArrayBuckets [common.RollupConstNumBuckets]RollupUpdateBucketsParameters
SafeMode bool
}
// RollupEventUpdateTokenExchange is an event of the Rollup Smart Contract
@ -798,6 +799,20 @@ func (c *RollupClient) RollupEventsByBlock(blockNum int64) (*RollupEvents, *ethC
case logHermezSafeMode:
var safeMode RollupEventSafeMode
rollupEvents.SafeMode = append(rollupEvents.SafeMode, safeMode)
// Also add an UpdateBucketsParameter with
// SafeMode=true to keep the order between `safeMode`
// and `UpdateBucketsParameters`
bucketsParameters := RollupEventUpdateBucketsParameters{
SafeMode: true,
}
for i := range bucketsParameters.ArrayBuckets {
bucketsParameters.ArrayBuckets[i].CeilUSD = big.NewInt(0)
bucketsParameters.ArrayBuckets[i].Withdrawals = big.NewInt(0)
bucketsParameters.ArrayBuckets[i].BlockWithdrawalRate = big.NewInt(0)
bucketsParameters.ArrayBuckets[i].MaxWithdrawals = big.NewInt(0)
}
rollupEvents.UpdateBucketsParameters = append(rollupEvents.UpdateBucketsParameters,
bucketsParameters)
}
}
return &rollupEvents, blockHash, nil

+ 78
- 23
synchronizer/synchronizer.go

@ -313,7 +313,7 @@ func (s *Synchronizer) updateCurrentSlotIfSync(batchesLen int) error {
slot.SlotNum = slotNum
slot.StartBlock, slot.EndBlock = s.consts.Auction.SlotBlocks(slot.SlotNum)
// If Synced, update the current coordinator
if s.stats.Synced() {
if s.stats.Synced() && blockNum >= s.consts.Auction.GenesisBlockNum {
bidCoord, err := s.historyDB.GetBestBidCoordinator(slot.SlotNum)
if err != nil && tracerr.Unwrap(err) != sql.ErrNoRows {
return tracerr.Wrap(err)
@ -897,29 +897,74 @@ func (s *Synchronizer) rollupSync(ethBlock *common.Block) (*common.RollupData, e
rollupData.AddedTokens = append(rollupData.AddedTokens, token)
}
for _, evt := range rollupEvents.UpdateBucketWithdraw {
rollupData.UpdateBucketWithdraw = append(rollupData.UpdateBucketWithdraw,
common.BucketUpdate{
EthBlockNum: blockNum,
NumBucket: evt.NumBucket,
BlockStamp: evt.BlockStamp,
Withdrawals: evt.Withdrawals,
})
}
for _, evt := range rollupEvents.Withdraw {
rollupData.Withdrawals = append(rollupData.Withdrawals, common.WithdrawInfo{
Idx: common.Idx(evt.Idx),
NumExitRoot: common.BatchNum(evt.NumExitRoot),
InstantWithdraw: evt.InstantWithdraw,
TxHash: evt.TxHash,
})
}
for _, evt := range rollupEvents.UpdateTokenExchange {
if len(evt.AddressArray) != len(evt.ValueArray) {
return nil, tracerr.Wrap(fmt.Errorf("in RollupEventUpdateTokenExchange "+
"len(AddressArray) != len(ValueArray) (%v != %v)",
len(evt.AddressArray), len(evt.ValueArray)))
}
for i := range evt.AddressArray {
rollupData.TokenExchanges = append(rollupData.TokenExchanges,
common.TokenExchange{
EthBlockNum: blockNum,
Address: evt.AddressArray[i],
ValueUSD: int64(evt.ValueArray[i]),
})
}
}
varsUpdate := false
for _, evtUpdateForgeL1L2BatchTimeout := range rollupEvents.UpdateForgeL1L2BatchTimeout {
s.vars.Rollup.ForgeL1L2BatchTimeout = evtUpdateForgeL1L2BatchTimeout.NewForgeL1L2BatchTimeout
for _, evt := range rollupEvents.UpdateForgeL1L2BatchTimeout {
s.vars.Rollup.ForgeL1L2BatchTimeout = evt.NewForgeL1L2BatchTimeout
varsUpdate = true
}
for _, evtUpdateFeeAddToken := range rollupEvents.UpdateFeeAddToken {
s.vars.Rollup.FeeAddToken = evtUpdateFeeAddToken.NewFeeAddToken
for _, evt := range rollupEvents.UpdateFeeAddToken {
s.vars.Rollup.FeeAddToken = evt.NewFeeAddToken
varsUpdate = true
}
// NOTE: WithdrawDelay update doesn't have event, so we can't track changes
for _, evt := range rollupEvents.UpdateWithdrawalDelay {
s.vars.Rollup.WithdrawalDelay = evt.NewWithdrawalDelay
varsUpdate = true
}
// NOTE: Buckets update dones't have event, so we can't track changes
// NOTE: We skip the event rollupEvents.SafeMode because the
// implementation RollupEventsByBlock already inserts a non-existing
// RollupEventUpdateBucketsParameters into UpdateBucketsParameters with
// all the bucket values at 0 and SafeMode = true
for _, evtWithdraw := range rollupEvents.Withdraw {
rollupData.Withdrawals = append(rollupData.Withdrawals, common.WithdrawInfo{
Idx: common.Idx(evtWithdraw.Idx),
NumExitRoot: common.BatchNum(evtWithdraw.NumExitRoot),
InstantWithdraw: evtWithdraw.InstantWithdraw,
TxHash: evtWithdraw.TxHash,
})
for _, evt := range rollupEvents.UpdateBucketsParameters {
for i, bucket := range evt.ArrayBuckets {
s.vars.Rollup.Buckets[i] = common.BucketParams{
CeilUSD: bucket.CeilUSD,
Withdrawals: bucket.Withdrawals,
BlockWithdrawalRate: bucket.BlockWithdrawalRate,
MaxWithdrawals: bucket.MaxWithdrawals,
}
}
s.vars.Rollup.SafeMode = evt.SafeMode
varsUpdate = true
}
if varsUpdate {
@ -958,22 +1003,22 @@ func (s *Synchronizer) auctionSync(ethBlock *common.Block) (*common.AuctionData,
}
// Get bids
for _, evtNewBid := range auctionEvents.NewBid {
for _, evt := range auctionEvents.NewBid {
bid := common.Bid{
SlotNum: evtNewBid.Slot,
BidValue: evtNewBid.BidAmount,
Bidder: evtNewBid.Bidder,
SlotNum: evt.Slot,
BidValue: evt.BidAmount,
Bidder: evt.Bidder,
EthBlockNum: blockNum,
}
auctionData.Bids = append(auctionData.Bids, bid)
}
// Get Coordinators
for _, evtSetCoordinator := range auctionEvents.SetCoordinator {
for _, evt := range auctionEvents.SetCoordinator {
coordinator := common.Coordinator{
Bidder: evtSetCoordinator.BidderAddress,
Forger: evtSetCoordinator.ForgerAddress,
URL: evtSetCoordinator.CoordinatorURL,
Bidder: evt.BidderAddress,
Forger: evt.ForgerAddress,
URL: evt.CoordinatorURL,
EthBlockNum: blockNum,
}
auctionData.Coordinators = append(auctionData.Coordinators, coordinator)
@ -999,6 +1044,7 @@ func (s *Synchronizer) auctionSync(ethBlock *common.Block) (*common.AuctionData,
}
for _, evt := range auctionEvents.NewBootCoordinator {
s.vars.Auction.BootCoordinator = evt.NewBootCoordinator
s.vars.Auction.BootCoordinatorURL = evt.NewBootCoordinatorURL
varsUpdate = true
}
for _, evt := range auctionEvents.NewOpenAuctionSlots {
@ -1069,10 +1115,19 @@ func (s *Synchronizer) wdelayerSync(ethBlock *common.Block) (*common.WDelayerDat
Amount: evt.Amount,
})
}
for _, evt := range wDelayerEvents.EscapeHatchWithdrawal {
wDelayerData.EscapeHatchWithdrawals = append(wDelayerData.EscapeHatchWithdrawals,
common.WDelayerEscapeHatchWithdrawal{
EthBlockNum: blockNum,
Who: evt.Who,
To: evt.To,
TokenAddr: evt.Token,
Amount: evt.Amount,
})
}
varsUpdate := false
// TODO EscapeHatchWithdrawal
for range wDelayerEvents.EmergencyModeEnabled {
s.vars.WDelayer.EmergencyMode = true
varsUpdate = true

+ 11
- 7
test/ethclient.go

@ -123,8 +123,10 @@ func (a *AuctionBlock) getMinBidBySlot(slot int64) (*big.Int, error) {
slotState = eth.NewSlotState()
a.State.Slots[slot] = slotState
}
// If the bidAmount for a slot is 0 it means that it has not yet been bid, so the midBid will be the minimum
// bid for the slot time plus the outbidding set, otherwise it will be the bidAmount plus the outbidding
// If the bidAmount for a slot is 0 it means that it has not yet been
// bid, so the midBid will be the minimum bid for the slot time plus
// the outbidding set, otherwise it will be the bidAmount plus the
// outbidding
if slotState.BidAmount.Cmp(big.NewInt(0)) == 0 {
prevBid = a.Vars.DefaultSlotSetBid[slotSet]
} else {
@ -157,7 +159,6 @@ func (a *AuctionBlock) forge(forger ethCommon.Address) error {
// Get the relativeBlock to check if the slotDeadline has been exceeded
relativeBlock := a.Eth.BlockNum - (a.Constants.GenesisBlockNum +
(slotToForge * int64(a.Constants.BlocksPerSlot)))
if relativeBlock < int64(a.Vars.SlotDeadline) {
slotState.ForgerCommitment = true
}
@ -182,8 +183,9 @@ func (a *AuctionBlock) canForge(forger ethCommon.Address, blockNum int64) (bool,
// Get the relativeBlock to check if the slotDeadline has been exceeded
relativeBlock := blockNum - (a.Constants.GenesisBlockNum + (slotToForge * int64(a.Constants.BlocksPerSlot)))
// If the closedMinBid is 0 it means that we have to take as minBid the one that is set for this slot set,
// otherwise the one that has been saved will be used
// If the closedMinBid is 0 it means that we have to take as minBid the
// one that is set for this slot set, otherwise the one that has been
// saved will be used
var minBid *big.Int
slotState, ok := a.State.Slots[slotToForge]
if !ok {
@ -197,14 +199,16 @@ func (a *AuctionBlock) canForge(forger ethCommon.Address, blockNum int64) (bool,
}
if !slotState.ForgerCommitment && (relativeBlock >= int64(a.Vars.SlotDeadline)) {
// if the relative block has exceeded the slotDeadline and no batch has been forged, anyone can forge
// if the relative block has exceeded the slotDeadline and no
// batch has been forged, anyone can forge
return true, nil
} else if coord, ok := a.State.Coordinators[slotState.Bidder]; ok &&
coord.Forger == forger && slotState.BidAmount.Cmp(minBid) >= 0 {
// if forger bidAmount has exceeded the minBid it can forge
return true, nil
} else if a.Vars.BootCoordinator == forger && slotState.BidAmount.Cmp(minBid) == -1 {
// if it's the boot coordinator and it has not been bid or the bid is below the minimum it can forge
// if it's the boot coordinator and it has not been bid or the
// bid is below the minimum it can forge
return true, nil
} else {
return false, nil

Loading…
Cancel
Save