@ -3,9 +3,8 @@ package synchronizer
import (
import (
"context"
"context"
"database/sql"
"database/sql"
"errors"
"sync"
"github.com/ethereum/go-ethereum"
"github.com/hermeznetwork/hermez-node/common"
"github.com/hermeznetwork/hermez-node/common"
"github.com/hermeznetwork/hermez-node/db/historydb"
"github.com/hermeznetwork/hermez-node/db/historydb"
"github.com/hermeznetwork/hermez-node/db/statedb"
"github.com/hermeznetwork/hermez-node/db/statedb"
@ -14,26 +13,46 @@ import (
)
)
var (
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" )
// 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")
)
)
// // SyncronizerState describes the synchronization progress of the smart contracts
// type SyncronizerState struct {
// LastUpdate time.Time // last time this information was updated
// CurrentBatchNum BatchNum // Last batch that was forged on the blockchain
// CurrentBlockNum uint64 // Last block that was mined on Ethereum
// CurrentToForgeL1TxsNum uint32
// LastSyncedBatchNum BatchNum // last batch synchronized by the coordinator
// LastSyncedBlockNum uint64 // last Ethereum block synchronized by the coordinator
// LastSyncedToForgeL1TxsNum uint32
// }
// // SyncStatus is returned by the Status method of the Synchronizer
// type SyncStatus struct {
// CurrentBlock int64
// CurrentBatch BatchNum
// CurrentForgerAddr ethCommon.Address
// NextForgerAddr ethCommon.Address
// Synchronized bool
// }
// rollupData contains information returned by the Rollup SC
// rollupData contains information returned by the Rollup SC
type rollupData struct {
type rollupData struct {
l1UserTxs [ ] common . L1Tx
l1UserTxs [ ] common . L1Tx
batches [ ] historydb . BatchData
batches [ ] common . BatchData
// withdrawals []*common.ExitInfo
// withdrawals []*common.ExitInfo
registeredTokens [ ] common . Token
vars * common . RollupVars
ad dTokens [ ] common . Token
vars * common . RollupVars
}
}
// NewRollupData creates an empty rollupData with the slices initialized.
// NewRollupData creates an empty rollupData with the slices initialized.
func newRollupData ( ) rollupData {
func newRollupData ( ) rollupData {
return rollupData {
return rollupData {
l1UserTxs : make ( [ ] common . L1Tx , 0 ) ,
l1UserTxs : make ( [ ] common . L1Tx , 0 ) ,
batches : make ( [ ] historydb . BatchData , 0 ) ,
batches : make ( [ ] common . BatchData , 0 ) ,
// withdrawals: make([]*common.ExitInfo, 0),
// withdrawals: make([]*common.ExitInfo, 0),
registeredTokens : make ( [ ] common . Token , 0 ) ,
ad dTokens: make ( [ ] common . Token , 0 ) ,
}
}
}
}
@ -56,58 +75,21 @@ type wdelayerData struct {
vars * common . WithdrawDelayerVars
vars * common . WithdrawDelayerVars
}
}
// BatchData contains information about Batches from the contracts
// type BatchData struct {
// 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),
// createdAccounts: make([]*common.Account, 0),
// exitTree: make([]*common.ExitInfo, 0),
// }
// }
// BlockData contains information about Blocks from the contracts
// type blockData struct {
// Block *common.Block
// // Rollup
// 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
// Bids []*common.Bid
// Coordinators []*common.Coordinator
// AuctionVars *common.AuctionVars
// // WithdrawalDelayer
// WithdrawalDelayerVars *common.WithdrawalDelayerVars
// }
// Synchronizer implements the Synchronizer type
// Synchronizer implements the Synchronizer type
type Synchronizer struct {
type Synchronizer struct {
ethClient eth . ClientInterface
ethClient eth . ClientInterface
auctionConstants eth . AuctionConstants
auctionConstants eth . AuctionConstants
historyDB * historydb . HistoryDB
historyDB * historydb . HistoryDB
stateDB * statedb . StateDB
stateDB * statedb . StateDB
firstSavedBlock * common . Block
mux sync . Mutex
// firstSavedBlock *common.Block
// mux sync.Mutex
}
}
// NewSynchronizer creates a new Synchronizer
// NewSynchronizer creates a new Synchronizer
func NewSynchronizer ( ethClient eth . ClientInterface , historyDB * historydb . HistoryDB , stateDB * statedb . StateDB ) ( * Synchronizer , error ) {
func NewSynchronizer ( ethClient eth . ClientInterface , historyDB * historydb . HistoryDB , stateDB * statedb . StateDB ) ( * Synchronizer , error ) {
auctionConstants , err := ethClient . AuctionConstants ( )
auctionConstants , err := ethClient . AuctionConstants ( )
if err != nil {
if err != nil {
log . Errorw ( "NewSynchronizer" , "err" , err )
return nil , err
return nil , err
}
}
return & Synchronizer {
return & Synchronizer {
@ -118,128 +100,101 @@ func NewSynchronizer(ethClient eth.ClientInterface, historyDB *historydb.History
} , nil
} , nil
}
}
// Sync2 attems to synchronize an ethereum block starting from lastSavedBlock.
// If lastSavedBlock is nil, the lastSavedBlock value is obtained from de DB.
// If a block is synched, it will be returned and also stored in the DB. If a
// reorg is detected, the number of discarded blocks will be returned and no
// synchronization will be made.
// TODO: Be smart about locking: only lock during the read/write operations
// 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 ( ctx context . Context ) error {
// Avoid new sync while performing one
s . mux . Lock ( )
defer s . mux . Unlock ( )
func ( s * Synchronizer ) Sync2 ( ctx context . Context , lastSavedBlock * common . Block ) ( * common . BlockData , * int64 , error ) {
var nextBlockNum int64 // next block number to sync
var nextBlockNum int64 // next block number to sync
if lastSavedBlock == nil {
var err error
// Get lastSavedBlock from History DB
lastSavedBlock , err = s . historyDB . GetLastBlock ( )
if err != nil && err != sql . ErrNoRows {
return nil , nil , err
}
// If we don't have any stored block, we must do a full sync starting from the rollup genesis block
if err == sql . ErrNoRows {
nextBlockNum = s . auctionConstants . GenesisBlockNum
}
}
if lastSavedBlock != nil {
nextBlockNum = lastSavedBlock . EthBlockNum + 1
}
// Get lastSavedBlock from History DB
lastSavedBlock , err := s . historyDB . GetLastBlock ( )
if err != nil && err != sql . ErrNoRows {
return err
ethBlock , err := s . ethClient . EthBlockByNumber ( ctx , nextBlockNum )
if err == ethereum . NotFound {
return nil , nil , nil
} else if err != nil {
return nil , nil , err
}
}
// If we don't have any stored block, we must do a full sync starting from the rollup genesis block
if err == sql . ErrNoRows {
nextBlockNum = s . auctionConstants . GenesisBlockNum
} else {
// Get the latest block we have in History DB from blockchain to detect a reorg
ethBlock , err := s . ethClient . EthBlockByNumber ( ctx , lastSavedBlock . EthBlockNum )
if err != nil {
return err
}
if ethBlock . Hash != lastSavedBlock . Hash {
// Reorg detected
log . Debugf ( "Reorg Detected..." )
_ , err := s . reorg ( lastSavedBlock )
if err != nil {
return err
}
log . Debugw ( "Syncing..." , "block" , nextBlockNum )
lastSavedBlock , err = s . historyDB . GetLastBlock ( )
// Check that the obtianed ethBlock.ParentHash == prevEthBlock.Hash; if not, reorg!
if lastSavedBlock != nil {
if lastSavedBlock . Hash != ethBlock . ParentHash {
// Reorg detected
log . Debugw ( "Reorg Detected" ,
"blockNum" , ethBlock . EthBlockNum ,
"block.parent" , ethBlock . ParentHash , "parent.hash" , lastSavedBlock . Hash )
lastDBBlockNum , err := s . reorg ( lastSavedBlock )
if err != nil {
if err != nil {
return err
return nil , nil , err
}
}
discarded := lastSavedBlock . EthBlockNum - lastDBBlockNum
return nil , & discarded , nil
}
}
nextBlockNum = lastSavedBlock . EthBlockNum + 1
}
}
log . Debugf ( "Syncing..." )
// Get latest blockNum in blockchain
latestBlockNum , err := s . ethClient . EthCurrentBlock ( )
// Get data from the rollup contract
rollupData , err := s . rollupSync ( ethBlock )
if err != nil {
if err != nil {
return err
return nil , nil , err
}
}
log . Debugf ( "Blocks to sync: %v (firstBlockToSync: %v, latestBlock: %v)" , latestBlockNum - nextBlockNum + 1 , nextBlockNum , latestBlockNum )
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 ( nextBlockNum )
if err != nil {
return err
}
// Get data from the auction contract
auctionData , err := s . auctionSync ( nextBlockNum )
if err != nil {
return err
}
// Get data from the auction contract
auctionData , err := s . auctionSync ( ethBlock )
if err != nil {
return nil , nil , err
}
// Get data from the WithdrawalDelayer contract
wdelayerData , err := s . wdelayerSync ( n ex tBlockNum )
if err != nil {
return err
}
// Get data from the WithdrawalDelayer contract
wdelayerData , err := s . wdelayerSync ( ethBlock )
if err != nil {
return nil , nil , err
}
// Group all the block data into the structs to save into HistoryDB
var blockData historydb . BlockData
// Group all the block data into the structs to save into HistoryDB
var blockData common . BlockData
blockData . Block = ethBlock
blockData . Block = * ethBlock
if rollupData != nil {
blockData . L1UserTxs = rollupData . l1UserTxs
blockData . Batches = rollupData . batches
// blockData.withdrawals = rollupData.withdrawals // TODO
blockData . RegisteredTokens = rollupData . registeredTokens
blockData . RollupVars = rollupData . vars
}
blockData . L1UserTxs = rollupData . l1UserTxs
blockData . Batches = rollupData . batches
// blockData.withdrawals = rollupData.withdrawals // TODO
blockData . AddedTokens = rollupData . addTokens
blockData . RollupVars = rollupData . vars
if auctionData != nil {
blockData . Bids = auctionData . bids
blockData . Coordinators = auctionData . coordinators
blockData . AuctionVars = auctionData . vars
}
blockData . Bids = auctionData . bids
blockData . Coordinators = auctionData . coordinators
blockData . AuctionVars = auctionData . vars
if wdelayerData != nil {
blockData . WithdrawDelayerVars = wdelayerData . vars
}
blockData . WithdrawDelayerVars = wdelayerData . vars
// Add rollupData and auctionData once the method is updated
// TODO: Save Whole Struct -> AddBlockSCData(blockData)
log . Debugw ( "Sync()" , "block" , blockData )
// err = s.historyDB.AddBlock(blockData.Block)
// if err != nil {
// return err
// }
err = s . historyDB . AddBlockSCData ( & blockData )
if err != nil {
return err
}
nextBlockNum ++
// log.Debugw("Sync()", "block", blockData)
// err = s.historyDB.AddBlock(blockData.Block)
// if err != nil {
// return err
// }
err = s . historyDB . AddBlockSCData ( & blockData )
if err != nil {
return nil , nil , err
}
}
return nil
return & blockData , nil , nil
}
}
// reorg manages a reorg, updating History and State DB as needed. Keeps
// reorg manages a reorg, updating History and State DB as needed. Keeps
@ -250,12 +205,8 @@ func (s *Synchronizer) Sync(ctx context.Context) error {
func ( s * Synchronizer ) reorg ( uncleBlock * common . Block ) ( int64 , error ) {
func ( s * Synchronizer ) reorg ( uncleBlock * common . Block ) ( int64 , error ) {
var block * common . Block
var block * common . Block
blockNum := uncleBlock . EthBlockNum
blockNum := uncleBlock . EthBlockNum
found := false
log . Debugf ( "Reorg first uncle block: %v" , blockNum )
// Iterate History DB and the blokchain looking for the latest valid block
for ! found && blockNum > s . firstSavedBlock . EthBlockNum {
for blockNum >= s . auctionConstants . GenesisBlockNum {
ethBlock , err := s . ethClient . EthBlockByNumber ( context . Background ( ) , blockNum )
ethBlock , err := s . ethClient . EthBlockByNumber ( context . Background ( ) , blockNum )
if err != nil {
if err != nil {
return 0 , err
return 0 , err
@ -266,39 +217,36 @@ func (s *Synchronizer) reorg(uncleBlock *common.Block) (int64, error) {
return 0 , err
return 0 , err
}
}
if block . Hash == ethBlock . Hash {
if block . Hash == ethBlock . Hash {
found = true
log . Debugf ( "Found valid block: %v" , blockNum )
log . Debugf ( "Found valid block: %v" , blockNum )
} else {
log . Debugf ( "Discarding block: %v" , blockNum )
break
}
}
blockNum --
blockNum --
}
}
total := uncleBlock . EthBlockNum - blockNum
log . Debugw ( "Discarding blocks" , "total" , total , "from" , uncleBlock . EthBlockNum , "to" , blockNum + 1 )
if found {
// Set History DB and State DB to the correct state
err := s . historyDB . Reorg ( block . EthBlockNum )
if err != nil {
return 0 , err
}
// Set History DB and State DB to the correct state
err := s . historyDB . Reorg ( block . EthBlockNum )
if err != nil {
return 0 , err
}
batchNum , err := s . historyDB . GetLastBatchNum ( )
if err != nil && err != sql . ErrNoRows {
batchNum , err := s . historyDB . GetLastBatchNum ( )
if err != nil && err != sql . ErrNoRows {
return 0 , err
}
if batchNum != 0 {
err = s . stateDB . Reset ( batchNum )
if err != nil {
return 0 , err
return 0 , err
}
}
if batchNum != 0 {
err = s . stateDB . Reset ( batchNum )
if err != nil {
return 0 , err
}
}
return block . EthBlockNum , nil
}
}
return 0 , ErrNotAbleToSync
return block . EthBlockNum , nil
}
}
// TODO: Figure out who will use the Status output, and only return what's strictly need
/ *
// Status returns current status values from the Synchronizer
// Status returns current status values from the Synchronizer
func ( s * Synchronizer ) Status ( ) ( * common . SyncStatus , error ) {
func ( s * Synchronizer ) Status ( ) ( * common . SyncStatus , error ) {
// Avoid possible inconsistencies
// Avoid possible inconsistencies
@ -340,21 +288,27 @@ func (s *Synchronizer) Status() (*common.SyncStatus, error) {
status . Synchronized = status . CurrentBlock == latestBlockNum
status . Synchronized = status . CurrentBlock == latestBlockNum
return status , nil
return status , nil
}
}
* /
// rollupSync gets information from the Rollup Contract
func ( s * Synchronizer ) rollupSync ( blockNum int64 ) ( * rollupData , error ) {
// rollupSync retreives all the Rollup Smart Contract Data that happened at
// ethBlock.blockNum with ethBlock.Hash.
func ( s * Synchronizer ) rollupSync ( ethBlock * common . Block ) ( * rollupData , error ) {
blockNum := ethBlock . EthBlockNum
var rollupData = newRollupData ( )
var rollupData = newRollupData ( )
// var forgeL1TxsNum int64
// var forgeL1TxsNum int64
var numAccounts int
var numAccounts int
// Get rollup events in the block
rollupEvents , _ , err := s . ethClient . RollupEventsByBlock ( blockNum )
// Get rollup events in the block, and make sure the block hash matches
// the expected one.
rollupEvents , blockHash , err := s . ethClient . RollupEventsByBlock ( blockNum )
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
if * blockHash != ethBlock . Hash {
return nil , eth . ErrBlockHashMismatchEvent
}
// TODO: Replace GetLastL1TxsNum by GetNextL1TxsNum
var nextForgeL1TxsNum int64
var nextForgeL1TxsNum int64 // forgeL1TxsNum for the next L1Batch
nextForgeL1TxsNumPtr , err := s . historyDB . GetLastL1TxsNum ( )
nextForgeL1TxsNumPtr , err := s . historyDB . GetLastL1TxsNum ( )
if err != nil {
if err != nil {
return nil , err
return nil , err
@ -379,7 +333,7 @@ func (s *Synchronizer) rollupSync(blockNum int64) (*rollupData, error) {
// Get ForgeBatch events to get the L1CoordinatorTxs
// Get ForgeBatch events to get the L1CoordinatorTxs
for _ , evtForgeBatch := range rollupEvents . ForgeBatch {
for _ , evtForgeBatch := range rollupEvents . ForgeBatch {
batchData := historydb . NewBatchData ( )
batchData := common . NewBatchData ( )
position := 0
position := 0
// Get the input for each Tx
// Get the input for each Tx
@ -387,70 +341,70 @@ func (s *Synchronizer) rollupSync(blockNum int64) (*rollupData, error) {
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
batchNum := common . BatchNum ( evtForgeBatch . BatchNum )
forgeL1TxsNum := nextForgeL1TxsNum
forgeL1TxsNum := nextForgeL1TxsNum
var l1UserTxs [ ] common . L1Tx
// Check if this is a L1Batch to get L1 Tx from it
// Check if this is a L1Batch to get L1 Tx from it
if forgeBatchArgs . L1Batch {
if forgeBatchArgs . L1Batch {
// 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 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 { }
// Get L1UserTxs with toForgeL1TxsNum, which correspond
// to the L1UserTxs that are forged in this batch, so
// that stateDB can process them.
// First try to find them in HistoryDB.
l1UserTxs , err := s . historyDB . GetL1UserTxs ( forgeL1TxsNum )
if len ( l1UserTxs ) == 0 {
// If not found in the DB, try to find them in
// this block. This could happen because in a
// block there could be multiple batches with
// L1Batch = true (although it's a very rare
// case).
// If not found in the DB and the block doesn't
// contain the l1UserTxs, it means that the
// L1UserTxs queue with toForgeL1TxsNum was
// closed empty, so we leave `l1UserTxs` as an
// empty slice.
for _ , l1UserTx := range rollupData . l1UserTxs {
if * l1UserTx . ToForgeL1TxsNum == forgeL1TxsNum {
l1UserTxs = append ( l1UserTxs , l1UserTx )
}
}
}
if err != nil {
return nil , err
}
position = len ( l1UserTxs )
position = len ( l1UserTxs )
// Get L1 Coordinator Txs
// Get L1 Coordinator Txs
for i := 0 ; i < len ( forgeBatchArgs . L1CoordinatorTxs ) ; i ++ {
for i := range forgeBatchArgs . L1CoordinatorTxs {
l1CoordinatorTx := forgeBatchArgs . L1CoordinatorTxs [ i ]
l1CoordinatorTx := forgeBatchArgs . L1CoordinatorTxs [ i ]
l1CoordinatorTx . Position = position
l1CoordinatorTx . Position = position
l1CoordinatorTx . ToForgeL1TxsNum = & forgeL1TxsNum
l1CoordinatorTx . ToForgeL1TxsNum = & forgeL1TxsNum
l1CoordinatorTx . UserOrigin = false
l1CoordinatorTx . UserOrigin = false
l1CoordinatorTx . EthBlockNum = blockNum
l1CoordinatorTx . EthBlockNum = blockNum
bn := new ( common . BatchNum )
* bn = common . BatchNum ( evtForgeBatch . BatchNum )
l1CoordinatorTx . BatchNum = bn
l1CoordinatorTx . BatchNum = & batchNum
l1Tx , err := common . NewL1Tx ( & l1CoordinatorTx )
l1Tx , err := common . NewL1Tx ( & l1CoordinatorTx )
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
batchData . L1CoordinatorTxs = append ( batchData . L1CoordinatorTxs , * l1Tx )
batchData . L1CoordinatorTxs = append ( batchData . L1CoordinatorTxs , * l1Tx )
// 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.createdAccounts = append(batchData.createdAccounts, &account)
// numAccounts++
// }
position ++
position ++
}
}
nextForgeL1TxsNum ++
nextForgeL1TxsNum ++
}
}
// Get L2Txs
poolL2Txs := common . L2TxsToPoolL2Txs ( forgeBatchArgs . L2TxsData ) // TODO: This is a big uggly, find a better way
// Insert all the txs forged in this batch (l1UserTxs,
// L1CoordinatorTxs, PoolL2Txs) into stateDB so that they are
// processed.
poolL2Txs := common . L2TxsToPoolL2Txs ( forgeBatchArgs . L2TxsData ) // TODO: This is a big ugly, find a better way
// Get exitTree
// TODO: Get createdAccounts from ProcessTxs()
// TODO: Get createdAccounts from ProcessTxs()
// TODO: Get CollectedFees from ProcessTxs()
// TODO: Get CollectedFees from ProcessTxs()
// TODO: Pass forgeBatchArgs.FeeIdxCoordinator to ProcessTxs()
// TODO: Pass forgeBatchArgs.FeeIdxCoordinator to ProcessTxs()
_ , exitInfo , err := s . stateDB . ProcessTxs ( batchData . L1UserTxs , batchData . L1CoordinatorTxs , poolL2Txs )
// ProcessTxs updates poolL2Txs adding: Nonce, TokenID
_ , exitInfo , err := s . stateDB . ProcessTxs ( l1UserTxs , batchData . L1CoordinatorTxs , poolL2Txs )
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
@ -459,21 +413,42 @@ func (s *Synchronizer) rollupSync(blockNum int64) (*rollupData, error) {
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
batchData . L2Txs = append ( batchData . L2Txs , l2Txs ... )
for i := range l2Txs {
_l2Tx := l2Txs [ i ]
_l2Tx . Position = position
_l2Tx . EthBlockNum = blockNum
_l2Tx . BatchNum = batchNum
l2Tx , err := common . NewL2Tx ( & _l2Tx )
if err != nil {
return nil , err
}
batchData . L2Txs = append ( batchData . L2Txs , * l2Tx )
position ++
}
batchData . ExitTree = exitInfo
batchData . ExitTree = exitInfo
slotNum := int64 ( 0 )
if ethBlock . EthBlockNum >= s . auctionConstants . GenesisBlockNum {
slotNum = ( ethBlock . EthBlockNum - s . auctionConstants . GenesisBlockNum ) /
int64 ( s . auctionConstants . BlocksPerSlot )
}
// Get Batch information
// Get Batch information
batch := & common . Batch {
BatchNum : common . BatchNum ( evtForgeBatch . BatchNum ) ,
batch := common . Batch {
BatchNum : batchNum ,
EthBlockNum : blockNum ,
EthBlockNum : blockNum ,
ForgerAddr : * sender ,
ForgerAddr : * sender ,
// CollectedFees: , TODO: Clarify where to get them if they are still needed
// 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 // calculate from blockNum + ethClient Constants
StateRoot : forgeBatchArgs . NewStRoot ,
NumAccounts : numAccounts ,
ExitRoot : forgeBatchArgs . NewExitRoot ,
SlotNum : slotNum ,
}
if forgeBatchArgs . L1Batch {
batch . ForgeL1TxsNum = & forgeL1TxsNum
}
}
batchData . Batch = batch
batchData . Batch = batch
rollupData . batches = append ( rollupData . batches , * batchData )
rollupData . batches = append ( rollupData . batches , * batchData )
@ -487,12 +462,19 @@ func (s *Synchronizer) rollupSync(blockNum int64) (*rollupData, error) {
token . EthAddr = evtAddToken . TokenAddress
token . EthAddr = evtAddToken . TokenAddress
token . EthBlockNum = blockNum
token . EthBlockNum = blockNum
// TODO: Add external information consulting SC about it using Address
token . Name = "TODO"
token . Symbol = "TODO"
token . Decimals = 8 // TODO
if consts , err := s . ethClient . EthERC20Consts ( evtAddToken . TokenAddress ) ; err != nil {
log . Warnw ( "Error retreiving ERC20 token constants" , "addr" , evtAddToken . TokenAddress )
// TODO: Add external information consulting SC about it using Address
token . Name = "ERC20_ETH_ERROR"
token . Symbol = "ERROR"
token . Decimals = 1
} else {
token . Name = cutStringMax ( consts . Name , 20 )
token . Symbol = cutStringMax ( consts . Symbol , 10 )
token . Decimals = consts . Decimals
}
rollupData . registeredTokens = append ( rollupData . registeredTokens , token )
rollupData . ad dTokens = append ( rollupData . ad dTokens, token )
}
}
// TODO: rollupEvents.UpdateForgeL1L2BatchTimeout
// TODO: rollupEvents.UpdateForgeL1L2BatchTimeout
@ -506,20 +488,31 @@ func (s *Synchronizer) rollupSync(blockNum int64) (*rollupData, error) {
return & rollupData , nil
return & rollupData , nil
}
}
func cutStringMax ( s string , max int ) string {
if len ( s ) > max {
return s [ : max ]
}
return s
}
// auctionSync gets information from the Auction Contract
// auctionSync gets information from the Auction Contract
func ( s * Synchronizer ) auctionSync ( blockNum int64 ) ( * auctionData , error ) {
func ( s * Synchronizer ) auctionSync ( ethBlock * common . Block ) ( * auctionData , error ) {
blockNum := ethBlock . EthBlockNum
var auctionData = newAuctionData ( )
var auctionData = newAuctionData ( )
// Get auction events in the block
// Get auction events in the block
auctionEvents , _ , err := s . ethClient . AuctionEventsByBlock ( blockNum )
auctionEvents , blockHash , err := s . ethClient . AuctionEventsByBlock ( blockNum )
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
if * blockHash != ethBlock . Hash {
return nil , eth . ErrBlockHashMismatchEvent
}
// Get bids
// Get bids
for _ , evtNewBid := range auctionEvents . NewBid {
for _ , evtNewBid := range auctionEvents . NewBid {
bid := common . Bid {
bid := common . Bid {
SlotNum : common . SlotNum ( evtNewBid . Slot ) ,
SlotNum : evtNewBid . Slot ,
BidValue : evtNewBid . BidAmount ,
BidValue : evtNewBid . BidAmount ,
Bidder : evtNewBid . Bidder ,
Bidder : evtNewBid . Bidder ,
EthBlockNum : blockNum ,
EthBlockNum : blockNum ,
@ -556,11 +549,14 @@ func (s *Synchronizer) auctionSync(blockNum int64) (*auctionData, error) {
}
}
// wdelayerSync gets information from the Withdrawal Delayer Contract
// wdelayerSync gets information from the Withdrawal Delayer Contract
func ( s * Synchronizer ) wdelayerSync ( blockNum int64 ) ( * wdelayerData , error ) {
func ( s * Synchronizer ) wdelayerSync ( ethBlock * common . Block ) ( * wdelayerData , error ) {
// blockNum := ethBlock.EthBlockNum
// TODO: VARS
// TODO: VARS
// TODO: CONSTANTS
// TODO: CONSTANTS
return nil , nil
return & wdelayerData {
vars : nil ,
} , nil
}
}
// func (s *Synchronizer) getIdx(rollupEvents *eth.RollupEvents) (int64, error) {
// func (s *Synchronizer) getIdx(rollupEvents *eth.RollupEvents) (int64, error) {
@ -579,17 +575,15 @@ func (s *Synchronizer) wdelayerSync(blockNum int64) (*wdelayerData, error) {
// }
// }
func getL1UserTx ( eventsL1UserTx [ ] eth . RollupEventL1UserTx , blockNum int64 ) ( [ ] common . L1Tx , error ) {
func getL1UserTx ( eventsL1UserTx [ ] eth . RollupEventL1UserTx , blockNum int64 ) ( [ ] common . L1Tx , error ) {
l1Txs := make ( [ ] common . L1Tx , 0 )
for _ , evtL1UserTx := range eventsL1UserTx {
evtL1UserTx . L1UserTx . EthBlockNum = blockNum
nL 1Tx, err := common . NewL1Tx ( & evtL1UserTx . L1UserTx )
l1Txs := make ( [ ] common . L1Tx , len ( eventsL1UserTx ) )
for i := range eventsL1UserTx {
eventsL1UserTx [ i ] . L1UserTx . EthBlockNum = blockNum
// Check validity of L1UserTx
l 1Tx, err := common . NewL1Tx ( & even ts L1UserTx [ i ] . L1UserTx )
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
evtL1UserTx . L1UserTx = * nL1Tx
l1Txs = append ( l1Txs , evtL1UserTx . L1UserTx )
l1Txs [ i ] = * l1Tx
}
}
return l1Txs , nil
return l1Txs , nil
}
}