@ -25,71 +25,98 @@ var ErrDone = fmt.Errorf("done")
// Config contains the Coordinator configuration
// Config contains the Coordinator configuration
type Config struct {
type Config struct {
// ForgerAddress is the address under which this coordinator is forging
ForgerAddress ethCommon . Address
ForgerAddress ethCommon . Address
// ConfirmBlocks is the number of confirmation blocks to wait for sent
// ethereum transactions before forgetting about them
ConfirmBlocks int64
ConfirmBlocks int64
// L1BatchTimeoutPerc is the portion of the range before the L1Batch
// timeout that will trigger a schedule to forge an L1Batch
L1BatchTimeoutPerc float64
// EthClientAttempts is the number of attempts to do an eth client RPC
// call before giving up
EthClientAttempts int
// EthClientAttemptsDelay is delay between attempts do do an eth client
// RPC call
EthClientAttemptsDelay time . Duration
// TxManagerCheckInterval is the waiting interval between receipt
// checks of ethereum transactions in the TxManager
TxManagerCheckInterval time . Duration
// DebugBatchPath if set, specifies the path where batchInfo is stored
// in JSON in every step/update of the pipeline
DebugBatchPath string
}
func ( c * Config ) debugBatchStore ( batchInfo * BatchInfo ) {
if c . DebugBatchPath != "" {
if err := batchInfo . DebugStore ( c . DebugBatchPath ) ; err != nil {
log . Warnw ( "Error storing debug BatchInfo" ,
"path" , c . DebugBatchPath , "err" , err )
}
}
}
}
// Coordinator implements the Coordinator type
// Coordinator implements the Coordinator type
type Coordinator struct {
type Coordinator struct {
// State
// State
forging bool
batchNum common . BatchNum
serverProofPool * ServerProofPool
consts synchronizer . SCConsts
vars synchronizer . SCVariables
batchNum common . BatchNum
serverProofs [ ] ServerProofInterface
consts synchronizer . SCConsts
vars synchronizer . SCVariable s
started bool
cfg Config
cfg Config
hdb * historydb . HistoryDB
txsel * txselector . TxSelector
historyDB * historydb . HistoryDB
txSelector * txselector . TxSelector
batchBuilder * batchbuilder . BatchBuilder
batchBuilder * batchbuilder . BatchBuilder
ethClient eth . ClientInterface
msgCh chan interface { }
msgCh chan interface { }
ctx context . Context
ctx context . Context
wg sync . WaitGroup
wg sync . WaitGroup
cancel context . CancelFunc
cancel context . CancelFunc
pipelineCtx context . Context
pipelineWg sync . WaitGroup
pipelineCancel context . CancelFunc
pipeline * Pipeline
txManager * TxManager
txManager * TxManager
}
}
// NewCoordinator creates a new Coordinator
// NewCoordinator creates a new Coordinator
func NewCoordinator ( cfg Config ,
func NewCoordinator ( cfg Config ,
hdb * historydb . HistoryDB ,
txsel * txselector . TxSelector ,
bb * batchbuilder . BatchBuilder ,
historyDB * historydb . HistoryDB ,
txSelector * txselector . TxSelector ,
batchBuilder * batchbuilder . BatchBuilder ,
serverProofs [ ] ServerProofInterface ,
serverProofs [ ] ServerProofInterface ,
ethClient eth . ClientInterface ,
ethClient eth . ClientInterface ,
scConsts * synchronizer . SCConsts ,
scConsts * synchronizer . SCConsts ,
initSCVars * synchronizer . SCVariables ,
initSCVars * synchronizer . SCVariables ,
) * Coordinator { // once synchronizer is ready, synchronizer.Synchronizer will be passed as parameter here
serverProofPool := NewServerProofPool ( len ( serverProofs ) )
for _ , serverProof := range serverProofs {
serverProofPool . Add ( serverProof )
) ( * Coordinator , error ) {
// nolint reason: hardcoded `1.0`, by design the percentage can't be over 100%
if cfg . L1BatchTimeoutPerc >= 1.0 { //nolint:gomnd
return nil , fmt . Errorf ( "invalid value for Config.L1BatchTimeoutPerc (%v >= 1.0)" ,
cfg . L1BatchTimeoutPerc )
}
if cfg . EthClientAttempts < 1 {
return nil , fmt . Errorf ( "invalid value for Config.EthClientAttempts (%v < 1)" ,
cfg . EthClientAttempts )
}
}
txManager := NewTxManager ( ethClient , cfg . ConfirmBlocks )
txManager := NewTxManager ( & cfg , ethClient )
ctx , cancel := context . WithCancel ( context . Background ( ) )
ctx , cancel := context . WithCancel ( context . Background ( ) )
c := Coordinator {
c := Coordinator {
forging : false ,
batchNum : - 1 ,
serverProofPool : serverProofPool ,
consts : * scConsts ,
vars : * initSCVars ,
batchNum : - 1 ,
serverProofs : serverProofs ,
consts : * scConsts ,
vars : * initSCVars ,
cfg : cfg ,
cfg : cfg ,
hdb : hdb ,
txsel : txsel ,
batchBuilder : bb ,
historyDB : historyDB ,
txSelector : txSelector ,
batchBuilder : batchBuilder ,
ethClient : ethClient ,
// ethClient: ethClient,
msgCh : make ( chan interface { } ) ,
msgCh : make ( chan interface { } ) ,
ctx : ctx ,
ctx : ctx ,
@ -98,7 +125,12 @@ func NewCoordinator(cfg Config,
txManager : txManager ,
txManager : txManager ,
}
}
return & c
return & c , nil
}
func ( c * Coordinator ) newPipeline ( ) * Pipeline {
return NewPipeline ( c . cfg , c . historyDB , c . txSelector , c . batchBuilder ,
c . txManager , c . serverProofs , & c . consts )
}
}
// MsgSyncStats indicates an update to the Synchronizer stats
// MsgSyncStats indicates an update to the Synchronizer stats
@ -134,50 +166,53 @@ func (c *Coordinator) handleMsgSyncSCVars(msg *MsgSyncSCVars) {
}
}
}
}
func ( c * Coordinator ) handleMsgSyncStats ( stats * synchronizer . Stats ) error {
if ! stats . Synced ( ) {
return nil
}
c . txManager . SetLastBlock ( stats . Eth . LastBlock )
func ( c * Coordinator ) canForge ( stats * synchronizer . Stats ) bool {
anyoneForge := false
anyoneForge := false
if stats . Sync . Auction . CurrentSlot . BatchesLen == 0 &&
if stats . Sync . Auction . CurrentSlot . BatchesLen == 0 &&
c . consts . Auction . RelativeBlock ( stats . Eth . LastBlock ) > int64 ( c . vars . Auction . SlotDeadline ) {
c . consts . Auction . RelativeBlock ( stats . Eth . LastBlock . Num + 1 ) > int64 ( c . vars . Auction . SlotDeadline ) {
log . Debug ( "Coordinator: anyone can forge in the current slot (slotDeadline passed)" )
log . Debug ( "Coordinator: anyone can forge in the current slot (slotDeadline passed)" )
anyoneForge = true
anyoneForge = true
}
}
if stats . Sync . Auction . CurrentSlot . Forger != c . cfg . ForgerAddress && ! anyoneForge {
if c . forging {
log . Info ( "Coordinator: forging state end" )
c . forging = false
c . PipelineStop ( )
}
// log.Debug("Coordinator: not in forge time") // DBG
if stats . Sync . Auction . CurrentSlot . Forger == c . cfg . ForgerAddress || anyoneForge {
return true
}
return false
}
func ( c * Coordinator ) handleMsgSyncStats ( stats * synchronizer . Stats ) error {
if ! stats . Synced ( ) {
return nil
return nil
}
}
// log.Debug("Coordinator: forge time") // DBG
if ! c . forging {
// Start pipeline from a batchNum state taken from synchronizer
log . Info ( "Coordinator: forging state begin" )
c . batchNum = common . BatchNum ( stats . Sync . LastBatch )
err := c . txsel . Reset ( c . batchNum )
if err != nil {
log . Errorw ( "Coordinator: TxSelector.Reset" , "error" , err )
return err
c . txManager . SetLastBlock ( stats . Eth . LastBlock . Num )
canForge := c . canForge ( stats )
if c . pipeline == nil {
if canForge {
log . Info ( "Coordinator: forging state begin" )
batchNum := common . BatchNum ( stats . Sync . LastBatch )
c . pipeline = c . newPipeline ( )
if err := c . pipeline . Start ( batchNum , stats , & c . vars ) ; err != nil {
return err
}
}
}
err = c . batchBuilder . Reset ( c . batchNum , true )
if err != nil {
log . Errorw ( "Coordinator: BatchBuilder.Reset" , "error" , err )
return err
} else {
if canForge {
c . pipeline . SetSyncStats ( stats )
} else {
log . Info ( "Coordinator: forging state end" )
c . pipeline . Stop ( )
c . pipeline = nil
}
}
c . forging = true
c . PipelineStart ( )
}
}
return nil
return nil
}
}
// Start the coordinator
// Start the coordinator
func ( c * Coordinator ) Start ( ) {
func ( c * Coordinator ) Start ( ) {
if c . started {
log . Fatal ( "Coordinator already started" )
}
c . started = true
c . wg . Add ( 1 )
c . wg . Add ( 1 )
go func ( ) {
go func ( ) {
c . txManager . Run ( c . ctx )
c . txManager . Run ( c . ctx )
@ -198,10 +233,12 @@ func (c *Coordinator) Start() {
stats := msg . Stats
stats := msg . Stats
if err := c . handleMsgSyncStats ( & stats ) ; err != nil {
if err := c . handleMsgSyncStats ( & stats ) ; err != nil {
log . Errorw ( "Coordinator.handleMsgSyncStats error" , "err" , err )
log . Errorw ( "Coordinator.handleMsgSyncStats error" , "err" , err )
continue
}
}
case MsgSyncReorg :
case MsgSyncReorg :
if err := c . handleReorg ( ) ; err != nil {
if err := c . handleReorg ( ) ; err != nil {
log . Errorw ( "Coordinator.handleReorg error" , "err" , err )
log . Errorw ( "Coordinator.handleReorg error" , "err" , err )
continue
}
}
case MsgSyncSCVars :
case MsgSyncSCVars :
c . handleMsgSyncSCVars ( & msg )
c . handleMsgSyncSCVars ( & msg )
@ -215,97 +252,45 @@ func (c *Coordinator) Start() {
// Stop the coordinator
// Stop the coordinator
func ( c * Coordinator ) Stop ( ) {
func ( c * Coordinator ) Stop ( ) {
log . Infow ( "Stopping coordinator..." )
if ! c . started {
log . Fatal ( "Coordinator already stopped" )
}
c . started = false
log . Infow ( "Stopping Coordinator..." )
c . cancel ( )
c . cancel ( )
c . wg . Wait ( )
c . wg . Wait ( )
if c . forging {
c . forging = false
c . PipelineStop ( )
if c . pipeline != nil {
c . pipeline . Stop ( )
c . pipeline = nil
}
}
}
}
// PipelineStart starts the forging pipeline
func ( c * Coordinator ) PipelineStart ( ) {
c . pipelineCtx , c . pipelineCancel = context . WithCancel ( context . Background ( ) )
queueSize := 1
batchChSentServerProof := make ( chan * BatchInfo , queueSize )
c . pipelineWg . Add ( 1 )
go func ( ) {
for {
select {
case <- c . pipelineCtx . Done ( ) :
log . Debug ( "Pipeline forgeSendServerProof loop done" )
c . pipelineWg . Done ( )
return
default :
c . batchNum = c . batchNum + 1
batchInfo , err := c . forgeSendServerProof ( c . pipelineCtx , c . batchNum )
if err == ErrDone {
continue
}
if err != nil {
log . Errorw ( "forgeSendServerProof" , "err" , err )
continue
}
batchChSentServerProof <- batchInfo
}
}
} ( )
c . pipelineWg . Add ( 1 )
go func ( ) {
for {
select {
case <- c . pipelineCtx . Done ( ) :
log . Debug ( "Pipeline waitServerProofSendEth loop done" )
c . pipelineWg . Done ( )
return
case batchInfo := <- batchChSentServerProof :
err := c . waitServerProof ( c . pipelineCtx , batchInfo )
if err == ErrDone {
continue
}
if err != nil {
log . Errorw ( "waitServerProof" , "err" , err )
continue
}
c . txManager . AddBatch ( batchInfo )
}
}
} ( )
}
// PipelineStop stops the forging pipeline
func ( c * Coordinator ) PipelineStop ( ) {
log . Debug ( "Stopping pipeline..." )
c . pipelineCancel ( )
c . pipelineWg . Wait ( )
func ( c * Coordinator ) handleReorg ( ) error {
return nil // TODO
}
}
// TxManager handles everything related to ethereum transactions: It makes the
// TxManager handles everything related to ethereum transactions: It makes the
// call to forge, waits for transaction confirmation, and keeps checking them
// call to forge, waits for transaction confirmation, and keeps checking them
// until a number of confirmed blocks have passed.
// until a number of confirmed blocks have passed.
type TxManager struct {
type TxManager struct {
ethClient eth . ClientInterface
batchCh chan * BatchInfo
lastBlockCh chan int64
queue [ ] * BatchInfo
confirmation int64
lastBlock int64
cfg Config
ethClient eth . ClientInterface
batchCh chan * BatchInfo
lastBlockCh chan int64
queue [ ] * BatchInfo
lastBlock int64
}
}
// NewTxManager creates a new TxManager
// NewTxManager creates a new TxManager
func NewTxManager ( ethClient eth . ClientInterface , confirmation int64 ) * TxManager {
func NewTxManager ( cfg * Config , ethClient eth . ClientInterface ) * TxManager {
return & TxManager {
return & TxManager {
cfg : * cfg ,
ethClient : ethClient ,
ethClient : ethClient ,
// TODO: Find best queue size
// TODO: Find best queue size
batchCh : make ( chan * BatchInfo , 16 ) , //nolint:gomnd
batchCh : make ( chan * BatchInfo , 16 ) , //nolint:gomnd
// TODO: Find best queue size
// TODO: Find best queue size
lastBlockCh : make ( chan int64 , 16 ) , //nolint:gomnd
confirmation : confirmation ,
lastBlock : - 1 ,
lastBlockCh : make ( chan int64 , 16 ) , //nolint:gomnd
lastBlock : - 1 ,
}
}
}
}
@ -320,13 +305,77 @@ func (t *TxManager) SetLastBlock(lastBlock int64) {
t . lastBlockCh <- lastBlock
t . lastBlockCh <- lastBlock
}
}
const waitTime = 200 * time . Millisecond
func ( t * TxManager ) rollupForgeBatch ( ctx context . Context , batchInfo * BatchInfo ) error {
var ethTx * types . Transaction
var err error
for attempt := 0 ; attempt < t . cfg . EthClientAttempts ; attempt ++ {
ethTx , err = t . ethClient . RollupForgeBatch ( batchInfo . ForgeBatchArgs )
if err != nil {
log . Errorw ( "TxManager ethClient.RollupForgeBatch" ,
"attempt" , attempt , "err" , err )
} else {
break
}
select {
case <- ctx . Done ( ) :
return ErrDone
case <- time . After ( t . cfg . EthClientAttemptsDelay ) :
}
}
if err != nil {
return fmt . Errorf ( "reached max attempts for ethClient.RollupForgeBatch: %w" , err )
}
batchInfo . EthTx = ethTx
t . cfg . debugBatchStore ( batchInfo )
return nil
}
func ( t * TxManager ) ethTransactionReceipt ( ctx context . Context , batchInfo * BatchInfo ) error {
txHash := batchInfo . EthTx . Hash ( )
var receipt * types . Receipt
var err error
for attempt := 0 ; attempt < t . cfg . EthClientAttempts ; attempt ++ {
receipt , err = t . ethClient . EthTransactionReceipt ( ctx , txHash )
if err != nil {
log . Errorw ( "TxManager ethClient.EthTransactionReceipt" ,
"attempt" , attempt , "err" , err )
} else {
break
}
select {
case <- ctx . Done ( ) :
return ErrDone
case <- time . After ( t . cfg . EthClientAttemptsDelay ) :
}
}
if err != nil {
return fmt . Errorf ( "reached max attempts for ethClient.EthTransactionReceipt: %w" , err )
}
batchInfo . Receipt = receipt
t . cfg . debugBatchStore ( batchInfo )
return nil
}
func ( t * TxManager ) handleReceipt ( batchInfo * BatchInfo ) ( * int64 , error ) {
receipt := batchInfo . Receipt
if receipt != nil {
if receipt . Status == types . ReceiptStatusFailed {
log . Errorw ( "TxManager receipt status is failed" , "receipt" , receipt )
return nil , fmt . Errorf ( "ethereum transaction receipt statis is failed" )
} else if receipt . Status == types . ReceiptStatusSuccessful {
confirm := t . lastBlock - receipt . BlockNumber . Int64 ( )
return & confirm , nil
}
}
return nil , nil
}
const longWaitTime = 999 * time . Hour
const longWaitTime = 999 * time . Hour
// Run the TxManager
// Run the TxManager
func ( t * TxManager ) Run ( ctx context . Context ) {
func ( t * TxManager ) Run ( ctx context . Context ) {
next := 0
next := 0
d := time . Duration ( longWaitTime )
waitTime := time . Duration ( longWaitTime )
for {
for {
select {
select {
case <- ctx . Done ( ) :
case <- ctx . Done ( ) :
@ -335,41 +384,41 @@ func (t *TxManager) Run(ctx context.Context) {
case lastBlock := <- t . lastBlockCh :
case lastBlock := <- t . lastBlockCh :
t . lastBlock = lastBlock
t . lastBlock = lastBlock
case batchInfo := <- t . batchCh :
case batchInfo := <- t . batchCh :
ethTx , err := t . ethClient . RollupForgeBatch ( batchInfo . ForgeBatchArgs )
if err != nil {
// TODO: Figure out different error cases and handle them properly
log . Errorw ( "TxManager ethClient.RollupForgeBatch" , "err" , err )
if err := t . rollupForgeBatch ( ctx , batchInfo ) ; err == ErrDone {
continue
} else if err != nil {
// TODO: Reset pipeline
continue
continue
}
}
log . Debugf ( "ethClient ForgeCall sent, batchNum: %d" , batchInfo . BatchNum )
log . Debugf ( "ethClient ForgeCall sent, batchNum: %d" , batchInfo . BatchNum )
batchInfo . EthTx = ethTx
t . queue = append ( t . queue , batchInfo )
t . queue = append ( t . queue , batchInfo )
d = waitTime
case <- time . After ( d ) :
waitTime = t . cfg . TxManagerCheckInterval
case <- time . After ( waitTime ) :
if len ( t . queue ) == 0 {
if len ( t . queue ) == 0 {
continue
continue
}
}
batchInfo := t . queue [ next ]
batchInfo := t . queue [ next ]
txID := batchInfo . EthTx . Hash ( )
receipt , err := t . ethClient . EthTransactionReceipt ( ctx , txID )
if err != nil {
log . Errorw ( "TxManager ethClient.EthTransactionReceipt" , "err" , err )
// TODO: Figure out different error cases and handle them properly
// TODO: Notify the Coordinator to maybe reset the pipeline
err := t . ethTransactionReceipt ( ctx , batchInfo )
if err == ErrDone {
continue
continue
} else if err != nil { //nolint:staticcheck
// We can't get the receipt for the
// transaction, so we can't confirm if it was
// mined
// TODO: Reset pipeline
}
}
if receipt != nil {
if receipt . Status == types . ReceiptStatusFailed {
log . Errorw ( "TxManager receipt status is failed" , "receipt" , receipt )
} else if receipt . Status == types . ReceiptStatusSuccessful {
if t . lastBlock - receipt . BlockNumber . Int64 ( ) >= t . confirmation {
log . Debugw ( "TxManager tx for RollupForgeBatch confirmed" , "batchNum" , batchInfo . BatchNum )
t . queue = t . queue [ 1 : ]
if len ( t . queue ) == 0 {
d = longWaitTime
}
}
confirm , err := t . handleReceipt ( batchInfo )
if err != nil { //nolint:staticcheck
// Transaction was rejected
// TODO: Reset pipeline
}
if confirm != nil && * confirm >= t . cfg . ConfirmBlocks {
log . Debugw ( "TxManager tx for RollupForgeBatch confirmed" ,
"batchNum" , batchInfo . BatchNum )
t . queue = t . queue [ 1 : ]
if len ( t . queue ) == 0 {
waitTime = longWaitTime
}
}
}
}
if len ( t . queue ) == 0 {
if len ( t . queue ) == 0 {
@ -381,11 +430,157 @@ func (t *TxManager) Run(ctx context.Context) {
}
}
}
}
// Pipeline manages the forging of batches with parallel server proofs
type Pipeline struct {
cfg Config
consts synchronizer . SCConsts
// state
batchNum common . BatchNum
vars synchronizer . SCVariables
lastScheduledL1BatchBlockNum int64
started bool
serverProofPool * ServerProofPool
txManager * TxManager
historyDB * historydb . HistoryDB
txSelector * txselector . TxSelector
batchBuilder * batchbuilder . BatchBuilder
stats synchronizer . Stats
statsCh chan synchronizer . Stats
ctx context . Context
wg sync . WaitGroup
cancel context . CancelFunc
}
// NewPipeline creates a new Pipeline
func NewPipeline ( cfg Config ,
historyDB * historydb . HistoryDB ,
txSelector * txselector . TxSelector ,
batchBuilder * batchbuilder . BatchBuilder ,
txManager * TxManager ,
serverProofs [ ] ServerProofInterface ,
scConsts * synchronizer . SCConsts ,
) * Pipeline {
serverProofPool := NewServerProofPool ( len ( serverProofs ) )
for _ , serverProof := range serverProofs {
serverProofPool . Add ( serverProof )
}
return & Pipeline {
cfg : cfg ,
historyDB : historyDB ,
txSelector : txSelector ,
batchBuilder : batchBuilder ,
serverProofPool : serverProofPool ,
txManager : txManager ,
consts : * scConsts ,
// TODO: Find best queue size
statsCh : make ( chan synchronizer . Stats , 16 ) , //nolint:gomnd
}
}
// SetSyncStats is a thread safe method to sets the synchronizer Stats
func ( p * Pipeline ) SetSyncStats ( stats * synchronizer . Stats ) {
p . statsCh <- * stats
}
// Start the forging pipeline
func ( p * Pipeline ) Start ( batchNum common . BatchNum ,
syncStats * synchronizer . Stats , initSCVars * synchronizer . SCVariables ) error {
if p . started {
log . Fatal ( "Pipeline already started" )
}
p . started = true
// Reset pipeline state
p . batchNum = batchNum
p . vars = * initSCVars
p . lastScheduledL1BatchBlockNum = 0
p . ctx , p . cancel = context . WithCancel ( context . Background ( ) )
err := p . txSelector . Reset ( p . batchNum )
if err != nil {
log . Errorw ( "Pipeline: TxSelector.Reset" , "error" , err )
return err
}
err = p . batchBuilder . Reset ( p . batchNum , true )
if err != nil {
log . Errorw ( "Pipeline: BatchBuilder.Reset" , "error" , err )
return err
}
queueSize := 1
batchChSentServerProof := make ( chan * BatchInfo , queueSize )
p . wg . Add ( 1 )
go func ( ) {
for {
select {
case <- p . ctx . Done ( ) :
log . Debug ( "Pipeline forgeSendServerProof loop done" )
p . wg . Done ( )
return
case syncStats := <- p . statsCh :
p . stats = syncStats
default :
p . batchNum = p . batchNum + 1
batchInfo , err := p . forgeSendServerProof ( p . ctx , p . batchNum )
if err == ErrDone {
continue
}
if err != nil {
log . Errorw ( "forgeSendServerProof" , "err" , err )
continue
}
batchChSentServerProof <- batchInfo
}
}
} ( )
p . wg . Add ( 1 )
go func ( ) {
for {
select {
case <- p . ctx . Done ( ) :
log . Debug ( "Pipeline waitServerProofSendEth loop done" )
p . wg . Done ( )
return
case batchInfo := <- batchChSentServerProof :
err := p . waitServerProof ( p . ctx , batchInfo )
if err == ErrDone {
continue
}
if err != nil {
log . Errorw ( "waitServerProof" , "err" , err )
continue
}
p . txManager . AddBatch ( batchInfo )
}
}
} ( )
return nil
}
// Stop the forging pipeline
func ( p * Pipeline ) Stop ( ) {
if ! p . started {
log . Fatal ( "Pipeline already stopped" )
}
p . started = false
log . Debug ( "Stopping Pipeline..." )
p . cancel ( )
p . wg . Wait ( )
// TODO: Cancel all proofServers with pending proofs
}
// forgeSendServerProof the next batch, wait for a proof server to be available and send the
// forgeSendServerProof the next batch, wait for a proof server to be available and send the
// circuit inputs to the proof server.
// circuit inputs to the proof server.
func ( c * Coordinator ) forgeSendServerProof ( ctx context . Context , batchNum common . BatchNum ) ( * BatchInfo , error ) {
func ( p * Pipeline ) forgeSendServerProof ( ctx context . Context , batchNum common . BatchNum ) ( * BatchInfo , error ) {
// remove transactions from the pool that have been there for too long
// remove transactions from the pool that have been there for too long
err := c . purgeRemoveByTimeout ( )
err := p . purgeRemoveByTimeout ( )
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
@ -396,17 +591,18 @@ func (c *Coordinator) forgeSendServerProof(ctx context.Context, batchNum common.
// var feesInfo
// var feesInfo
var l1UserTxsExtra , l1OperatorTxs [ ] common . L1Tx
var l1UserTxsExtra , l1OperatorTxs [ ] common . L1Tx
// 1. Decide if we forge L2Tx or L1+L2Tx
// 1. Decide if we forge L2Tx or L1+L2Tx
if c . shouldL1L2Batch ( ) {
if p . shouldL1L2Batch ( ) {
p . lastScheduledL1BatchBlockNum = p . stats . Eth . LastBatch
// 2a: L1+L2 txs
// 2a: L1+L2 txs
// l1UserTxs, toForgeL1TxsNumber := c.hdb .GetNextL1UserTxs() // TODO once HistoryDB is ready, uncomment
var l1UserTxs [ ] common . L1Tx = nil // tmp, depends on HistoryDB
l1UserTxsExtra , l1OperatorTxs , poolL2Txs , err = c . txsel . GetL1L2TxSelection ( [ ] common . Idx { } , batchNum , l1UserTxs ) // TODO once feesInfo is added to method return, add the var
// l1UserTxs, toForgeL1TxsNumber := c.historyDB .GetNextL1UserTxs() // TODO once HistoryDB is ready, uncomment
var l1UserTxs [ ] common . L1Tx = nil // tmp, depends on HistoryDB
l1UserTxsExtra , l1OperatorTxs , poolL2Txs , err = p . txSelector . GetL1L2TxSelection ( [ ] common . Idx { } , batchNum , l1UserTxs ) // TODO once feesInfo is added to method return, add the var
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
} else {
} else {
// 2b: only L2 txs
// 2b: only L2 txs
_ , poolL2Txs , err = c . txsel . GetL2TxSelection ( [ ] common . Idx { } , batchNum ) // TODO once feesInfo is added to method return, add the var
_ , poolL2Txs , err = p . txSelector . GetL2TxSelection ( [ ] common . Idx { } , batchNum ) // TODO once feesInfo is added to method return, add the var
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
@ -418,7 +614,7 @@ func (c *Coordinator) forgeSendServerProof(ctx context.Context, batchNum common.
// the poolL2Txs selected. Will mark as invalid the txs that have a
// the poolL2Txs selected. Will mark as invalid the txs that have a
// (fromIdx, nonce) which already appears in the selected txs (includes
// (fromIdx, nonce) which already appears in the selected txs (includes
// all the nonces smaller than the current one)
// all the nonces smaller than the current one)
err = c . purgeInvalidDueToL2TxsSelection ( poolL2Txs )
err = p . purgeInvalidDueToL2TxsSelection ( poolL2Txs )
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
@ -431,18 +627,19 @@ func (c *Coordinator) forgeSendServerProof(ctx context.Context, batchNum common.
// 4. Call BatchBuilder with TxSelector output
// 4. Call BatchBuilder with TxSelector output
configBatch := & batchbuilder . ConfigBatch {
configBatch := & batchbuilder . ConfigBatch {
ForgerAddress : c . cfg . ForgerAddress ,
ForgerAddress : p . cfg . ForgerAddress ,
}
}
zkInputs , err := c . batchBuilder . BuildBatch ( [ ] common . Idx { } , configBatch , l1UserTxsExtra , l1OperatorTxs , poolL2Txs , nil ) // TODO []common.TokenID --> feesInfo
zkInputs , err := p . batchBuilder . BuildBatch ( [ ] common . Idx { } , configBatch , l1UserTxsExtra , l1OperatorTxs , poolL2Txs , nil ) // TODO []common.TokenID --> feesInfo
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
// 5. Save metadata from BatchBuilder output for BatchNum
// 5. Save metadata from BatchBuilder output for BatchNum
batchInfo . ZKInputs = zkInputs
batchInfo . ZKInputs = zkInputs
p . cfg . debugBatchStore ( & batchInfo )
// 6. Wait for an available server proof blocking call
// 6. Wait for an available server proof blocking call
serverProof , err := c . serverProofPool . Get ( ctx )
serverProof , err := p . serverProofPool . Get ( ctx )
if err != nil {
if err != nil {
return nil , err
return nil , err
}
}
@ -451,9 +648,10 @@ func (c *Coordinator) forgeSendServerProof(ctx context.Context, batchNum common.
// If there's an error further on, add the serverProof back to
// If there's an error further on, add the serverProof back to
// the pool
// the pool
if err != nil {
if err != nil {
c . serverProofPool . Add ( serverProof )
p . serverProofPool . Add ( serverProof )
}
}
} ( )
} ( )
p . cfg . debugBatchStore ( & batchInfo )
// 7. Call the selected idle server proof with BatchBuilder output,
// 7. Call the selected idle server proof with BatchBuilder output,
// save server proof info for batchNum
// save server proof info for batchNum
@ -466,35 +664,20 @@ func (c *Coordinator) forgeSendServerProof(ctx context.Context, batchNum common.
}
}
// waitServerProof gets the generated zkProof & sends it to the SmartContract
// waitServerProof gets the generated zkProof & sends it to the SmartContract
func ( c * Coordinator ) waitServerProof ( ctx context . Context , batchInfo * BatchInfo ) error {
func ( p * Pipeline ) waitServerProof ( ctx context . Context , batchInfo * BatchInfo ) error {
proof , err := batchInfo . ServerProof . GetProof ( ctx ) // blocking call, until not resolved don't continue. Returns when the proof server has calculated the proof
proof , err := batchInfo . ServerProof . GetProof ( ctx ) // blocking call, until not resolved don't continue. Returns when the proof server has calculated the proof
if err != nil {
if err != nil {
return err
return err
}
}
c . serverProofPool . Add ( batchInfo . ServerProof )
p . serverProofPool . Add ( batchInfo . ServerProof )
batchInfo . ServerProof = nil
batchInfo . ServerProof = nil
batchInfo . Proof = proof
batchInfo . Proof = proof
batchInfo . ForgeBatchArgs = c . prepareForgeBatchArgs ( batchInfo )
batchInfo . ForgeBatchArgs = p . prepareForgeBatchArgs ( batchInfo )
batchInfo . TxStatus = TxStatusPending
batchInfo . TxStatus = TxStatusPending
// TODO(FUTURE) once tx data type is defined, store ethTx (returned by ForgeCall)
// TBD if use ethTxStore as a disk k-v database, or use a Queue
// tx, err := c.ethTxStore.NewTx()
// if err != nil {
// return err
// }
// tx.Put(ethTx.Hash(), ethTx.Bytes())
// if err := tx.Commit(); err!=nil {
// return nil
// }
p . cfg . debugBatchStore ( batchInfo )
return nil
return nil
}
}
func ( c * Coordinator ) handleReorg ( ) error {
return nil // TODO
}
// isForgeSequence returns true if the node is the Forger in the current ethereum block
// isForgeSequence returns true if the node is the Forger in the current ethereum block
// func (c *Coordinator) isForgeSequence() (bool, error) {
// func (c *Coordinator) isForgeSequence() (bool, error) {
// // TODO: Consider checking if we can forge by quering the Synchronizer instead of using ethClient
// // TODO: Consider checking if we can forge by quering the Synchronizer instead of using ethClient
@ -509,19 +692,31 @@ func (c *Coordinator) handleReorg() error {
// return c.ethClient.AuctionCanForge(*addr, blockNum+1)
// return c.ethClient.AuctionCanForge(*addr, blockNum+1)
// }
// }
func ( c * Coordinator ) purgeRemoveByTimeout ( ) error {
func ( p * Pipeline ) purgeRemoveByTimeout ( ) error {
return nil // TODO
return nil // TODO
}
}
func ( c * Coordinator ) purgeInvalidDueToL2TxsSelection ( l2Txs [ ] common . PoolL2Tx ) error {
func ( p * Pipeline ) purgeInvalidDueToL2TxsSelection ( l2Txs [ ] common . PoolL2Tx ) error {
return nil // TODO
return nil // TODO
}
}
func ( c * Coordinator ) shouldL1L2Batch ( ) bool {
return false // TODO
func ( p * Pipeline ) shouldL1L2Batch ( ) bool {
// Take the lastL1BatchBlockNum as the biggest between the last
// scheduled one, and the synchronized one.
lastL1BatchBlockNum := p . lastScheduledL1BatchBlockNum
if p . stats . Sync . LastL1BatchBlock > lastL1BatchBlockNum {
lastL1BatchBlockNum = p . stats . Sync . LastL1BatchBlock
}
// Return true if we have passed the l1BatchTimeoutPerc portion of the
// range before the l1batch timeout.
if p . stats . Eth . LastBlock . Num - lastL1BatchBlockNum >=
int64 ( float64 ( p . vars . Rollup . ForgeL1L2BatchTimeout ) * p . cfg . L1BatchTimeoutPerc ) {
return true
}
return false
}
}
func ( c * Coordinator ) prepareForgeBatchArgs ( batchInfo * BatchInfo ) * eth . RollupForgeBatchArgs {
func ( p * Pipeline ) prepareForgeBatchArgs ( batchInfo * BatchInfo ) * eth . RollupForgeBatchArgs {
// TODO
// TODO
return & eth . RollupForgeBatchArgs { }
return & eth . RollupForgeBatchArgs { }
}
}