Browse Source

Review synchronizer, rearange some stuff, add notes and TODOs.

feature/sql-semaphore1
Eduard S 4 years ago
parent
commit
9f83fcf520
8 changed files with 170 additions and 144 deletions
  1. +1
    -1
      common/batch.go
  2. +1
    -1
      common/l1tx.go
  3. +1
    -1
      common/tx.go
  4. +4
    -2
      db/historydb/historydb.go
  5. +2
    -0
      db/l2db/l2db.go
  6. +2
    -0
      db/statedb/statedb.go
  7. +158
    -138
      synchronizer/synchronizer.go
  8. +1
    -1
      test/historydb.go

+ 1
- 1
common/batch.go

@ -19,7 +19,7 @@ type Batch struct {
StateRoot Hash `meddler:"state_root"`
NumAccounts int `meddler:"num_accounts"`
ExitRoot Hash `meddler:"exit_root"`
ForgeL1TxsNum uint32 `meddler:"forge_l1_txs_num"` // optional, Only when the batch forges L1 txs. Identifier that corresponds to the group of L1 txs forged in the current batch.
ForgeL1TxsNum int64 `meddler:"forge_l1_txs_num"` // optional, Only when the batch forges L1 txs. Identifier that corresponds to the group of L1 txs forged in the current batch.
SlotNum SlotNum `meddler:"slot_num"` // Slot in which the batch is forged
}

+ 1
- 1
common/l1tx.go

