diff --git a/common/batch.go b/common/batch.go index 9a43503..e8baf01 100644 --- a/common/batch.go +++ b/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 } diff --git a/common/l1tx.go b/common/l1tx.go index 3c674e8..6f5d4a9 100644 --- a/common/l1tx.go +++ b/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) diff --git a/common/scvars.go b/common/scvars.go index bf7f537..8bc4eea 100644 --- a/common/scvars.go +++ b/common/scvars.go @@ -2,6 +2,7 @@ package common import ( "math/big" + "time" eth "github.com/ethereum/go-ethereum/common" ) @@ -30,6 +31,16 @@ type AuctionVars struct { AllocationRatio AllocationRatio } +// WithdrawalDelayerVars contains the Withdrawal Delayer smart contract variables +type WithdrawalDelayerVars struct { + HermezRollupAddress eth.Address + HermezGovernanceDAOAddress eth.Address + WhiteHackGroupAddress eth.Address + WithdrawalDelay uint + EmergencyModeStartingTime time.Time + EmergencyModeEnabled bool +} + // MinBidSlots TODO type MinBidSlots [6]uint diff --git a/common/syncstatus.go b/common/syncstatus.go new file mode 100644 index 0000000..de65b94 --- /dev/null +++ b/common/syncstatus.go @@ -0,0 +1,12 @@ +package common + +import ethCommon "github.com/ethereum/go-ethereum/common" + +// 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 +} diff --git a/common/tx.go b/common/tx.go index da324c8..440ba07 100644 --- a/common/tx.go +++ b/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"` diff --git a/db/historydb/historydb.go b/db/historydb/historydb.go index 73eea17..60f24bf 100644 --- a/db/historydb/historydb.go +++ b/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 @@ -127,10 +129,11 @@ func (hdb *HistoryDB) GetLastBatchNum() (common.BatchNum, error) { return batchNum, row.Scan(&batchNum) } -// GetLastL1TxsNum returns the greatest ForgeL1TxsNum in the DB -func (hdb *HistoryDB) GetLastL1TxsNum() (uint32, 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() (*int64, error) { row := hdb.db.QueryRow("SELECT MAX(forge_l1_txs_num) FROM batch;") - var lastL1TxsNum uint32 + lastL1TxsNum := new(int64) return lastL1TxsNum, row.Scan(&lastL1TxsNum) } diff --git a/db/historydb/historydb_test.go b/db/historydb/historydb_test.go index 3a4c3f7..878cc2c 100644 --- a/db/historydb/historydb_test.go +++ b/db/historydb/historydb_test.go @@ -89,8 +89,12 @@ func TestBatches(t *testing.T) { // Generate fake batches const nBatches = 9 batches := test.GenBatches(nBatches, blocks) + // Test GetLastL1TxsNum with no batches + fetchedLastL1TxsNum, err := historyDB.GetLastL1TxsNum() + assert.NoError(t, err) + assert.Nil(t, fetchedLastL1TxsNum) // Add batches to the DB - err := historyDB.AddBatches(batches) + err = historyDB.AddBatches(batches) assert.NoError(t, err) // Get batches from the DB fetchedBatches, err := historyDB.GetBatches(0, common.BatchNum(nBatches)) @@ -103,9 +107,9 @@ func TestBatches(t *testing.T) { assert.NoError(t, err) assert.Equal(t, batches[len(batches)-1].BatchNum, fetchedLastBatchNum) // Test GetLastL1TxsNum - fetchedLastL1TxsNum, err := historyDB.GetLastL1TxsNum() + fetchedLastL1TxsNum, err = historyDB.GetLastL1TxsNum() assert.NoError(t, err) - assert.Equal(t, batches[nBatches-1].ForgeL1TxsNum, fetchedLastL1TxsNum) + assert.Equal(t, batches[nBatches-1].ForgeL1TxsNum, *fetchedLastL1TxsNum) } func TestBids(t *testing.T) { diff --git a/db/l2db/l2db.go b/db/l2db/l2db.go index 6af711c..7d0d5f5 100644 --- a/db/l2db/l2db.go +++ b/db/l2db/l2db.go @@ -16,6 +16,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 { diff --git a/db/statedb/statedb.go b/db/statedb/statedb.go index 008e8b0..8c58561 100644 --- a/db/statedb/statedb.go +++ b/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") diff --git a/eth/rollup.go b/eth/rollup.go index 61336c0..954b1e9 100644 --- a/eth/rollup.go +++ b/eth/rollup.go @@ -91,9 +91,9 @@ type RollupState struct { // RollupEventL1UserTx is an event of the Rollup Smart Contract type RollupEventL1UserTx struct { - L1Tx common.L1Tx - QueueIndex *big.Int - TransactionIndex *big.Int + L1Tx common.L1Tx + ToForgeL1TxsNum int64 // QueueIndex *big.Int + Position int // TransactionIndex *big.Int } // RollupEventAddToken is an event of the Rollup Smart Contract diff --git a/synchronizer/synchronizer.go b/synchronizer/synchronizer.go index b297f65..5bd63b7 100644 --- a/synchronizer/synchronizer.go +++ b/synchronizer/synchronizer.go @@ -4,9 +4,9 @@ import ( "context" "database/sql" "errors" + "strconv" "sync" - ethCommon "github.com/ethereum/go-ethereum/common" "github.com/hermeznetwork/hermez-node/common" "github.com/hermeznetwork/hermez-node/db/historydb" "github.com/hermeznetwork/hermez-node/db/statedb" @@ -14,46 +14,81 @@ 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") ) +// rollupData contains information returned by the Rollup SC +type rollupData struct { + l1Txs []*common.L1Tx + batches []*BatchData + // withdrawals []*common.ExitInfo + registeredTokens []*common.Token + rollupVars *common.RollupVars +} + +// NewRollupData creates an empty rollupData with the slices initialized. +func newRollupData() rollupData { + return rollupData{ + l1Txs: make([]*common.L1Tx, 0), + batches: make([]*BatchData, 0), + // withdrawals: make([]*common.ExitInfo, 0), + registeredTokens: make([]*common.Token, 0), + } +} + +// auctionData contains information returned by the Action SC +type auctionData struct { + bids []*common.Bid + coordinators []*common.Coordinator + auctionVars *common.AuctionVars +} + +// newAuctionData creates an empty auctionData with the slices initialized. +func newAuctionData() *auctionData { + return &auctionData{ + bids: make([]*common.Bid, 0), + coordinators: make([]*common.Coordinator, 0), + } +} + // BatchData contains information about Batches from the contracts -//nolint:structcheck,unused type BatchData struct { - l1txs []common.L1Tx - l2txs []common.L2Tx - registeredAccounts []common.Account - exitTree []common.ExitInfo + 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 -//nolint:structcheck,unused type BlockData struct { block *common.Block // Rollup - batches []BatchData - withdrawals []common.ExitInfo - registeredTokens []common.Token + 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 + bids []*common.Bid + coordinators []*common.Coordinator auctionVars *common.AuctionVars -} - -// Status is returned by the Status method -type Status struct { - CurrentBlock int64 - CurrentBatch common.BatchNum - CurrentForgerAddr ethCommon.Address - NextForgerAddr ethCommon.Address - Synchronized bool + // WithdrawalDelayer + withdrawalDelayerVars *common.WithdrawalDelayerVars } // Synchronizer implements the Synchronizer type @@ -75,35 +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() - var lastStoredForgeL1TxsNum int64 - - // 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) @@ -114,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 } @@ -124,6 +153,7 @@ func (s *Synchronizer) Sync() error { return err } } + nextBlockNum = lastSavedBlock.EthBlockNum + 1 } log.Debugf("Syncing...") @@ -134,34 +164,63 @@ 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 - blockData, batchData, err := s.rollupSync(ethBlock, lastStoredForgeL1TxsNum) + rollupData, err := s.rollupSync(nextBlockNum) if err != nil { return err } // Get data from the auction contract - err = s.auctionSync(blockData, batchData) + auctionData, err := s.auctionSync(nextBlockNum) if err != nil { return err } - // Add rollupData and auctionData once the method is updated - err = s.historyDB.AddBlock(ethBlock) + // Get data from the WithdrawalDelayer contract + wdelayerData, err := s.wdelayerSync(nextBlockNum) if err != nil { return err } - // We get the block on every iteration - lastSavedBlock, err = s.historyDB.GetLastBlock() + // Group all the block data into the structs to save into HistoryDB + var blockData BlockData + + blockData.block = ethBlock + + if rollupData != nil { + blockData.l1Txs = rollupData.l1Txs + blockData.batches = rollupData.batches + // blockData.withdrawals = rollupData.withdrawals // TODO + blockData.registeredTokens = rollupData.registeredTokens + blockData.rollupVars = rollupData.rollupVars + } + + if auctionData != nil { + blockData.bids = auctionData.bids + blockData.coordinators = auctionData.coordinators + blockData.auctionVars = auctionData.auctionVars + } + + if wdelayerData != nil { + blockData.withdrawalDelayerVars = wdelayerData + } + + // Add rollupData and auctionData once the method is updated + // TODO: Save Whole Struct -> AddBlockSCData(blockData) + err = s.historyDB.AddBlock(blockData.block) if err != nil { return err } @@ -170,8 +229,12 @@ func (s *Synchronizer) Sync() error { 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 @@ -182,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 @@ -203,33 +266,40 @@ 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 -func (s *Synchronizer) Status() (*Status, error) { +func (s *Synchronizer) Status() (*common.SyncStatus, error) { // Avoid possible inconsistencies s.mux.Lock() defer s.mux.Unlock() - var status *Status + 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() @@ -251,7 +321,7 @@ func (s *Synchronizer) Status() (*Status, error) { return nil, err } - // TODO: Get CurrentForgerAddr & NextForgerAddr + // TODO: Get CurrentForgerAddr & NextForgerAddr from the Auction SC / Or from the HistoryDB // Check if Synchronizer is synchronized status.Synchronized = status.CurrentBlock == latestBlockNum @@ -259,13 +329,251 @@ func (s *Synchronizer) Status() (*Status, error) { } // rollupSync gets information from the Rollup Contract -func (s *Synchronizer) rollupSync(block *common.Block, lastStoredForgeL1TxsNum int64) (*BlockData, []*BatchData, error) { - // To be implemented - return nil, nil, nil +func (s *Synchronizer) rollupSync(blockNum int64) (*rollupData, error) { + var rollupData = newRollupData() + // var forgeL1TxsNum int64 + var numAccounts int + + // Get rollup events in the block + rollupEvents, _, err := s.ethClient.RollupEventsByBlock(blockNum) + if err != nil { + return nil, err + } + + // 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 + // } + + // Get L1UserTX + rollupData.l1Txs = getL1UserTx(rollupEvents.L1UserTx, blockNum) + + // Get ForgeBatch events to get the L1CoordinatorTxs + for _, fbEvent := range rollupEvents.ForgeBatch { + batchData := NewBatchData() + position := 0 + + // Get the input for each Tx + forgeBatchArgs, err := s.ethClient.RollupForgeBatchArgs(fbEvent.EthTxHash) + if err != nil { + return nil, err + } + 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 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 = 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 = blockNum + l1CoordinatorTx.BatchNum = common.BatchNum(fbEvent.BatchNum) + + batchData.l1CoordinatorTxs = append(batchData.l1CoordinatorTxs, l1CoordinatorTx) + + // 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++ + } + 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 + } + + l2Txs := common.PoolL2TxsToL2Txs(poolL2Txs) // TODO: This is a big uggly, find a better way + batchData.l2Txs = append(batchData.l2Txs, l2Txs...) + + batchData.exitTree = exitInfo + + // Get Batch information + batch := &common.Batch{ + BatchNum: common.BatchNum(fbEvent.BatchNum), + 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 // calculate from blockNum + ethClient Constants + } + batchData.batch = batch + rollupData.batches = append(rollupData.batches, batchData) + } + + // Get Registered Tokens + for _, eAddToken := range rollupEvents.AddToken { + var token *common.Token + + token.TokenID = common.TokenID(eAddToken.TokenID) + token.EthAddr = eAddToken.Address + 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 + + return &rollupData, nil } // auctionSync gets information from the Auction Contract -func (s *Synchronizer) auctionSync(blockData *BlockData, batchData []*BatchData) error { - // To be implemented - return nil +func (s *Synchronizer) auctionSync(blockNum int64) (*auctionData, error) { + var auctionData = newAuctionData() + + // Get auction events in the block + auctionEvents, _, err := s.ethClient.AuctionEventsByBlock(blockNum) + if err != nil { + return nil, err + } + + // Get bids + for _, eNewBid := range auctionEvents.NewBid { + bid := &common.Bid{ + SlotNum: common.SlotNum(eNewBid.Slot), + BidValue: eNewBid.BidAmount, + ForgerAddr: eNewBid.CoordinatorForger, + EthBlockNum: blockNum, + } + auctionData.bids = append(auctionData.bids, bid) + } + + // Get Coordinators + for _, eNewCoordinator := range auctionEvents.NewCoordinator { + coordinator := &common.Coordinator{ + Forger: eNewCoordinator.ForgerAddress, + Withdraw: eNewCoordinator.WithdrawalAddress, + URL: eNewCoordinator.URL, + } + 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{ + Forger: eCoordinatorUpdated.ForgerAddress, + Withdraw: eCoordinatorUpdated.WithdrawalAddress, + URL: eCoordinatorUpdated.URL, + } + auctionData.coordinators = append(auctionData.coordinators, coordinator) + } + + // TODO: VARS + // TODO: CONSTANTS + + return auctionData, nil +} + +// wdelayerSync gets information from the Withdrawal Delayer Contract +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) { +// // 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 = eL1UserTx.ToForgeL1TxsNum + eL1UserTx.L1Tx.Position = eL1UserTx.Position + eL1UserTx.L1Tx.UserOrigin = true + eL1UserTx.L1Tx.EthBlockNum = blockNum + eL1UserTx.L1Tx.BatchNum = 0 + + l1Txs = append(l1Txs, &eL1UserTx.L1Tx) + } + return l1Txs } diff --git a/synchronizer/synchronizer_test.go b/synchronizer/synchronizer_test.go index 287b390..1d3fef8 100644 --- a/synchronizer/synchronizer_test.go +++ b/synchronizer/synchronizer_test.go @@ -38,10 +38,11 @@ func Test(t *testing.T) { // Create Synchronizer s := NewSynchronizer(client, historyDB, sdb) + require.NotNil(t, s) // Test Sync - err = s.Sync() - require.Nil(t, err) + // err = s.Sync() + // require.Nil(t, err) // TODO: Reorg will be properly tested once we have the mock ethClient implemented /* diff --git a/test/historydb.go b/test/historydb.go index 6887bdf..7b3e0b5 100644 --- a/test/historydb.go +++ b/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) }