Browse Source

Merge pull request #129 from hermeznetwork/feature/integration

Feature/integration
feature/sql-semaphore1
arnau 4 years ago
committed by GitHub
parent
commit
a40689f05e
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 625 additions and 267 deletions
  1. +18
    -12
      config/config.go
  2. +5
    -32
      coordinator/batch.go
  3. +0
    -27
      coordinator/batch_test.go
  4. +63
    -40
      coordinator/coordinator.go
  5. +27
    -4
      coordinator/coordinator_test.go
  6. +64
    -11
      coordinator/proofpool.go
  7. +100
    -62
      eth/auction.go
  8. +21
    -0
      eth/rollup.go
  9. +2
    -1
      go.mod
  10. +10
    -0
      go.sum
  11. +11
    -6
      node/node.go
  12. +292
    -67
      test/ethclient.go
  13. +12
    -5
      test/ethclient_test.go

+ 18
- 12
config/config.go

@ -25,10 +25,28 @@ func (d *Duration) UnmarshalText(data []byte) error {
return nil return nil
} }
// ServerProof is the server proof configuration data.
type ServerProof struct {
URL string `validate:"required"`
}
// Coordinator is the coordinator specific configuration. // Coordinator is the coordinator specific configuration.
type Coordinator struct { type Coordinator struct {
ForgerAddress ethCommon.Address `validate:"required"` ForgerAddress ethCommon.Address `validate:"required"`
ForgeLoopInterval Duration `validate:"required"` ForgeLoopInterval Duration `validate:"required"`
L2DB struct {
Name string `validate:"required"`
SafetyPeriod uint16 `validate:"required"`
MaxTxs uint32 `validate:"required"`
TTL Duration `validate:"required"`
} `validate:"required"`
TxSelector struct {
Path string `validate:"required"`
} `validate:"required"`
BatchBuilder struct {
Path string `validate:"required"`
} `validate:"required"`
ServerProofs []ServerProof `validate:"required"`
} }
// Node is the hermez node configuration. // Node is the hermez node configuration.
@ -42,24 +60,12 @@ type Node struct {
User string `validate:"required"` User string `validate:"required"`
Password string `validate:"required"` Password string `validate:"required"`
} `validate:"required"` } `validate:"required"`
L2DB struct {
Name string `validate:"required"`
SafetyPeriod uint16 `validate:"required"`
MaxTxs uint32 `validate:"required"`
TTL Duration `validate:"required"`
} `validate:"required"`
HistoryDB struct { HistoryDB struct {
Name string `validate:"required"` Name string `validate:"required"`
} `validate:"required"` } `validate:"required"`
Web3 struct { Web3 struct {
URL string `validate:"required"` URL string `validate:"required"`
} `validate:"required"` } `validate:"required"`
TxSelector struct {
Path string `validate:"required"`
} `validate:"required"`
BatchBuilder struct {
Path string `validate:"required"`
} `validate:"required"`
Synchronizer struct { Synchronizer struct {
SyncLoopInterval Duration `validate:"required"` SyncLoopInterval Duration `validate:"required"`
} `validate:"required"` } `validate:"required"`

+ 5
- 32
coordinator/batch.go