@ -19,7 +19,7 @@ const (
type L1Tx struct {
// Stored in DB: mandatory fileds
TxID TxID
ToForgeL1TxsNum uint32 // toForgeL1TxsNum in which the tx was forged / will be forged
ToForgeL1TxsNum int64 // toForgeL1TxsNum in which the tx was forged / will be forged
Position int
UserOrigin bool // true if the tx was originated by a user, false if it was aoriginated by a coordinator. Note that this differ from the spec for implementation simplification purpposes
FromIdx Idx // FromIdx is used by L1Tx/Deposit to indicate the Idx receiver of the L1Tx.LoadAmount (deposit)

+ 1
- 1
common/tx.go

@ -54,7 +54,7 @@ type Tx struct {
BatchNum BatchNum `meddler:"batch_num,zeroisnull"` // batchNum in which this tx was forged. If the tx is L2, this must be != 0
EthBlockNum int64 `meddler:"eth_block_num"` // Ethereum Block Number in which this L1Tx was added to the queue
// L1
ToForgeL1TxsNum uint32 `meddler:"to_forge_l1_txs_num"` // toForgeL1TxsNum in which the tx was forged / will be forged
ToForgeL1TxsNum int64 `meddler:"to_forge_l1_txs_num"` // toForgeL1TxsNum in which the tx was forged / will be forged
UserOrigin bool `meddler:"user_origin"` // true if the tx was originated by a user, false if it was aoriginated by a coordinator. Note that this differ from the spec for implementation simplification purpposes
FromEthAddr ethCommon.Address `meddler:"from_eth_addr"`
FromBJJ *babyjub.PublicKey `meddler:"from_bjj"`

+ 4
- 2
db/historydb/historydb.go

@ -14,6 +14,8 @@ import (
"github.com/russross/meddler"
)
// TODO(Edu): Document here how HistoryDB is kept consistent
// HistoryDB persist the historic of the rollup
type HistoryDB struct {
db *sqlx.DB
@ -129,9 +131,9 @@ func (hdb *HistoryDB) GetLastBatchNum() (common.BatchNum, error) {
// GetLastL1TxsNum returns the greatest ForgeL1TxsNum in the DB. If there's no
// batch in the DB (nil, nil) is returned.
func (hdb *HistoryDB) GetLastL1TxsNum() (*uint32, error) {
func (hdb *HistoryDB) GetLastL1TxsNum() (*int64, error) {
row := hdb.db.QueryRow("SELECT MAX(forge_l1_txs_num) FROM batch;")
lastL1TxsNum := new(uint32)
lastL1TxsNum := new(int64)
return lastL1TxsNum, row.Scan(&lastL1TxsNum)
}

+ 2
- 0
db/l2db/l2db.go

@ -17,6 +17,8 @@ import (
"github.com/russross/meddler"
)
// TODO(Edu): Check DB consistency while there's concurrent use from Coordinator/TxSelector & API
// L2DB stores L2 txs and authorization registers received by the coordinator and keeps them until they are no longer relevant
// due to them being forged or invalid after a safety period
type L2DB struct {

+ 2
- 0
db/statedb/statedb.go

@ -13,6 +13,8 @@ import (
"github.com/iden3/go-merkletree/db/pebble"
)
// TODO(Edu): Document here how StateDB is kept consistent
// ErrStateDBWithoutMT is used when a method that requires a MerkleTree is
// called in a StateDB that does not have a MerkleTree defined
var ErrStateDBWithoutMT = errors.New("Can not call method to use MerkleTree in a StateDB without MerkleTree")

+ 158
- 138
synchronizer/synchronizer.go

@ -14,10 +14,6 @@ import (
"github.com/hermeznetwork/hermez-node/log"
)
const (
blocksToSync = 20 // TODO: This will be deleted once we can get the firstSavedBlock from the ethClient
)
var (
// ErrNotAbleToSync is used when there is not possible to find a valid block to sync
ErrNotAbleToSync = errors.New("it has not been possible to synchronize any block")
@ -59,22 +55,22 @@ func newAuctionData() *auctionData {
// BatchData contains information about Batches from the contracts
type BatchData struct {
l1UserTxs []*common.L1Tx
l1CoordinatorTxs []*common.L1Tx
l2Txs []*common.L2Tx
registeredAccounts []*common.Account
exitTree []*common.ExitInfo
batch *common.Batch
l1UserTxs []*common.L1Tx
l1CoordinatorTxs []*common.L1Tx
l2Txs []*common.L2Tx
createdAccounts []*common.Account
exitTree []*common.ExitInfo
batch *common.Batch
}
// NewBatchData creates an empty BatchData with the slices initialized.
func NewBatchData() *BatchData {
return &BatchData{
l1UserTxs: make([]*common.L1Tx, 0),
l1CoordinatorTxs: make([]*common.L1Tx, 0),
l2Txs: make([]*common.L2Tx, 0),
registeredAccounts: make([]*common.Account, 0),
exitTree: make([]*common.ExitInfo, 0),
l1UserTxs: make([]*common.L1Tx, 0),
l1CoordinatorTxs: make([]*common.L1Tx, 0),
l2Txs: make([]*common.L2Tx, 0),
createdAccounts: make([]*common.Account, 0),
exitTree: make([]*common.ExitInfo, 0),
}
}
@ -82,9 +78,9 @@ func NewBatchData() *BatchData {
type BlockData struct {
block *common.Block
// Rollup
l1Txs []*common.L1Tx
batches []*BatchData
// withdrawals []*common.ExitInfo
l1Txs []*common.L1Tx // TODO: Answer: User? Coordinator? Both?
batches []*BatchData // TODO: Also contains L1Txs!
// withdrawals []*common.ExitInfo // TODO
registeredTokens []*common.Token
rollupVars *common.RollupVars
// Auction
@ -114,33 +110,29 @@ func NewSynchronizer(ethClient *eth.Client, historyDB *historydb.HistoryDB, stat
return s
}
// TODO: Be smart about locking: only lock during the read/write operations
// Sync updates History and State DB with information from the blockchain
// TODO: Return true if a new block was processed
// TODO: Add argument: maximum number of blocks to process
// TODO: Check reorgs in the middle of syncing a block. Probably make
// rollupSync, auctionSync and withdrawalSync return the block hash.
func (s *Synchronizer) Sync() error {
// Avoid new sync while performing one
s.mux.Lock()
defer s.mux.Unlock()
// TODO: Get this information from ethClient once it's implemented
// for the moment we will get the latestblock - 20 as firstSavedBlock
latestBlock, err := s.ethClient.EthBlockByNumber(context.Background(), 0)
if err != nil {
return err
}
s.firstSavedBlock, err = s.ethClient.EthBlockByNumber(context.Background(), latestBlock.EthBlockNum-blocksToSync)
if err != nil {
return err
}
var nextBlockNum int64 // next block number to sync
// Get lastSavedBlock from History DB
lastSavedBlock, err := s.historyDB.GetLastBlock()
if err != nil && err != sql.ErrNoRows {
return err
}
// Check if we got a block or nil
// In case of nil we must do a full sync
if lastSavedBlock == nil || lastSavedBlock.EthBlockNum == 0 {
lastSavedBlock = s.firstSavedBlock
// If we don't have any stored block, we must do a full sync starting from the rollup genesis block
if err == sql.ErrNoRows {
// TODO: Query rollup constants and genesis information, store them
nextBlockNum = 1234 // TODO: Replace this with genesisBlockNum
} else {
// Get the latest block we have in History DB from blockchain to detect a reorg
ethBlock, err := s.ethClient.EthBlockByNumber(context.Background(), lastSavedBlock.EthBlockNum)
@ -151,7 +143,7 @@ func (s *Synchronizer) Sync() error {
if ethBlock.Hash != lastSavedBlock.Hash {
// Reorg detected
log.Debugf("Reorg Detected...")
err := s.reorg(lastSavedBlock)
_, err := s.reorg(lastSavedBlock)
if err != nil {
return err
}
@ -161,6 +153,7 @@ func (s *Synchronizer) Sync() error {
return err
}
}
nextBlockNum = lastSavedBlock.EthBlockNum + 1
}
log.Debugf("Syncing...")
@ -171,28 +164,33 @@ func (s *Synchronizer) Sync() error {
return err
}
log.Debugf("Blocks to sync: %v (lastSavedBlock: %v, latestBlock: %v)", latestBlockNum-lastSavedBlock.EthBlockNum, lastSavedBlock.EthBlockNum, latestBlockNum)
log.Debugf("Blocks to sync: %v (firstBlockToSync: %v, latestBlock: %v)", latestBlockNum-nextBlockNum+1, nextBlockNum, latestBlockNum)
for lastSavedBlock.EthBlockNum < latestBlockNum {
ethBlock, err := s.ethClient.EthBlockByNumber(context.Background(), lastSavedBlock.EthBlockNum+1)
for nextBlockNum < latestBlockNum {
ethBlock, err := s.ethClient.EthBlockByNumber(context.Background(), nextBlockNum)
if err != nil {
return err
}
// TODO: Check that the obtianed ethBlock.ParentHash == prevEthBlock.Hash; if not, reorg!
// TODO: Send the ethHash in rollupSync(), auctionSync() and
// wdelayerSync() and make sure they all use the same block
// hash.
// Get data from the rollup contract
rollupData, err := s.rollupSync(ethBlock)
rollupData, err := s.rollupSync(nextBlockNum)
if err != nil {
return err
}
// Get data from the auction contract
auctionData, err := s.auctionSync(ethBlock)
auctionData, err := s.auctionSync(nextBlockNum)
if err != nil {
return err
}
// Get data from the WithdrawalDelayer contract
wdelayerData, err := s.wdelayerSync(ethBlock)
wdelayerData, err := s.wdelayerSync(nextBlockNum)
if err != nil {
return err
}
@ -205,7 +203,7 @@ func (s *Synchronizer) Sync() error {
if rollupData != nil {
blockData.l1Txs = rollupData.l1Txs
blockData.batches = rollupData.batches
// blockData.withdrawals = rollupData.withdrawals
// blockData.withdrawals = rollupData.withdrawals // TODO
blockData.registeredTokens = rollupData.registeredTokens
blockData.rollupVars = rollupData.rollupVars
}
@ -226,19 +224,17 @@ func (s *Synchronizer) Sync() error {
if err != nil {
return err
}
// We get the block on every iteration
lastSavedBlock, err = s.historyDB.GetLastBlock()
if err != nil {
return err
}
}
return nil
}
// reorg manages a reorg, updating History and State DB as needed
func (s *Synchronizer) reorg(uncleBlock *common.Block) error {
// reorg manages a reorg, updating History and State DB as needed. Keeps
// checking previous blocks from the HistoryDB against the blockchain until a
// block hash match is found. All future blocks in the HistoryDB and
// corresponding batches in StateBD are discarded. Returns the last valid
// blockNum from the HistoryDB.
func (s *Synchronizer) reorg(uncleBlock *common.Block) (int64, error) {
var block *common.Block
blockNum := uncleBlock.EthBlockNum
found := false
@ -249,12 +245,12 @@ func (s *Synchronizer) reorg(uncleBlock *common.Block) error {
for !found && blockNum > s.firstSavedBlock.EthBlockNum {
ethBlock, err := s.ethClient.EthBlockByNumber(context.Background(), blockNum)
if err != nil {
return err
return 0, err
}
block, err = s.historyDB.GetBlock(blockNum)
if err != nil {
return err
return 0, err
}
if block.Hash == ethBlock.Hash {
found = true
@ -270,24 +266,24 @@ func (s *Synchronizer) reorg(uncleBlock *common.Block) error {
// Set History DB and State DB to the correct state
err := s.historyDB.Reorg(block.EthBlockNum)
if err != nil {
return err
return 0, err
}
batchNum, err := s.historyDB.GetLastBatchNum()
if err != nil && err != sql.ErrNoRows {
return err
return 0, err
}
if batchNum != 0 {
err = s.stateDB.Reset(batchNum)
if err != nil {
return err
return 0, err
}
}
return nil
return block.EthBlockNum, nil
}
return ErrNotAbleToSync
return 0, ErrNotAbleToSync
}
// Status returns current status values from the Synchronizer
@ -298,6 +294,13 @@ func (s *Synchronizer) Status() (*common.SyncStatus, error) {
var status *common.SyncStatus
// TODO: Join all queries to the DB into a single transaction so that
// we can remove the mutex locking here:
// - HistoryDB.GetLastBlock
// - HistoryDB.GetLastBatchNum
// - HistoryDB.GetCurrentForgerAddr
// - HistoryDB.GetNextForgerAddr
// Get latest block in History DB
lastSavedBlock, err := s.historyDB.GetLastBlock()
if err != nil {
@ -318,7 +321,7 @@ func (s *Synchronizer) Status() (*common.SyncStatus, error) {
return nil, err
}
// TODO: Get CurrentForgerAddr & NextForgerAddr from the Auction SC
// TODO: Get CurrentForgerAddr & NextForgerAddr from the Auction SC / Or from the HistoryDB
// Check if Synchronizer is synchronized
status.Synchronized = status.CurrentBlock == latestBlockNum
@ -326,104 +329,105 @@ func (s *Synchronizer) Status() (*common.SyncStatus, error) {
}
// rollupSync gets information from the Rollup Contract
func (s *Synchronizer) rollupSync(block *common.Block) (*rollupData, error) {
func (s *Synchronizer) rollupSync(blockNum int64) (*rollupData, error) {
var rollupData = newRollupData()
var forgeL1TxsNum uint32
// var forgeL1TxsNum int64
var numAccounts int
// using GetLastL1TxsNum as GetNextL1TxsNum
lastStoredForgeL1TxsNum := uint32(0)
lastStoredForgeL1TxsNumPtr, err := s.historyDB.GetLastL1TxsNum()
// Get rollup events in the block
rollupEvents, _, err := s.ethClient.RollupEventsByBlock(blockNum)
if err != nil {
return nil, err
}
if lastStoredForgeL1TxsNumPtr != nil {
lastStoredForgeL1TxsNum = *lastStoredForgeL1TxsNumPtr + 1
}
// }
// Get rollup events in the block
rollupEvents, _, err := s.ethClient.RollupEventsByBlock(block.EthBlockNum)
// TODO: Replace GetLastL1TxsNum by GetNextL1TxsNum
nextForgeL1TxsNum := int64(0)
nextForgeL1TxsNumPtr, err := s.historyDB.GetLastL1TxsNum()
if err != nil {
return nil, err
}
if nextForgeL1TxsNumPtr != nil {
nextForgeL1TxsNum = *nextForgeL1TxsNumPtr + 1
}
// Get newLastIdx that will be used to complete the accounts
idx, err := s.getIdx(rollupEvents)
if err != nil {
return nil, err
}
// idx, err := s.getIdx(rollupEvents)
// if err != nil {
// return nil, err
// }
// Get L1UserTX
rollupData.l1Txs = s.getL1UserTx(rollupEvents.L1UserTx, block)
rollupData.l1Txs = getL1UserTx(rollupEvents.L1UserTx, blockNum)
// Get ForgeBatch events to get the L1CoordinatorTxs
for _, fbEvent := range rollupEvents.ForgeBatch {
batchData := NewBatchData()
// TODO: Get position from HistoryDB filtering by
// to_forge_l1_txs_num and batch_num and latest position, then add 1
position := 1
position := 0
// Get the input for each Tx
forgeBatchArgs, err := s.ethClient.RollupForgeBatchArgs(fbEvent.EthTxHash)
if err != nil {
return nil, err
}
// Check if this is a L1Bath to get L1 Tx from it
forgeL1TxsNum := int64(0)
// Check if this is a L1Batch to get L1 Tx from it
if forgeBatchArgs.L1Batch {
forgeL1TxsNum = nextForgeL1TxsNum
// Get L1 User Txs from History DB
// TODO: Get L1TX from HistoryDB filtered by toforgeL1txNum & fromidx = 0 and
// update batch number and add accounts to registeredAccounts updating idx
// l1UserTxs, err := s.historyDB.GetL1UserTxs(lastStoredForgeL1TxsNum)
// update batch number and add accounts to createdAccounts updating idx
// l1UserTxs, err := s.historyDB.GetL1UserTxs(nextForgeL1TxsNum)
// If HistoryDB doesn't have L1UserTxs at
// nextForgeL1TxsNum, check if they exist in
// rollupData.l1Txs. This could happen because in a
// block there could be multiple batches with L1Batch =
// true (although it's a very rare case). If the
// L1UserTxs are not in rollupData.l1Txs, use an empty
// array (this happens when the L1UserTxs queue is
// frozen but didn't store any tx).
l1UserTxs := []common.L1Tx{}
position = len(l1UserTxs)
// Get L1 Coordinator Txs
for _, l1CoordinatorTx := range forgeBatchArgs.L1CoordinatorTxs {
l1CoordinatorTx.Position = position
l1CoordinatorTx.ToForgeL1TxsNum = uint32(lastStoredForgeL1TxsNum)
l1CoordinatorTx.TxID = common.TxID(common.Hash([]byte("0x01" + strconv.FormatInt(int64(lastStoredForgeL1TxsNum), 10) + strconv.FormatInt(int64(l1CoordinatorTx.Position), 10) + "00")))
l1CoordinatorTx.ToForgeL1TxsNum = nextForgeL1TxsNum
l1CoordinatorTx.TxID = common.TxID(common.Hash([]byte("0x01" + strconv.FormatInt(int64(nextForgeL1TxsNum), 10) + strconv.FormatInt(int64(l1CoordinatorTx.Position), 10) + "00")))
l1CoordinatorTx.UserOrigin = false
l1CoordinatorTx.EthBlockNum = block.EthBlockNum
l1CoordinatorTx.EthBlockNum = blockNum
l1CoordinatorTx.BatchNum = common.BatchNum(fbEvent.BatchNum)
batchData.l1CoordinatorTxs = append(batchData.l1CoordinatorTxs, l1CoordinatorTx)
forgeL1TxsNum++
// Check if we have to register an account
if l1CoordinatorTx.FromIdx == 0 {
account := common.Account{
// TODO: Uncommnent when common.account has IDx
// IDx: common.Idx(idx),
TokenID: l1CoordinatorTx.TokenID,
Nonce: 0,
Balance: l1CoordinatorTx.LoadAmount,
PublicKey: l1CoordinatorTx.FromBJJ,
EthAddr: l1CoordinatorTx.FromEthAddr,
}
idx++
batchData.registeredAccounts = append(batchData.registeredAccounts, &account)
numAccounts++
}
// if l1CoordinatorTx.FromIdx == 0 {
// account := common.Account{
// // TODO: Uncommnent when common.account has IDx
// // IDx: common.Idx(idx),
// TokenID: l1CoordinatorTx.TokenID,
// Nonce: 0,
// Balance: l1CoordinatorTx.LoadAmount,
// PublicKey: l1CoordinatorTx.FromBJJ,
// EthAddr: l1CoordinatorTx.FromEthAddr,
// }
// idx++
// batchData.createdAccounts = append(batchData.createdAccounts, &account)
// numAccounts++
// }
position++
}
lastStoredForgeL1TxsNum++
nextForgeL1TxsNum++
}
// Get L2Txs
poolL2Txs := common.L2TxsToPoolL2Txs(forgeBatchArgs.L2Txs) // TODO: This is a big uggly, find a better way
// Get exitTree
// TODO: Get createdAccounts from ProcessTxs()
// TODO: Get CollectedFees from ProcessTxs()
// TODO: Pass forgeBatchArgs.FeeIdxCoordinator to ProcessTxs()
_, exitInfo, err := s.stateDB.ProcessTxs(true, false, batchData.l1UserTxs, batchData.l1CoordinatorTxs, poolL2Txs)
if err != nil {
return nil, err
@ -437,18 +441,16 @@ func (s *Synchronizer) rollupSync(block *common.Block) (*rollupData, error) {
// Get Batch information
batch := &common.Batch{
BatchNum: common.BatchNum(fbEvent.BatchNum),
EthBlockNum: block.EthBlockNum,
// ForgerAddr: , TODO: Get it from ethClient
EthBlockNum: blockNum,
// ForgerAddr: , TODO: Get it from ethClient -> Add ForgerAddr to RollupEventForgeBatch
// CollectedFees: , TODO: Clarify where to get them if they are still needed
StateRoot: common.Hash(forgeBatchArgs.NewStRoot.Bytes()),
NumAccounts: numAccounts,
ExitRoot: common.Hash(forgeBatchArgs.NewExitRoot.Bytes()),
ForgeL1TxsNum: forgeL1TxsNum,
// SlotNum: TODO: Calculate once ethClient provides the info
// SlotNum: TODO: Calculate once ethClient provides the info // calculate from blockNum + ethClient Constants
}
batchData.batch = batch
rollupData.batches = append(rollupData.batches, batchData)
}
@ -458,12 +460,16 @@ func (s *Synchronizer) rollupSync(block *common.Block) (*rollupData, error) {
token.TokenID = common.TokenID(eAddToken.TokenID)
token.EthAddr = eAddToken.Address
token.EthBlockNum = block.EthBlockNum
token.EthBlockNum = blockNum
// TODO: Add external information consulting SC about it using Address
rollupData.registeredTokens = append(rollupData.registeredTokens, token)
}
// TODO: rollupEvents.UpdateForgeL1L2BatchTimeout
// TODO: rollupEvents.UpdateFeeAddToken
// TODO: rollupEvents.WithdrawEvent
// TODO: Emergency Mechanism
// TODO: Variables
// TODO: Constants
@ -472,12 +478,11 @@ func (s *Synchronizer) rollupSync(block *common.Block) (*rollupData, error) {
}
// auctionSync gets information from the Auction Contract
func (s *Synchronizer) auctionSync(block *common.Block) (*auctionData, error) {
func (s *Synchronizer) auctionSync(blockNum int64) (*auctionData, error) {
var auctionData = newAuctionData()
// Get auction events in the block
auctionEvents, _, err := s.ethClient.AuctionEventsByBlock(block.EthBlockNum)
auctionEvents, _, err := s.ethClient.AuctionEventsByBlock(blockNum)
if err != nil {
return nil, err
}
@ -485,11 +490,10 @@ func (s *Synchronizer) auctionSync(block *common.Block) (*auctionData, error) {
// Get bids
for _, eNewBid := range auctionEvents.NewBid {
bid := &common.Bid{
SlotNum: common.SlotNum(eNewBid.Slot),
BidValue: eNewBid.BidAmount,
ForgerAddr: eNewBid.CoordinatorForger,
EthBlockNum: block.EthBlockNum,
EthBlockNum: blockNum,
}
auctionData.bids = append(auctionData.bids, bid)
}
@ -504,6 +508,20 @@ func (s *Synchronizer) auctionSync(block *common.Block) (*auctionData, error) {
auctionData.coordinators = append(auctionData.coordinators, coordinator)
}
// TODO: NewSlotDeadline
// TODO: NewClosedAuctionSlots
// TODO: NewOutbidding
// TODO: NewDonationAddress
// TODO: NewBootCoordinator
// TODO: NewOpenAuctionSlots
// TODO: NewAllocationRatio
// TODO: NewForgeAllocated
// TODO: NewDefaultSlotSetBid
// TODO: NewForge
// TODO: HEZClaimed
// TODO: Think about separating new coordinaors from coordinator updated
// Get Coordinators from updates
for _, eCoordinatorUpdated := range auctionEvents.CoordinatorUpdated {
coordinator := &common.Coordinator{
@ -521,36 +539,38 @@ func (s *Synchronizer) auctionSync(block *common.Block) (*auctionData, error) {
}
// wdelayerSync gets information from the Withdrawal Delayer Contract
func (s *Synchronizer) wdelayerSync(block *common.Block) (*common.WithdrawalDelayerVars, error) {
func (s *Synchronizer) wdelayerSync(blockNum int64) (*common.WithdrawalDelayerVars, error) {
// TODO: VARS
// TODO: CONSTANTS
return nil, nil
}
func (s *Synchronizer) getIdx(rollupEvents *eth.RollupEvents) (int64, error) {
lastForgeBatch := rollupEvents.ForgeBatch[len(rollupEvents.ForgeBatch)-1]
// Get the input for forgeBatch
forgeBatchArgs, err := s.ethClient.RollupForgeBatchArgs(lastForgeBatch.EthTxHash)
if err != nil {
return 0, err
}
return forgeBatchArgs.NewLastIdx + 1, nil
}
func (s *Synchronizer) getL1UserTx(l1UserTxEvents []eth.RollupEventL1UserTx, block *common.Block) []*common.L1Tx {
// func (s *Synchronizer) getIdx(rollupEvents *eth.RollupEvents) (int64, error) {
// // TODO: FIXME: There will be an error here when `len(rollupEvents.ForgeBatch) == 0`
// lastForgeBatch := rollupEvents.ForgeBatch[len(rollupEvents.ForgeBatch)-1]
//
// // TODO: RollupForgeBatchArgs is already called in `rollupSync`.
// // Ideally it should not need to be called twice for the same batch.
// // Get the input for forgeBatch
// forgeBatchArgs, err := s.ethClient.RollupForgeBatchArgs(lastForgeBatch.EthTxHash)
// if err != nil {
// return 0, err
// }
//
// return forgeBatchArgs.NewLastIdx + 1, nil
// }
func getL1UserTx(l1UserTxEvents []eth.RollupEventL1UserTx, blockNum int64) []*common.L1Tx {
l1Txs := make([]*common.L1Tx, 0)
for _, eL1UserTx := range l1UserTxEvents {
// Fill aditional Tx fields
eL1UserTx.L1Tx.TxID = common.TxID(common.Hash([]byte("0x00" + strconv.FormatInt(int64(eL1UserTx.ToForgeL1TxsNum), 10) + strconv.FormatInt(int64(eL1UserTx.Position), 10) + "00")))
eL1UserTx.L1Tx.ToForgeL1TxsNum = uint32(eL1UserTx.ToForgeL1TxsNum)
eL1UserTx.L1Tx.ToForgeL1TxsNum = eL1UserTx.ToForgeL1TxsNum
eL1UserTx.L1Tx.Position = eL1UserTx.Position
eL1UserTx.L1Tx.UserOrigin = true
eL1UserTx.L1Tx.EthBlockNum = block.EthBlockNum
eL1UserTx.L1Tx.EthBlockNum = blockNum
eL1UserTx.L1Tx.BatchNum = 0
l1Txs = append(l1Txs, &eL1UserTx.L1Tx)

+ 1
- 1
test/historydb.go

@ -71,7 +71,7 @@ func GenBatches(nBatches int, blocks []common.Block) []common.Batch {
SlotNum: common.SlotNum(i),
}
if i%2 == 0 {
batch.ForgeL1TxsNum = uint32(i)
batch.ForgeL1TxsNum = int64(i)
}
batches = append(batches, batch)
}

Loading…
Cancel
Save