Browse Source

Merge pull request #52 from hermeznetwork/feature/init-coordinator

Initial Coordinator (OpManager) structure
feature/sql-semaphore1
Eduard S 4 years ago
committed by GitHub
parent
commit
58cf35f6ea
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 404 additions and 13 deletions
  1. +4
    -4
      batchbuilder/batchbuilder.go
  2. +3
    -0
      common/errors.go
  3. +4
    -0
      common/zk.go
  4. +80
    -0
      coordinator/batch.go
  5. +26
    -0
      coordinator/batch_test.go
  6. +225
    -0
      coordinator/coordinator.go
  7. +25
    -0
      coordinator/proofpool.go
  8. +4
    -4
      db/statedb/statedb.go
  9. +15
    -0
      eth/client.go
  10. +13
    -0
      node/node.go
  11. +5
    -5
      txselector/txselector.go

+ 4
- 4
batchbuilder/batchbuilder.go

@ -26,12 +26,12 @@ type BatchBuilder struct {
// ConfigBatch contains the batch configuration // ConfigBatch contains the batch configuration
type ConfigBatch struct { type ConfigBatch struct {
CoordinatorAddress ethCommon.Address
ForgerAddress ethCommon.Address
} }
// NewBatchBuilder constructs a new BatchBuilder, and executes the bb.Reset // NewBatchBuilder constructs a new BatchBuilder, and executes the bb.Reset
// method // method
func NewBatchBuilder(synchronizerStateDB *statedb.StateDB, configCircuits []ConfigCircuit, batchNum int, idx, nLevels uint64) (*BatchBuilder, error) {
func NewBatchBuilder(synchronizerStateDB *statedb.StateDB, configCircuits []ConfigCircuit, batchNum uint64, idx, nLevels uint64) (*BatchBuilder, error) {
localStateDB, err := statedb.NewLocalStateDB(synchronizerStateDB, true, int(nLevels)) localStateDB, err := statedb.NewLocalStateDB(synchronizerStateDB, true, int(nLevels))
if err != nil { if err != nil {
return nil, err return nil, err
@ -51,7 +51,7 @@ func NewBatchBuilder(synchronizerStateDB *statedb.StateDB, configCircuits []Conf
// `batchNum`. If `fromSynchronizer` is true, the BatchBuilder must take a // `batchNum`. If `fromSynchronizer` is true, the BatchBuilder must take a
// copy of the rollup state from the Synchronizer at that `batchNum`, otherwise // copy of the rollup state from the Synchronizer at that `batchNum`, otherwise
// it can just roll back the internal copy. // it can just roll back the internal copy.
func (bb *BatchBuilder) Reset(batchNum int, fromSynchronizer bool) error {
func (bb *BatchBuilder) Reset(batchNum uint64, fromSynchronizer bool) error {
err := bb.localStateDB.Reset(batchNum, fromSynchronizer) err := bb.localStateDB.Reset(batchNum, fromSynchronizer)
if err != nil { if err != nil {
return err return err
@ -61,7 +61,7 @@ func (bb *BatchBuilder) Reset(batchNum int, fromSynchronizer bool) error {
} }
// BuildBatch takes the transactions and returns the common.ZKInputs of the next batch // BuildBatch takes the transactions and returns the common.ZKInputs of the next batch
func (bb *BatchBuilder) BuildBatch(configBatch ConfigBatch, l1usertxs, l1coordinatortxs []common.L1Tx, l2txs []common.L2Tx, tokenIDs []common.TokenID) (*common.ZKInputs, error) {
func (bb *BatchBuilder) BuildBatch(configBatch ConfigBatch, l1usertxs, l1coordinatortxs []common.L1Tx, l2txs []common.PoolL2Tx, tokenIDs []common.TokenID) (*common.ZKInputs, error) {
for _, tx := range l1usertxs { for _, tx := range l1usertxs {
err := bb.processL1Tx(tx) err := bb.processL1Tx(tx)
if err != nil { if err != nil {

+ 3
- 0
common/errors.go

@ -7,3 +7,6 @@ var ErrNotInFF = errors.New("BigInt not inside the Finite Field")
// ErrNumOverflow is used when a given value overflows the maximum capacity of the parameter // ErrNumOverflow is used when a given value overflows the maximum capacity of the parameter
var ErrNumOverflow = errors.New("Value overflows the type") var ErrNumOverflow = errors.New("Value overflows the type")
// ErrBatchQueueEmpty is used when the coordinator.BatchQueue.Pop() is called and has no elements
var ErrBatchQueueEmpty = errors.New("BatchQueue empty")

+ 4
- 0
common/zk.go

@ -54,3 +54,7 @@ type ZKInputs struct {
OldKey2 []*big.Int OldKey2 []*big.Int
OldValue2 []*big.Int OldValue2 []*big.Int
} }
type CallDataForge struct {
// TBD
}

+ 80
- 0
coordinator/batch.go

@ -0,0 +1,80 @@
package coordinator
import (
"github.com/hermeznetwork/hermez-node/common"
)
type Proof struct {
// TBD this type will be got from the proof server
}
// BatchInfo contans the Batch information
type BatchInfo struct {
batchNum uint64
serverProof *ServerProofInfo
zkInputs *common.ZKInputs
proof *Proof
L1UserTxsExtra []common.L1Tx
L1OperatorTxs []common.L1Tx
L2Txs []common.PoolL2Tx
// FeesInfo
}
// NewBatchInfo creates a new BatchInfo with the given batchNum &
// ServerProofInfo
func NewBatchInfo(batchNum uint64, serverProof *ServerProofInfo) BatchInfo {
return BatchInfo{
batchNum: batchNum,
serverProof: serverProof,
}
}
// SetTxsInfo sets the l1UserTxs, l1OperatorTxs and l2Txs to the BatchInfo data
// structure
func (bi *BatchInfo) SetTxsInfo(l1UserTxsExtra, l1OperatorTxs []common.L1Tx, l2Txs []common.PoolL2Tx) {
// TBD parameter: feesInfo
bi.L1UserTxsExtra = l1UserTxsExtra
bi.L1OperatorTxs = l1OperatorTxs
bi.L2Txs = l2Txs
}
// SetZKInputs sets the ZKInputs to the BatchInfo data structure
func (bi *BatchInfo) SetZKInputs(zkInputs *common.ZKInputs) {
bi.zkInputs = zkInputs
}
// SetServerProof sets the ServerProofInfo to the BatchInfo data structure
func (bi *BatchInfo) SetServerProof(serverProof *ServerProofInfo) {
bi.serverProof = serverProof
}
// SetProof sets the Proof to the BatchInfo data structure
func (bi *BatchInfo) SetProof(proof *Proof) {
bi.proof = proof
}
// BatchQueue implements a FIFO queue of BatchInfo
type BatchQueue struct {
queue []*BatchInfo
}
func NewBatchQueue() *BatchQueue {
return &BatchQueue{
queue: []*BatchInfo{},
}
}
// Push adds the given BatchInfo to the BatchQueue
func (bq *BatchQueue) Push(b *BatchInfo) {
bq.queue = append(bq.queue, b)
}
// Pop pops the first BatchInfo from the BatchQueue
func (bq *BatchQueue) Pop() *BatchInfo {
if len(bq.queue) == 0 {
return nil
}
b := bq.queue[0]
bq.queue = bq.queue[1:]
return b
}

+ 26
- 0
coordinator/batch_test.go

@ -0,0 +1,26 @@
package coordinator
import (
"testing"
"github.com/stretchr/testify/assert"
)
func TestBatchQueue(t *testing.T) {
bq := BatchQueue{}
bq.Push(&BatchInfo{
batchNum: 0,
})
bq.Push(&BatchInfo{
batchNum: 2,
})
bq.Push(&BatchInfo{
batchNum: 1,
})
assert.Equal(t, uint64(0), bq.Pop().batchNum)
assert.Equal(t, uint64(2), bq.Pop().batchNum)
assert.Equal(t, uint64(1), bq.Pop().batchNum)
assert.Nil(t, bq.Pop())
}

+ 225
- 0
coordinator/coordinator.go

@ -0,0 +1,225 @@
package coordinator
import (
"time"
ethCommon "github.com/ethereum/go-ethereum/common"
"github.com/hermeznetwork/hermez-node/batchbuilder"
"github.com/hermeznetwork/hermez-node/common"
"github.com/hermeznetwork/hermez-node/eth"
"github.com/hermeznetwork/hermez-node/txselector"
kvdb "github.com/iden3/go-merkletree/db"
"github.com/iden3/go-merkletree/db/memory"
)
// CoordinatorConfig contains the Coordinator configuration
type CoordinatorConfig struct {
ForgerAddress ethCommon.Address
}
// Coordinator implements the Coordinator type
type Coordinator struct {
config CoordinatorConfig
batchNum uint64
batchQueue *BatchQueue
serverProofPool ServerProofPool
// synchronizer *synchronizer.Synchronizer
txsel *txselector.TxSelector
batchBuilder *batchbuilder.BatchBuilder
ethClient *eth.EthClient
ethTxStore kvdb.Storage
}
// NewCoordinator creates a new Coordinator
func NewCoordinator() *Coordinator { // once synchronizer is ready, synchronizer.Synchronizer will be passed as parameter here
var c *Coordinator
c.ethClient = eth.NewEthClient() // TBD
c.ethTxStore = memory.NewMemoryStorage()
return c
}
// Start starts the Coordinator service
func (c *Coordinator) Start() {
// TODO TBD note: the sequences & loops & errors & logging & goroutines
// & channels approach still needs to be defined, the current code is a
// wip draft
// TBD: goroutines strategy
// if in Forge Sequence:
if c.isForgeSequence() {
// c.batchNum = c.synchronizer.LastBatchNum()
_ = c.txsel.Reset(c.batchNum)
_ = c.batchBuilder.Reset(c.batchNum, true)
c.batchQueue = NewBatchQueue()
go func() {
for {
_ = c.forgeSequence()
time.Sleep(1 * time.Second)
}
}()
go func() {
for {
_ = c.proveSequence()
time.Sleep(1 * time.Second)
}
}()
go func() {
for {
_ = c.forgeConfirmationSequence()
time.Sleep(1 * time.Second)
}
}()
}
}
func (c *Coordinator) forgeSequence() error {
// TODO once synchronizer has this method ready:
// If there's been a reorg, handle it
// handleReorg() function decides if the reorg must restart the pipeline or not
// if c.synchronizer.Reorg():
_ = c.handleReorg()
// 0. If there's an available server proof: Start pipeline for batchNum = batchNum + 1
serverProofInfo, err := c.serverProofPool.GetNextAvailable() // blocking call, returns when a server proof is available
if err != nil {
return err
}
// remove transactions from the pool that have been there for too long
err = c.purgeRemoveByTimeout()
if err != nil {
return err
}
c.batchNum = c.batchNum + 1
batchInfo := NewBatchInfo(c.batchNum, serverProofInfo) // to accumulate metadata of the batch
var l2Txs []common.PoolL2Tx
// var feesInfo
var l1UserTxsExtra, l1OperatorTxs []common.L1Tx
// 1. Decide if we forge L2Tx or L1+L2Tx
if c.shouldL1L2Batch() {
// 2a: L1+L2 txs
// l1UserTxs, toForgeL1TxsNumber := c.synchronizer.GetNextL1UserTxs() // TODO once synchronizer is ready, uncomment
var l1UserTxs []common.L1Tx = nil // tmp, depends on synchronizer
l1UserTxsExtra, l1OperatorTxs, l2Txs, err = c.txsel.GetL1L2TxSelection(c.batchNum, l1UserTxs) // TODO once feesInfo is added to method return, add the var
if err != nil {
return err
}
} else {
// 2b: only L2 txs
l2Txs, err = c.txsel.GetL2TxSelection(c.batchNum) // TODO once feesInfo is added to method return, add the var
if err != nil {
return err
}
l1UserTxsExtra = nil
l1OperatorTxs = nil
}
// Run purger to invalidate transactions that become invalid beause of
// the l2Txs selected. Will mark as invalid the txs that have a
// (fromIdx, nonce) which already appears in the selected txs (includes
// all the nonces smaller than the current one)
err = c.purgeInvalidDueToL2TxsSelection(l2Txs)
if err != nil {
return err
}
// 3. Save metadata from TxSelector output for BatchNum
batchInfo.SetTxsInfo(l1UserTxsExtra, l1OperatorTxs, l2Txs) // TODO feesInfo
// 4. Call BatchBuilder with TxSelector output
configBatch := batchbuilder.ConfigBatch{
ForgerAddress: c.config.ForgerAddress,
}
zkInputs, err := c.batchBuilder.BuildBatch(configBatch, l1UserTxsExtra, l1OperatorTxs, l2Txs, nil) // TODO []common.TokenID --> feesInfo
if err != nil {
return err
}
// 5. Save metadata from BatchBuilder output for BatchNum
batchInfo.SetZKInputs(zkInputs)
// 6. Call an idle server proof with BatchBuilder output, save server proof info for batchNum
err = batchInfo.serverProof.CalculateProof(zkInputs)
if err != nil {
return err
}
c.batchQueue.Push(&batchInfo)
return nil
}
// proveSequence gets the generated zkProof & sends it to the SmartContract
func (c *Coordinator) proveSequence() error {
batchInfo := c.batchQueue.Pop()
if batchInfo == nil {
// no batches in queue, return
return common.ErrBatchQueueEmpty
}
serverProofInfo := batchInfo.serverProof
proof, err := serverProofInfo.GetProof() // blocking call, until not resolved don't continue. Returns when the proof server has calculated the proof
if err != nil {
return err
}
batchInfo.SetProof(proof)
callData := c.prepareCallDataForge(batchInfo)
_, err = c.ethClient.ForgeCall(callData)
if err != nil {
return err
}
// TODO 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
// }
return nil
}
func (c *Coordinator) forgeConfirmationSequence() error {
// TODO strategy of this sequence TBD
// confirm eth txs and mark them as accepted sequence
// ethTx := ethTxStore.GetFirstPending()
// waitForAccepted(ethTx) // blocking call, returns once the ethTx is mined
// ethTxStore.MarkAccepted(ethTx)
return nil
}
func (c *Coordinator) handleReorg() error {
return nil
}
// isForgeSequence returns true if the node is the Forger in the current ethereum block
func (c *Coordinator) isForgeSequence() bool {
return false
}
func (c *Coordinator) purgeRemoveByTimeout() error {
return nil
}
func (c *Coordinator) purgeInvalidDueToL2TxsSelection(l2Txs []common.PoolL2Tx) error {
return nil
}
func (c *Coordinator) shouldL1L2Batch() bool {
return false
}
func (c *Coordinator) prepareCallDataForge(batchInfo *BatchInfo) *common.CallDataForge {
return nil
}

+ 25
- 0
coordinator/proofpool.go

@ -0,0 +1,25 @@
package coordinator
import "github.com/hermeznetwork/hermez-node/common"
type ServerProofInfo struct {
// TODO
Available bool
}
func (p *ServerProofInfo) CalculateProof(zkInputs *common.ZKInputs) error {
return nil
}
func (p *ServerProofInfo) GetProof() (*Proof, error) {
return nil, nil
}
type ServerProofPool struct {
// pool []ServerProofInfo
}
func (p *ServerProofPool) GetNextAvailable() (*ServerProofInfo, error) {
return nil, nil
}

+ 4
- 4
db/statedb/statedb.go

@ -50,21 +50,21 @@ func NewStateDB(path string, inDisk bool, withMT bool, nLevels int) (*StateDB, e
} }
// CheckPointAt does a checkpoint at the given batchNum in the defined path // CheckPointAt does a checkpoint at the given batchNum in the defined path
func (s *StateDB) CheckPointAt(batchNum int, path string) error {
func (s *StateDB) CheckPointAt(batchNum uint64, path string) error {
// TODO // TODO
return nil return nil
} }
// Reset resets the StateDB to the checkpoint at the given batchNum // Reset resets the StateDB to the checkpoint at the given batchNum
func (s *StateDB) Reset(batchNum int) error {
func (s *StateDB) Reset(batchNum uint64) error {
// TODO // TODO
return nil return nil
} }
// Checkpoints returns a list of the checkpoints (batchNums) // Checkpoints returns a list of the checkpoints (batchNums)
func (s *StateDB) Checkpoints() ([]int, error) {
func (s *StateDB) Checkpoints() ([]uint64, error) {
// TODO // TODO
//batchnums, err //batchnums, err
@ -208,7 +208,7 @@ func NewLocalStateDB(synchronizerDB *StateDB, withMT bool, nLevels int) (*LocalS
// Reset performs a reset, getting the state from // Reset performs a reset, getting the state from
// LocalStateDB.synchronizerStateDB for the given batchNum // LocalStateDB.synchronizerStateDB for the given batchNum
func (l *LocalStateDB) Reset(batchNum int, fromSynchronizer bool) error {
func (l *LocalStateDB) Reset(batchNum uint64, fromSynchronizer bool) error {
// TODO // TODO
// if fromSynchronizer==true: // if fromSynchronizer==true:
// make copy from l.synchronizerStateDB at the batchNum to the localStateDB // make copy from l.synchronizerStateDB at the batchNum to the localStateDB

+ 15
- 0
eth/client.go

@ -0,0 +1,15 @@
package eth
import "github.com/hermeznetwork/hermez-node/common"
type EthClient struct {
}
func NewEthClient() *EthClient {
// TODO
return &EthClient{}
}
func (ec *EthClient) ForgeCall(callData *common.CallDataForge) ([]byte, error) {
// TODO this depends on the smart contracts, once are ready this will be updated
return nil, nil
}

+ 13
- 0
node/node.go

@ -0,0 +1,13 @@
package node
type mode string
// ModeCoordinator defines the mode of the HermezNode as Coordinator, which
// means that the node is set to forge (which also will be synchronizing with
// the L1 blockchain state)
const ModeCoordinator mode = "coordinator"
// ModeSynchronizer defines the mode of the HermezNode as Synchronizer, which
// means that the node is set to only synchronize with the L1 blockchain state
// and will not forge
const ModeSynchronizer mode = "synchronizer"

+ 5
- 5
txselector/txselector.go

@ -52,7 +52,7 @@ func NewTxSelector(synchronizerStateDB *statedb.StateDB, l2 *l2db.L2DB, maxL1Use
// Reset tells the TxSelector to get it's internal AccountsDB // Reset tells the TxSelector to get it's internal AccountsDB
// from the required `batchNum` // from the required `batchNum`
func (txsel *TxSelector) Reset(batchNum int) error {
func (txsel *TxSelector) Reset(batchNum uint64) error {
err := txsel.localAccountsDB.Reset(batchNum, true) err := txsel.localAccountsDB.Reset(batchNum, true)
if err != nil { if err != nil {
return err return err
@ -61,7 +61,7 @@ func (txsel *TxSelector) Reset(batchNum int) error {
} }
// GetL2TxSelection returns a selection of the L2Txs for the next batch, from the L2DB pool // GetL2TxSelection returns a selection of the L2Txs for the next batch, from the L2DB pool
func (txsel *TxSelector) GetL2TxSelection(batchNum int) ([]common.PoolL2Tx, error) {
func (txsel *TxSelector) GetL2TxSelection(batchNum uint64) ([]common.PoolL2Tx, error) {
// get pending l2-tx from tx-pool // get pending l2-tx from tx-pool
l2TxsRaw, err := txsel.l2db.GetPendingTxs() // once l2db ready, maybe use parameter 'batchNum' l2TxsRaw, err := txsel.l2db.GetPendingTxs() // once l2db ready, maybe use parameter 'batchNum'
if err != nil { if err != nil {
@ -90,7 +90,7 @@ func (txsel *TxSelector) GetL2TxSelection(batchNum int) ([]common.PoolL2Tx, erro
} }
// GetL1L2TxSelection returns the selection of L1 + L2 txs // GetL1L2TxSelection returns the selection of L1 + L2 txs
func (txsel *TxSelector) GetL1L2TxSelection(batchNum int, l1txs []common.Tx) ([]common.Tx, []common.PoolL2Tx, []common.Tx, error) {
func (txsel *TxSelector) GetL1L2TxSelection(batchNum uint64, l1txs []common.L1Tx) ([]common.L1Tx, []common.L1Tx, []common.PoolL2Tx, error) {
// apply l1-user-tx to localAccountDB // apply l1-user-tx to localAccountDB
// create new leaves // create new leaves
// update balances // update balances
@ -134,7 +134,7 @@ func (txsel *TxSelector) GetL1L2TxSelection(batchNum int, l1txs []common.Tx) ([]
maxL2Txs := txsel.MaxTxs - uint64(len(l1OperatorTxs)) // - len(l1UserTxs) maxL2Txs := txsel.MaxTxs - uint64(len(l1OperatorTxs)) // - len(l1UserTxs)
l2txs := txsel.getL2Profitable(validTxs, maxL2Txs) l2txs := txsel.getL2Profitable(validTxs, maxL2Txs)
return l1txs, l2txs, l1OperatorTxs, nil
return l1txs, l1OperatorTxs, l2txs, nil
} }
func (txsel *TxSelector) checkIfAccountExistOrPending(idx common.Idx) bool { func (txsel *TxSelector) checkIfAccountExistOrPending(idx common.Idx) bool {
@ -155,7 +155,7 @@ func (txsel *TxSelector) getL2Profitable(txs txs, max uint64) txs {
sort.Sort(txs) sort.Sort(txs)
return txs[:max] return txs[:max]
} }
func (txsel *TxSelector) createL1OperatorTxForL2Tx(accounts []common.Account) []common.Tx {
func (txsel *TxSelector) createL1OperatorTxForL2Tx(accounts []common.Account) []common.L1Tx {
// //
return nil return nil
} }

Loading…
Cancel
Save