@ -11,7 +11,7 @@ type Proof struct {
// BatchInfo contans the Batch information // BatchInfo contans the Batch information
type BatchInfo struct { type BatchInfo struct {
batchNum common.BatchNum batchNum common.BatchNum
serverProof *ServerProofInfo
serverProof ServerProofInterface
zkInputs *common.ZKInputs zkInputs *common.ZKInputs
proof *Proof proof *Proof
L1UserTxsExtra []*common.L1Tx L1UserTxsExtra []*common.L1Tx
@ -21,8 +21,8 @@ type BatchInfo struct {
} }
// NewBatchInfo creates a new BatchInfo with the given batchNum & // NewBatchInfo creates a new BatchInfo with the given batchNum &
// ServerProofInfo
func NewBatchInfo(batchNum common.BatchNum, serverProof *ServerProofInfo) BatchInfo {
// ServerProof
func NewBatchInfo(batchNum common.BatchNum, serverProof ServerProofInterface) BatchInfo {
return BatchInfo{ return BatchInfo{
batchNum: batchNum, batchNum: batchNum,
serverProof: serverProof, serverProof: serverProof,
@ -43,8 +43,8 @@ func (bi *BatchInfo) SetZKInputs(zkInputs *common.ZKInputs) {
bi.zkInputs = zkInputs bi.zkInputs = zkInputs
} }
// SetServerProof sets the ServerProofInfo to the BatchInfo data structure
func (bi *BatchInfo) SetServerProof(serverProof *ServerProofInfo) {
// SetServerProof sets the ServerProof to the BatchInfo data structure
func (bi *BatchInfo) SetServerProof(serverProof ServerProofInterface) {
bi.serverProof = serverProof bi.serverProof = serverProof
} }
@ -52,30 +52,3 @@ func (bi *BatchInfo) SetServerProof(serverProof *ServerProofInfo) {
func (bi *BatchInfo) SetProof(proof *Proof) { func (bi *BatchInfo) SetProof(proof *Proof) {
bi.proof = proof bi.proof = proof
} }
// BatchQueue implements a FIFO queue of BatchInfo
type BatchQueue struct {
queue []*BatchInfo
}
// NewBatchQueue returns a new *BatchQueue
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
}

+ 0
- 27
coordinator/batch_test.go

@ -1,27 +0,0 @@
package coordinator
import (
"testing"
"github.com/hermeznetwork/hermez-node/common"
"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, common.BatchNum(0), bq.Pop().batchNum)
assert.Equal(t, common.BatchNum(2), bq.Pop().batchNum)
assert.Equal(t, common.BatchNum(1), bq.Pop().batchNum)
assert.Nil(t, bq.Pop())
}

+ 63
- 40
coordinator/coordinator.go

@ -2,6 +2,7 @@ package coordinator
import ( import (
"fmt" "fmt"
"sync"
ethCommon "github.com/ethereum/go-ethereum/common" ethCommon "github.com/ethereum/go-ethereum/common"
"github.com/hermeznetwork/hermez-node/batchbuilder" "github.com/hermeznetwork/hermez-node/batchbuilder"
@ -14,6 +15,8 @@ import (
"github.com/iden3/go-merkletree/db/memory" "github.com/iden3/go-merkletree/db/memory"
) )
var errTODO = fmt.Errorf("TODO")
// ErrStop is returned when the function is stopped asynchronously via the stop // ErrStop is returned when the function is stopped asynchronously via the stop
// channel. It doesn't indicate an error. // channel. It doesn't indicate an error.
var ErrStop = fmt.Errorf("Stopped") var ErrStop = fmt.Errorf("Stopped")
@ -26,12 +29,13 @@ type Config struct {
// Coordinator implements the Coordinator type // Coordinator implements the Coordinator type
type Coordinator struct { type Coordinator struct {
forging bool forging bool
rw *sync.RWMutex
isForgeSeq bool // WIP just for testing while implementing isForgeSeq bool // WIP just for testing while implementing
config Config config Config
batchNum common.BatchNum batchNum common.BatchNum
serverProofPool ServerProofPool
serverProofPool *ServerProofPool
// synchronizer *synchronizer.Synchronizer // synchronizer *synchronizer.Synchronizer
hdb *historydb.HistoryDB hdb *historydb.HistoryDB
@ -47,14 +51,21 @@ func NewCoordinator(conf Config,
hdb *historydb.HistoryDB, hdb *historydb.HistoryDB,
txsel *txselector.TxSelector, txsel *txselector.TxSelector,
bb *batchbuilder.BatchBuilder, bb *batchbuilder.BatchBuilder,
ethClient *eth.Client) *Coordinator { // once synchronizer is ready, synchronizer.Synchronizer will be passed as parameter here
serverProofs []ServerProofInterface,
ethClient eth.ClientInterface) *Coordinator { // once synchronizer is ready, synchronizer.Synchronizer will be passed as parameter here
serverProofPool := NewServerProofPool(len(serverProofs))
for _, serverProof := range serverProofs {
serverProofPool.Add(serverProof)
}
c := Coordinator{ c := Coordinator{
config: conf,
hdb: hdb,
txsel: txsel,
batchBuilder: bb,
ethClient: ethClient,
ethTxStore: memory.NewMemoryStorage(),
config: conf,
serverProofPool: serverProofPool,
hdb: hdb,
txsel: txsel,
batchBuilder: bb,
ethClient: ethClient,
ethTxStore: memory.NewMemoryStorage(),
rw: &sync.RWMutex{},
} }
return &c return &c
} }
@ -62,27 +73,28 @@ func NewCoordinator(conf Config,
// ForgeLoopFn is the function ran in a loop that checks if it's time to forge // ForgeLoopFn is the function ran in a loop that checks if it's time to forge
// and forges a batch if so and sends it to outBatchCh. Returns true if it's // and forges a batch if so and sends it to outBatchCh. Returns true if it's
// the coordinator turn to forge. // the coordinator turn to forge.
func (c *Coordinator) ForgeLoopFn(outBatchCh chan *BatchInfo, stopCh chan bool) (bool, error) {
func (c *Coordinator) ForgeLoopFn(outBatchCh chan *BatchInfo, stopCh chan bool) (forgetime bool, err error) {
if !c.isForgeSequence() { if !c.isForgeSequence() {
if c.forging { if c.forging {
log.Info("stop forging")
log.Info("ForgeLoopFn: forging state end")
c.forging = false c.forging = false
} }
log.Debug("not in forge time")
log.Debug("ForgeLoopFn: not in forge time")
return false, nil return false, nil
} }
log.Debug("forge time")
log.Debug("ForgeLoopFn: forge time")
if !c.forging { if !c.forging {
log.Info("start forging")
// Start pipeline from a batchNum state taken from synchronizer
log.Info("ForgeLoopFn: forging state begin")
// c.batchNum = c.hdb.GetLastBatchNum() // uncomment when HistoryDB is ready // c.batchNum = c.hdb.GetLastBatchNum() // uncomment when HistoryDB is ready
err := c.txsel.Reset(c.batchNum) err := c.txsel.Reset(c.batchNum)
if err != nil { if err != nil {
log.Errorw("TxSelector.Reset", "error", err)
log.Errorw("ForgeLoopFn: TxSelector.Reset", "error", err)
return true, err return true, err
} }
err = c.batchBuilder.Reset(c.batchNum, true) err = c.batchBuilder.Reset(c.batchNum, true)
if err != nil { if err != nil {
log.Errorw("BatchBuilder.Reset", "error", err)
log.Errorw("ForgeLoopFn: BatchBuilder.Reset", "error", err)
return true, err return true, err
} }
// c.batchQueue = NewBatchQueue() // c.batchQueue = NewBatchQueue()
@ -94,21 +106,27 @@ func (c *Coordinator) ForgeLoopFn(outBatchCh chan *BatchInfo, stopCh chan bool)
// if c.synchronizer.Reorg(): // if c.synchronizer.Reorg():
_ = c.handleReorg() _ = c.handleReorg()
// 0. If there's an available server proof: Start pipeline for batchNum = batchNum + 1.
// non-blocking call, returns nil if a server proof is
// not available, or non-nil otherwise.
serverProofInfo, err := c.serverProofPool.GetNextAvailable(stopCh)
// 0. Wait for an available server proof
// blocking call
serverProof, err := c.serverProofPool.Get(stopCh)
if err != nil { if err != nil {
return true, err return true, err
} }
defer func() {
if !forgetime || err != nil {
c.serverProofPool.Add(serverProof)
}
}()
log.Debugw("start forge")
batchInfo, err := c.forge(serverProofInfo)
log.Debugw("ForgeLoopFn: using serverProof", "server", serverProof)
log.Debugw("ForgeLoopFn: forge start")
// forge for batchNum = batchNum + 1.
batchInfo, err := c.forge(serverProof)
if err != nil { if err != nil {
log.Errorw("forge", "error", err) log.Errorw("forge", "error", err)
return true, err return true, err
} }
log.Debugw("end forge", "batchNum", batchInfo.batchNum)
log.Debugw("ForgeLoopFn: forge end", "batchNum", batchInfo.batchNum)
outBatchCh <- batchInfo outBatchCh <- batchInfo
return true, nil return true, nil
} }
@ -119,14 +137,14 @@ func (c *Coordinator) ForgeLoopFn(outBatchCh chan *BatchInfo, stopCh chan bool)
func (c *Coordinator) GetProofCallForgeLoopFn(inBatchCh, outBatchCh chan *BatchInfo, stopCh chan bool) error { func (c *Coordinator) GetProofCallForgeLoopFn(inBatchCh, outBatchCh chan *BatchInfo, stopCh chan bool) error {
select { select {
case <-stopCh: case <-stopCh:
log.Info("forgeLoopFn stopped")
log.Info("GetProofCallForgeLoopFn: forgeLoopFn stopped")
return ErrStop return ErrStop
case batchInfo := <-inBatchCh: case batchInfo := <-inBatchCh:
log.Debugw("start getProofCallForge", "batchNum", batchInfo.batchNum)
if err := c.getProofCallForge(batchInfo); err != nil {
log.Debugw("GetProofCallForgeLoopFn: getProofCallForge start", "batchNum", batchInfo.batchNum)
if err := c.getProofCallForge(batchInfo, stopCh); err != nil {
return err return err
} }
log.Debugw("end getProofCallForge", "batchNum", batchInfo.batchNum)
log.Debugw("GetProofCallForgeLoopFn: getProofCallForge end", "batchNum", batchInfo.batchNum)
outBatchCh <- batchInfo outBatchCh <- batchInfo
} }
return nil return nil
@ -138,19 +156,19 @@ func (c *Coordinator) GetProofCallForgeLoopFn(inBatchCh, outBatchCh chan *BatchI
func (c *Coordinator) ForgeCallConfirmLoopFn(inBatchCh chan *BatchInfo, stopCh chan bool) error { func (c *Coordinator) ForgeCallConfirmLoopFn(inBatchCh chan *BatchInfo, stopCh chan bool) error {
select { select {
case <-stopCh: case <-stopCh:
log.Info("forgeConfirmLoopFn stopped")
log.Info("ForgeCallConfirmLoopFn: forgeConfirmLoopFn stopped")
return ErrStop return ErrStop
case batchInfo := <-inBatchCh: case batchInfo := <-inBatchCh:
log.Debugw("start forgeCallConfirm", "batchNum", batchInfo.batchNum)
log.Debugw("ForgeCallConfirmLoopFn: forgeCallConfirm start", "batchNum", batchInfo.batchNum)
if err := c.forgeCallConfirm(batchInfo); err != nil { if err := c.forgeCallConfirm(batchInfo); err != nil {
return err return err
} }
log.Debugw("end forgeCallConfirm", "batchNum", batchInfo.batchNum)
log.Debugw("ForgeCallConfirmLoopFn: forgeCallConfirm end", "batchNum", batchInfo.batchNum)
} }
return nil return nil
} }
func (c *Coordinator) forge(serverProofInfo *ServerProofInfo) (*BatchInfo, error) {
func (c *Coordinator) forge(serverProof ServerProofInterface) (*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 := c.purgeRemoveByTimeout()
if err != nil { if err != nil {
@ -158,7 +176,7 @@ func (c *Coordinator) forge(serverProofInfo *ServerProofInfo) (*BatchInfo, error
} }
c.batchNum = c.batchNum + 1 c.batchNum = c.batchNum + 1
batchInfo := NewBatchInfo(c.batchNum, serverProofInfo) // to accumulate metadata of the batch
batchInfo := NewBatchInfo(c.batchNum, serverProof) // to accumulate metadata of the batch
var poolL2Txs []*common.PoolL2Tx var poolL2Txs []*common.PoolL2Tx
// var feesInfo // var feesInfo
@ -216,9 +234,11 @@ func (c *Coordinator) forge(serverProofInfo *ServerProofInfo) (*BatchInfo, error
} }
// getProofCallForge gets the generated zkProof & sends it to the SmartContract // getProofCallForge gets the generated zkProof & sends it to the SmartContract
func (c *Coordinator) getProofCallForge(batchInfo *BatchInfo) error {
serverProofInfo := batchInfo.serverProof
proof, err := serverProofInfo.GetProof() // blocking call, until not resolved don't continue. Returns when the proof server has calculated the proof
func (c *Coordinator) getProofCallForge(batchInfo *BatchInfo, stopCh chan bool) error {
serverProof := batchInfo.serverProof
proof, err := serverProof.GetProof(stopCh) // blocking call, until not resolved don't continue. Returns when the proof server has calculated the proof
c.serverProofPool.Add(serverProof)
batchInfo.serverProof = nil
if err != nil { if err != nil {
return err return err
} }
@ -254,26 +274,29 @@ func (c *Coordinator) forgeCallConfirm(batchInfo *BatchInfo) error {
} }
func (c *Coordinator) handleReorg() error { func (c *Coordinator) handleReorg() error {
return nil
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 { func (c *Coordinator) isForgeSequence() bool {
return c.isForgeSeq
c.rw.RLock()
defer c.rw.RUnlock()
return c.isForgeSeq // TODO
} }
func (c *Coordinator) purgeRemoveByTimeout() error { func (c *Coordinator) purgeRemoveByTimeout() error {
return nil
return nil // TODO
} }
func (c *Coordinator) purgeInvalidDueToL2TxsSelection(l2Txs []*common.PoolL2Tx) error { func (c *Coordinator) purgeInvalidDueToL2TxsSelection(l2Txs []*common.PoolL2Tx) error {
return nil
return nil // TODO
} }
func (c *Coordinator) shouldL1L2Batch() bool { func (c *Coordinator) shouldL1L2Batch() bool {
return false
return false // TODO
} }
func (c *Coordinator) prepareForgeBatchArgs(batchInfo *BatchInfo) *eth.RollupForgeBatchArgs { func (c *Coordinator) prepareForgeBatchArgs(batchInfo *BatchInfo) *eth.RollupForgeBatchArgs {
return nil
// TODO
return &eth.RollupForgeBatchArgs{}
} }

+ 27
- 4
coordinator/coordinator_test.go

@ -10,8 +10,8 @@ import (
"github.com/hermeznetwork/hermez-node/db/historydb" "github.com/hermeznetwork/hermez-node/db/historydb"
"github.com/hermeznetwork/hermez-node/db/l2db" "github.com/hermeznetwork/hermez-node/db/l2db"
"github.com/hermeznetwork/hermez-node/db/statedb" "github.com/hermeznetwork/hermez-node/db/statedb"
"github.com/hermeznetwork/hermez-node/eth"
"github.com/hermeznetwork/hermez-node/log" "github.com/hermeznetwork/hermez-node/log"
"github.com/hermeznetwork/hermez-node/test"
"github.com/hermeznetwork/hermez-node/txselector" "github.com/hermeznetwork/hermez-node/txselector"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
@ -77,7 +77,7 @@ func (cn *CoordNode) Start() {
} else if err != nil { } else if err != nil {
log.Errorw("CoordNode ForgeLoopFn", "error", err) log.Errorw("CoordNode ForgeLoopFn", "error", err)
} else if !forge { } else if !forge {
time.Sleep(500 * time.Millisecond)
time.Sleep(200 * time.Millisecond)
} }
} }
} }
@ -121,30 +121,53 @@ func (cn *CoordNode) Stop() {
cn.stopForgeCallConfirm <- true cn.stopForgeCallConfirm <- true
} }
type timer struct {
time int64
}
func (t *timer) Time() int64 {
currentTime := t.time
t.time++
return currentTime
}
func TestCoordinator(t *testing.T) { func TestCoordinator(t *testing.T) {
txsel, bb := newTestModules(t) txsel, bb := newTestModules(t)
conf := Config{} conf := Config{}
hdb := &historydb.HistoryDB{} hdb := &historydb.HistoryDB{}
c := NewCoordinator(conf, hdb, txsel, bb, &eth.Client{})
serverProofs := []ServerProofInterface{&ServerProof{}, &ServerProof{}}
var timer timer
ethClientSetup := test.NewClientSetupExample()
addr := ethClientSetup.AuctionVariables.BootCoordinator
ethClient := test.NewClient(true, &timer, addr, ethClientSetup)
c := NewCoordinator(conf, hdb, txsel, bb, serverProofs, ethClient)
cn := NewCoordNode(c) cn := NewCoordNode(c)
cn.Start() cn.Start()
time.Sleep(1 * time.Second) time.Sleep(1 * time.Second)
// simulate forgeSequence time // simulate forgeSequence time
log.Info("simulate entering in forge time") log.Info("simulate entering in forge time")
c.rw.Lock()
c.isForgeSeq = true c.isForgeSeq = true
c.rw.Unlock()
time.Sleep(1 * time.Second) time.Sleep(1 * time.Second)
// simulate going out from forgeSequence // simulate going out from forgeSequence
log.Info("simulate going out from forge time") log.Info("simulate going out from forge time")
c.rw.Lock()
c.isForgeSeq = false c.isForgeSeq = false
c.rw.Unlock()
time.Sleep(1 * time.Second) time.Sleep(1 * time.Second)
// simulate entering forgeSequence time again // simulate entering forgeSequence time again
log.Info("simulate entering in forge time again") log.Info("simulate entering in forge time again")
c.rw.Lock()
c.isForgeSeq = true c.isForgeSeq = true
time.Sleep(1 * time.Second)
c.rw.Unlock()
time.Sleep(2 * time.Second)
// simulate stopping forgerLoop by channel // simulate stopping forgerLoop by channel
log.Info("simulate stopping forgerLoop by closing coordinator stopch") log.Info("simulate stopping forgerLoop by closing coordinator stopch")

+ 64
- 11
coordinator/proofpool.go

@ -1,39 +1,92 @@
package coordinator package coordinator
import ( import (
"time"
"github.com/hermeznetwork/hermez-node/common" "github.com/hermeznetwork/hermez-node/common"
"github.com/hermeznetwork/hermez-node/log" "github.com/hermeznetwork/hermez-node/log"
) )
// ServerProofInfo contains the data related to a ServerProof
type ServerProofInfo struct {
// ServerProofInterface is the interface to a ServerProof that calculates zk proofs
type ServerProofInterface interface {
CalculateProof(zkInputs *common.ZKInputs) error
GetProof(stopCh chan bool) (*Proof, error)
}
// ServerProof contains the data related to a ServerProof
type ServerProof struct {
// TODO // TODO
URL string
Available bool Available bool
} }
// NewServerProof creates a new ServerProof
func NewServerProof(URL string) *ServerProof {
return &ServerProof{URL: URL}
}
// CalculateProof sends the *common.ZKInputs to the ServerProof to compute the // CalculateProof sends the *common.ZKInputs to the ServerProof to compute the
// Proof // Proof
func (p *ServerProofInfo) CalculateProof(zkInputs *common.ZKInputs) error {
func (p *ServerProof) CalculateProof(zkInputs *common.ZKInputs) error {
return errTODO
}
// GetProof retreives the Proof from the ServerProof
func (p *ServerProof) GetProof(stopCh chan bool) (*Proof, error) {
return nil, errTODO
}
// ServerProofMock is a mock ServerProof to be used in tests. It doesn't calculate anything
type ServerProofMock struct {
}
// CalculateProof sends the *common.ZKInputs to the ServerProof to compute the
// Proof
func (p *ServerProofMock) CalculateProof(zkInputs *common.ZKInputs) error {
return nil return nil
} }
// GetProof retreives the Proof from the ServerProof // GetProof retreives the Proof from the ServerProof
func (p *ServerProofInfo) GetProof() (*Proof, error) {
return nil, nil
func (p *ServerProofMock) GetProof(stopCh chan bool) (*Proof, error) {
// Simulate a delay
select {
case <-time.After(200 * time.Millisecond): //nolint:gomnd
return &Proof{}, nil
case <-stopCh:
return nil, ErrStop
}
} }
// ServerProofPool contains the multiple ServerProofInfo
// ServerProofPool contains the multiple ServerProof
type ServerProofPool struct { type ServerProofPool struct {
// pool []ServerProofInfo
pool chan ServerProofInterface
}
// NewServerProofPool creates a new pool of ServerProofs.
func NewServerProofPool(maxServerProofs int) *ServerProofPool {
return &ServerProofPool{
pool: make(chan ServerProofInterface, maxServerProofs),
}
}
// Add a ServerProof to the pool
func (p *ServerProofPool) Add(serverProof ServerProofInterface) {
p.pool <- serverProof
} }
// GetNextAvailable returns the available ServerProofInfo
func (p *ServerProofPool) GetNextAvailable(stopCh chan bool) (*ServerProofInfo, error) {
// Get returns the next available ServerProof
func (p *ServerProofPool) Get(stopCh chan bool) (ServerProofInterface, error) {
select { select {
case <-stopCh: case <-stopCh:
log.Info("ServerProofPool.GetNextAvailable stopped")
log.Info("ServerProofPool.Get stopped")
return nil, ErrStop return nil, ErrStop
default: default:
select {
case <-stopCh:
log.Info("ServerProofPool.Get stopped")
return nil, ErrStop
case serverProof := <-p.pool:
return serverProof, nil
}
} }
return nil, nil
} }

+ 100
- 62
eth/auction.go

@ -7,6 +7,7 @@ import (
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/ethclient" "github.com/ethereum/go-ethereum/ethclient"
HermezAuctionProtocol "github.com/hermeznetwork/hermez-node/eth/contracts/auction" HermezAuctionProtocol "github.com/hermeznetwork/hermez-node/eth/contracts/auction"
"github.com/hermeznetwork/hermez-node/log"
) )
// AuctionConstants are the constants of the Rollup Smart Contract // AuctionConstants are the constants of the Rollup Smart Contract
@ -35,6 +36,16 @@ type SlotState struct {
Fulfilled bool Fulfilled bool
} }
// NewSlotState returns an empty SlotState
func NewSlotState() *SlotState {
return &SlotState{
Forger: ethCommon.Address{},
BidAmount: big.NewInt(0),
ClosedMinBid: big.NewInt(0),
Fulfilled: false,
}
}
// Coordinator is the details of the Coordinator identified by the forger address // Coordinator is the details of the Coordinator identified by the forger address
type Coordinator struct { type Coordinator struct {
WithdrawalAddress ethCommon.Address WithdrawalAddress ethCommon.Address
@ -48,15 +59,15 @@ type AuctionVariables struct {
// Boot Coordinator Address // Boot Coordinator Address
BootCoordinator ethCommon.Address BootCoordinator ethCommon.Address
// The minimum bid value in a series of 6 slots // The minimum bid value in a series of 6 slots
MinBidEpoch [6]*big.Int
DefaultSlotSetBid [6]*big.Int
// Distance (#slots) to the closest slot to which you can bid ( 2 Slots = 2 * 40 Blocks = 20 min ) // Distance (#slots) to the closest slot to which you can bid ( 2 Slots = 2 * 40 Blocks = 20 min )
ClosedAuctionSlots uint16 ClosedAuctionSlots uint16
// Distance (#slots) to the farthest slot to which you can bid (30 days = 4320 slots ) // Distance (#slots) to the farthest slot to which you can bid (30 days = 4320 slots )
OpenAuctionSlots uint16 OpenAuctionSlots uint16
// How the HEZ tokens deposited by the slot winner are distributed (Burn: 40% - Donation: 40% - HGT: 20%) // How the HEZ tokens deposited by the slot winner are distributed (Burn: 40% - Donation: 40% - HGT: 20%)
AllocationRatio [3]uint8
AllocationRatio [3]uint16
// Minimum outbid (percentage) over the previous one to consider it valid // Minimum outbid (percentage) over the previous one to consider it valid
Outbidding uint8
Outbidding uint16
// Number of blocks at the end of a slot in which any coordinator can forge if the winner has not forged one before // Number of blocks at the end of a slot in which any coordinator can forge if the winner has not forged one before
SlotDeadline uint8 SlotDeadline uint8
} }
@ -90,7 +101,7 @@ type AuctionEventNewClosedAuctionSlots struct {
// AuctionEventNewOutbidding is an event of the Auction Smart Contract // AuctionEventNewOutbidding is an event of the Auction Smart Contract
type AuctionEventNewOutbidding struct { type AuctionEventNewOutbidding struct {
NewOutbidding uint8
NewOutbidding uint16
} }
// AuctionEventNewDonationAddress is an event of the Auction Smart Contract // AuctionEventNewDonationAddress is an event of the Auction Smart Contract
@ -110,7 +121,7 @@ type AuctionEventNewOpenAuctionSlots struct {
// AuctionEventNewAllocationRatio is an event of the Auction Smart Contract // AuctionEventNewAllocationRatio is an event of the Auction Smart Contract
type AuctionEventNewAllocationRatio struct { type AuctionEventNewAllocationRatio struct {
NewAllocationRatio [3]uint8
NewAllocationRatio [3]uint16
} }
// AuctionEventNewCoordinator is an event of the Auction Smart Contract // AuctionEventNewCoordinator is an event of the Auction Smart Contract
@ -136,9 +147,9 @@ type AuctionEventNewForgeAllocated struct {
GovernanceAmount *big.Int GovernanceAmount *big.Int
} }
// AuctionEventNewMinBidEpoch is an event of the Auction Smart Contract
type AuctionEventNewMinBidEpoch struct {
SlotEpoch int64
// AuctionEventNewDefaultSlotSetBid is an event of the Auction Smart Contract
type AuctionEventNewDefaultSlotSetBid struct {
SlotSet int64
NewInitialMinBid *big.Int NewInitialMinBid *big.Int
} }
@ -167,7 +178,7 @@ type AuctionEvents struct { //nolint:structcheck
NewCoordinator []AuctionEventNewCoordinator NewCoordinator []AuctionEventNewCoordinator
CoordinatorUpdated []AuctionEventCoordinatorUpdated CoordinatorUpdated []AuctionEventCoordinatorUpdated
NewForgeAllocated []AuctionEventNewForgeAllocated NewForgeAllocated []AuctionEventNewForgeAllocated
NewMinBidEpoch []AuctionEventNewMinBidEpoch
NewDefaultSlotSetBid []AuctionEventNewDefaultSlotSetBid
NewForge []AuctionEventNewForge NewForge []AuctionEventNewForge
HEZClaimed []AuctionEventHEZClaimed HEZClaimed []AuctionEventHEZClaimed
} }
@ -186,7 +197,7 @@ func NewAuctionEvents() AuctionEvents {
NewCoordinator: make([]AuctionEventNewCoordinator, 0), NewCoordinator: make([]AuctionEventNewCoordinator, 0),
CoordinatorUpdated: make([]AuctionEventCoordinatorUpdated, 0), CoordinatorUpdated: make([]AuctionEventCoordinatorUpdated, 0),
NewForgeAllocated: make([]AuctionEventNewForgeAllocated, 0), NewForgeAllocated: make([]AuctionEventNewForgeAllocated, 0),
NewMinBidEpoch: make([]AuctionEventNewMinBidEpoch, 0),
NewDefaultSlotSetBid: make([]AuctionEventNewDefaultSlotSetBid, 0),
NewForge: make([]AuctionEventNewForge, 0), NewForge: make([]AuctionEventNewForge, 0),
HEZClaimed: make([]AuctionEventHEZClaimed, 0), HEZClaimed: make([]AuctionEventHEZClaimed, 0),
} }
@ -205,15 +216,15 @@ type AuctionInterface interface {
AuctionGetOpenAuctionSlots() (uint16, error) AuctionGetOpenAuctionSlots() (uint16, error)
AuctionSetClosedAuctionSlots(newClosedAuctionSlots uint16) (*types.Transaction, error) AuctionSetClosedAuctionSlots(newClosedAuctionSlots uint16) (*types.Transaction, error)
AuctionGetClosedAuctionSlots() (uint16, error) AuctionGetClosedAuctionSlots() (uint16, error)
AuctionSetOutbidding(newOutbidding uint8) (*types.Transaction, error)
AuctionGetOutbidding() (uint8, error)
AuctionSetAllocationRatio(newAllocationRatio [3]uint8) (*types.Transaction, error)
AuctionGetAllocationRatio() ([3]uint8, error)
AuctionSetOutbidding(newOutbidding uint16) (*types.Transaction, error)
AuctionGetOutbidding() (uint16, error)
AuctionSetAllocationRatio(newAllocationRatio [3]uint16) (*types.Transaction, error)
AuctionGetAllocationRatio() ([3]uint16, error)
AuctionSetDonationAddress(newDonationAddress ethCommon.Address) (*types.Transaction, error) AuctionSetDonationAddress(newDonationAddress ethCommon.Address) (*types.Transaction, error)
AuctionGetDonationAddress() (*ethCommon.Address, error) AuctionGetDonationAddress() (*ethCommon.Address, error)
AuctionSetBootCoordinator(newBootCoordinator ethCommon.Address) (*types.Transaction, error) AuctionSetBootCoordinator(newBootCoordinator ethCommon.Address) (*types.Transaction, error)
AuctionGetBootCoordinator() (*ethCommon.Address, error) AuctionGetBootCoordinator() (*ethCommon.Address, error)
AuctionChangeEpochMinBid(slotEpoch int64, newInitialMinBid *big.Int) (*types.Transaction, error)
AuctionChangeDefaultSlotSetBid(slotSet int64, newInitialMinBid *big.Int) (*types.Transaction, error)
// Coordinator Management // Coordinator Management
AuctionRegisterCoordinator(forgerAddress ethCommon.Address, URL string) (*types.Transaction, error) AuctionRegisterCoordinator(forgerAddress ethCommon.Address, URL string) (*types.Transaction, error)
@ -223,13 +234,13 @@ type AuctionInterface interface {
// Slot Info // Slot Info
AuctionGetCurrentSlotNumber() (int64, error) AuctionGetCurrentSlotNumber() (int64, error)
AuctionGetMinBidBySlot(slot int64) (*big.Int, error) AuctionGetMinBidBySlot(slot int64) (*big.Int, error)
AuctionGetMinBidEpoch(epoch uint8) (*big.Int, error)
AuctionGetDefaultSlotSetBid(slotSet uint8) (*big.Int, error)
// Bidding // Bidding
// AuctionTokensReceived(operator, from, to ethCommon.Address, amount *big.Int, // AuctionTokensReceived(operator, from, to ethCommon.Address, amount *big.Int,
// userData, operatorData []byte) error // Only called from another smart contract // userData, operatorData []byte) error // Only called from another smart contract
AuctionBid(slot int64, bidAmount *big.Int, forger ethCommon.Address) (*types.Transaction, error) AuctionBid(slot int64, bidAmount *big.Int, forger ethCommon.Address) (*types.Transaction, error)
AuctionMultiBid(startingSlot int64, endingSlot int64, slotEpoch [6]bool,
AuctionMultiBid(startingSlot int64, endingSlot int64, slotSet [6]bool,
maxBid, closedMinBid, budget *big.Int, forger ethCommon.Address) (*types.Transaction, error) maxBid, closedMinBid, budget *big.Int, forger ethCommon.Address) (*types.Transaction, error)
// Forge // Forge
@ -267,6 +278,7 @@ func NewAuctionClient(client *EthereumClient, address ethCommon.Address) *Auctio
// AuctionSetSlotDeadline is the interface to call the smart contract function // AuctionSetSlotDeadline is the interface to call the smart contract function
func (c *AuctionClient) AuctionSetSlotDeadline(newDeadline uint8) (*types.Transaction, error) { func (c *AuctionClient) AuctionSetSlotDeadline(newDeadline uint8) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -288,6 +300,7 @@ func (c *AuctionClient) AuctionGetSlotDeadline() (uint8, error) {
// AuctionSetOpenAuctionSlots is the interface to call the smart contract function // AuctionSetOpenAuctionSlots is the interface to call the smart contract function
func (c *AuctionClient) AuctionSetOpenAuctionSlots(newOpenAuctionSlots uint16) (*types.Transaction, error) { func (c *AuctionClient) AuctionSetOpenAuctionSlots(newOpenAuctionSlots uint16) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -309,6 +322,7 @@ func (c *AuctionClient) AuctionGetOpenAuctionSlots() (uint16, error) {
// AuctionSetClosedAuctionSlots is the interface to call the smart contract function // AuctionSetClosedAuctionSlots is the interface to call the smart contract function
func (c *AuctionClient) AuctionSetClosedAuctionSlots(newClosedAuctionSlots uint16) (*types.Transaction, error) { func (c *AuctionClient) AuctionSetClosedAuctionSlots(newClosedAuctionSlots uint16) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -329,49 +343,58 @@ func (c *AuctionClient) AuctionGetClosedAuctionSlots() (uint16, error) {
} }
// AuctionSetOutbidding is the interface to call the smart contract function // AuctionSetOutbidding is the interface to call the smart contract function
func (c *AuctionClient) AuctionSetOutbidding(newOutbidding uint8) (*types.Transaction, error) {
func (c *AuctionClient) AuctionSetOutbidding(newOutbidding uint16) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// AuctionGetOutbidding is the interface to call the smart contract function // AuctionGetOutbidding is the interface to call the smart contract function
func (c *AuctionClient) AuctionGetOutbidding() (uint8, error) {
var outbidding uint8
if err := c.client.Call(func(ec *ethclient.Client) error {
auction, err := HermezAuctionProtocol.NewHermezAuctionProtocol(c.address, ec)
if err != nil {
return err
}
outbidding, err = auction.GetOutbidding(nil)
return err
}); err != nil {
return 0, err
}
return outbidding, nil
func (c *AuctionClient) AuctionGetOutbidding() (uint16, error) {
// TODO: Update
// var outbidding uint8
// if err := c.client.Call(func(ec *ethclient.Client) error {
// auction, err := HermezAuctionProtocol.NewHermezAuctionProtocol(c.address, ec)
// if err != nil {
// return err
// }
// outbidding, err = auction.GetOutbidding(nil)
// return err
// }); err != nil {
// return 0, err
// }
// return outbidding, nil
log.Error("TODO")
return 0, errTODO
} }
// AuctionSetAllocationRatio is the interface to call the smart contract function // AuctionSetAllocationRatio is the interface to call the smart contract function
func (c *AuctionClient) AuctionSetAllocationRatio(newAllocationRatio [3]uint8) (*types.Transaction, error) {
func (c *AuctionClient) AuctionSetAllocationRatio(newAllocationRatio [3]uint16) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// AuctionGetAllocationRatio is the interface to call the smart contract function // AuctionGetAllocationRatio is the interface to call the smart contract function
func (c *AuctionClient) AuctionGetAllocationRatio() ([3]uint8, error) {
var allocationRation [3]uint8
if err := c.client.Call(func(ec *ethclient.Client) error {
auction, err := HermezAuctionProtocol.NewHermezAuctionProtocol(c.address, ec)
if err != nil {
return err
}
allocationRation, err = auction.GetAllocationRatio(nil)
return err
}); err != nil {
return [3]uint8{}, err
}
return allocationRation, nil
func (c *AuctionClient) AuctionGetAllocationRatio() ([3]uint16, error) {
// TODO: Update
// var allocationRation [3]uint8
// if err := c.client.Call(func(ec *ethclient.Client) error {
// auction, err := HermezAuctionProtocol.NewHermezAuctionProtocol(c.address, ec)
// if err != nil {
// return err
// }
// allocationRation, err = auction.GetAllocationRatio(nil)
// return err
// }); err != nil {
// return [3]uint8{}, err
// }
// return allocationRation, nil
log.Error("TODO")
return [3]uint16{}, errTODO
} }
// AuctionSetDonationAddress is the interface to call the smart contract function // AuctionSetDonationAddress is the interface to call the smart contract function
func (c *AuctionClient) AuctionSetDonationAddress(newDonationAddress ethCommon.Address) (*types.Transaction, error) { func (c *AuctionClient) AuctionSetDonationAddress(newDonationAddress ethCommon.Address) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -393,6 +416,7 @@ func (c *AuctionClient) AuctionGetDonationAddress() (*ethCommon.Address, error)
// AuctionSetBootCoordinator is the interface to call the smart contract function // AuctionSetBootCoordinator is the interface to call the smart contract function
func (c *AuctionClient) AuctionSetBootCoordinator(newBootCoordinator ethCommon.Address) (*types.Transaction, error) { func (c *AuctionClient) AuctionSetBootCoordinator(newBootCoordinator ethCommon.Address) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -412,23 +436,27 @@ func (c *AuctionClient) AuctionGetBootCoordinator() (*ethCommon.Address, error)
return &bootCoordinator, nil return &bootCoordinator, nil
} }
// AuctionChangeEpochMinBid is the interface to call the smart contract function
func (c *AuctionClient) AuctionChangeEpochMinBid(slotEpoch int64, newInitialMinBid *big.Int) (*types.Transaction, error) {
// AuctionChangeDefaultSlotSetBid is the interface to call the smart contract function
func (c *AuctionClient) AuctionChangeDefaultSlotSetBid(slotSet int64, newInitialMinBid *big.Int) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// AuctionRegisterCoordinator is the interface to call the smart contract function // AuctionRegisterCoordinator is the interface to call the smart contract function
func (c *AuctionClient) AuctionRegisterCoordinator(forgerAddress ethCommon.Address, URL string) (*types.Transaction, error) { func (c *AuctionClient) AuctionRegisterCoordinator(forgerAddress ethCommon.Address, URL string) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// AuctionIsRegisteredCoordinator is the interface to call the smart contract function // AuctionIsRegisteredCoordinator is the interface to call the smart contract function
func (c *AuctionClient) AuctionIsRegisteredCoordinator(forgerAddress ethCommon.Address) (bool, error) { func (c *AuctionClient) AuctionIsRegisteredCoordinator(forgerAddress ethCommon.Address) (bool, error) {
log.Error("TODO")
return false, errTODO return false, errTODO
} }
// AuctionUpdateCoordinatorInfo is the interface to call the smart contract function // AuctionUpdateCoordinatorInfo is the interface to call the smart contract function
func (c *AuctionClient) AuctionUpdateCoordinatorInfo(forgerAddress ethCommon.Address, newWithdrawAddress ethCommon.Address, newURL string) (*types.Transaction, error) { func (c *AuctionClient) AuctionUpdateCoordinatorInfo(forgerAddress ethCommon.Address, newWithdrawAddress ethCommon.Address, newURL string) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -466,20 +494,24 @@ func (c *AuctionClient) AuctionGetMinBidBySlot(slot int64) (*big.Int, error) {
return minBid, nil return minBid, nil
} }
// AuctionGetMinBidEpoch is the interface to call the smart contract function
func (c *AuctionClient) AuctionGetMinBidEpoch(epoch uint8) (*big.Int, error) {
var minBidEpoch *big.Int
if err := c.client.Call(func(ec *ethclient.Client) error {
auction, err := HermezAuctionProtocol.NewHermezAuctionProtocol(c.address, ec)
if err != nil {
return err
}
minBidEpoch, err = auction.GetMinBidEpoch(nil, epoch)
return err
}); err != nil {
return big.NewInt(0), err
}
return minBidEpoch, nil
// AuctionGetDefaultSlotSetBid is the interface to call the smart contract function
func (c *AuctionClient) AuctionGetDefaultSlotSetBid(slotSet uint8) (*big.Int, error) {
// TODO: Update
// var DefaultSlotSetBid *big.Int
// if err := c.client.Call(func(ec *ethclient.Client) error {
// auction, err := HermezAuctionProtocol.NewHermezAuctionProtocol(c.address, ec)
// if err != nil {
// return err
// }
// defaultSlotSetBid, err = auction.GetDefaultSlotSetBid(nil, slotSet)
// return err
// }); err != nil {
// return big.NewInt(0), err
// }
// return defaultSlotSetBid, nil
log.Error("TODO")
return nil, errTODO
} }
// AuctionTokensReceived is the interface to call the smart contract function // AuctionTokensReceived is the interface to call the smart contract function
@ -489,16 +521,19 @@ func (c *AuctionClient) AuctionGetMinBidEpoch(epoch uint8) (*big.Int, error) {
// AuctionBid is the interface to call the smart contract function // AuctionBid is the interface to call the smart contract function
func (c *AuctionClient) AuctionBid(slot int64, bidAmount *big.Int, forger ethCommon.Address) (*types.Transaction, error) { func (c *AuctionClient) AuctionBid(slot int64, bidAmount *big.Int, forger ethCommon.Address) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// AuctionMultiBid is the interface to call the smart contract function // AuctionMultiBid is the interface to call the smart contract function
func (c *AuctionClient) AuctionMultiBid(startingSlot int64, endingSlot int64, slotEpoch [6]bool, maxBid, closedMinBid, budget *big.Int, forger ethCommon.Address) (*types.Transaction, error) {
func (c *AuctionClient) AuctionMultiBid(startingSlot int64, endingSlot int64, slotSet [6]bool, maxBid, closedMinBid, budget *big.Int, forger ethCommon.Address) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// AuctionCanForge is the interface to call the smart contract function // AuctionCanForge is the interface to call the smart contract function
func (c *AuctionClient) AuctionCanForge(forger ethCommon.Address) (bool, error) { func (c *AuctionClient) AuctionCanForge(forger ethCommon.Address) (bool, error) {
log.Error("TODO")
return false, errTODO return false, errTODO
} }
@ -509,15 +544,18 @@ func (c *AuctionClient) AuctionCanForge(forger ethCommon.Address) (bool, error)
// AuctionClaimHEZ is the interface to call the smart contract function // AuctionClaimHEZ is the interface to call the smart contract function
func (c *AuctionClient) AuctionClaimHEZ() (*types.Transaction, error) { func (c *AuctionClient) AuctionClaimHEZ() (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// AuctionConstants returns the Constants of the Auction Smart Contract // AuctionConstants returns the Constants of the Auction Smart Contract
func (c *AuctionClient) AuctionConstants() (*AuctionConstants, error) { func (c *AuctionClient) AuctionConstants() (*AuctionConstants, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// AuctionEventsByBlock returns the events in a block that happened in the Auction Smart Contract // AuctionEventsByBlock returns the events in a block that happened in the Auction Smart Contract
func (c *AuctionClient) AuctionEventsByBlock(blockNum int64) (*AuctionEvents, *ethCommon.Hash, error) { func (c *AuctionClient) AuctionEventsByBlock(blockNum int64) (*AuctionEvents, *ethCommon.Hash, error) {
log.Error("TODO")
return nil, nil, errTODO return nil, nil, errTODO
} }

+ 21
- 0
eth/rollup.go

@ -6,6 +6,7 @@ import (
ethCommon "github.com/ethereum/go-ethereum/common" ethCommon "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/hermeznetwork/hermez-node/common" "github.com/hermeznetwork/hermez-node/common"
"github.com/hermeznetwork/hermez-node/log"
"github.com/hermeznetwork/hermez-node/utils" "github.com/hermeznetwork/hermez-node/utils"
"github.com/iden3/go-iden3-crypto/babyjub" "github.com/iden3/go-iden3-crypto/babyjub"
) )
@ -219,11 +220,13 @@ type RollupClient struct {
// RollupForgeBatch is the interface to call the smart contract function // RollupForgeBatch is the interface to call the smart contract function
func (c *RollupClient) RollupForgeBatch(args *RollupForgeBatchArgs) (*types.Transaction, error) { func (c *RollupClient) RollupForgeBatch(args *RollupForgeBatchArgs) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupAddToken is the interface to call the smart contract function // RollupAddToken is the interface to call the smart contract function
func (c *RollupClient) RollupAddToken(tokenAddress ethCommon.Address) (*types.Transaction, error) { func (c *RollupClient) RollupAddToken(tokenAddress ethCommon.Address) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -234,76 +237,91 @@ func (c *RollupClient) RollupAddToken(tokenAddress ethCommon.Address) (*types.Tr
// RollupWithdrawMerkleProof is the interface to call the smart contract function // RollupWithdrawMerkleProof is the interface to call the smart contract function
func (c *RollupClient) RollupWithdrawMerkleProof(tokenID int64, balance *big.Int, babyPubKey *babyjub.PublicKey, numExitRoot int64, siblings []*big.Int, idx int64, instantWithdraw bool) (*types.Transaction, error) { func (c *RollupClient) RollupWithdrawMerkleProof(tokenID int64, balance *big.Int, babyPubKey *babyjub.PublicKey, numExitRoot int64, siblings []*big.Int, idx int64, instantWithdraw bool) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupForceExit is the interface to call the smart contract function // RollupForceExit is the interface to call the smart contract function
func (c *RollupClient) RollupForceExit(fromIdx int64, amountF utils.Float16, tokenID int64) (*types.Transaction, error) { func (c *RollupClient) RollupForceExit(fromIdx int64, amountF utils.Float16, tokenID int64) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupForceTransfer is the interface to call the smart contract function // RollupForceTransfer is the interface to call the smart contract function
func (c *RollupClient) RollupForceTransfer(fromIdx int64, amountF utils.Float16, tokenID, toIdx int64) (*types.Transaction, error) { func (c *RollupClient) RollupForceTransfer(fromIdx int64, amountF utils.Float16, tokenID, toIdx int64) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupCreateAccountDepositTransfer is the interface to call the smart contract function // RollupCreateAccountDepositTransfer is the interface to call the smart contract function
func (c *RollupClient) RollupCreateAccountDepositTransfer(babyPubKey babyjub.PublicKey, loadAmountF, amountF utils.Float16, tokenID int64, toIdx int64) (*types.Transaction, error) { func (c *RollupClient) RollupCreateAccountDepositTransfer(babyPubKey babyjub.PublicKey, loadAmountF, amountF utils.Float16, tokenID int64, toIdx int64) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupDepositTransfer is the interface to call the smart contract function // RollupDepositTransfer is the interface to call the smart contract function
func (c *RollupClient) RollupDepositTransfer(fromIdx int64, loadAmountF, amountF utils.Float16, tokenID int64, toIdx int64) (*types.Transaction, error) { func (c *RollupClient) RollupDepositTransfer(fromIdx int64, loadAmountF, amountF utils.Float16, tokenID int64, toIdx int64) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupDeposit is the interface to call the smart contract function // RollupDeposit is the interface to call the smart contract function
func (c *RollupClient) RollupDeposit(fromIdx int64, loadAmountF utils.Float16, tokenID int64) (*types.Transaction, error) { func (c *RollupClient) RollupDeposit(fromIdx int64, loadAmountF utils.Float16, tokenID int64) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupCreateAccountDepositFromRelayer is the interface to call the smart contract function // RollupCreateAccountDepositFromRelayer is the interface to call the smart contract function
func (c *RollupClient) RollupCreateAccountDepositFromRelayer(accountCreationAuthSig []byte, babyPubKey babyjub.PublicKey, loadAmountF utils.Float16) (*types.Transaction, error) { func (c *RollupClient) RollupCreateAccountDepositFromRelayer(accountCreationAuthSig []byte, babyPubKey babyjub.PublicKey, loadAmountF utils.Float16) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupCreateAccountDeposit is the interface to call the smart contract function // RollupCreateAccountDeposit is the interface to call the smart contract function
func (c *RollupClient) RollupCreateAccountDeposit(babyPubKey babyjub.PublicKey, loadAmountF utils.Float16, tokenID int64) (*types.Transaction, error) { func (c *RollupClient) RollupCreateAccountDeposit(babyPubKey babyjub.PublicKey, loadAmountF utils.Float16, tokenID int64) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupGetTokenAddress is the interface to call the smart contract function // RollupGetTokenAddress is the interface to call the smart contract function
func (c *RollupClient) RollupGetTokenAddress(tokenID int64) (*ethCommon.Address, error) { func (c *RollupClient) RollupGetTokenAddress(tokenID int64) (*ethCommon.Address, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupGetL1TxFromQueue is the interface to call the smart contract function // RollupGetL1TxFromQueue is the interface to call the smart contract function
func (c *RollupClient) RollupGetL1TxFromQueue(queue int64, position int64) ([]byte, error) { func (c *RollupClient) RollupGetL1TxFromQueue(queue int64, position int64) ([]byte, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupGetQueue is the interface to call the smart contract function // RollupGetQueue is the interface to call the smart contract function
func (c *RollupClient) RollupGetQueue(queue int64) ([]byte, error) { func (c *RollupClient) RollupGetQueue(queue int64) ([]byte, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupUpdateForgeL1Timeout is the interface to call the smart contract function // RollupUpdateForgeL1Timeout is the interface to call the smart contract function
func (c *RollupClient) RollupUpdateForgeL1Timeout(newForgeL1Timeout int64) (*types.Transaction, error) { func (c *RollupClient) RollupUpdateForgeL1Timeout(newForgeL1Timeout int64) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupUpdateFeeL1UserTx is the interface to call the smart contract function // RollupUpdateFeeL1UserTx is the interface to call the smart contract function
func (c *RollupClient) RollupUpdateFeeL1UserTx(newFeeL1UserTx *big.Int) (*types.Transaction, error) { func (c *RollupClient) RollupUpdateFeeL1UserTx(newFeeL1UserTx *big.Int) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupUpdateFeeAddToken is the interface to call the smart contract function // RollupUpdateFeeAddToken is the interface to call the smart contract function
func (c *RollupClient) RollupUpdateFeeAddToken(newFeeAddToken *big.Int) (*types.Transaction, error) { func (c *RollupClient) RollupUpdateFeeAddToken(newFeeAddToken *big.Int) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupUpdateTokensHEZ is the interface to call the smart contract function // RollupUpdateTokensHEZ is the interface to call the smart contract function
func (c *RollupClient) RollupUpdateTokensHEZ(newTokenHEZ ethCommon.Address) (*types.Transaction, error) { func (c *RollupClient) RollupUpdateTokensHEZ(newTokenHEZ ethCommon.Address) (*types.Transaction, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -314,11 +332,13 @@ func (c *RollupClient) RollupUpdateTokensHEZ(newTokenHEZ ethCommon.Address) (*ty
// RollupConstants returns the Constants of the Rollup Smart Contract // RollupConstants returns the Constants of the Rollup Smart Contract
func (c *RollupClient) RollupConstants() (*RollupConstants, error) { func (c *RollupClient) RollupConstants() (*RollupConstants, error) {
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupEventsByBlock returns the events in a block that happened in the Rollup Smart Contract // RollupEventsByBlock returns the events in a block that happened in the Rollup Smart Contract
func (c *RollupClient) RollupEventsByBlock(blockNum int64) (*RollupEvents, *ethCommon.Hash, error) { func (c *RollupClient) RollupEventsByBlock(blockNum int64) (*RollupEvents, *ethCommon.Hash, error) {
log.Error("TODO")
return nil, nil, errTODO return nil, nil, errTODO
} }
@ -329,5 +349,6 @@ func (c *RollupClient) RollupForgeBatchArgs(ethTxHash ethCommon.Hash) (*RollupFo
// m := abi.MethodById(txData) -> Method // m := abi.MethodById(txData) -> Method
// m.Inputs.Unpack(txData) -> Args // m.Inputs.Unpack(txData) -> Args
// client.TransactionReceipt()? // client.TransactionReceipt()?
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }

+ 2
- 1
go.mod

@ -20,6 +20,7 @@ require (
github.com/stretchr/testify v1.6.1 github.com/stretchr/testify v1.6.1
github.com/urfave/cli v1.22.1 github.com/urfave/cli v1.22.1
github.com/urfave/cli/v2 v2.2.0 github.com/urfave/cli/v2 v2.2.0
go.uber.org/zap v1.13.0
go.uber.org/multierr v1.6.0 // indirect
go.uber.org/zap v1.16.0
gopkg.in/go-playground/validator.v9 v9.29.1 gopkg.in/go-playground/validator.v9 v9.29.1
) )

+ 10
- 0
go.sum

@ -601,13 +601,23 @@ go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
go.uber.org/atomic v1.5.0 h1:OI5t8sDa1Or+q8AeE+yKeB/SDYioSHAgcVljj9JIETY= go.uber.org/atomic v1.5.0 h1:OI5t8sDa1Or+q8AeE+yKeB/SDYioSHAgcVljj9JIETY=
go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk=
go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw=
go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc=
go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
go.uber.org/multierr v1.3.0 h1:sFPn2GLc3poCkfrpIXGhBD2X0CMIo4Q/zSULXrj/+uc= go.uber.org/multierr v1.3.0 h1:sFPn2GLc3poCkfrpIXGhBD2X0CMIo4Q/zSULXrj/+uc=
go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4=
go.uber.org/multierr v1.5.0 h1:KCa4XfM8CWFCpxXRGok+Q0SS/0XBhMDbHHGABQLvD2A=
go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU=
go.uber.org/multierr v1.6.0 h1:y6IPFStTAIT5Ytl7/XYmHvzXQ7S3g/IeZW9hyZ5thw4=
go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU=
go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA=
go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
go.uber.org/zap v1.13.0 h1:nR6NoDBgAf67s68NhaXbsojM+2gxp3S1hWkHDl27pVU= go.uber.org/zap v1.13.0 h1:nR6NoDBgAf67s68NhaXbsojM+2gxp3S1hWkHDl27pVU=
go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM=
go.uber.org/zap v1.16.0 h1:uFRZXykJGK9lLY4HtgSw44DnIcAM+kRBP7x5m+NpAOM=
go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ=
golang.org/x/crypto v0.0.0-20170930174604-9419663f5a44/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20170930174604-9419663f5a44/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=

+ 11
- 6
node/node.go

@ -86,28 +86,32 @@ func NewNode(mode Mode, cfg *config.Node, coordCfg *config.Coordinator) (*Node,
cfg.PostgreSQL.Host, cfg.PostgreSQL.Host,
cfg.PostgreSQL.User, cfg.PostgreSQL.User,
cfg.PostgreSQL.Password, cfg.PostgreSQL.Password,
cfg.L2DB.Name,
cfg.L2DB.SafetyPeriod,
cfg.L2DB.MaxTxs,
cfg.L2DB.TTL.Duration,
coordCfg.L2DB.Name,
coordCfg.L2DB.SafetyPeriod,
coordCfg.L2DB.MaxTxs,
coordCfg.L2DB.TTL.Duration,
) )
if err != nil { if err != nil {
return nil, err return nil, err
} }
// TODO: Get (maxL1UserTxs, maxL1OperatorTxs, maxTxs) from the smart contract // TODO: Get (maxL1UserTxs, maxL1OperatorTxs, maxTxs) from the smart contract
txSelector, err := txselector.NewTxSelector(cfg.TxSelector.Path, stateDB, l2DB, 10, 10, 10)
txSelector, err := txselector.NewTxSelector(coordCfg.TxSelector.Path, stateDB, l2DB, 10, 10, 10)
if err != nil { if err != nil {
return nil, err return nil, err
} }
// TODO: Get (configCircuits []ConfigCircuit, batchNum common.BatchNum, nLevels uint64) from smart contract // TODO: Get (configCircuits []ConfigCircuit, batchNum common.BatchNum, nLevels uint64) from smart contract
nLevels := uint64(32) //nolint:gomnd nLevels := uint64(32) //nolint:gomnd
batchBuilder, err := batchbuilder.NewBatchBuilder(cfg.BatchBuilder.Path, stateDB, nil, 0, nLevels)
batchBuilder, err := batchbuilder.NewBatchBuilder(coordCfg.BatchBuilder.Path, stateDB, nil, 0, nLevels)
if err != nil { if err != nil {
return nil, err return nil, err
} }
if err != nil { if err != nil {
return nil, err return nil, err
} }
serverProofs := make([]coordinator.ServerProofInterface, len(coordCfg.ServerProofs))
for i, serverProofCfg := range coordCfg.ServerProofs {
serverProofs[i] = coordinator.NewServerProof(serverProofCfg.URL)
}
coord = coordinator.NewCoordinator( coord = coordinator.NewCoordinator(
coordinator.Config{ coordinator.Config{
ForgerAddress: coordCfg.ForgerAddress, ForgerAddress: coordCfg.ForgerAddress,
@ -115,6 +119,7 @@ func NewNode(mode Mode, cfg *config.Node, coordCfg *config.Coordinator) (*Node,
historyDB, historyDB,
txSelector, txSelector,
batchBuilder, batchBuilder,
serverProofs,
client, client,
) )
} }

+ 292
- 67
test/ethclient.go

@ -65,8 +65,8 @@ func (a *AuctionBlock) getCurrentSlotNumber() int64 {
return a.getSlotNumber(a.Eth.BlockNum) return a.getSlotNumber(a.Eth.BlockNum)
} }
func (a *AuctionBlock) getEpoch(slot int64) int64 {
return slot % int64(len(a.Vars.MinBidEpoch))
func (a *AuctionBlock) getSlotSet(slot int64) int64 {
return slot % int64(len(a.Vars.DefaultSlotSetBid))
} }
func (a *AuctionBlock) getMinBidBySlot(slot int64) (*big.Int, error) { func (a *AuctionBlock) getMinBidBySlot(slot int64) (*big.Int, error) {
@ -74,23 +74,89 @@ func (a *AuctionBlock) getMinBidBySlot(slot int64) (*big.Int, error) {
return nil, errBidClosed return nil, errBidClosed
} }
epoch := a.getEpoch(slot)
slotSet := a.getSlotSet(slot)
// fmt.Println("slot:", slot, "slotSet:", slotSet)
var prevBid *big.Int var prevBid *big.Int
slotState, ok := a.State.Slots[slot] slotState, ok := a.State.Slots[slot]
if !ok {
slotState = eth.NewSlotState()
a.State.Slots[slot] = slotState
}
// If the bidAmount for a slot is 0 it means that it has not yet been bid, so the midBid will be the minimum // If the bidAmount for a slot is 0 it means that it has not yet been bid, so the midBid will be the minimum
// bid for the slot time plus the outbidding set, otherwise it will be the bidAmount plus the outbidding // bid for the slot time plus the outbidding set, otherwise it will be the bidAmount plus the outbidding
if !ok || slotState.BidAmount.Cmp(big.NewInt(0)) == 0 {
prevBid = a.Vars.MinBidEpoch[epoch]
if slotState.BidAmount.Cmp(big.NewInt(0)) == 0 {
prevBid = a.Vars.DefaultSlotSetBid[slotSet]
} else { } else {
prevBid = slotState.BidAmount prevBid = slotState.BidAmount
} }
outBid := new(big.Int).Set(prevBid) outBid := new(big.Int).Set(prevBid)
// fmt.Println("outBid:", outBid)
outBid.Mul(outBid, big.NewInt(int64(a.Vars.Outbidding))) outBid.Mul(outBid, big.NewInt(int64(a.Vars.Outbidding)))
outBid.Div(outBid, big.NewInt(100)) //nolint:gomnd
outBid.Div(outBid, big.NewInt(10000)) //nolint:gomnd
outBid.Add(prevBid, outBid) outBid.Add(prevBid, outBid)
// fmt.Println("minBid:", outBid)
return outBid, nil return outBid, nil
} }
func (a *AuctionBlock) forge(forger ethCommon.Address) error {
if ok, err := a.canForge(forger, a.Eth.BlockNum); err != nil {
return err
} else if !ok {
return fmt.Errorf("Can't forge")
}
slotToForge := a.getSlotNumber(a.Eth.BlockNum)
slotState, ok := a.State.Slots[slotToForge]
if !ok {
slotState = eth.NewSlotState()
a.State.Slots[slotToForge] = slotState
}
slotState.Fulfilled = true
a.Events.NewForge = append(a.Events.NewForge, eth.AuctionEventNewForge{
Forger: forger,
CurrentSlot: slotToForge,
})
return nil
}
func (a *AuctionBlock) canForge(forger ethCommon.Address, blockNum int64) (bool, error) {
if blockNum < a.Constants.GenesisBlockNum {
return false, fmt.Errorf("Auction has not started yet")
}
slotToForge := a.getSlotNumber(blockNum)
// Get the relativeBlock to check if the slotDeadline has been exceeded
relativeBlock := blockNum - (a.Constants.GenesisBlockNum + (slotToForge * int64(a.Constants.BlocksPerSlot)))
// If the closedMinBid is 0 it means that we have to take as minBid the one that is set for this slot set,
// otherwise the one that has been saved will be used
var minBid *big.Int
slotState, ok := a.State.Slots[slotToForge]
if !ok {
slotState = eth.NewSlotState()
a.State.Slots[slotToForge] = slotState
}
if slotState.ClosedMinBid.Cmp(big.NewInt(0)) == 0 {
minBid = a.Vars.DefaultSlotSetBid[a.getSlotSet(slotToForge)]
} else {
minBid = slotState.ClosedMinBid
}
if !slotState.Fulfilled && (relativeBlock >= int64(a.Vars.SlotDeadline)) {
// if the relative block has exceeded the slotDeadline and no batch has been forged, anyone can forge
return true, nil
} else if slotState.Forger == forger && slotState.BidAmount.Cmp(minBid) >= 0 {
// if forger bidAmount has exceeded the minBid it can forge
return true, nil
} else if a.Vars.BootCoordinator == forger && slotState.BidAmount.Cmp(minBid) == -1 {
// if it's the boot coordinator and it has not been bid or the bid is below the minimum it can forge
return true, nil
} else {
return false, nil
}
}
// EthereumBlock stores all the generic data related to the an ethereum block // EthereumBlock stores all the generic data related to the an ethereum block
type EthereumBlock struct { type EthereumBlock struct {
BlockNum int64 BlockNum int64
@ -107,13 +173,18 @@ type Block struct {
Eth *EthereumBlock Eth *EthereumBlock
} }
// Next prepares the successive block.
func (b *Block) Next() *Block {
blockNextRaw, err := copystructure.Copy(b)
func (b *Block) copy() *Block {
bCopyRaw, err := copystructure.Copy(b)
if err != nil { if err != nil {
panic(err) panic(err)
} }
blockNext := blockNextRaw.(*Block)
bCopy := bCopyRaw.(*Block)
return bCopy
}
// Next prepares the successive block.
func (b *Block) Next() *Block {
blockNext := b.copy()
blockNext.Rollup.Events = eth.NewRollupEvents() blockNext.Rollup.Events = eth.NewRollupEvents()
blockNext.Auction.Events = eth.NewAuctionEvents() blockNext.Auction.Events = eth.NewAuctionEvents()
blockNext.Eth = &EthereumBlock{ blockNext.Eth = &EthereumBlock{
@ -138,34 +209,59 @@ type ClientSetup struct {
} }
// NewClientSetupExample returns a ClientSetup example with hardcoded realistic values. // NewClientSetupExample returns a ClientSetup example with hardcoded realistic values.
// TODO: Fill all values that are currently default.
//nolint:gomnd //nolint:gomnd
func NewClientSetupExample() *ClientSetup { func NewClientSetupExample() *ClientSetup {
rollupConstants := &eth.RollupConstants{}
rfield, ok := new(big.Int).SetString("21888242871839275222246405745257275088548364400416034343698204186575808495617", 10)
if !ok {
panic("bad rfield")
}
initialMinimalBidding, ok := new(big.Int).SetString("10000000000000000000", 10) // 10 * (1e18)
if !ok {
panic("bad initialMinimalBidding")
}
tokenHEZ := ethCommon.HexToAddress("0x51D243D62852Bba334DD5cc33f242BAc8c698074")
governanceAddress := ethCommon.HexToAddress("0x688EfD95BA4391f93717CF02A9aED9DBD2855cDd")
rollupConstants := &eth.RollupConstants{
MaxAmountDeposit: new(big.Int).Lsh(big.NewInt(1), 128),
MaxAmountL2: new(big.Int).Lsh(big.NewInt(1), 192),
MaxTokens: 0xffffffff,
MaxL1Tx: 256,
MaxL1UserTx: 128,
Rfield: rfield,
L1CoordinatorBytes: 101,
L1UserBytes: 68,
L2Bytes: 11,
}
rollupVariables := &eth.RollupVariables{ rollupVariables := &eth.RollupVariables{
MaxTxVerifiers: make([]int, 0),
TokenHEZ: ethCommon.Address{},
GovernanceAddress: ethCommon.Address{},
SafetyBot: ethCommon.Address{},
ConsensusContract: ethCommon.Address{},
WithdrawalContract: ethCommon.Address{},
FeeAddToken: big.NewInt(1),
ForgeL1Timeout: 16,
FeeL1UserTx: big.NewInt(2),
MaxTxVerifiers: []int{512, 1024, 2048},
TokenHEZ: tokenHEZ,
GovernanceAddress: governanceAddress,
SafetyBot: ethCommon.HexToAddress("0x84d8B79E84fe87B14ad61A554e740f6736bF4c20"),
ConsensusContract: ethCommon.HexToAddress("0x8E442975805fb1908f43050c9C1A522cB0e28D7b"),
WithdrawalContract: ethCommon.HexToAddress("0x5CB7979cBdbf65719BEE92e4D15b7b7Ed3D79114"),
FeeAddToken: big.NewInt(11),
ForgeL1Timeout: 9,
FeeL1UserTx: big.NewInt(22),
} }
auctionConstants := &eth.AuctionConstants{ auctionConstants := &eth.AuctionConstants{
BlocksPerSlot: 40,
DelayGenesis: 0,
BlocksPerSlot: 40,
InitialMinimalBidding: initialMinimalBidding,
GenesisBlockNum: 0,
GovernanceAddress: governanceAddress,
TokenHEZ: tokenHEZ,
HermezRollup: ethCommon.HexToAddress("0x474B6e29852257491cf283EfB1A9C61eBFe48369"),
} }
auctionVariables := &eth.AuctionVariables{ auctionVariables := &eth.AuctionVariables{
DonationAddress: ethCommon.Address{},
BootCoordinator: ethCommon.Address{},
MinBidEpoch: [6]*big.Int{
big.NewInt(10), big.NewInt(11), big.NewInt(12),
big.NewInt(13), big.NewInt(14), big.NewInt(15)},
DonationAddress: ethCommon.HexToAddress("0x61Ed87CF0A1496b49A420DA6D84B58196b98f2e7"),
BootCoordinator: ethCommon.HexToAddress("0xE39fEc6224708f0772D2A74fd3f9055A90E0A9f2"),
DefaultSlotSetBid: [6]*big.Int{
big.NewInt(1000), big.NewInt(1100), big.NewInt(1200),
big.NewInt(1300), big.NewInt(1400), big.NewInt(1500)},
ClosedAuctionSlots: 2, ClosedAuctionSlots: 2,
OpenAuctionSlots: 100,
AllocationRatio: [3]uint8{},
Outbidding: 10,
OpenAuctionSlots: 4320,
AllocationRatio: [3]uint16{4000, 4000, 2000},
Outbidding: 1000,
SlotDeadline: 20, SlotDeadline: 20,
} }
return &ClientSetup{ return &ClientSetup{
@ -277,6 +373,17 @@ func NewClient(l bool, timer Timer, addr ethCommon.Address, setup *ClientSetup)
// Mock Control // Mock Control
// //
func (c *Client) setNextBlock(block *Block) {
c.blocks[c.blockNum+1] = block
}
func (c *Client) revertIfErr(err error, block *Block) {
if err != nil {
log.Infow("TestClient revert", "block", block.Eth.BlockNum, "err", err)
c.setNextBlock(block)
}
}
// Debugf calls log.Debugf if c.log is true // Debugf calls log.Debugf if c.log is true
func (c *Client) Debugf(template string, args ...interface{}) { func (c *Client) Debugf(template string, args ...interface{}) {
if c.log { if c.log {
@ -439,11 +546,30 @@ func (c *Client) newTransaction(name string, value interface{}) *types.Transacti
} }
// RollupForgeBatch is the interface to call the smart contract function // RollupForgeBatch is the interface to call the smart contract function
func (c *Client) RollupForgeBatch(*eth.RollupForgeBatchArgs) (*types.Transaction, error) {
func (c *Client) RollupForgeBatch(args *eth.RollupForgeBatchArgs) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
return nil, errTODO
a := c.nextBlock().Auction
ok, err := a.canForge(c.addr, a.Eth.BlockNum)
if err != nil {
return nil, err
}
if !ok {
return nil, fmt.Errorf("incorrect slot")
}
// TODO: Verify proof
// Auction
err = a.forge(c.addr)
if err != nil {
return nil, err
}
return c.addBatch(args)
} }
// CtlAddBatch adds forged batch to the Rollup, without checking any ZKProof // CtlAddBatch adds forged batch to the Rollup, without checking any ZKProof
@ -451,11 +577,17 @@ func (c *Client) CtlAddBatch(args *eth.RollupForgeBatchArgs) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
if _, err := c.addBatch(args); err != nil {
panic(err)
}
}
func (c *Client) addBatch(args *eth.RollupForgeBatchArgs) (*types.Transaction, error) {
nextBlock := c.nextBlock() nextBlock := c.nextBlock()
r := nextBlock.Rollup r := nextBlock.Rollup
r.State.StateRoot = args.NewStRoot r.State.StateRoot = args.NewStRoot
if args.NewLastIdx < r.State.CurrentIdx { if args.NewLastIdx < r.State.CurrentIdx {
panic("args.NewLastIdx < r.State.CurrentIdx")
return nil, fmt.Errorf("args.NewLastIdx < r.State.CurrentIdx")
} }
r.State.CurrentIdx = args.NewLastIdx r.State.CurrentIdx = args.NewLastIdx
r.State.ExitRoots = append(r.State.ExitRoots, args.NewExitRoot) r.State.ExitRoots = append(r.State.ExitRoots, args.NewExitRoot)
@ -472,12 +604,16 @@ func (c *Client) CtlAddBatch(args *eth.RollupForgeBatchArgs) {
BatchNum: int64(len(r.State.ExitRoots)), BatchNum: int64(len(r.State.ExitRoots)),
EthTxHash: ethTx.Hash(), EthTxHash: ethTx.Hash(),
}) })
return ethTx, nil
} }
// RollupAddToken is the interface to call the smart contract function // RollupAddToken is the interface to call the smart contract function
func (c *Client) RollupAddToken(tokenAddress ethCommon.Address) (*types.Transaction, error) {
func (c *Client) RollupAddToken(tokenAddress ethCommon.Address) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
nextBlock := c.nextBlock() nextBlock := c.nextBlock()
r := nextBlock.Rollup r := nextBlock.Rollup
@ -498,66 +634,90 @@ func (c *Client) RollupAddToken(tokenAddress ethCommon.Address) (*types.Transact
// } // }
// RollupWithdrawMerkleProof is the interface to call the smart contract function // RollupWithdrawMerkleProof is the interface to call the smart contract function
func (c *Client) RollupWithdrawMerkleProof(tokenID int64, balance *big.Int, babyPubKey *babyjub.PublicKey, numExitRoot int64, siblings []*big.Int, idx int64, instantWithdraw bool) (*types.Transaction, error) {
func (c *Client) RollupWithdrawMerkleProof(tokenID int64, balance *big.Int, babyPubKey *babyjub.PublicKey, numExitRoot int64, siblings []*big.Int, idx int64, instantWithdraw bool) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupForceExit is the interface to call the smart contract function // RollupForceExit is the interface to call the smart contract function
func (c *Client) RollupForceExit(fromIdx int64, amountF utils.Float16, tokenID int64) (*types.Transaction, error) {
func (c *Client) RollupForceExit(fromIdx int64, amountF utils.Float16, tokenID int64) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupForceTransfer is the interface to call the smart contract function // RollupForceTransfer is the interface to call the smart contract function
func (c *Client) RollupForceTransfer(fromIdx int64, amountF utils.Float16, tokenID, toIdx int64) (*types.Transaction, error) {
func (c *Client) RollupForceTransfer(fromIdx int64, amountF utils.Float16, tokenID, toIdx int64) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupCreateAccountDepositTransfer is the interface to call the smart contract function // RollupCreateAccountDepositTransfer is the interface to call the smart contract function
func (c *Client) RollupCreateAccountDepositTransfer(babyPubKey babyjub.PublicKey, loadAmountF, amountF utils.Float16, tokenID int64, toIdx int64) (*types.Transaction, error) {
func (c *Client) RollupCreateAccountDepositTransfer(babyPubKey babyjub.PublicKey, loadAmountF, amountF utils.Float16, tokenID int64, toIdx int64) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupDepositTransfer is the interface to call the smart contract function // RollupDepositTransfer is the interface to call the smart contract function
func (c *Client) RollupDepositTransfer(fromIdx int64, loadAmountF, amountF utils.Float16, tokenID int64, toIdx int64) (*types.Transaction, error) {
func (c *Client) RollupDepositTransfer(fromIdx int64, loadAmountF, amountF utils.Float16, tokenID int64, toIdx int64) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupDeposit is the interface to call the smart contract function // RollupDeposit is the interface to call the smart contract function
func (c *Client) RollupDeposit(fromIdx int64, loadAmountF utils.Float16, tokenID int64) (*types.Transaction, error) {
func (c *Client) RollupDeposit(fromIdx int64, loadAmountF utils.Float16, tokenID int64) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupCreateAccountDepositFromRelayer is the interface to call the smart contract function // RollupCreateAccountDepositFromRelayer is the interface to call the smart contract function
func (c *Client) RollupCreateAccountDepositFromRelayer(accountCreationAuthSig []byte, babyPubKey babyjub.PublicKey, loadAmountF utils.Float16) (*types.Transaction, error) {
func (c *Client) RollupCreateAccountDepositFromRelayer(accountCreationAuthSig []byte, babyPubKey babyjub.PublicKey, loadAmountF utils.Float16) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupCreateAccountDeposit is the interface to call the smart contract function // RollupCreateAccountDeposit is the interface to call the smart contract function
func (c *Client) RollupCreateAccountDeposit(babyPubKey babyjub.PublicKey, loadAmountF utils.Float16, tokenID int64) (*types.Transaction, error) {
func (c *Client) RollupCreateAccountDeposit(babyPubKey babyjub.PublicKey, loadAmountF utils.Float16, tokenID int64) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -566,6 +726,7 @@ func (c *Client) RollupGetTokenAddress(tokenID int64) (*ethCommon.Address, error
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -574,6 +735,7 @@ func (c *Client) RollupGetL1TxFromQueue(queue int64, position int64) ([]byte, er
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -582,38 +744,51 @@ func (c *Client) RollupGetQueue(queue int64) ([]byte, error) {
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupUpdateForgeL1Timeout is the interface to call the smart contract function // RollupUpdateForgeL1Timeout is the interface to call the smart contract function
func (c *Client) RollupUpdateForgeL1Timeout(newForgeL1Timeout int64) (*types.Transaction, error) {
func (c *Client) RollupUpdateForgeL1Timeout(newForgeL1Timeout int64) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupUpdateFeeL1UserTx is the interface to call the smart contract function // RollupUpdateFeeL1UserTx is the interface to call the smart contract function
func (c *Client) RollupUpdateFeeL1UserTx(newFeeL1UserTx *big.Int) (*types.Transaction, error) {
func (c *Client) RollupUpdateFeeL1UserTx(newFeeL1UserTx *big.Int) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupUpdateFeeAddToken is the interface to call the smart contract function // RollupUpdateFeeAddToken is the interface to call the smart contract function
func (c *Client) RollupUpdateFeeAddToken(newFeeAddToken *big.Int) (*types.Transaction, error) {
func (c *Client) RollupUpdateFeeAddToken(newFeeAddToken *big.Int) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// RollupUpdateTokensHEZ is the interface to call the smart contract function // RollupUpdateTokensHEZ is the interface to call the smart contract function
func (c *Client) RollupUpdateTokensHEZ(newTokenHEZ ethCommon.Address) (*types.Transaction, error) {
func (c *Client) RollupUpdateTokensHEZ(newTokenHEZ ethCommon.Address) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -627,6 +802,7 @@ func (c *Client) RollupConstants() (*eth.RollupConstants, error) {
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -659,10 +835,13 @@ func (c *Client) RollupForgeBatchArgs(ethTxHash ethCommon.Hash) (*eth.RollupForg
// //
// AuctionSetSlotDeadline is the interface to call the smart contract function // AuctionSetSlotDeadline is the interface to call the smart contract function
func (c *Client) AuctionSetSlotDeadline(newDeadline uint8) (*types.Transaction, error) {
func (c *Client) AuctionSetSlotDeadline(newDeadline uint8) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -671,14 +850,18 @@ func (c *Client) AuctionGetSlotDeadline() (uint8, error) {
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
log.Error("TODO")
return 0, errTODO return 0, errTODO
} }
// AuctionSetOpenAuctionSlots is the interface to call the smart contract function // AuctionSetOpenAuctionSlots is the interface to call the smart contract function
func (c *Client) AuctionSetOpenAuctionSlots(newOpenAuctionSlots uint16) (*types.Transaction, error) {
func (c *Client) AuctionSetOpenAuctionSlots(newOpenAuctionSlots uint16) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -687,14 +870,18 @@ func (c *Client) AuctionGetOpenAuctionSlots() (uint16, error) {
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
log.Error("TODO")
return 0, errTODO return 0, errTODO
} }
// AuctionSetClosedAuctionSlots is the interface to call the smart contract function // AuctionSetClosedAuctionSlots is the interface to call the smart contract function
func (c *Client) AuctionSetClosedAuctionSlots(newClosedAuctionSlots uint16) (*types.Transaction, error) {
func (c *Client) AuctionSetClosedAuctionSlots(newClosedAuctionSlots uint16) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -703,46 +890,58 @@ func (c *Client) AuctionGetClosedAuctionSlots() (uint16, error) {
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
log.Error("TODO")
return 0, errTODO return 0, errTODO
} }
// AuctionSetOutbidding is the interface to call the smart contract function // AuctionSetOutbidding is the interface to call the smart contract function
func (c *Client) AuctionSetOutbidding(newOutbidding uint8) (*types.Transaction, error) {
func (c *Client) AuctionSetOutbidding(newOutbidding uint16) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// AuctionGetOutbidding is the interface to call the smart contract function // AuctionGetOutbidding is the interface to call the smart contract function
func (c *Client) AuctionGetOutbidding() (uint8, error) {
func (c *Client) AuctionGetOutbidding() (uint16, error) {
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
log.Error("TODO")
return 0, errTODO return 0, errTODO
} }
// AuctionSetAllocationRatio is the interface to call the smart contract function // AuctionSetAllocationRatio is the interface to call the smart contract function
func (c *Client) AuctionSetAllocationRatio(newAllocationRatio [3]uint8) (*types.Transaction, error) {
func (c *Client) AuctionSetAllocationRatio(newAllocationRatio [3]uint16) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// AuctionGetAllocationRatio is the interface to call the smart contract function // AuctionGetAllocationRatio is the interface to call the smart contract function
func (c *Client) AuctionGetAllocationRatio() ([3]uint8, error) {
func (c *Client) AuctionGetAllocationRatio() ([3]uint16, error) {
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
return [3]uint8{}, errTODO
log.Error("TODO")
return [3]uint16{}, errTODO
} }
// AuctionSetDonationAddress is the interface to call the smart contract function // AuctionSetDonationAddress is the interface to call the smart contract function
func (c *Client) AuctionSetDonationAddress(newDonationAddress ethCommon.Address) (*types.Transaction, error) {
func (c *Client) AuctionSetDonationAddress(newDonationAddress ethCommon.Address) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -751,14 +950,18 @@ func (c *Client) AuctionGetDonationAddress() (*ethCommon.Address, error) {
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// AuctionSetBootCoordinator is the interface to call the smart contract function // AuctionSetBootCoordinator is the interface to call the smart contract function
func (c *Client) AuctionSetBootCoordinator(newBootCoordinator ethCommon.Address) (*types.Transaction, error) {
func (c *Client) AuctionSetBootCoordinator(newBootCoordinator ethCommon.Address) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -773,18 +976,23 @@ func (c *Client) AuctionGetBootCoordinator() (*ethCommon.Address, error) {
return &a.Vars.BootCoordinator, nil return &a.Vars.BootCoordinator, nil
} }
// AuctionChangeEpochMinBid is the interface to call the smart contract function
func (c *Client) AuctionChangeEpochMinBid(slotEpoch int64, newInitialMinBid *big.Int) (*types.Transaction, error) {
// AuctionChangeDefaultSlotSetBid is the interface to call the smart contract function
func (c *Client) AuctionChangeDefaultSlotSetBid(slotSet int64, newInitialMinBid *big.Int) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// AuctionRegisterCoordinator is the interface to call the smart contract function // AuctionRegisterCoordinator is the interface to call the smart contract function
func (c *Client) AuctionRegisterCoordinator(forgerAddress ethCommon.Address, URL string) (*types.Transaction, error) {
func (c *Client) AuctionRegisterCoordinator(forgerAddress ethCommon.Address, URL string) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
nextBlock := c.nextBlock() nextBlock := c.nextBlock()
a := nextBlock.Auction a := nextBlock.Auction
@ -816,14 +1024,18 @@ func (c *Client) AuctionIsRegisteredCoordinator(forgerAddress ethCommon.Address)
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
log.Error("TODO")
return false, errTODO return false, errTODO
} }
// AuctionUpdateCoordinatorInfo is the interface to call the smart contract function // AuctionUpdateCoordinatorInfo is the interface to call the smart contract function
func (c *Client) AuctionUpdateCoordinatorInfo(forgerAddress ethCommon.Address, newWithdrawAddress ethCommon.Address, newURL string) (*types.Transaction, error) {
func (c *Client) AuctionUpdateCoordinatorInfo(forgerAddress ethCommon.Address, newWithdrawAddress ethCommon.Address, newURL string) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -832,6 +1044,7 @@ func (c *Client) AuctionGetCurrentSlotNumber() (int64, error) {
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
log.Error("TODO")
return 0, errTODO return 0, errTODO
} }
@ -840,14 +1053,16 @@ func (c *Client) AuctionGetMinBidBySlot(slot int64) (*big.Int, error) {
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
// AuctionGetMinBidEpoch is the interface to call the smart contract function
func (c *Client) AuctionGetMinBidEpoch(epoch uint8) (*big.Int, error) {
// AuctionGetDefaultSlotSetBid is the interface to call the smart contract function
func (c *Client) AuctionGetDefaultSlotSetBid(slotSet uint8) (*big.Int, error) {
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -857,9 +1072,11 @@ func (c *Client) AuctionGetMinBidEpoch(epoch uint8) (*big.Int, error) {
// } // }
// AuctionBid is the interface to call the smart contract function // AuctionBid is the interface to call the smart contract function
func (c *Client) AuctionBid(slot int64, bidAmount *big.Int, forger ethCommon.Address) (*types.Transaction, error) {
func (c *Client) AuctionBid(slot int64, bidAmount *big.Int, forger ethCommon.Address) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { func() { c.revertIfErr(err, cpy) }() }()
nextBlock := c.nextBlock() nextBlock := c.nextBlock()
a := nextBlock.Auction a := nextBlock.Auction
@ -886,7 +1103,7 @@ func (c *Client) AuctionBid(slot int64, bidAmount *big.Int, forger ethCommon.Add
slotState, ok := a.State.Slots[slot] slotState, ok := a.State.Slots[slot]
if !ok { if !ok {
slotState = &eth.SlotState{}
slotState = eth.NewSlotState()
a.State.Slots[slot] = slotState a.State.Slots[slot] = slotState
} }
slotState.Forger = forger slotState.Forger = forger
@ -904,10 +1121,13 @@ func (c *Client) AuctionBid(slot int64, bidAmount *big.Int, forger ethCommon.Add
} }
// AuctionMultiBid is the interface to call the smart contract function // AuctionMultiBid is the interface to call the smart contract function
func (c *Client) AuctionMultiBid(startingSlot int64, endingSlot int64, slotEpoch [6]bool, maxBid, closedMinBid, budget *big.Int, forger ethCommon.Address) (*types.Transaction, error) {
func (c *Client) AuctionMultiBid(startingSlot int64, endingSlot int64, slotSet [6]bool, maxBid, closedMinBid, budget *big.Int, forger ethCommon.Address) (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }
@ -916,7 +1136,9 @@ func (c *Client) AuctionCanForge(forger ethCommon.Address) (bool, error) {
c.rw.RLock() c.rw.RLock()
defer c.rw.RUnlock() defer c.rw.RUnlock()
return false, errTODO
currentBlock := c.currentBlock()
a := currentBlock.Auction
return a.canForge(forger, a.Eth.BlockNum)
} }
// AuctionForge is the interface to call the smart contract function // AuctionForge is the interface to call the smart contract function
@ -925,10 +1147,13 @@ func (c *Client) AuctionCanForge(forger ethCommon.Address) (bool, error) {
// } // }
// AuctionClaimHEZ is the interface to call the smart contract function // AuctionClaimHEZ is the interface to call the smart contract function
func (c *Client) AuctionClaimHEZ() (*types.Transaction, error) {
func (c *Client) AuctionClaimHEZ() (tx *types.Transaction, err error) {
c.rw.Lock() c.rw.Lock()
defer c.rw.Unlock() defer c.rw.Unlock()
cpy := c.nextBlock().copy()
defer func() { c.revertIfErr(err, cpy) }()
log.Error("TODO")
return nil, errTODO return nil, errTODO
} }

+ 12
- 5
test/ethclient_test.go

@ -72,16 +72,22 @@ func TestClientAuction(t *testing.T) {
var timer timer var timer timer
clientSetup := NewClientSetupExample() clientSetup := NewClientSetupExample()
clientSetup.AuctionVariables.ClosedAuctionSlots = 2 clientSetup.AuctionVariables.ClosedAuctionSlots = 2
clientSetup.AuctionVariables.OpenAuctionSlots = 100
clientSetup.AuctionVariables.OpenAuctionSlots = 4320
clientSetup.AuctionVariables.DefaultSlotSetBid = [6]*big.Int{
big.NewInt(1000), big.NewInt(1100), big.NewInt(1200),
big.NewInt(1300), big.NewInt(1400), big.NewInt(1500)}
c := NewClient(true, &timer, addrWithdraw, clientSetup) c := NewClient(true, &timer, addrWithdraw, clientSetup)
// Check several cases in which bid doesn't succed, and also do 2 successful bids.
_, err := c.AuctionBid(0, big.NewInt(1), addrForge) _, err := c.AuctionBid(0, big.NewInt(1), addrForge)
assert.Equal(t, errBidClosed, err) assert.Equal(t, errBidClosed, err)
_, err = c.AuctionBid(102, big.NewInt(1), addrForge)
_, err = c.AuctionBid(4322, big.NewInt(1), addrForge)
assert.Equal(t, errBidNotOpen, err) assert.Equal(t, errBidNotOpen, err)
_, err = c.AuctionBid(101, big.NewInt(16), addrForge)
// 101 % 6 = 5; defaultSlotSetBid[5] = 1500; 1500 + 10% = 1650
_, err = c.AuctionBid(101, big.NewInt(1650), addrForge)
assert.Equal(t, errCoordNotReg, err) assert.Equal(t, errCoordNotReg, err)
_, err = c.AuctionRegisterCoordinator(addrForge, "https://foo.bar") _, err = c.AuctionRegisterCoordinator(addrForge, "https://foo.bar")
@ -90,7 +96,7 @@ func TestClientAuction(t *testing.T) {
_, err = c.AuctionBid(3, big.NewInt(1), addrForge) _, err = c.AuctionBid(3, big.NewInt(1), addrForge)
assert.Equal(t, errBidBelowMin, err) assert.Equal(t, errBidBelowMin, err)
_, err = c.AuctionBid(3, big.NewInt(16), addrForge)
_, err = c.AuctionBid(3, big.NewInt(1650), addrForge)
assert.Nil(t, err) assert.Nil(t, err)
_, err = c.AuctionRegisterCoordinator(addrForge2, "https://foo2.bar") _, err = c.AuctionRegisterCoordinator(addrForge2, "https://foo2.bar")
@ -99,7 +105,8 @@ func TestClientAuction(t *testing.T) {
_, err = c.AuctionBid(3, big.NewInt(16), addrForge2) _, err = c.AuctionBid(3, big.NewInt(16), addrForge2)
assert.Equal(t, errBidBelowMin, err) assert.Equal(t, errBidBelowMin, err)
_, err = c.AuctionBid(3, big.NewInt(17), addrForge2)
// 1650 + 10% = 1815
_, err = c.AuctionBid(3, big.NewInt(1815), addrForge2)
assert.Nil(t, err) assert.Nil(t, err)
c.CtlMineBlock() c.CtlMineBlock()

Loading…
Cancel
Save