Browse Source

Smart Contracts Data Synchronization

feature/sql-semaphore1
ToniRamirezM 4 years ago
committed by Eduard S
parent
commit
145f37b135
4 changed files with 342 additions and 38 deletions
  1. +11
    -0
      common/scvars.go
  2. +12
    -0
      common/syncstatus.go
  3. +3
    -3
      eth/rollup.go
  4. +316
    -35
      synchronizer/synchronizer.go

+ 11
- 0
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

+ 12
- 0
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
}

+ 3
- 3
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

+ 316
- 35
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"
@ -23,37 +23,76 @@ var (
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
registeredAccounts []*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),
}
}
// 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
batches []*BatchData
// withdrawals []*common.ExitInfo
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
@ -81,8 +120,6 @@ func (s *Synchronizer) Sync() error {
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)
@ -143,19 +180,49 @@ func (s *Synchronizer) Sync() error {
}
// Get data from the rollup contract
blockData, batchData, err := s.rollupSync(ethBlock, lastStoredForgeL1TxsNum)
rollupData, err := s.rollupSync(ethBlock)
if err != nil {
return err
}
// Get data from the auction contract
err = s.auctionSync(blockData, batchData)
auctionData, err := s.auctionSync(ethBlock)
if err != nil {
return err
}
// Get data from the WithdrawalDelayer contract
wdelayerData, err := s.wdelayerSync(ethBlock)
if err != nil {
return err
}
// 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
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
err = s.historyDB.AddBlock(ethBlock)
// TODO: Save Whole Struct -> AddBlockSCData(blockData)
err = s.historyDB.AddBlock(blockData.block)
if err != nil {
return err
}
@ -224,12 +291,12 @@ func (s *Synchronizer) reorg(uncleBlock *common.Block) error {
}
// 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
// Get latest block in History DB
lastSavedBlock, err := s.historyDB.GetLastBlock()
@ -251,7 +318,7 @@ func (s *Synchronizer) Status() (*Status, error) {
return nil, err
}
// TODO: Get CurrentForgerAddr & NextForgerAddr
// TODO: Get CurrentForgerAddr & NextForgerAddr from the Auction SC
// Check if Synchronizer is synchronized
status.Synchronized = status.CurrentBlock == latestBlockNum
@ -259,13 +326,227 @@ 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(block *common.Block) (*rollupData, error) {
var rollupData = newRollupData()
var forgeL1TxsNum uint32
var numAccounts int
lastStoredForgeL1TxsNum, err := s.historyDB.GetLastL1TxsNum()
if err != nil {
return nil, err
}
// Get rollup events in the block
rollupEvents, _, err := s.ethClient.RollupEventsByBlock(block.EthBlockNum)
if err != nil {
return nil, err
}
// 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 = s.getL1UserTx(rollupEvents.L1UserTx, block)
// 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
// 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
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 registeredAccounts updating idx
// l1UserTxs, err := s.historyDB.GetL1UserTxs(lastStoredForgeL1TxsNum)
// 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.UserOrigin = false
l1CoordinatorTx.EthBlockNum = uint64(block.EthBlockNum)
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++
}
position++
}
lastStoredForgeL1TxsNum++
}
// Get L2Txs
batchData.l2Txs = append(batchData.l2Txs, forgeBatchArgs.L2Txs...)
// Get exitTree
_, exitInfo, err := s.stateDB.ProcessTxs(true, batchData.l1UserTxs, batchData.l1CoordinatorTxs, batchData.l2Txs)
if err != nil {
return nil, err
}
batchData.exitTree = exitInfo
// Get Batch information
batch := &common.Batch{
BatchNum: common.BatchNum(fbEvent.BatchNum),
EthBlockNum: block.EthBlockNum,
// ForgerAddr: , TODO: Get it from ethClient
// 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
}
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 = uint64(block.EthBlockNum)
// TODO: Add external information consulting SC about it using Address
rollupData.registeredTokens = append(rollupData.registeredTokens, token)
}
// 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(block *common.Block) (*auctionData, error) {
var auctionData = newAuctionData()
// Get auction events in the block
auctionEvents, _, err := s.ethClient.AuctionEventsByBlock(block.EthBlockNum)
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: block.EthBlockNum,
}
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)
}
// 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(block *common.Block) (*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 {
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.Position = eL1UserTx.Position
eL1UserTx.L1Tx.UserOrigin = true
eL1UserTx.L1Tx.EthBlockNum = uint64(block.EthBlockNum)
eL1UserTx.L1Tx.BatchNum = 0
l1Txs = append(l1Txs, &eL1UserTx.L1Tx)
}
return l1Txs
}

Loading…
Cancel
Save