Browse Source

Merge pull request #631 from hermeznetwork/feature/serveapicli2

Allow serving API only via new cli command
feature/update-smart-contracts
arnau 3 years ago
committed by GitHub
parent
commit
6c6d1ea7b8
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 1403 additions and 836 deletions
  1. +10
    -33
      api/api.go
  2. +38
    -31
      api/api_test.go
  3. +6
    -2
      api/slots_test.go
  4. +4
    -308
      api/state.go
  5. +77
    -63
      api/state_test.go
  6. +24
    -0
      cli/node/cfg.api.toml
  7. +79
    -15
      cli/node/main.go
  8. +33
    -0
      common/eth.go
  9. +3
    -3
      common/fee.go
  10. +101
    -49
      config/config.go
  11. +8
    -8
      coordinator/coordinator.go
  12. +3
    -3
      coordinator/coordinator_test.go
  13. +8
    -8
      coordinator/pipeline.go
  14. +1
    -5
      coordinator/pipeline_test.go
  15. +5
    -5
      coordinator/txmanager.go
  16. +246
    -137
      db/historydb/apiqueries.go
  17. +69
    -11
      db/historydb/historydb.go
  18. +65
    -9
      db/historydb/historydb_test.go
  19. +165
    -0
      db/historydb/nodeinfo.go
  20. +48
    -13
      db/historydb/views.go
  21. +11
    -0
      db/migrations/0001.sql
  22. +221
    -88
      node/node.go
  23. +155
    -0
      stateapiupdater/stateapiupdater.go
  24. +11
    -33
      synchronizer/synchronizer.go
  25. +12
    -12
      synchronizer/synchronizer_test.go

+ 10
- 33
api/api.go

@ -2,40 +2,19 @@ package api
import (
"errors"
"sync"
ethCommon "github.com/ethereum/go-ethereum/common"
"github.com/gin-gonic/gin"
"github.com/hermeznetwork/hermez-node/common"
"github.com/hermeznetwork/hermez-node/db/historydb"
"github.com/hermeznetwork/hermez-node/db/l2db"
"github.com/hermeznetwork/tracerr"
)
// TODO: Add correct values to constants
const (
createAccountExtraFeePercentage float64 = 2
createAccountInternalExtraFeePercentage float64 = 2.5
)
// Status define status of the network
type Status struct {
sync.RWMutex
NodeConfig NodeConfig `json:"nodeConfig"`
Network Network `json:"network"`
Metrics historydb.Metrics `json:"metrics"`
Rollup historydb.RollupVariablesAPI `json:"rollup"`
Auction historydb.AuctionVariablesAPI `json:"auction"`
WithdrawalDelayer common.WDelayerVariables `json:"withdrawalDelayer"`
RecommendedFee common.RecommendedFee `json:"recommendedFee"`
}
// API serves HTTP requests to allow external interaction with the Hermez node
type API struct {
h *historydb.HistoryDB
cg *configAPI
l2 *l2db.L2DB
status Status
chainID uint16
hermezAddress ethCommon.Address
}
@ -46,8 +25,6 @@ func NewAPI(
server *gin.Engine,
hdb *historydb.HistoryDB,
l2db *l2db.L2DB,
config *Config,
nodeConfig *NodeConfig,
) (*API, error) {
// Check input
// TODO: is stateDB only needed for explorer endpoints or for both?
@ -57,20 +34,20 @@ func NewAPI(
if explorerEndpoints && hdb == nil {
return nil, tracerr.Wrap(errors.New("cannot serve Explorer endpoints without HistoryDB"))
}
consts, err := hdb.GetConstants()
if err != nil {
return nil, err
}
a := &API{
h: hdb,
cg: &configAPI{
RollupConstants: *newRollupConstants(config.RollupConstants),
AuctionConstants: config.AuctionConstants,
WDelayerConstants: config.WDelayerConstants,
},
l2: l2db,
status: Status{
NodeConfig: *nodeConfig,
RollupConstants: *newRollupConstants(consts.Rollup),
AuctionConstants: consts.Auction,
WDelayerConstants: consts.WDelayer,
},
chainID: config.ChainID,
hermezAddress: config.HermezAddress,
l2: l2db,
chainID: consts.ChainID,
hermezAddress: consts.HermezAddress,
}
// Add coordinator endpoints

+ 38
- 31
api/api_test.go

@ -24,6 +24,7 @@ import (
"github.com/hermeznetwork/hermez-node/db/historydb"
"github.com/hermeznetwork/hermez-node/db/l2db"
"github.com/hermeznetwork/hermez-node/log"
"github.com/hermeznetwork/hermez-node/stateapiupdater"
"github.com/hermeznetwork/hermez-node/test"
"github.com/hermeznetwork/hermez-node/test/til"
"github.com/hermeznetwork/hermez-node/test/txsets"
@ -180,12 +181,13 @@ type testCommon struct {
auctionVars common.AuctionVariables
rollupVars common.RollupVariables
wdelayerVars common.WDelayerVariables
nextForgers []NextForger
nextForgers []historydb.NextForgerAPI
}
var tc testCommon
var config configAPI
var api *API
var stateAPIUpdater *stateapiupdater.Updater
// TestMain initializes the API server, and fill HistoryDB and StateDB with fake data,
// emulating the task of the synchronizer in order to have data to be returned
@ -206,16 +208,6 @@ func TestMain(m *testing.M) {
if err != nil {
panic(err)
}
// StateDB
dir, err := ioutil.TempDir("", "tmpdb")
if err != nil {
panic(err)
}
defer func() {
if err := os.RemoveAll(dir); err != nil {
panic(err)
}
}()
// L2DB
l2DB := l2db.NewL2DB(database, database, 10, 1000, 0.0, 24*time.Hour, apiConnCon)
test.WipeDB(l2DB.DB()) // this will clean HistoryDB and L2DB
@ -230,18 +222,38 @@ func TestMain(m *testing.M) {
// API
apiGin := gin.Default()
// Reset DB
test.WipeDB(hdb.DB())
constants := &historydb.Constants{
SCConsts: common.SCConsts{
Rollup: _config.RollupConstants,
Auction: _config.AuctionConstants,
WDelayer: _config.WDelayerConstants,
},
ChainID: chainID,
HermezAddress: _config.HermezAddress,
}
if err := hdb.SetConstants(constants); err != nil {
panic(err)
}
nodeConfig := &historydb.NodeConfig{
MaxPoolTxs: 10,
MinFeeUSD: 0,
}
if err := hdb.SetNodeConfig(nodeConfig); err != nil {
panic(err)
}
api, err = NewAPI(
true,
true,
apiGin,
hdb,
l2DB,
&_config,
&NodeConfig{
ForgeDelay: 180,
},
)
if err != nil {
log.Error(err)
panic(err)
}
// Start server
@ -257,9 +269,6 @@ func TestMain(m *testing.M) {
}
}()
// Reset DB
test.WipeDB(api.h.DB())
// Generate blockchain data with til
tcc := til.NewContext(chainID, common.RollupConstMaxL1UserTx)
tilCfgExtra := til.ConfigExtra{
@ -460,19 +469,19 @@ func TestMain(m *testing.M) {
if err = api.h.AddBids(bids); err != nil {
panic(err)
}
bootForger := NextForger{
bootForger := historydb.NextForgerAPI{
Coordinator: historydb.CoordinatorAPI{
Forger: auctionVars.BootCoordinator,
URL: auctionVars.BootCoordinatorURL,
},
}
// Set next forgers: set all as boot coordinator then replace the non boot coordinators
nextForgers := []NextForger{}
nextForgers := []historydb.NextForgerAPI{}
var initBlock int64 = 140
var deltaBlocks int64 = 40
for i := 1; i < int(auctionVars.ClosedAuctionSlots)+2; i++ {
fromBlock := initBlock + deltaBlocks*int64(i-1)
bootForger.Period = Period{
bootForger.Period = historydb.Period{
SlotNum: int64(i),
FromBlock: fromBlock,
ToBlock: fromBlock + deltaBlocks - 1,
@ -512,7 +521,13 @@ func TestMain(m *testing.M) {
WithdrawalDelay: uint64(3000),
}
// Generate test data, as expected to be received/sent from/to the API
stateAPIUpdater = stateapiupdater.NewUpdater(hdb, nodeConfig, &common.SCVariables{
Rollup: rollupVars,
Auction: auctionVars,
WDelayer: wdelayerVars,
}, constants)
// Generate test data, as expected to be received/sended from/to the API
testCoords := genTestCoordinators(commonCoords)
testBids := genTestBids(commonBlocks, testCoords, bids)
testExits := genTestExits(commonExitTree, testTokens, commonAccounts)
@ -589,15 +604,12 @@ func TestMain(m *testing.M) {
if err := database.Close(); err != nil {
panic(err)
}
if err := os.RemoveAll(dir); err != nil {
panic(err)
}
os.Exit(result)
}
func TestTimeout(t *testing.T) {
pass := os.Getenv("POSTGRES_PASS")
databaseTO, err := db.InitSQLDB(5432, "localhost", "hermez", pass, "hermez")
databaseTO, err := db.ConnectSQLDB(5432, "localhost", "hermez", pass, "hermez")
require.NoError(t, err)
apiConnConTO := db.NewAPIConnectionController(1, 100*time.Millisecond)
hdbTO := historydb.NewHistoryDB(databaseTO, databaseTO, apiConnConTO)
@ -627,17 +639,12 @@ func TestTimeout(t *testing.T) {
require.NoError(t, err)
}
}()
_config := getConfigTest(0)
_, err = NewAPI(
true,
true,
apiGinTO,
hdbTO,
l2DBTO,
&_config,
&NodeConfig{
ForgeDelay: 180,
},
)
require.NoError(t, err)

+ 6
- 2
api/slots_test.go

@ -99,7 +99,9 @@ func TestGetSlot(t *testing.T) {
nil, &fetchedSlot,
),
)
emptySlot := api.getEmptyTestSlot(slotNum, api.status.Network.LastSyncBlock, tc.auctionVars)
// ni, err := api.h.GetNodeInfoAPI()
// assert.NoError(t, err)
emptySlot := api.getEmptyTestSlot(slotNum, 0, tc.auctionVars)
assertSlot(t, emptySlot, fetchedSlot)
// Invalid slotNum
@ -127,8 +129,10 @@ func TestGetSlots(t *testing.T) {
err := doGoodReqPaginated(path, historydb.OrderAsc, &testSlotsResponse{}, appendIter)
assert.NoError(t, err)
allSlots := tc.slots
// ni, err := api.h.GetNodeInfoAPI()
// assert.NoError(t, err)
for i := tc.slots[len(tc.slots)-1].SlotNum; i < maxSlotNum; i++ {
emptySlot := api.getEmptyTestSlot(i+1, api.status.Network.LastSyncBlock, tc.auctionVars)
emptySlot := api.getEmptyTestSlot(i+1, 0, tc.auctionVars)
allSlots = append(allSlots, emptySlot)
}
assertSlots(t, allSlots, fetchedSlots)

+ 4
- 308
api/state.go

@ -1,320 +1,16 @@
package api
import (
"database/sql"
"fmt"
"math"
"math/big"
"net/http"
"time"
"github.com/gin-gonic/gin"
"github.com/hermeznetwork/hermez-node/apitypes"
"github.com/hermeznetwork/hermez-node/common"
"github.com/hermeznetwork/hermez-node/db/historydb"
"github.com/hermeznetwork/tracerr"
)
// Network define status of the network
type Network struct {
LastEthBlock int64 `json:"lastEthereumBlock"`
LastSyncBlock int64 `json:"lastSynchedBlock"`
LastBatch *historydb.BatchAPI `json:"lastBatch"`
CurrentSlot int64 `json:"currentSlot"`
NextForgers []NextForger `json:"nextForgers"`
}
// NodeConfig is the configuration of the node that is exposed via API
type NodeConfig struct {
// ForgeDelay in seconds
ForgeDelay float64 `json:"forgeDelay"`
}
// NextForger is a representation of the information of a coordinator and the period will forge
type NextForger struct {
Coordinator historydb.CoordinatorAPI `json:"coordinator"`
Period Period `json:"period"`
}
// Period is a representation of a period
type Period struct {
SlotNum int64 `json:"slotNum"`
FromBlock int64 `json:"fromBlock"`
ToBlock int64 `json:"toBlock"`
FromTimestamp time.Time `json:"fromTimestamp"`
ToTimestamp time.Time `json:"toTimestamp"`
}
func (a *API) getState(c *gin.Context) {
// TODO: There are no events for the buckets information, so now this information will be 0
a.status.RLock()
status := a.status //nolint
a.status.RUnlock()
c.JSON(http.StatusOK, status) //nolint
}
// SC Vars
// SetRollupVariables set Status.Rollup variables
func (a *API) SetRollupVariables(rollupVariables common.RollupVariables) {
a.status.Lock()
var rollupVAPI historydb.RollupVariablesAPI
rollupVAPI.EthBlockNum = rollupVariables.EthBlockNum
rollupVAPI.FeeAddToken = apitypes.NewBigIntStr(rollupVariables.FeeAddToken)
rollupVAPI.ForgeL1L2BatchTimeout = rollupVariables.ForgeL1L2BatchTimeout
rollupVAPI.WithdrawalDelay = rollupVariables.WithdrawalDelay
for i, bucket := range rollupVariables.Buckets {
var apiBucket historydb.BucketParamsAPI
apiBucket.CeilUSD = apitypes.NewBigIntStr(bucket.CeilUSD)
apiBucket.Withdrawals = apitypes.NewBigIntStr(bucket.Withdrawals)
apiBucket.BlockWithdrawalRate = apitypes.NewBigIntStr(bucket.BlockWithdrawalRate)
apiBucket.MaxWithdrawals = apitypes.NewBigIntStr(bucket.MaxWithdrawals)
rollupVAPI.Buckets[i] = apiBucket
}
rollupVAPI.SafeMode = rollupVariables.SafeMode
a.status.Rollup = rollupVAPI
a.status.Unlock()
}
// SetWDelayerVariables set Status.WithdrawalDelayer variables
func (a *API) SetWDelayerVariables(wDelayerVariables common.WDelayerVariables) {
a.status.Lock()
a.status.WithdrawalDelayer = wDelayerVariables
a.status.Unlock()
}
// SetAuctionVariables set Status.Auction variables
func (a *API) SetAuctionVariables(auctionVariables common.AuctionVariables) {
a.status.Lock()
var auctionAPI historydb.AuctionVariablesAPI
auctionAPI.EthBlockNum = auctionVariables.EthBlockNum
auctionAPI.DonationAddress = auctionVariables.DonationAddress
auctionAPI.BootCoordinator = auctionVariables.BootCoordinator
auctionAPI.BootCoordinatorURL = auctionVariables.BootCoordinatorURL
auctionAPI.DefaultSlotSetBidSlotNum = auctionVariables.DefaultSlotSetBidSlotNum
auctionAPI.ClosedAuctionSlots = auctionVariables.ClosedAuctionSlots
auctionAPI.OpenAuctionSlots = auctionVariables.OpenAuctionSlots
auctionAPI.Outbidding = auctionVariables.Outbidding
auctionAPI.SlotDeadline = auctionVariables.SlotDeadline
for i, slot := range auctionVariables.DefaultSlotSetBid {
auctionAPI.DefaultSlotSetBid[i] = apitypes.NewBigIntStr(slot)
}
for i, ratio := range auctionVariables.AllocationRatio {
auctionAPI.AllocationRatio[i] = ratio
}
a.status.Auction = auctionAPI
a.status.Unlock()
}
// Network
// UpdateNetworkInfoBlock update Status.Network block related information
func (a *API) UpdateNetworkInfoBlock(
lastEthBlock, lastSyncBlock common.Block,
) {
a.status.Network.LastSyncBlock = lastSyncBlock.Num
a.status.Network.LastEthBlock = lastEthBlock.Num
}
// UpdateNetworkInfo update Status.Network information
func (a *API) UpdateNetworkInfo(
lastEthBlock, lastSyncBlock common.Block,
lastBatchNum common.BatchNum, currentSlot int64,
) error {
lastBatch, err := a.h.GetBatchAPI(lastBatchNum)
if tracerr.Unwrap(err) == sql.ErrNoRows {
lastBatch = nil
} else if err != nil {
return tracerr.Wrap(err)
}
lastClosedSlot := currentSlot + int64(a.status.Auction.ClosedAuctionSlots)
nextForgers, err := a.getNextForgers(lastSyncBlock, currentSlot, lastClosedSlot)
if tracerr.Unwrap(err) == sql.ErrNoRows {
nextForgers = nil
} else if err != nil {
return tracerr.Wrap(err)
}
a.status.Lock()
a.status.Network.LastSyncBlock = lastSyncBlock.Num
a.status.Network.LastEthBlock = lastEthBlock.Num
a.status.Network.LastBatch = lastBatch
a.status.Network.CurrentSlot = currentSlot
a.status.Network.NextForgers = nextForgers
// Update buckets withdrawals
bucketsUpdate, err := a.h.GetBucketUpdatesAPI()
if tracerr.Unwrap(err) == sql.ErrNoRows {
bucketsUpdate = nil
} else if err != nil {
return tracerr.Wrap(err)
}
for i, bucketParams := range a.status.Rollup.Buckets {
for _, bucketUpdate := range bucketsUpdate {
if bucketUpdate.NumBucket == i {
bucketParams.Withdrawals = bucketUpdate.Withdrawals
a.status.Rollup.Buckets[i] = bucketParams
break
}
}
}
a.status.Unlock()
return nil
}
// apiSlotToBigInts converts from [6]*apitypes.BigIntStr to [6]*big.Int
func apiSlotToBigInts(defaultSlotSetBid [6]*apitypes.BigIntStr) ([6]*big.Int, error) {
var slots [6]*big.Int
for i, slot := range defaultSlotSetBid {
bigInt, ok := new(big.Int).SetString(string(*slot), 10)
if !ok {
return slots, tracerr.Wrap(fmt.Errorf("can't convert %T into big.Int", slot))
}
slots[i] = bigInt
}
return slots, nil
}
// getNextForgers returns next forgers
func (a *API) getNextForgers(lastBlock common.Block, currentSlot, lastClosedSlot int64) ([]NextForger, error) {
secondsPerBlock := int64(15) //nolint:gomnd
// currentSlot and lastClosedSlot included
limit := uint(lastClosedSlot - currentSlot + 1)
bids, _, err := a.h.GetBestBidsAPI(&currentSlot, &lastClosedSlot, nil, &limit, "ASC")
if err != nil && tracerr.Unwrap(err) != sql.ErrNoRows {
return nil, tracerr.Wrap(err)
}
nextForgers := []NextForger{}
// Get min bid info
var minBidInfo []historydb.MinBidInfo
if currentSlot >= a.status.Auction.DefaultSlotSetBidSlotNum {
// All min bids can be calculated with the last update of AuctionVariables
bigIntSlots, err := apiSlotToBigInts(a.status.Auction.DefaultSlotSetBid)
if err != nil {
return nil, tracerr.Wrap(err)
}
minBidInfo = []historydb.MinBidInfo{{
DefaultSlotSetBid: bigIntSlots,
DefaultSlotSetBidSlotNum: a.status.Auction.DefaultSlotSetBidSlotNum,
}}
} else {
// Get all the relevant updates from the DB
minBidInfo, err = a.h.GetAuctionVarsUntilSetSlotNumAPI(lastClosedSlot, int(lastClosedSlot-currentSlot)+1)
if err != nil {
return nil, tracerr.Wrap(err)
}
}
// Create nextForger for each slot
for i := currentSlot; i <= lastClosedSlot; i++ {
fromBlock := i*int64(a.cg.AuctionConstants.BlocksPerSlot) + a.cg.AuctionConstants.GenesisBlockNum
toBlock := (i+1)*int64(a.cg.AuctionConstants.BlocksPerSlot) + a.cg.AuctionConstants.GenesisBlockNum - 1
nextForger := NextForger{
Period: Period{
SlotNum: i,
FromBlock: fromBlock,
ToBlock: toBlock,
FromTimestamp: lastBlock.Timestamp.Add(time.Second * time.Duration(secondsPerBlock*(fromBlock-lastBlock.Num))),
ToTimestamp: lastBlock.Timestamp.Add(time.Second * time.Duration(secondsPerBlock*(toBlock-lastBlock.Num))),
},
}
foundForger := false
// If there is a bid for a slot, get forger (coordinator)
for j := range bids {
slotNum := bids[j].SlotNum
if slotNum == i {
// There's a bid for the slot
// Check if the bid is greater than the minimum required
for i := 0; i < len(minBidInfo); i++ {
// Find the most recent update
if slotNum >= minBidInfo[i].DefaultSlotSetBidSlotNum {
// Get min bid
minBidSelector := slotNum % int64(len(a.status.Auction.DefaultSlotSetBid))
minBid := minBidInfo[i].DefaultSlotSetBid[minBidSelector]
// Check if the bid has beaten the minimum
bid, ok := new(big.Int).SetString(string(bids[j].BidValue), 10)
if !ok {
return nil, tracerr.New("Wrong bid value, error parsing it as big.Int")
}
if minBid.Cmp(bid) == 1 {
// Min bid is greater than bid, the slot will be forged by boot coordinator
break
}
foundForger = true
break
}
}
if !foundForger { // There is no bid or it's smaller than the minimum
break
}
coordinator, err := a.h.GetCoordinatorAPI(bids[j].Bidder)
if err != nil {
return nil, tracerr.Wrap(err)
}
nextForger.Coordinator = *coordinator
break
}
}
// If there is no bid, the coordinator that will forge is boot coordinator
if !foundForger {
nextForger.Coordinator = historydb.CoordinatorAPI{
Forger: a.status.Auction.BootCoordinator,
URL: a.status.Auction.BootCoordinatorURL,
}
}
nextForgers = append(nextForgers, nextForger)
}
return nextForgers, nil
}
// Metrics
// UpdateMetrics update Status.Metrics information
func (a *API) UpdateMetrics() error {
a.status.RLock()
if a.status.Network.LastBatch == nil {
a.status.RUnlock()
return nil
}
batchNum := a.status.Network.LastBatch.BatchNum
a.status.RUnlock()
metrics, err := a.h.GetMetricsAPI(batchNum)
stateAPI, err := a.h.GetStateAPI()
if err != nil {
return tracerr.Wrap(err)
}
a.status.Lock()
a.status.Metrics = *metrics
a.status.Unlock()
return nil
}
// Recommended fee
// UpdateRecommendedFee update Status.RecommendedFee information
func (a *API) UpdateRecommendedFee() error {
feeExistingAccount, err := a.h.GetAvgTxFeeAPI()
if err != nil {
return tracerr.Wrap(err)
}
var minFeeUSD float64
if a.l2 != nil {
minFeeUSD = a.l2.MinFeeUSD()
retBadReq(err, c)
return
}
a.status.Lock()
a.status.RecommendedFee.ExistingAccount =
math.Max(feeExistingAccount, minFeeUSD)
a.status.RecommendedFee.CreatesAccount =
math.Max(createAccountExtraFeePercentage*feeExistingAccount, minFeeUSD)
a.status.RecommendedFee.CreatesAccountAndRegister =
math.Max(createAccountInternalExtraFeePercentage*feeExistingAccount, minFeeUSD)
a.status.Unlock()
return nil
c.JSON(http.StatusOK, stateAPI)
}

+ 77
- 63
api/state_test.go

@ -13,7 +13,7 @@ import (
type testStatus struct {
Network testNetwork `json:"network"`
Metrics historydb.Metrics `json:"metrics"`
Metrics historydb.MetricsAPI `json:"metrics"`
Rollup historydb.RollupVariablesAPI `json:"rollup"`
Auction historydb.AuctionVariablesAPI `json:"auction"`
WithdrawalDelayer common.WDelayerVariables `json:"withdrawalDelayer"`
@ -21,18 +21,19 @@ type testStatus struct {
}
type testNetwork struct {
LastEthBlock int64 `json:"lastEthereumBlock"`
LastSyncBlock int64 `json:"lastSynchedBlock"`
LastBatch testBatch `json:"lastBatch"`
CurrentSlot int64 `json:"currentSlot"`
NextForgers []NextForger `json:"nextForgers"`
LastEthBlock int64 `json:"lastEthereumBlock"`
LastSyncBlock int64 `json:"lastSynchedBlock"`
LastBatch testBatch `json:"lastBatch"`
CurrentSlot int64 `json:"currentSlot"`
NextForgers []historydb.NextForgerAPI `json:"nextForgers"`
}
func TestSetRollupVariables(t *testing.T) {
rollupVars := &common.RollupVariables{}
assertEqualRollupVariables(t, *rollupVars, api.status.Rollup, true)
api.SetRollupVariables(tc.rollupVars)
assertEqualRollupVariables(t, tc.rollupVars, api.status.Rollup, true)
stateAPIUpdater.SetSCVars(&common.SCVariablesPtr{Rollup: &tc.rollupVars})
require.NoError(t, stateAPIUpdater.Store())
ni, err := api.h.GetNodeInfoAPI()
require.NoError(t, err)
assertEqualRollupVariables(t, tc.rollupVars, ni.StateAPI.Rollup, true)
}
func assertEqualRollupVariables(t *testing.T, rollupVariables common.RollupVariables, apiVariables historydb.RollupVariablesAPI, checkBuckets bool) {
@ -51,17 +52,19 @@ func assertEqualRollupVariables(t *testing.T, rollupVariables common.RollupVaria
}
func TestSetWDelayerVariables(t *testing.T) {
wdelayerVars := &common.WDelayerVariables{}
assert.Equal(t, *wdelayerVars, api.status.WithdrawalDelayer)
api.SetWDelayerVariables(tc.wdelayerVars)
assert.Equal(t, tc.wdelayerVars, api.status.WithdrawalDelayer)
stateAPIUpdater.SetSCVars(&common.SCVariablesPtr{WDelayer: &tc.wdelayerVars})
require.NoError(t, stateAPIUpdater.Store())
ni, err := api.h.GetNodeInfoAPI()
require.NoError(t, err)
assert.Equal(t, tc.wdelayerVars, ni.StateAPI.WithdrawalDelayer)
}
func TestSetAuctionVariables(t *testing.T) {
auctionVars := &common.AuctionVariables{}
assertEqualAuctionVariables(t, *auctionVars, api.status.Auction)
api.SetAuctionVariables(tc.auctionVars)
assertEqualAuctionVariables(t, tc.auctionVars, api.status.Auction)
stateAPIUpdater.SetSCVars(&common.SCVariablesPtr{Auction: &tc.auctionVars})
require.NoError(t, stateAPIUpdater.Store())
ni, err := api.h.GetNodeInfoAPI()
require.NoError(t, err)
assertEqualAuctionVariables(t, tc.auctionVars, ni.StateAPI.Auction)
}
func assertEqualAuctionVariables(t *testing.T, auctionVariables common.AuctionVariables, apiVariables historydb.AuctionVariablesAPI) {
@ -85,11 +88,6 @@ func assertEqualAuctionVariables(t *testing.T, auctionVariables common.AuctionVa
}
func TestUpdateNetworkInfo(t *testing.T) {
status := &Network{}
assert.Equal(t, status.LastSyncBlock, api.status.Network.LastSyncBlock)
assert.Equal(t, status.LastBatch, api.status.Network.LastBatch)
assert.Equal(t, status.CurrentSlot, api.status.Network.CurrentSlot)
assert.Equal(t, status.NextForgers, api.status.Network.NextForgers)
lastBlock := tc.blocks[3]
lastBatchNum := common.BatchNum(3)
currentSlotNum := int64(1)
@ -118,14 +116,17 @@ func TestUpdateNetworkInfo(t *testing.T) {
err := api.h.AddBucketUpdatesTest(api.h.DB(), bucketUpdates)
require.NoError(t, err)
err = api.UpdateNetworkInfo(lastBlock, lastBlock, lastBatchNum, currentSlotNum)
assert.NoError(t, err)
assert.Equal(t, lastBlock.Num, api.status.Network.LastSyncBlock)
assert.Equal(t, lastBatchNum, api.status.Network.LastBatch.BatchNum)
assert.Equal(t, currentSlotNum, api.status.Network.CurrentSlot)
assert.Equal(t, int(api.status.Auction.ClosedAuctionSlots)+1, len(api.status.Network.NextForgers))
assert.Equal(t, api.status.Rollup.Buckets[0].Withdrawals, apitypes.NewBigIntStr(big.NewInt(123)))
assert.Equal(t, api.status.Rollup.Buckets[2].Withdrawals, apitypes.NewBigIntStr(big.NewInt(43)))
err = stateAPIUpdater.UpdateNetworkInfo(lastBlock, lastBlock, lastBatchNum, currentSlotNum)
require.NoError(t, err)
require.NoError(t, stateAPIUpdater.Store())
ni, err := api.h.GetNodeInfoAPI()
require.NoError(t, err)
assert.Equal(t, lastBlock.Num, ni.StateAPI.Network.LastSyncBlock)
assert.Equal(t, lastBatchNum, ni.StateAPI.Network.LastBatch.BatchNum)
assert.Equal(t, currentSlotNum, ni.StateAPI.Network.CurrentSlot)
assert.Equal(t, int(ni.StateAPI.Auction.ClosedAuctionSlots)+1, len(ni.StateAPI.Network.NextForgers))
assert.Equal(t, ni.StateAPI.Rollup.Buckets[0].Withdrawals, apitypes.NewBigIntStr(big.NewInt(123)))
assert.Equal(t, ni.StateAPI.Rollup.Buckets[2].Withdrawals, apitypes.NewBigIntStr(big.NewInt(43)))
}
func TestUpdateMetrics(t *testing.T) {
@ -133,51 +134,62 @@ func TestUpdateMetrics(t *testing.T) {
lastBlock := tc.blocks[3]
lastBatchNum := common.BatchNum(12)
currentSlotNum := int64(1)
err := api.UpdateNetworkInfo(lastBlock, lastBlock, lastBatchNum, currentSlotNum)
assert.NoError(t, err)
err = api.UpdateMetrics()
assert.NoError(t, err)
assert.Greater(t, api.status.Metrics.TransactionsPerBatch, float64(0))
assert.Greater(t, api.status.Metrics.BatchFrequency, float64(0))
assert.Greater(t, api.status.Metrics.TransactionsPerSecond, float64(0))
assert.Greater(t, api.status.Metrics.TotalAccounts, int64(0))
assert.Greater(t, api.status.Metrics.TotalBJJs, int64(0))
assert.Greater(t, api.status.Metrics.AvgTransactionFee, float64(0))
err := stateAPIUpdater.UpdateNetworkInfo(lastBlock, lastBlock, lastBatchNum, currentSlotNum)
require.NoError(t, err)
err = stateAPIUpdater.UpdateMetrics()
require.NoError(t, err)
require.NoError(t, stateAPIUpdater.Store())
ni, err := api.h.GetNodeInfoAPI()
require.NoError(t, err)
assert.Greater(t, ni.StateAPI.Metrics.TransactionsPerBatch, float64(0))
assert.Greater(t, ni.StateAPI.Metrics.BatchFrequency, float64(0))
assert.Greater(t, ni.StateAPI.Metrics.TransactionsPerSecond, float64(0))
assert.Greater(t, ni.StateAPI.Metrics.TotalAccounts, int64(0))
assert.Greater(t, ni.StateAPI.Metrics.TotalBJJs, int64(0))
assert.Greater(t, ni.StateAPI.Metrics.AvgTransactionFee, float64(0))
}
func TestUpdateRecommendedFee(t *testing.T) {
err := api.UpdateRecommendedFee()
assert.NoError(t, err)
err := stateAPIUpdater.UpdateRecommendedFee()
require.NoError(t, err)
require.NoError(t, stateAPIUpdater.Store())
var minFeeUSD float64
if api.l2 != nil {
minFeeUSD = api.l2.MinFeeUSD()
}
assert.Greater(t, api.status.RecommendedFee.ExistingAccount, minFeeUSD)
assert.Equal(t, api.status.RecommendedFee.CreatesAccount,
api.status.RecommendedFee.ExistingAccount*createAccountExtraFeePercentage)
assert.Equal(t, api.status.RecommendedFee.CreatesAccountAndRegister,
api.status.RecommendedFee.ExistingAccount*createAccountInternalExtraFeePercentage)
ni, err := api.h.GetNodeInfoAPI()
require.NoError(t, err)
assert.Greater(t, ni.StateAPI.RecommendedFee.ExistingAccount, minFeeUSD)
assert.Equal(t, ni.StateAPI.RecommendedFee.CreatesAccount,
ni.StateAPI.RecommendedFee.ExistingAccount*
historydb.CreateAccountExtraFeePercentage)
assert.Equal(t, ni.StateAPI.RecommendedFee.CreatesAccountInternal,
ni.StateAPI.RecommendedFee.ExistingAccount*
historydb.CreateAccountInternalExtraFeePercentage)
}
func TestGetState(t *testing.T) {
lastBlock := tc.blocks[3]
lastBatchNum := common.BatchNum(12)
currentSlotNum := int64(1)
api.SetRollupVariables(tc.rollupVars)
api.SetWDelayerVariables(tc.wdelayerVars)
api.SetAuctionVariables(tc.auctionVars)
err := api.UpdateNetworkInfo(lastBlock, lastBlock, lastBatchNum, currentSlotNum)
assert.NoError(t, err)
err = api.UpdateMetrics()
assert.NoError(t, err)
err = api.UpdateRecommendedFee()
assert.NoError(t, err)
stateAPIUpdater.SetSCVars(&common.SCVariablesPtr{
Rollup: &tc.rollupVars,
Auction: &tc.auctionVars,
WDelayer: &tc.wdelayerVars,
})
err := stateAPIUpdater.UpdateNetworkInfo(lastBlock, lastBlock, lastBatchNum, currentSlotNum)
require.NoError(t, err)
err = stateAPIUpdater.UpdateMetrics()
require.NoError(t, err)
err = stateAPIUpdater.UpdateRecommendedFee()
require.NoError(t, err)
require.NoError(t, stateAPIUpdater.Store())
endpoint := apiURL + "state"
var status testStatus
assert.NoError(t, doGoodReq("GET", endpoint, nil, &status))
require.NoError(t, doGoodReq("GET", endpoint, nil, &status))
// SC vars
// UpdateNetworkInfo will overwrite buckets withdrawal values
@ -205,12 +217,14 @@ func TestGetState(t *testing.T) {
// TODO: perform real asserts (not just greater than 0)
assert.Greater(t, status.RecommendedFee.ExistingAccount, float64(0))
assert.Equal(t, status.RecommendedFee.CreatesAccount,
status.RecommendedFee.ExistingAccount*createAccountExtraFeePercentage)
assert.Equal(t, status.RecommendedFee.CreatesAccountAndRegister,
status.RecommendedFee.ExistingAccount*createAccountInternalExtraFeePercentage)
status.RecommendedFee.ExistingAccount*
historydb.CreateAccountExtraFeePercentage)
assert.Equal(t, status.RecommendedFee.CreatesAccountInternal,
status.RecommendedFee.ExistingAccount*
historydb.CreateAccountInternalExtraFeePercentage)
}
func assertNextForgers(t *testing.T, expected, actual []NextForger) {
func assertNextForgers(t *testing.T, expected, actual []historydb.NextForgerAPI) {
assert.Equal(t, len(expected), len(actual))
for i := range expected {
// ignore timestamps and other metadata

+ 24
- 0
cli/node/cfg.api.toml

@ -0,0 +1,24 @@
[API]
Address = "localhost:8386"
Explorer = true
MaxSQLConnections = 10
SQLConnectionTimeout = "2s"
[PostgreSQL]
PortWrite = 5432
HostWrite = "localhost"
UserWrite = "hermez"
PasswordWrite = "yourpasswordhere"
NameWrite = "hermez"
[Coordinator.L2DB]
SafetyPeriod = 10
MaxTxs = 512
TTL = "24h"
PurgeBatchDelay = 10
InvalidateBatchDelay = 20
PurgeBlockDelay = 10
InvalidateBlockDelay = 20
[Coordinator.API]
Coordinator = true

+ 79
- 15
cli/node/main.go

@ -196,17 +196,7 @@ func cmdWipeSQL(c *cli.Context) error {
return nil
}
func cmdRun(c *cli.Context) error {
cfg, err := parseCli(c)
if err != nil {
return tracerr.Wrap(fmt.Errorf("error parsing flags and config: %w", err))
}
node, err := node.NewNode(cfg.mode, cfg.node)
if err != nil {
return tracerr.Wrap(fmt.Errorf("error starting node: %w", err))
}
node.Start()
func waitSigInt() {
stopCh := make(chan interface{})
// catch ^C to send the stop signal
@ -227,11 +217,40 @@ func cmdRun(c *cli.Context) error {
}
}()
<-stopCh
}
func cmdRun(c *cli.Context) error {
cfg, err := parseCli(c)
if err != nil {
return tracerr.Wrap(fmt.Errorf("error parsing flags and config: %w", err))
}
node, err := node.NewNode(cfg.mode, cfg.node)
if err != nil {
return tracerr.Wrap(fmt.Errorf("error starting node: %w", err))
}
node.Start()
waitSigInt()
node.Stop()
return nil
}
func cmdServeAPI(c *cli.Context) error {
cfg, err := parseCliAPIServer(c)
if err != nil {
return tracerr.Wrap(fmt.Errorf("error parsing flags and config: %w", err))
}
srv, err := node.NewAPIServer(cfg.mode, cfg.server)
if err != nil {
return tracerr.Wrap(fmt.Errorf("error starting api server: %w", err))
}
srv.Start()
waitSigInt()
srv.Stop()
return nil
}
func cmdDiscard(c *cli.Context) error {
_cfg, err := parseCli(c)
if err != nil {
@ -319,20 +338,59 @@ func getConfig(c *cli.Context) (*Config, error) {
var cfg Config
mode := c.String(flagMode)
nodeCfgPath := c.String(flagCfg)
if nodeCfgPath == "" {
return nil, tracerr.Wrap(fmt.Errorf("required flag \"%v\" not set", flagCfg))
var err error
switch mode {
case modeSync:
cfg.mode = node.ModeSynchronizer
cfg.node, err = config.LoadNode(nodeCfgPath, false)
if err != nil {
return nil, tracerr.Wrap(err)
}
case modeCoord:
cfg.mode = node.ModeCoordinator
cfg.node, err = config.LoadNode(nodeCfgPath, true)
if err != nil {
return nil, tracerr.Wrap(err)
}
default:
return nil, tracerr.Wrap(fmt.Errorf("invalid mode \"%v\"", mode))
}
return &cfg, nil
}
// ConfigAPIServer is the configuration of the api server execution
type ConfigAPIServer struct {
mode node.Mode
server *config.APIServer
}
func parseCliAPIServer(c *cli.Context) (*ConfigAPIServer, error) {
cfg, err := getConfigAPIServer(c)
if err != nil {
if err := cli.ShowAppHelp(c); err != nil {
panic(err)
}
return nil, tracerr.Wrap(err)
}
return cfg, nil
}
func getConfigAPIServer(c *cli.Context) (*ConfigAPIServer, error) {
var cfg ConfigAPIServer
mode := c.String(flagMode)
nodeCfgPath := c.String(flagCfg)
var err error
switch mode {
case modeSync:
cfg.mode = node.ModeSynchronizer
cfg.node, err = config.LoadNode(nodeCfgPath)
cfg.server, err = config.LoadAPIServer(nodeCfgPath, false)
if err != nil {
return nil, tracerr.Wrap(err)
}
case modeCoord:
cfg.mode = node.ModeCoordinator
cfg.node, err = config.LoadCoordinator(nodeCfgPath)
cfg.server, err = config.LoadAPIServer(nodeCfgPath, true)
if err != nil {
return nil, tracerr.Wrap(err)
}
@ -398,6 +456,12 @@ func main() {
Usage: "Run the hermez-node in the indicated mode",
Action: cmdRun,
},
{
Name: "serveapi",
Aliases: []string{},
Usage: "Serve the API only",
Action: cmdServeAPI,
},
{
Name: "discard",
Aliases: []string{},

+ 33
- 0
common/eth.go

@ -0,0 +1,33 @@
package common
// SCVariables joins all the smart contract variables in a single struct
type SCVariables struct {
Rollup RollupVariables `validate:"required"`
Auction AuctionVariables `validate:"required"`
WDelayer WDelayerVariables `validate:"required"`
}
// AsPtr returns the SCVariables as a SCVariablesPtr using pointers to the
// original SCVariables
func (v *SCVariables) AsPtr() *SCVariablesPtr {
return &SCVariablesPtr{
Rollup: &v.Rollup,
Auction: &v.Auction,
WDelayer: &v.WDelayer,
}
}
// SCVariablesPtr joins all the smart contract variables as pointers in a single
// struct
type SCVariablesPtr struct {
Rollup *RollupVariables `validate:"required"`
Auction *AuctionVariables `validate:"required"`
WDelayer *WDelayerVariables `validate:"required"`
}
// SCConsts joins all the smart contract constants in a single struct
type SCConsts struct {
Rollup RollupConstants
Auction AuctionConstants
WDelayer WDelayerConstants
}

+ 3
- 3
common/fee.go

@ -22,9 +22,9 @@ var FeeFactorLsh60 [256]*big.Int
// the coordinator according to the tx type (if the tx requires to create an
// account and register, only register or he account already esists)
type RecommendedFee struct {
ExistingAccount float64 `json:"existingAccount"`
CreatesAccount float64 `json:"createAccount"`
CreatesAccountAndRegister float64 `json:"createAccountInternal"`
ExistingAccount float64 `json:"existingAccount"`
CreatesAccount float64 `json:"createAccount"`
CreatesAccountInternal float64 `json:"createAccountInternal"`
}
// FeeSelector is used to select a percentage from the FeePlan.

+ 101
- 49
config/config.go

@ -45,6 +45,13 @@ type ForgeBatchGasCost struct {
L2Tx uint64 `validate:"required"`
}
// CoordinatorAPI specifies the configuration parameters of the API in mode
// coordinator
type CoordinatorAPI struct {
// Coordinator enables the coordinator API endpoints
Coordinator bool
}
// Coordinator is the coordinator specific configuration.
type Coordinator struct {
// ForgerAddress is the address under which this coordinator is forging
@ -206,10 +213,7 @@ type Coordinator struct {
// ForgeBatch transaction.
ForgeBatchGasCost ForgeBatchGasCost `validate:"required"`
} `validate:"required"`
API struct {
// Coordinator enables the coordinator API endpoints
Coordinator bool
} `validate:"required"`
API CoordinatorAPI `validate:"required"`
Debug struct {
// BatchPath if set, specifies the path where batchInfo is stored
// in JSON in every step/update of the pipeline
@ -224,6 +228,45 @@ type Coordinator struct {
}
}
// PostgreSQL is the postgreSQL configuration parameters. It's possible to use
// diferentiated SQL connections for read/write. If the read configuration is
// not provided, the write one it's going to be used for both reads and writes
type PostgreSQL struct {
// Port of the PostgreSQL write server
PortWrite int `validate:"required"`
// Host of the PostgreSQL write server
HostWrite string `validate:"required"`
// User of the PostgreSQL write server
UserWrite string `validate:"required"`
// Password of the PostgreSQL write server
PasswordWrite string `validate:"required"`
// Name of the PostgreSQL write server database
NameWrite string `validate:"required"`
// Port of the PostgreSQL read server
PortRead int
// Host of the PostgreSQL read server
HostRead string
// User of the PostgreSQL read server
UserRead string
// Password of the PostgreSQL read server
PasswordRead string
// Name of the PostgreSQL read server database
NameRead string
}
// NodeDebug specifies debug configuration parameters
type NodeDebug struct {
// APIAddress is the address where the debugAPI will listen if
// set
APIAddress string
// MeddlerLogs enables meddler debug mode, where unused columns and struct
// fields will be logged
MeddlerLogs bool
// GinDebugMode sets Gin-Gonic (the web framework) to run in
// debug mode
GinDebugMode bool
}
// Node is the hermez node configuration.
type Node struct {
PriceUpdater struct {
@ -244,32 +287,8 @@ type Node struct {
// Keep is the number of checkpoints to keep
Keep int `validate:"required"`
} `validate:"required"`
// It's possible to use diferentiated SQL connections for read/write.
// If the read configuration is not provided, the write one it's going to be used
// for both reads and writes
PostgreSQL struct {
// Port of the PostgreSQL write server
PortWrite int `validate:"required"`
// Host of the PostgreSQL write server
HostWrite string `validate:"required"`
// User of the PostgreSQL write server
UserWrite string `validate:"required"`
// Password of the PostgreSQL write server
PasswordWrite string `validate:"required"`
// Name of the PostgreSQL write server database
NameWrite string `validate:"required"`
// Port of the PostgreSQL read server
PortRead int
// Host of the PostgreSQL read server
HostRead string
// User of the PostgreSQL read server
UserRead string
// Password of the PostgreSQL read server
PasswordRead string
// Name of the PostgreSQL read server database
NameRead string
} `validate:"required"`
Web3 struct {
PostgreSQL PostgreSQL `validate:"required"`
Web3 struct {
// URL is the URL of the web3 ethereum-node RPC server
URL string `validate:"required"`
} `validate:"required"`
@ -299,6 +318,7 @@ type Node struct {
// TokenHEZ address
TokenHEZName string `validate:"required"`
} `validate:"required"`
// API specifies the configuration parameters of the API
API struct {
// Address where the API will listen if set
Address string
@ -316,20 +336,45 @@ type Node struct {
// can wait to stablish a SQL connection
SQLConnectionTimeout Duration
} `validate:"required"`
Debug struct {
// APIAddress is the address where the debugAPI will listen if
// set
APIAddress string
// MeddlerLogs enables meddler debug mode, where unused columns and struct
// fields will be logged
MeddlerLogs bool
// GinDebugMode sets Gin-Gonic (the web framework) to run in
// debug mode
GinDebugMode bool
}
Debug NodeDebug `validate:"required"`
Coordinator Coordinator `validate:"-"`
}
// APIServer is the api server configuration parameters
type APIServer struct {
// NodeAPI specifies the configuration parameters of the API
API struct {
// Address where the API will listen if set
Address string `validate:"required"`
// Explorer enables the Explorer API endpoints
Explorer bool
// Maximum concurrent connections allowed between API and SQL
MaxSQLConnections int `validate:"required"`
// SQLConnectionTimeout is the maximum amount of time that an API request
// can wait to stablish a SQL connection
SQLConnectionTimeout Duration
} `validate:"required"`
PostgreSQL PostgreSQL `validate:"required"`
Coordinator struct {
API struct {
// Coordinator enables the coordinator API endpoints
Coordinator bool
} `validate:"required"`
L2DB struct {
// MaxTxs is the maximum number of pending L2Txs that can be
// stored in the pool. Once this number of pending L2Txs is
// reached, inserts to the pool will be denied until some of
// the pending txs are forged.
MaxTxs uint32 `validate:"required"`
// MinFeeUSD is the minimum fee in USD that a tx must pay in
// order to be accepted into the pool. Txs with lower than
// minimum fee will be rejected at the API level.
MinFeeUSD float64
} `validate:"required"`
}
Debug NodeDebug `validate:"required"`
}
// Load loads a generic config.
func Load(path string, cfg interface{}) error {
bs, err := ioutil.ReadFile(path) //nolint:gosec
@ -343,8 +388,8 @@ func Load(path string, cfg interface{}) error {
return nil
}
// LoadCoordinator loads the Coordinator configuration from path.
func LoadCoordinator(path string) (*Node, error) {
// LoadNode loads the Node configuration from path.
func LoadNode(path string, coordinator bool) (*Node, error) {
var cfg Node
if err := Load(path, &cfg); err != nil {
return nil, tracerr.Wrap(fmt.Errorf("error loading node configuration file: %w", err))
@ -353,21 +398,28 @@ func LoadCoordinator(path string) (*Node, error) {
if err := validate.Struct(cfg); err != nil {
return nil, tracerr.Wrap(fmt.Errorf("error validating configuration file: %w", err))
}
if err := validate.Struct(cfg.Coordinator); err != nil {
return nil, tracerr.Wrap(fmt.Errorf("error validating configuration file: %w", err))
if coordinator {
if err := validate.Struct(cfg.Coordinator); err != nil {
return nil, tracerr.Wrap(fmt.Errorf("error validating configuration file: %w", err))
}
}
return &cfg, nil
}
// LoadNode loads the Node configuration from path.
func LoadNode(path string) (*Node, error) {
var cfg Node
// LoadAPIServer loads the APIServer configuration from path.
func LoadAPIServer(path string, coordinator bool) (*APIServer, error) {
var cfg APIServer
if err := Load(path, &cfg); err != nil {
return nil, tracerr.Wrap(fmt.Errorf("error loading node configuration file: %w", err))
return nil, tracerr.Wrap(fmt.Errorf("error loading apiServer configuration file: %w", err))
}
validate := validator.New()
if err := validate.Struct(cfg); err != nil {
return nil, tracerr.Wrap(fmt.Errorf("error validating configuration file: %w", err))
}
if coordinator {
if err := validate.Struct(cfg.Coordinator); err != nil {
return nil, tracerr.Wrap(fmt.Errorf("error validating configuration file: %w", err))
}
}
return &cfg, nil
}

+ 8
- 8
coordinator/coordinator.go

@ -153,8 +153,8 @@ type Coordinator struct {
pipelineNum int // Pipeline sequential number. The first pipeline is 1
pipelineFromBatch fromBatch // batch from which we started the pipeline
provers []prover.Client
consts synchronizer.SCConsts
vars synchronizer.SCVariables
consts common.SCConsts
vars common.SCVariables
stats synchronizer.Stats
started bool
@ -194,8 +194,8 @@ func NewCoordinator(cfg Config,
batchBuilder *batchbuilder.BatchBuilder,
serverProofs []prover.Client,
ethClient eth.ClientInterface,
scConsts *synchronizer.SCConsts,
initSCVars *synchronizer.SCVariables,
scConsts *common.SCConsts,
initSCVars *common.SCVariables,
) (*Coordinator, error) {
// nolint reason: hardcoded `1.0`, by design the percentage can't be over 100%
if cfg.L1BatchTimeoutPerc >= 1.0 { //nolint:gomnd
@ -284,13 +284,13 @@ type MsgSyncBlock struct {
Batches []common.BatchData
// Vars contains each Smart Contract variables if they are updated, or
// nil if they haven't changed.
Vars synchronizer.SCVariablesPtr
Vars common.SCVariablesPtr
}
// MsgSyncReorg indicates a reorg
type MsgSyncReorg struct {
Stats synchronizer.Stats
Vars synchronizer.SCVariablesPtr
Vars common.SCVariablesPtr
}
// MsgStopPipeline indicates a signal to reset the pipeline
@ -309,7 +309,7 @@ func (c *Coordinator) SendMsg(ctx context.Context, msg interface{}) {
}
}
func updateSCVars(vars *synchronizer.SCVariables, update synchronizer.SCVariablesPtr) {
func updateSCVars(vars *common.SCVariables, update common.SCVariablesPtr) {
if update.Rollup != nil {
vars.Rollup = *update.Rollup
}
@ -321,7 +321,7 @@ func updateSCVars(vars *synchronizer.SCVariables, update synchronizer.SCVariable
}
}
func (c *Coordinator) syncSCVars(vars synchronizer.SCVariablesPtr) {
func (c *Coordinator) syncSCVars(vars common.SCVariablesPtr) {
updateSCVars(&c.vars, vars)
}

+ 3
- 3
coordinator/coordinator_test.go

@ -189,12 +189,12 @@ func newTestCoordinator(t *testing.T, forgerAddr ethCommon.Address, ethClient *t
&prover.MockClient{Delay: 400 * time.Millisecond},
}
scConsts := &synchronizer.SCConsts{
scConsts := &common.SCConsts{
Rollup: *ethClientSetup.RollupConstants,
Auction: *ethClientSetup.AuctionConstants,
WDelayer: *ethClientSetup.WDelayerConstants,
}
initSCVars := &synchronizer.SCVariables{
initSCVars := &common.SCVariables{
Rollup: *ethClientSetup.RollupVariables,
Auction: *ethClientSetup.AuctionVariables,
WDelayer: *ethClientSetup.WDelayerVariables,
@ -534,7 +534,7 @@ func TestCoordinatorStress(t *testing.T) {
coord.SendMsg(ctx, MsgSyncBlock{
Stats: *stats,
Batches: blockData.Rollup.Batches,
Vars: synchronizer.SCVariablesPtr{
Vars: common.SCVariablesPtr{
Rollup: blockData.Rollup.Vars,
Auction: blockData.Auction.Vars,
WDelayer: blockData.WDelayer.Vars,

+ 8
- 8
coordinator/pipeline.go

@ -22,7 +22,7 @@ import (
type statsVars struct {
Stats synchronizer.Stats
Vars synchronizer.SCVariablesPtr
Vars common.SCVariablesPtr
}
type state struct {
@ -36,7 +36,7 @@ type state struct {
type Pipeline struct {
num int
cfg Config
consts synchronizer.SCConsts
consts common.SCConsts
// state
state state
@ -57,7 +57,7 @@ type Pipeline struct {
purger *Purger
stats synchronizer.Stats
vars synchronizer.SCVariables
vars common.SCVariables
statsVarsCh chan statsVars
ctx context.Context
@ -90,7 +90,7 @@ func NewPipeline(ctx context.Context,
coord *Coordinator,
txManager *TxManager,
provers []prover.Client,
scConsts *synchronizer.SCConsts,
scConsts *common.SCConsts,
) (*Pipeline, error) {
proversPool := NewProversPool(len(provers))
proversPoolSize := 0
@ -125,7 +125,7 @@ func NewPipeline(ctx context.Context,
// SetSyncStatsVars is a thread safe method to sets the synchronizer Stats
func (p *Pipeline) SetSyncStatsVars(ctx context.Context, stats *synchronizer.Stats,
vars *synchronizer.SCVariablesPtr) {
vars *common.SCVariablesPtr) {
select {
case p.statsVarsCh <- statsVars{Stats: *stats, Vars: *vars}:
case <-ctx.Done():
@ -134,7 +134,7 @@ func (p *Pipeline) SetSyncStatsVars(ctx context.Context, stats *synchronizer.Sta
// reset pipeline state
func (p *Pipeline) reset(batchNum common.BatchNum,
stats *synchronizer.Stats, vars *synchronizer.SCVariables) error {
stats *synchronizer.Stats, vars *common.SCVariables) error {
p.state = state{
batchNum: batchNum,
lastForgeL1TxsNum: stats.Sync.LastForgeL1TxsNum,
@ -195,7 +195,7 @@ func (p *Pipeline) reset(batchNum common.BatchNum,
return nil
}
func (p *Pipeline) syncSCVars(vars synchronizer.SCVariablesPtr) {
func (p *Pipeline) syncSCVars(vars common.SCVariablesPtr) {
updateSCVars(&p.vars, vars)
}
@ -256,7 +256,7 @@ func (p *Pipeline) handleForgeBatch(ctx context.Context,
// Start the forging pipeline
func (p *Pipeline) Start(batchNum common.BatchNum,
stats *synchronizer.Stats, vars *synchronizer.SCVariables) error {
stats *synchronizer.Stats, vars *common.SCVariables) error {
if p.started {
log.Fatal("Pipeline already started")
}

+ 1
- 5
coordinator/pipeline_test.go

@ -206,11 +206,7 @@ PoolTransfer(0) User2-User3: 300 (126)
require.NoError(t, err)
}
err = pipeline.reset(batchNum, syncStats, &synchronizer.SCVariables{
Rollup: *syncSCVars.Rollup,
Auction: *syncSCVars.Auction,
WDelayer: *syncSCVars.WDelayer,
})
err = pipeline.reset(batchNum, syncStats, syncSCVars)
require.NoError(t, err)
// Sanity check
sdbAccounts, err := pipeline.txSelector.LocalAccountsDB().TestGetAccounts()

+ 5
- 5
coordinator/txmanager.go

@ -31,10 +31,10 @@ type TxManager struct {
batchCh chan *BatchInfo
chainID *big.Int
account accounts.Account
consts synchronizer.SCConsts
consts common.SCConsts
stats synchronizer.Stats
vars synchronizer.SCVariables
vars common.SCVariables
statsVarsCh chan statsVars
discardPipelineCh chan int // int refers to the pipelineNum
@ -55,7 +55,7 @@ type TxManager struct {
// NewTxManager creates a new TxManager
func NewTxManager(ctx context.Context, cfg *Config, ethClient eth.ClientInterface, l2DB *l2db.L2DB,
coord *Coordinator, scConsts *synchronizer.SCConsts, initSCVars *synchronizer.SCVariables) (
coord *Coordinator, scConsts *common.SCConsts, initSCVars *common.SCVariables) (
*TxManager, error) {
chainID, err := ethClient.EthChainID()
if err != nil {
@ -104,7 +104,7 @@ func (t *TxManager) AddBatch(ctx context.Context, batchInfo *BatchInfo) {
// SetSyncStatsVars is a thread safe method to sets the synchronizer Stats
func (t *TxManager) SetSyncStatsVars(ctx context.Context, stats *synchronizer.Stats,
vars *synchronizer.SCVariablesPtr) {
vars *common.SCVariablesPtr) {
select {
case t.statsVarsCh <- statsVars{Stats: *stats, Vars: *vars}:
case <-ctx.Done():
@ -120,7 +120,7 @@ func (t *TxManager) DiscardPipeline(ctx context.Context, pipelineNum int) {
}
}
func (t *TxManager) syncSCVars(vars synchronizer.SCVariablesPtr) {
func (t *TxManager) syncSCVars(vars common.SCVariablesPtr) {
updateSCVars(&t.vars, vars)
}

+ 246
- 137
db/historydb/apiqueries.go

@ -1,8 +1,11 @@
package historydb
import (
"database/sql"
"errors"
"fmt"
"math/big"
"time"
ethCommon "github.com/ethereum/go-ethereum/common"
"github.com/hermeznetwork/hermez-node/common"
@ -32,9 +35,18 @@ func (hdb *HistoryDB) GetBatchAPI(batchNum common.BatchNum) (*BatchAPI, error) {
return nil, tracerr.Wrap(err)
}
defer hdb.apiConnCon.Release()
return hdb.getBatchAPI(hdb.dbRead, batchNum)
}
// GetBatchInternalAPI return the batch with the given batchNum
func (hdb *HistoryDB) GetBatchInternalAPI(batchNum common.BatchNum) (*BatchAPI, error) {
return hdb.getBatchAPI(hdb.dbRead, batchNum)
}
func (hdb *HistoryDB) getBatchAPI(d meddler.DB, batchNum common.BatchNum) (*BatchAPI, error) {
batch := &BatchAPI{}
return batch, tracerr.Wrap(meddler.QueryRow(
hdb.dbRead, batch,
d, batch,
`SELECT batch.item_id, batch.batch_num, batch.eth_block_num,
batch.forger_addr, batch.fees_collected, batch.total_fees_usd, batch.state_root,
batch.num_accounts, batch.exit_root, batch.forge_l1_txs_num, batch.slot_num,
@ -180,6 +192,14 @@ func (hdb *HistoryDB) GetBestBidsAPI(
return nil, 0, tracerr.Wrap(err)
}
defer hdb.apiConnCon.Release()
return hdb.getBestBidsAPI(hdb.dbRead, minSlotNum, maxSlotNum, bidderAddr, limit, order)
}
func (hdb *HistoryDB) getBestBidsAPI(
d meddler.DB,
minSlotNum, maxSlotNum *int64,
bidderAddr *ethCommon.Address,
limit *uint, order string,
) ([]BidAPI, uint64, error) {
var query string
var args []interface{}
// JOIN the best bid of each slot with the latest update of each coordinator
@ -214,7 +234,7 @@ func (hdb *HistoryDB) GetBestBidsAPI(
}
query = hdb.dbRead.Rebind(queryStr)
bidPtrs := []*BidAPI{}
if err := meddler.QueryAll(hdb.dbRead, &bidPtrs, query, args...); err != nil {
if err := meddler.QueryAll(d, &bidPtrs, query, args...); err != nil {
return nil, 0, tracerr.Wrap(err)
}
// log.Debug(query)
@ -697,25 +717,6 @@ func (hdb *HistoryDB) GetExitsAPI(
return db.SlicePtrsToSlice(exits).([]ExitAPI), exits[0].TotalItems - uint64(len(exits)), nil
}
// GetBucketUpdatesAPI retrieves latest values for each bucket
func (hdb *HistoryDB) GetBucketUpdatesAPI() ([]BucketUpdateAPI, error) {
cancel, err := hdb.apiConnCon.Acquire()
defer cancel()
if err != nil {
return nil, tracerr.Wrap(err)
}
defer hdb.apiConnCon.Release()
var bucketUpdates []*BucketUpdateAPI
err = meddler.QueryAll(
hdb.dbRead, &bucketUpdates,
`SELECT num_bucket, withdrawals FROM bucket_update
WHERE item_id in(SELECT max(item_id) FROM bucket_update
group by num_bucket)
ORDER BY num_bucket ASC;`,
)
return db.SlicePtrsToSlice(bucketUpdates).([]BucketUpdateAPI), tracerr.Wrap(err)
}
// GetCoordinatorsAPI returns a list of coordinators from the DB and pagination info
func (hdb *HistoryDB) GetCoordinatorsAPI(
bidderAddr, forgerAddr *ethCommon.Address,
@ -800,29 +801,6 @@ func (hdb *HistoryDB) GetAuctionVarsAPI() (*common.AuctionVariables, error) {
return auctionVars, tracerr.Wrap(err)
}
// GetAuctionVarsUntilSetSlotNumAPI returns all the updates of the auction vars
// from the last entry in which DefaultSlotSetBidSlotNum <= slotNum
func (hdb *HistoryDB) GetAuctionVarsUntilSetSlotNumAPI(slotNum int64, maxItems int) ([]MinBidInfo, error) {
cancel, err := hdb.apiConnCon.Acquire()
defer cancel()
if err != nil {
return nil, tracerr.Wrap(err)
}
defer hdb.apiConnCon.Release()
auctionVars := []*MinBidInfo{}
query := `
SELECT DISTINCT default_slot_set_bid, default_slot_set_bid_slot_num FROM auction_vars
WHERE default_slot_set_bid_slot_num < $1
ORDER BY default_slot_set_bid_slot_num DESC
LIMIT $2;
`
err = meddler.QueryAll(hdb.dbRead, &auctionVars, query, slotNum, maxItems)
if err != nil {
return nil, tracerr.Wrap(err)
}
return db.SlicePtrsToSlice(auctionVars).([]MinBidInfo), nil
}
// GetAccountAPI returns an account by its index
func (hdb *HistoryDB) GetAccountAPI(idx common.Idx) (*AccountAPI, error) {
cancel, err := hdb.apiConnCon.Acquire()
@ -941,137 +919,268 @@ func (hdb *HistoryDB) GetAccountsAPI(
accounts[0].TotalItems - uint64(len(accounts)), nil
}
// GetMetricsAPI returns metrics
func (hdb *HistoryDB) GetMetricsAPI(lastBatchNum common.BatchNum) (*Metrics, error) {
// GetCommonAccountAPI returns the account associated to an account idx
func (hdb *HistoryDB) GetCommonAccountAPI(idx common.Idx) (*common.Account, error) {
cancel, err := hdb.apiConnCon.Acquire()
defer cancel()
if err != nil {
return nil, tracerr.Wrap(err)
}
defer hdb.apiConnCon.Release()
metricsTotals := &MetricsTotals{}
metrics := &Metrics{}
account := &common.Account{}
err = meddler.QueryRow(
hdb.dbRead, metricsTotals, `SELECT
COALESCE (MIN(batch.batch_num), 0) as batch_num,
COALESCE (MIN(block.timestamp), NOW()) AS min_timestamp,
COALESCE (MAX(block.timestamp), NOW()) AS max_timestamp
FROM batch INNER JOIN block ON batch.eth_block_num = block.eth_block_num
WHERE block.timestamp >= NOW() - INTERVAL '24 HOURS' and batch.batch_num <= $1;`, lastBatchNum)
hdb.dbRead, account, `SELECT idx, token_id, batch_num, bjj, eth_addr
FROM account WHERE idx = $1;`, idx,
)
return account, tracerr.Wrap(err)
}
// GetCoordinatorAPI returns a coordinator by its bidderAddr
func (hdb *HistoryDB) GetCoordinatorAPI(bidderAddr ethCommon.Address) (*CoordinatorAPI, error) {
cancel, err := hdb.apiConnCon.Acquire()
defer cancel()
if err != nil {
return nil, tracerr.Wrap(err)
}
defer hdb.apiConnCon.Release()
return hdb.getCoordinatorAPI(hdb.dbRead, bidderAddr)
}
err = meddler.QueryRow(
hdb.dbRead, metricsTotals, `SELECT COUNT(*) as total_txs
FROM tx WHERE tx.batch_num between $1 AND $2;`, metricsTotals.FirstBatchNum, lastBatchNum)
func (hdb *HistoryDB) getCoordinatorAPI(d meddler.DB, bidderAddr ethCommon.Address) (*CoordinatorAPI, error) {
coordinator := &CoordinatorAPI{}
err := meddler.QueryRow(
d, coordinator,
"SELECT * FROM coordinator WHERE bidder_addr = $1 ORDER BY item_id DESC LIMIT 1;",
bidderAddr,
)
return coordinator, tracerr.Wrap(err)
}
// GetNodeInfoAPI retusnt he NodeInfo
func (hdb *HistoryDB) GetNodeInfoAPI() (*NodeInfo, error) {
cancel, err := hdb.apiConnCon.Acquire()
defer cancel()
if err != nil {
return nil, tracerr.Wrap(err)
}
defer hdb.apiConnCon.Release()
return hdb.GetNodeInfo()
}
seconds := metricsTotals.MaxTimestamp.Sub(metricsTotals.MinTimestamp).Seconds()
// Avoid dividing by 0
if seconds == 0 {
seconds++
}
metrics.TransactionsPerSecond = float64(metricsTotals.TotalTransactions) / seconds
if (lastBatchNum - metricsTotals.FirstBatchNum) > 0 {
metrics.TransactionsPerBatch = float64(metricsTotals.TotalTransactions) /
float64(lastBatchNum-metricsTotals.FirstBatchNum+1)
} else {
metrics.TransactionsPerBatch = float64(0)
}
// GetBucketUpdatesInternalAPI returns the latest bucket updates
func (hdb *HistoryDB) GetBucketUpdatesInternalAPI() ([]BucketUpdateAPI, error) {
var bucketUpdates []*BucketUpdateAPI
err := meddler.QueryAll(
hdb.dbRead, &bucketUpdates,
`SELECT num_bucket, withdrawals FROM bucket_update
WHERE item_id in(SELECT max(item_id) FROM bucket_update
group by num_bucket)
ORDER BY num_bucket ASC;`,
)
return db.SlicePtrsToSlice(bucketUpdates).([]BucketUpdateAPI), tracerr.Wrap(err)
}
err = meddler.QueryRow(
hdb.dbRead, metricsTotals, `SELECT COUNT(*) AS total_batches,
COALESCE (SUM(total_fees_usd), 0) AS total_fees FROM batch
WHERE batch_num between $1 and $2;`, metricsTotals.FirstBatchNum, lastBatchNum)
if err != nil {
// GetNextForgersInternalAPI returns next forgers
func (hdb *HistoryDB) GetNextForgersInternalAPI(auctionVars *common.AuctionVariables,
auctionConsts *common.AuctionConstants,
lastBlock common.Block, currentSlot, lastClosedSlot int64) ([]NextForgerAPI, error) {
secondsPerBlock := int64(15) //nolint:gomnd
// currentSlot and lastClosedSlot included
limit := uint(lastClosedSlot - currentSlot + 1)
bids, _, err := hdb.getBestBidsAPI(hdb.dbRead, &currentSlot, &lastClosedSlot, nil, &limit, "ASC")
if err != nil && tracerr.Unwrap(err) != sql.ErrNoRows {
return nil, tracerr.Wrap(err)
}
nextForgers := []NextForgerAPI{}
// Get min bid info
var minBidInfo []MinBidInfo
if currentSlot >= auctionVars.DefaultSlotSetBidSlotNum {
// All min bids can be calculated with the last update of AuctionVariables
if metricsTotals.TotalBatches > 0 {
metrics.BatchFrequency = seconds / float64(metricsTotals.TotalBatches)
minBidInfo = []MinBidInfo{{
DefaultSlotSetBid: auctionVars.DefaultSlotSetBid,
DefaultSlotSetBidSlotNum: auctionVars.DefaultSlotSetBidSlotNum,
}}
} else {
metrics.BatchFrequency = 0
// Get all the relevant updates from the DB
minBidInfo, err = hdb.getMinBidInfo(hdb.dbRead, currentSlot, lastClosedSlot)
if err != nil {
return nil, tracerr.Wrap(err)
}
}
if metricsTotals.TotalTransactions > 0 {
metrics.AvgTransactionFee = metricsTotals.TotalFeesUSD / float64(metricsTotals.TotalTransactions)
} else {
metrics.AvgTransactionFee = 0
// Create nextForger for each slot
for i := currentSlot; i <= lastClosedSlot; i++ {
fromBlock := i*int64(auctionConsts.BlocksPerSlot) +
auctionConsts.GenesisBlockNum
toBlock := (i+1)*int64(auctionConsts.BlocksPerSlot) +
auctionConsts.GenesisBlockNum - 1
nextForger := NextForgerAPI{
Period: Period{
SlotNum: i,
FromBlock: fromBlock,
ToBlock: toBlock,
FromTimestamp: lastBlock.Timestamp.Add(time.Second *
time.Duration(secondsPerBlock*(fromBlock-lastBlock.Num))),
ToTimestamp: lastBlock.Timestamp.Add(time.Second *
time.Duration(secondsPerBlock*(toBlock-lastBlock.Num))),
},
}
foundForger := false
// If there is a bid for a slot, get forger (coordinator)
for j := range bids {
slotNum := bids[j].SlotNum
if slotNum == i {
// There's a bid for the slot
// Check if the bid is greater than the minimum required
for i := 0; i < len(minBidInfo); i++ {
// Find the most recent update
if slotNum >= minBidInfo[i].DefaultSlotSetBidSlotNum {
// Get min bid
minBidSelector := slotNum % int64(len(auctionVars.DefaultSlotSetBid))
minBid := minBidInfo[i].DefaultSlotSetBid[minBidSelector]
// Check if the bid has beaten the minimum
bid, ok := new(big.Int).SetString(string(bids[j].BidValue), 10)
if !ok {
return nil, tracerr.New("Wrong bid value, error parsing it as big.Int")
}
if minBid.Cmp(bid) == 1 {
// Min bid is greater than bid, the slot will be forged by boot coordinator
break
}
foundForger = true
break
}
}
if !foundForger { // There is no bid or it's smaller than the minimum
break
}
coordinator, err := hdb.getCoordinatorAPI(hdb.dbRead, bids[j].Bidder)
if err != nil {
return nil, tracerr.Wrap(err)
}
nextForger.Coordinator = *coordinator
break
}
}
// If there is no bid, the coordinator that will forge is boot coordinator
if !foundForger {
nextForger.Coordinator = CoordinatorAPI{
Forger: auctionVars.BootCoordinator,
URL: auctionVars.BootCoordinatorURL,
}
}
nextForgers = append(nextForgers, nextForger)
}
err = meddler.QueryRow(
hdb.dbRead, metrics,
`SELECT COUNT(*) AS total_bjjs, COUNT(DISTINCT(bjj)) AS total_accounts FROM account;`)
if err != nil {
return nextForgers, nil
}
// GetMetricsInternalAPI returns the MetricsAPI
func (hdb *HistoryDB) GetMetricsInternalAPI(lastBatchNum common.BatchNum) (*MetricsAPI, error) {
var metrics MetricsAPI
// Get the first and last batch of the last 24h and their timestamps
// if u.state.Network.LastBatch == nil {
// return &metrics, nil
// }
type period struct {
FromBatchNum common.BatchNum `meddler:"from_batch_num"`
FromTimestamp time.Time `meddler:"from_timestamp"`
ToBatchNum common.BatchNum `meddler:"-"`
ToTimestamp time.Time `meddler:"to_timestamp"`
}
p := &period{
ToBatchNum: lastBatchNum,
}
if err := meddler.QueryRow(
hdb.dbRead, p, `SELECT
COALESCE (MIN(batch.batch_num), 0) as from_batch_num,
COALESCE (MIN(block.timestamp), NOW()) AS from_timestamp,
COALESCE (MAX(block.timestamp), NOW()) AS to_timestamp
FROM batch INNER JOIN block ON batch.eth_block_num = block.eth_block_num
WHERE block.timestamp >= NOW() - INTERVAL '24 HOURS';`,
); err != nil {
return nil, tracerr.Wrap(err)
}
err = meddler.QueryRow(
hdb.dbRead, metrics,
`SELECT COALESCE (AVG(EXTRACT(EPOCH FROM (forged.timestamp - added.timestamp))), 0)
AS estimated_time_to_forge_l1 FROM tx
INNER JOIN block AS added ON tx.eth_block_num = added.eth_block_num
INNER JOIN batch AS forged_batch ON tx.batch_num = forged_batch.batch_num
INNER JOIN block AS forged ON forged_batch.eth_block_num = forged.eth_block_num
WHERE tx.batch_num between $1 and $2 AND tx.is_l1 AND tx.user_origin;`,
metricsTotals.FirstBatchNum, lastBatchNum,
// Get the amount of txs of that period
row := hdb.dbRead.QueryRow(
`SELECT COUNT(*) as total_txs FROM tx WHERE tx.batch_num between $1 AND $2;`,
p.FromBatchNum, p.ToBatchNum,
)
if err != nil {
var nTxs int
if err := row.Scan(&nTxs); err != nil {
return nil, tracerr.Wrap(err)
}
return metrics, nil
}
// GetAvgTxFeeAPI returns average transaction fee of the last 1h
func (hdb *HistoryDB) GetAvgTxFeeAPI() (float64, error) {
cancel, err := hdb.apiConnCon.Acquire()
defer cancel()
if err != nil {
return 0, tracerr.Wrap(err)
// Set txs/s
seconds := p.ToTimestamp.Sub(p.FromTimestamp).Seconds()
if seconds == 0 { // Avoid dividing by 0
seconds++
}
defer hdb.apiConnCon.Release()
metricsTotals := &MetricsTotals{}
err = meddler.QueryRow(
hdb.dbRead, metricsTotals, `SELECT COUNT(tx.*) as total_txs,
COALESCE (MIN(tx.batch_num), 0) as batch_num
FROM tx INNER JOIN block ON tx.eth_block_num = block.eth_block_num
WHERE block.timestamp >= NOW() - INTERVAL '1 HOURS';`)
if err != nil {
return 0, tracerr.Wrap(err)
metrics.TransactionsPerSecond = float64(nTxs) / seconds
// Set txs/batch
nBatches := p.ToBatchNum - p.FromBatchNum + 1
if nBatches == 0 { // Avoid dividing by 0
nBatches++
}
err = meddler.QueryRow(
hdb.dbRead, metricsTotals, `SELECT COUNT(*) AS total_batches,
COALESCE (SUM(total_fees_usd), 0) AS total_fees FROM batch
WHERE batch_num > $1;`, metricsTotals.FirstBatchNum)
if err != nil {
return 0, tracerr.Wrap(err)
if (p.ToBatchNum - p.FromBatchNum) > 0 {
fmt.Printf("DBG ntxs: %v, nBatches: %v\n", nTxs, nBatches)
metrics.TransactionsPerBatch = float64(nTxs) /
float64(nBatches)
} else {
metrics.TransactionsPerBatch = 0
}
var avgTransactionFee float64
if metricsTotals.TotalTransactions > 0 {
avgTransactionFee = metricsTotals.TotalFeesUSD / float64(metricsTotals.TotalTransactions)
// Get total fee of that period
row = hdb.dbRead.QueryRow(
`SELECT COALESCE (SUM(total_fees_usd), 0) FROM batch WHERE batch_num between $1 AND $2;`,
p.FromBatchNum, p.ToBatchNum,
)
var totalFee float64
if err := row.Scan(&totalFee); err != nil {
return nil, tracerr.Wrap(err)
}
// Set batch frequency
metrics.BatchFrequency = seconds / float64(nBatches)
if nTxs > 0 {
metrics.AvgTransactionFee = totalFee / float64(nTxs)
} else {
avgTransactionFee = 0
metrics.AvgTransactionFee = 0
}
return avgTransactionFee, nil
// Get and set amount of registered accounts
type registeredAccounts struct {
TotalIdx int64 `meddler:"total_idx"`
TotalBJJ int64 `meddler:"total_bjj"`
}
ra := &registeredAccounts{}
if err := meddler.QueryRow(
hdb.dbRead, ra,
`SELECT COUNT(*) AS total_bjj, COUNT(DISTINCT(bjj)) AS total_idx FROM account;`,
); err != nil {
return nil, tracerr.Wrap(err)
}
metrics.TotalAccounts = ra.TotalIdx
metrics.TotalBJJs = ra.TotalBJJ
// Get and set estimated time to forge L1 tx
row = hdb.dbRead.QueryRow(
`SELECT COALESCE (AVG(EXTRACT(EPOCH FROM (forged.timestamp - added.timestamp))), 0) FROM tx
INNER JOIN block AS added ON tx.eth_block_num = added.eth_block_num
INNER JOIN batch AS forged_batch ON tx.batch_num = forged_batch.batch_num
INNER JOIN block AS forged ON forged_batch.eth_block_num = forged.eth_block_num
WHERE tx.batch_num between $1 and $2 AND tx.is_l1 AND tx.user_origin;`,
p.FromBatchNum, p.ToBatchNum,
)
var timeToForgeL1 float64
if err := row.Scan(&timeToForgeL1); err != nil {
return nil, tracerr.Wrap(err)
}
metrics.EstimatedTimeToForgeL1 = timeToForgeL1
return &metrics, nil
}
// GetCommonAccountAPI returns the account associated to an account idx
func (hdb *HistoryDB) GetCommonAccountAPI(idx common.Idx) (*common.Account, error) {
// GetStateAPI returns the StateAPI
func (hdb *HistoryDB) GetStateAPI() (*StateAPI, error) {
cancel, err := hdb.apiConnCon.Acquire()
defer cancel()
if err != nil {
return nil, tracerr.Wrap(err)
}
defer hdb.apiConnCon.Release()
account := &common.Account{}
err = meddler.QueryRow(
hdb.dbRead, account, `SELECT idx, token_id, batch_num, bjj, eth_addr
FROM account WHERE idx = $1;`, idx,
)
return account, tracerr.Wrap(err)
return hdb.getStateAPI(hdb.dbRead)
}

+ 69
- 11
db/historydb/historydb.go

@ -839,6 +839,18 @@ func (hdb *HistoryDB) GetAllBucketUpdates() ([]common.BucketUpdate, error) {
return db.SlicePtrsToSlice(bucketUpdates).([]common.BucketUpdate), tracerr.Wrap(err)
}
func (hdb *HistoryDB) getMinBidInfo(d meddler.DB,
currentSlot, lastClosedSlot int64) ([]MinBidInfo, error) {
minBidInfo := []*MinBidInfo{}
query := `
SELECT DISTINCT default_slot_set_bid, default_slot_set_bid_slot_num FROM auction_vars
WHERE default_slot_set_bid_slot_num < $1
ORDER BY default_slot_set_bid_slot_num DESC
LIMIT $2;`
err := meddler.QueryAll(d, &minBidInfo, query, lastClosedSlot, int(lastClosedSlot-currentSlot)+1)
return db.SlicePtrsToSlice(minBidInfo).([]MinBidInfo), tracerr.Wrap(err)
}
func (hdb *HistoryDB) addTokenExchanges(d meddler.DB, tokenExchanges []common.TokenExchange) error {
if len(tokenExchanges) == 0 {
return nil
@ -1137,17 +1149,6 @@ func (hdb *HistoryDB) AddBlockSCData(blockData *common.BlockData) (err error) {
return tracerr.Wrap(txn.Commit())
}
// GetCoordinatorAPI returns a coordinator by its bidderAddr
func (hdb *HistoryDB) GetCoordinatorAPI(bidderAddr ethCommon.Address) (*CoordinatorAPI, error) {
coordinator := &CoordinatorAPI{}
err := meddler.QueryRow(
hdb.dbRead, coordinator,
"SELECT * FROM coordinator WHERE bidder_addr = $1 ORDER BY item_id DESC LIMIT 1;",
bidderAddr,
)
return coordinator, tracerr.Wrap(err)
}
// AddAuctionVars insert auction vars into the DB
func (hdb *HistoryDB) AddAuctionVars(auctionVars *common.AuctionVariables) error {
return tracerr.Wrap(meddler.Insert(hdb.dbWrite, "auction_vars", auctionVars))
@ -1167,3 +1168,60 @@ func (hdb *HistoryDB) GetTokensTest() ([]TokenWithUSD, error) {
}
return db.SlicePtrsToSlice(tokens).([]TokenWithUSD), nil
}
const (
// CreateAccountExtraFeePercentage is the multiplication factor over
// the average fee for CreateAccount that is applied to obtain the
// recommended fee for CreateAccount
CreateAccountExtraFeePercentage float64 = 2.5
// CreateAccountInternalExtraFeePercentage is the multiplication factor
// over the average fee for CreateAccountInternal that is applied to
// obtain the recommended fee for CreateAccountInternal
CreateAccountInternalExtraFeePercentage float64 = 2.0
)
// GetRecommendedFee returns the RecommendedFee information
func (hdb *HistoryDB) GetRecommendedFee(minFeeUSD float64) (*common.RecommendedFee, error) {
var recommendedFee common.RecommendedFee
// Get total txs and the batch of the first selected tx of the last hour
type totalTxsSinceBatchNum struct {
TotalTxs int `meddler:"total_txs"`
FirstBatchNum common.BatchNum `meddler:"batch_num"`
}
ttsbn := &totalTxsSinceBatchNum{}
if err := meddler.QueryRow(
hdb.dbRead, ttsbn, `SELECT COUNT(tx.*) as total_txs,
COALESCE (MIN(tx.batch_num), 0) as batch_num
FROM tx INNER JOIN block ON tx.eth_block_num = block.eth_block_num
WHERE block.timestamp >= NOW() - INTERVAL '1 HOURS';`,
); err != nil {
return nil, tracerr.Wrap(err)
}
// Get the amount of batches and acumulated fees for the last hour
type totalBatchesAndFee struct {
TotalBatches int `meddler:"total_batches"`
TotalFees float64 `meddler:"total_fees"`
}
tbf := &totalBatchesAndFee{}
if err := meddler.QueryRow(
hdb.dbRead, tbf, `SELECT COUNT(*) AS total_batches,
COALESCE (SUM(total_fees_usd), 0) AS total_fees FROM batch
WHERE batch_num > $1;`, ttsbn.FirstBatchNum,
); err != nil {
return nil, tracerr.Wrap(err)
}
// Update NodeInfo struct
var avgTransactionFee float64
if ttsbn.TotalTxs > 0 {
avgTransactionFee = tbf.TotalFees / float64(ttsbn.TotalTxs)
} else {
avgTransactionFee = 0
}
recommendedFee.ExistingAccount =
math.Max(avgTransactionFee, minFeeUSD)
recommendedFee.CreatesAccount =
math.Max(CreateAccountExtraFeePercentage*avgTransactionFee, minFeeUSD)
recommendedFee.CreatesAccountInternal =
math.Max(CreateAccountInternalExtraFeePercentage*avgTransactionFee, minFeeUSD)
return &recommendedFee, nil
}

+ 65
- 9
db/historydb/historydb_test.go

@ -1176,7 +1176,7 @@ func TestGetMetricsAPI(t *testing.T) {
assert.NoError(t, err)
}
res, err := historyDBWithACC.GetMetricsAPI(common.BatchNum(numBatches))
res, err := historyDB.GetMetricsInternalAPI(common.BatchNum(numBatches))
assert.NoError(t, err)
assert.Equal(t, float64(numTx)/float64(numBatches), res.TransactionsPerBatch)
@ -1254,7 +1254,7 @@ func TestGetMetricsAPIMoreThan24Hours(t *testing.T) {
assert.NoError(t, err)
}
res, err := historyDBWithACC.GetMetricsAPI(common.BatchNum(numBatches))
res, err := historyDBWithACC.GetMetricsInternalAPI(common.BatchNum(numBatches))
assert.NoError(t, err)
assert.InEpsilon(t, 1.0, res.TransactionsPerBatch, 0.1)
@ -1269,13 +1269,7 @@ func TestGetMetricsAPIMoreThan24Hours(t *testing.T) {
func TestGetMetricsAPIEmpty(t *testing.T) {
test.WipeDB(historyDB.DB())
_, err := historyDBWithACC.GetMetricsAPI(0)
assert.NoError(t, err)
}
func TestGetAvgTxFeeEmpty(t *testing.T) {
test.WipeDB(historyDB.DB())
_, err := historyDBWithACC.GetAvgTxFeeAPI()
_, err := historyDBWithACC.GetMetricsInternalAPI(0)
assert.NoError(t, err)
}
@ -1464,3 +1458,65 @@ func setTestBlocks(from, to int64) []common.Block {
}
return blocks
}
func TestNodeInfo(t *testing.T) {
test.WipeDB(historyDB.DB())
err := historyDB.SetStateInternalAPI(&StateAPI{})
require.NoError(t, err)
clientSetup := test.NewClientSetupExample()
constants := &Constants{
SCConsts: common.SCConsts{
Rollup: *clientSetup.RollupConstants,
Auction: *clientSetup.AuctionConstants,
WDelayer: *clientSetup.WDelayerConstants,
},
ChainID: 42,
HermezAddress: clientSetup.AuctionConstants.HermezRollup,
}
err = historyDB.SetConstants(constants)
require.NoError(t, err)
// Test parameters
stateAPI := &StateAPI{
NodePublicConfig: NodePublicConfig{
ForgeDelay: 3.1,
},
Network: NetworkAPI{
LastEthBlock: 12,
LastSyncBlock: 34,
},
Metrics: MetricsAPI{
TransactionsPerBatch: 1.1,
TotalAccounts: 42,
},
Rollup: *NewRollupVariablesAPI(clientSetup.RollupVariables),
Auction: *NewAuctionVariablesAPI(clientSetup.AuctionVariables),
WithdrawalDelayer: *clientSetup.WDelayerVariables,
RecommendedFee: common.RecommendedFee{
ExistingAccount: 0.15,
},
}
err = historyDB.SetStateInternalAPI(stateAPI)
require.NoError(t, err)
nodeConfig := &NodeConfig{
MaxPoolTxs: 123,
MinFeeUSD: 0.5,
}
err = historyDB.SetNodeConfig(nodeConfig)
require.NoError(t, err)
dbConstants, err := historyDB.GetConstants()
require.NoError(t, err)
assert.Equal(t, constants, dbConstants)
dbNodeConfig, err := historyDB.GetNodeConfig()
require.NoError(t, err)
assert.Equal(t, nodeConfig, dbNodeConfig)
dbStateAPI, err := historyDB.getStateAPI(historyDB.dbRead)
require.NoError(t, err)
assert.Equal(t, stateAPI, dbStateAPI)
}

+ 165
- 0
db/historydb/nodeinfo.go

@ -0,0 +1,165 @@
package historydb
import (
"time"
ethCommon "github.com/ethereum/go-ethereum/common"
"github.com/hermeznetwork/hermez-node/common"
"github.com/hermeznetwork/tracerr"
"github.com/russross/meddler"
)
// Period represents a time period in ethereum
type Period struct {
SlotNum int64 `json:"slotNum"`
FromBlock int64 `json:"fromBlock"`
ToBlock int64 `json:"toBlock"`
FromTimestamp time.Time `json:"fromTimestamp"`
ToTimestamp time.Time `json:"toTimestamp"`
}
// NextForgerAPI represents the next forger exposed via the API
type NextForgerAPI struct {
Coordinator CoordinatorAPI `json:"coordinator"`
Period Period `json:"period"`
}
// NetworkAPI is the network state exposed via the API
type NetworkAPI struct {
LastEthBlock int64 `json:"lastEthereumBlock"`
LastSyncBlock int64 `json:"lastSynchedBlock"`
LastBatch *BatchAPI `json:"lastBatch"`
CurrentSlot int64 `json:"currentSlot"`
NextForgers []NextForgerAPI `json:"nextForgers"`
}
// NodePublicConfig is the configuration of the node that is exposed via API
type NodePublicConfig struct {
// ForgeDelay in seconds
ForgeDelay float64 `json:"forgeDelay"`
}
// StateAPI is an object representing the node and network state exposed via the API
type StateAPI struct {
// NodePublicConfig is the configuration of the node that is exposed via API
NodePublicConfig NodePublicConfig `json:"nodeConfig"`
Network NetworkAPI `json:"network"`
Metrics MetricsAPI `json:"metrics"`
Rollup RollupVariablesAPI `json:"rollup"`
Auction AuctionVariablesAPI `json:"auction"`
WithdrawalDelayer common.WDelayerVariables `json:"withdrawalDelayer"`
RecommendedFee common.RecommendedFee `json:"recommendedFee"`
}
// Constants contains network constants
type Constants struct {
common.SCConsts
ChainID uint16
HermezAddress ethCommon.Address
}
// NodeConfig contains the node config exposed in the API
type NodeConfig struct {
MaxPoolTxs uint32
MinFeeUSD float64
ForgeDelay float64
}
// NodeInfo contains information about he node used when serving the API
type NodeInfo struct {
ItemID int `meddler:"item_id,pk"`
StateAPI *StateAPI `meddler:"state,json"`
NodeConfig *NodeConfig `meddler:"config,json"`
Constants *Constants `meddler:"constants,json"`
}
// GetNodeInfo returns the NodeInfo
func (hdb *HistoryDB) GetNodeInfo() (*NodeInfo, error) {
ni := &NodeInfo{}
err := meddler.QueryRow(
hdb.dbRead, ni, `SELECT * FROM node_info WHERE item_id = 1;`,
)
return ni, tracerr.Wrap(err)
}
// GetConstants returns the Constats
func (hdb *HistoryDB) GetConstants() (*Constants, error) {
var nodeInfo NodeInfo
err := meddler.QueryRow(
hdb.dbRead, &nodeInfo,
"SELECT constants FROM node_info WHERE item_id = 1;",
)
return nodeInfo.Constants, tracerr.Wrap(err)
}
// SetConstants sets the Constants
func (hdb *HistoryDB) SetConstants(constants *Constants) error {
_constants := struct {
Constants *Constants `meddler:"constants,json"`
}{constants}
values, err := meddler.Default.Values(&_constants, false)
if err != nil {
return tracerr.Wrap(err)
}
_, err = hdb.dbWrite.Exec(
"UPDATE node_info SET constants = $1 WHERE item_id = 1;",
values[0],
)
return tracerr.Wrap(err)
}
// GetStateInternalAPI returns the StateAPI
func (hdb *HistoryDB) GetStateInternalAPI() (*StateAPI, error) {
return hdb.getStateAPI(hdb.dbRead)
}
func (hdb *HistoryDB) getStateAPI(d meddler.DB) (*StateAPI, error) {
var nodeInfo NodeInfo
err := meddler.QueryRow(
d, &nodeInfo,
"SELECT state FROM node_info WHERE item_id = 1;",
)
return nodeInfo.StateAPI, tracerr.Wrap(err)
}
// SetStateInternalAPI sets the StateAPI
func (hdb *HistoryDB) SetStateInternalAPI(stateAPI *StateAPI) error {
_stateAPI := struct {
StateAPI *StateAPI `meddler:"state,json"`
}{stateAPI}
values, err := meddler.Default.Values(&_stateAPI, false)
if err != nil {
return tracerr.Wrap(err)
}
_, err = hdb.dbWrite.Exec(
"UPDATE node_info SET state = $1 WHERE item_id = 1;",
values[0],
)
return tracerr.Wrap(err)
}
// GetNodeConfig returns the NodeConfig
func (hdb *HistoryDB) GetNodeConfig() (*NodeConfig, error) {
var nodeInfo NodeInfo
err := meddler.QueryRow(
hdb.dbRead, &nodeInfo,
"SELECT config FROM node_info WHERE item_id = 1;",
)
return nodeInfo.NodeConfig, tracerr.Wrap(err)
}
// SetNodeConfig sets the NodeConfig
func (hdb *HistoryDB) SetNodeConfig(nodeConfig *NodeConfig) error {
_nodeConfig := struct {
NodeConfig *NodeConfig `meddler:"config,json"`
}{nodeConfig}
values, err := meddler.Default.Values(&_nodeConfig, false)
if err != nil {
return tracerr.Wrap(err)
}
_, err = hdb.dbWrite.Exec(
"UPDATE node_info SET config = $1 WHERE item_id = 1;",
values[0],
)
return tracerr.Wrap(err)
}

+ 48
- 13
db/historydb/views.go

@ -308,8 +308,8 @@ type BatchAPI struct {
LastItem uint64 `json:"-" meddler:"last_item"`
}
// Metrics define metrics of the network
type Metrics struct {
// MetricsAPI define metrics of the network
type MetricsAPI struct {
TransactionsPerBatch float64 `json:"transactionsPerBatch"`
BatchFrequency float64 `json:"batchFrequency"`
TransactionsPerSecond float64 `json:"transactionsPerSecond"`
@ -319,17 +319,6 @@ type Metrics struct {
EstimatedTimeToForgeL1 float64 `json:"estimatedTimeToForgeL1" meddler:"estimated_time_to_forge_l1"`
}
// MetricsTotals is used to get temporal information from HistoryDB
// to calculate data to be stored into the Metrics struct
type MetricsTotals struct {
TotalTransactions uint64 `meddler:"total_txs"`
FirstBatchNum common.BatchNum `meddler:"batch_num"`
TotalBatches int64 `meddler:"total_batches"`
TotalFeesUSD float64 `meddler:"total_fees"`
MinTimestamp time.Time `meddler:"min_timestamp,utctime"`
MaxTimestamp time.Time `meddler:"max_timestamp,utctime"`
}
// BidAPI is a representation of a bid with additional information
// required by the API
type BidAPI struct {
@ -380,6 +369,27 @@ type RollupVariablesAPI struct {
SafeMode bool `json:"safeMode" meddler:"safe_mode"`
}
// NewRollupVariablesAPI creates a RollupVariablesAPI from common.RollupVariables
func NewRollupVariablesAPI(rollupVariables *common.RollupVariables) *RollupVariablesAPI {
rollupVars := RollupVariablesAPI{
EthBlockNum: rollupVariables.EthBlockNum,
FeeAddToken: apitypes.NewBigIntStr(rollupVariables.FeeAddToken),
ForgeL1L2BatchTimeout: rollupVariables.ForgeL1L2BatchTimeout,
WithdrawalDelay: rollupVariables.WithdrawalDelay,
SafeMode: rollupVariables.SafeMode,
}
for i, bucket := range rollupVariables.Buckets {
rollupVars.Buckets[i] = BucketParamsAPI{
CeilUSD: apitypes.NewBigIntStr(bucket.CeilUSD),
Withdrawals: apitypes.NewBigIntStr(bucket.Withdrawals),
BlockWithdrawalRate: apitypes.NewBigIntStr(bucket.BlockWithdrawalRate),
MaxWithdrawals: apitypes.NewBigIntStr(bucket.MaxWithdrawals),
}
}
return &rollupVars
}
// AuctionVariablesAPI are the variables of the Auction Smart Contract
type AuctionVariablesAPI struct {
EthBlockNum int64 `json:"ethereumBlockNum" meddler:"eth_block_num"`
@ -404,3 +414,28 @@ type AuctionVariablesAPI struct {
// SlotDeadline 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 `json:"slotDeadline" meddler:"slot_deadline" validate:"required"`
}
// NewAuctionVariablesAPI creates a AuctionVariablesAPI from common.AuctionVariables
func NewAuctionVariablesAPI(auctionVariables *common.AuctionVariables) *AuctionVariablesAPI {
auctionVars := AuctionVariablesAPI{
EthBlockNum: auctionVariables.EthBlockNum,
DonationAddress: auctionVariables.DonationAddress,
BootCoordinator: auctionVariables.BootCoordinator,
BootCoordinatorURL: auctionVariables.BootCoordinatorURL,
DefaultSlotSetBidSlotNum: auctionVariables.DefaultSlotSetBidSlotNum,
ClosedAuctionSlots: auctionVariables.ClosedAuctionSlots,
OpenAuctionSlots: auctionVariables.OpenAuctionSlots,
Outbidding: auctionVariables.Outbidding,
SlotDeadline: auctionVariables.SlotDeadline,
}
for i, slot := range auctionVariables.DefaultSlotSetBid {
auctionVars.DefaultSlotSetBid[i] = apitypes.NewBigIntStr(slot)
}
for i, ratio := range auctionVariables.AllocationRatio {
auctionVars.AllocationRatio[i] = ratio
}
return &auctionVars
}

+ 11
- 0
db/migrations/0001.sql

@ -661,6 +661,16 @@ CREATE TABLE account_creation_auth (
timestamp TIMESTAMP WITHOUT TIME ZONE NOT NULL DEFAULT timezone('utc', now())
);
CREATE TABLE node_info (
item_id SERIAL PRIMARY KEY,
state BYTEA, -- object returned by GET /state
config BYTEA, -- Node config
-- max_pool_txs BIGINT, -- L2DB config
-- min_fee NUMERIC, -- L2DB config
constants BYTEA -- info of the network that is constant
);
INSERT INTO node_info(item_id) VALUES (1); -- Always have a single row that we will update
-- +migrate Down
-- triggers
DROP TRIGGER IF EXISTS trigger_token_usd_update ON token;
@ -675,6 +685,7 @@ DROP FUNCTION IF EXISTS set_tx;
DROP FUNCTION IF EXISTS forge_l1_user_txs;
DROP FUNCTION IF EXISTS set_pool_tx;
-- drop tables IF EXISTS
DROP TABLE IF EXISTS node_info;
DROP TABLE IF EXISTS account_creation_auth;
DROP TABLE IF EXISTS tx_pool;
DROP TABLE IF EXISTS auction_vars;

+ 221
- 88
node/node.go

@ -27,6 +27,7 @@ import (
"github.com/hermeznetwork/hermez-node/log"
"github.com/hermeznetwork/hermez-node/priceupdater"
"github.com/hermeznetwork/hermez-node/prover"
"github.com/hermeznetwork/hermez-node/stateapiupdater"
"github.com/hermeznetwork/hermez-node/synchronizer"
"github.com/hermeznetwork/hermez-node/test/debugapi"
"github.com/hermeznetwork/hermez-node/txprocessor"
@ -53,9 +54,10 @@ const (
// Node is the Hermez Node
type Node struct {
nodeAPI *NodeAPI
debugAPI *debugapi.DebugAPI
priceUpdater *priceupdater.PriceUpdater
nodeAPI *NodeAPI
stateAPIUpdater *stateapiupdater.Updater
debugAPI *debugapi.DebugAPI
priceUpdater *priceupdater.PriceUpdater
// Coordinator
coord *coordinator.Coordinator
@ -67,6 +69,7 @@ type Node struct {
mode Mode
sqlConnRead *sqlx.DB
sqlConnWrite *sqlx.DB
historyDB *historydb.HistoryDB
ctx context.Context
wg sync.WaitGroup
cancel context.CancelFunc
@ -241,12 +244,35 @@ func NewNode(mode Mode, cfg *config.Node) (*Node, error) {
}
initSCVars := sync.SCVars()
scConsts := synchronizer.SCConsts{
scConsts := common.SCConsts{
Rollup: *sync.RollupConstants(),
Auction: *sync.AuctionConstants(),
WDelayer: *sync.WDelayerConstants(),
}
hdbNodeCfg := historydb.NodeConfig{
MaxPoolTxs: cfg.Coordinator.L2DB.MaxTxs,
MinFeeUSD: cfg.Coordinator.L2DB.MinFeeUSD,
ForgeDelay: cfg.Coordinator.ForgeDelay.Duration.Seconds(),
}
if err := historyDB.SetNodeConfig(&hdbNodeCfg); err != nil {
return nil, tracerr.Wrap(err)
}
hdbConsts := historydb.Constants{
SCConsts: common.SCConsts{
Rollup: scConsts.Rollup,
Auction: scConsts.Auction,
WDelayer: scConsts.WDelayer,
},
ChainID: chainIDU16,
HermezAddress: cfg.SmartContracts.Rollup,
}
if err := historyDB.SetConstants(&hdbConsts); err != nil {
return nil, tracerr.Wrap(err)
}
stateAPIUpdater := stateapiupdater.NewUpdater(historyDB, &hdbNodeCfg, initSCVars, &hdbConsts)
var coord *coordinator.Coordinator
if mode == ModeCoordinator {
// Unlock FeeAccount EthAddr in the keystore to generate the
@ -369,11 +395,7 @@ func NewNode(mode Mode, cfg *config.Node) (*Node, error) {
serverProofs,
client,
&scConsts,
&synchronizer.SCVariables{
Rollup: *initSCVars.Rollup,
Auction: *initSCVars.Auction,
WDelayer: *initSCVars.WDelayer,
},
initSCVars,
)
if err != nil {
return nil, tracerr.Wrap(err)
@ -405,23 +427,11 @@ func NewNode(mode Mode, cfg *config.Node) (*Node, error) {
coord, cfg.API.Explorer,
server,
historyDB,
stateDB,
l2DB,
&api.Config{
RollupConstants: scConsts.Rollup,
AuctionConstants: scConsts.Auction,
WDelayerConstants: scConsts.WDelayer,
ChainID: chainIDU16,
HermezAddress: cfg.SmartContracts.Rollup,
},
cfg.Coordinator.ForgeDelay.Duration,
)
if err != nil {
return nil, tracerr.Wrap(err)
}
nodeAPI.api.SetRollupVariables(*initSCVars.Rollup)
nodeAPI.api.SetAuctionVariables(*initSCVars.Auction)
nodeAPI.api.SetWDelayerVariables(*initSCVars.WDelayer)
}
var debugAPI *debugapi.DebugAPI
if cfg.Debug.APIAddress != "" {
@ -439,20 +449,138 @@ func NewNode(mode Mode, cfg *config.Node) (*Node, error) {
}
ctx, cancel := context.WithCancel(context.Background())
return &Node{
nodeAPI: nodeAPI,
debugAPI: debugAPI,
priceUpdater: priceUpdater,
coord: coord,
sync: sync,
cfg: cfg,
mode: mode,
sqlConnRead: dbRead,
sqlConnWrite: dbWrite,
ctx: ctx,
cancel: cancel,
stateAPIUpdater: stateAPIUpdater,
nodeAPI: nodeAPI,
debugAPI: debugAPI,
priceUpdater: priceUpdater,
coord: coord,
sync: sync,
cfg: cfg,
mode: mode,
sqlConnRead: dbRead,
sqlConnWrite: dbWrite,
historyDB: historyDB,
ctx: ctx,
cancel: cancel,
}, nil
}
// APIServer is a server that only runs the API
type APIServer struct {
nodeAPI *NodeAPI
mode Mode
ctx context.Context
wg sync.WaitGroup
cancel context.CancelFunc
}
// NewAPIServer creates a new APIServer
func NewAPIServer(mode Mode, cfg *config.APIServer) (*APIServer, error) {
meddler.Debug = cfg.Debug.MeddlerLogs
// Stablish DB connection
dbWrite, err := dbUtils.InitSQLDB(
cfg.PostgreSQL.PortWrite,
cfg.PostgreSQL.HostWrite,
cfg.PostgreSQL.UserWrite,
cfg.PostgreSQL.PasswordWrite,
cfg.PostgreSQL.NameWrite,
)
if err != nil {
return nil, tracerr.Wrap(fmt.Errorf("dbUtils.InitSQLDB: %w", err))
}
var dbRead *sqlx.DB
if cfg.PostgreSQL.HostRead == "" {
dbRead = dbWrite
} else if cfg.PostgreSQL.HostRead == cfg.PostgreSQL.HostWrite {
return nil, tracerr.Wrap(fmt.Errorf(
"PostgreSQL.HostRead and PostgreSQL.HostWrite must be different",
))
} else {
dbRead, err = dbUtils.InitSQLDB(
cfg.PostgreSQL.PortRead,
cfg.PostgreSQL.HostRead,
cfg.PostgreSQL.UserRead,
cfg.PostgreSQL.PasswordRead,
cfg.PostgreSQL.NameRead,
)
if err != nil {
return nil, tracerr.Wrap(fmt.Errorf("dbUtils.InitSQLDB: %w", err))
}
}
apiConnCon := dbUtils.NewAPIConnectionController(
cfg.API.MaxSQLConnections,
cfg.API.SQLConnectionTimeout.Duration,
)
historyDB := historydb.NewHistoryDB(dbRead, dbWrite, apiConnCon)
var l2DB *l2db.L2DB
if mode == ModeCoordinator {
l2DB = l2db.NewL2DB(
dbRead, dbWrite,
0,
cfg.Coordinator.L2DB.MaxTxs,
cfg.Coordinator.L2DB.MinFeeUSD,
0,
apiConnCon,
)
}
if cfg.Debug.GinDebugMode {
gin.SetMode(gin.DebugMode)
} else {
gin.SetMode(gin.ReleaseMode)
}
server := gin.Default()
coord := false
if mode == ModeCoordinator {
coord = cfg.Coordinator.API.Coordinator
}
nodeAPI, err := NewNodeAPI(
cfg.API.Address,
coord, cfg.API.Explorer,
server,
historyDB,
l2DB,
)
if err != nil {
return nil, tracerr.Wrap(err)
}
ctx, cancel := context.WithCancel(context.Background())
return &APIServer{
nodeAPI: nodeAPI,
mode: mode,
ctx: ctx,
cancel: cancel,
}, nil
}
// Start the APIServer
func (s *APIServer) Start() {
log.Infow("Starting api server...", "mode", s.mode)
log.Info("Starting NodeAPI...")
s.wg.Add(1)
go func() {
defer func() {
log.Info("NodeAPI routine stopped")
s.wg.Done()
}()
if err := s.nodeAPI.Run(s.ctx); err != nil {
if s.ctx.Err() != nil {
return
}
log.Fatalw("NodeAPI.Run", "err", err)
}
}()
}
// Stop the APIServer
func (s *APIServer) Stop() {
log.Infow("Stopping NodeAPI...")
s.cancel()
s.wg.Wait()
}
// NodeAPI holds the node http API
type NodeAPI struct { //nolint:golint
api *api.API
@ -472,10 +600,7 @@ func NewNodeAPI(
coordinatorEndpoints, explorerEndpoints bool,
server *gin.Engine,
hdb *historydb.HistoryDB,
sdb *statedb.StateDB,
l2db *l2db.L2DB,
config *api.Config,
forgeDelay time.Duration,
) (*NodeAPI, error) {
engine := gin.Default()
engine.NoRoute(handleNoRoute)
@ -485,10 +610,6 @@ func NewNodeAPI(
engine,
hdb,
l2db,
config,
&api.NodeConfig{
ForgeDelay: forgeDelay.Seconds(),
},
)
if err != nil {
return nil, tracerr.Wrap(err)
@ -534,58 +655,50 @@ func (a *NodeAPI) Run(ctx context.Context) error {
}
func (n *Node) handleNewBlock(ctx context.Context, stats *synchronizer.Stats,
vars synchronizer.SCVariablesPtr, batches []common.BatchData) {
vars *common.SCVariablesPtr, batches []common.BatchData) error {
if n.mode == ModeCoordinator {
n.coord.SendMsg(ctx, coordinator.MsgSyncBlock{
Stats: *stats,
Vars: vars,
Vars: *vars,
Batches: batches,
})
}
if n.nodeAPI != nil {
if vars.Rollup != nil {
n.nodeAPI.api.SetRollupVariables(*vars.Rollup)
}
if vars.Auction != nil {
n.nodeAPI.api.SetAuctionVariables(*vars.Auction)
}
if vars.WDelayer != nil {
n.nodeAPI.api.SetWDelayerVariables(*vars.WDelayer)
}
if stats.Synced() {
if err := n.nodeAPI.api.UpdateNetworkInfo(
stats.Eth.LastBlock, stats.Sync.LastBlock,
common.BatchNum(stats.Eth.LastBatchNum),
stats.Sync.Auction.CurrentSlot.SlotNum,
); err != nil {
log.Errorw("API.UpdateNetworkInfo", "err", err)
}
} else {
n.nodeAPI.api.UpdateNetworkInfoBlock(
stats.Eth.LastBlock, stats.Sync.LastBlock,
)
n.stateAPIUpdater.SetSCVars(vars)
if stats.Synced() {
if err := n.stateAPIUpdater.UpdateNetworkInfo(
stats.Eth.LastBlock, stats.Sync.LastBlock,
common.BatchNum(stats.Eth.LastBatchNum),
stats.Sync.Auction.CurrentSlot.SlotNum,
); err != nil {
log.Errorw("ApiStateUpdater.UpdateNetworkInfo", "err", err)
}
} else {
n.stateAPIUpdater.UpdateNetworkInfoBlock(
stats.Eth.LastBlock, stats.Sync.LastBlock,
)
}
if err := n.stateAPIUpdater.Store(); err != nil {
return tracerr.Wrap(err)
}
return nil
}
func (n *Node) handleReorg(ctx context.Context, stats *synchronizer.Stats,
vars synchronizer.SCVariablesPtr) {
vars *common.SCVariables) error {
if n.mode == ModeCoordinator {
n.coord.SendMsg(ctx, coordinator.MsgSyncReorg{
Stats: *stats,
Vars: vars,
Vars: *vars.AsPtr(),
})
}
if n.nodeAPI != nil {
vars := n.sync.SCVars()
n.nodeAPI.api.SetRollupVariables(*vars.Rollup)
n.nodeAPI.api.SetAuctionVariables(*vars.Auction)
n.nodeAPI.api.SetWDelayerVariables(*vars.WDelayer)
n.nodeAPI.api.UpdateNetworkInfoBlock(
stats.Eth.LastBlock, stats.Sync.LastBlock,
)
n.stateAPIUpdater.SetSCVars(vars.AsPtr())
n.stateAPIUpdater.UpdateNetworkInfoBlock(
stats.Eth.LastBlock, stats.Sync.LastBlock,
)
if err := n.stateAPIUpdater.Store(); err != nil {
return tracerr.Wrap(err)
}
return nil
}
// TODO(Edu): Consider keeping the `lastBlock` inside synchronizer so that we
@ -601,16 +714,20 @@ func (n *Node) syncLoopFn(ctx context.Context, lastBlock *common.Block) (*common
// case: reorg
log.Infow("Synchronizer.Sync reorg", "discarded", *discarded)
vars := n.sync.SCVars()
n.handleReorg(ctx, stats, vars)
if err := n.handleReorg(ctx, stats, vars); err != nil {
return nil, time.Duration(0), tracerr.Wrap(err)
}
return nil, time.Duration(0), nil
} else if blockData != nil {
// case: new block
vars := synchronizer.SCVariablesPtr{
vars := common.SCVariablesPtr{
Rollup: blockData.Rollup.Vars,
Auction: blockData.Auction.Vars,
WDelayer: blockData.WDelayer.Vars,
}
n.handleNewBlock(ctx, stats, vars, blockData.Rollup.Batches)
if err := n.handleNewBlock(ctx, stats, &vars, blockData.Rollup.Batches); err != nil {
return nil, time.Duration(0), tracerr.Wrap(err)
}
return &blockData.Block, time.Duration(0), nil
} else {
// case: no block
@ -629,7 +746,9 @@ func (n *Node) StartSynchronizer() {
// the last synced one) is synchronized
stats := n.sync.Stats()
vars := n.sync.SCVars()
n.handleNewBlock(n.ctx, stats, vars, []common.BatchData{})
if err := n.handleNewBlock(n.ctx, stats, vars.AsPtr(), []common.BatchData{}); err != nil {
log.Fatalw("Node.handleNewBlock", "err", err)
}
n.wg.Add(1)
go func() {
@ -716,18 +835,25 @@ func (n *Node) StartNodeAPI() {
n.wg.Add(1)
go func() {
// Do an initial update on startup
if err := n.nodeAPI.api.UpdateMetrics(); err != nil {
log.Errorw("API.UpdateMetrics", "err", err)
if err := n.stateAPIUpdater.UpdateMetrics(); err != nil {
log.Errorw("ApiStateUpdater.UpdateMetrics", "err", err)
}
if err := n.stateAPIUpdater.Store(); err != nil {
log.Errorw("ApiStateUpdater.Store", "err", err)
}
for {
select {
case <-n.ctx.Done():
log.Info("API.UpdateMetrics loop done")
log.Info("ApiStateUpdater.UpdateMetrics loop done")
n.wg.Done()
return
case <-time.After(n.cfg.API.UpdateMetricsInterval.Duration):
if err := n.nodeAPI.api.UpdateMetrics(); err != nil {
log.Errorw("API.UpdateMetrics", "err", err)
if err := n.stateAPIUpdater.UpdateMetrics(); err != nil {
log.Errorw("ApiStateUpdater.UpdateMetrics", "err", err)
continue
}
if err := n.stateAPIUpdater.Store(); err != nil {
log.Errorw("ApiStateUpdater.Store", "err", err)
}
}
}
@ -736,18 +862,25 @@ func (n *Node) StartNodeAPI() {
n.wg.Add(1)
go func() {
// Do an initial update on startup
if err := n.nodeAPI.api.UpdateRecommendedFee(); err != nil {
log.Errorw("API.UpdateRecommendedFee", "err", err)
if err := n.stateAPIUpdater.UpdateRecommendedFee(); err != nil {
log.Errorw("ApiStateUpdater.UpdateRecommendedFee", "err", err)
}
if err := n.stateAPIUpdater.Store(); err != nil {
log.Errorw("ApiStateUpdater.Store", "err", err)
}
for {
select {
case <-n.ctx.Done():
log.Info("API.UpdateRecommendedFee loop done")
log.Info("ApiStateUpdaterAPI.UpdateRecommendedFee loop done")
n.wg.Done()
return
case <-time.After(n.cfg.API.UpdateRecommendedFeeInterval.Duration):
if err := n.nodeAPI.api.UpdateRecommendedFee(); err != nil {
log.Errorw("API.UpdateRecommendedFee", "err", err)
if err := n.stateAPIUpdater.UpdateRecommendedFee(); err != nil {
log.Errorw("ApiStateUpdaterAPI.UpdateRecommendedFee", "err", err)
continue
}
if err := n.stateAPIUpdater.Store(); err != nil {
log.Errorw("ApiStateUpdater.Store", "err", err)
}
}
}

+ 155
- 0
stateapiupdater/stateapiupdater.go

@ -0,0 +1,155 @@
package stateapiupdater
import (
"database/sql"
"sync"
"github.com/hermeznetwork/hermez-node/common"
"github.com/hermeznetwork/hermez-node/db/historydb"
"github.com/hermeznetwork/tracerr"
)
// Updater is an utility object to facilitate updating the StateAPI
type Updater struct {
hdb *historydb.HistoryDB
state historydb.StateAPI
config historydb.NodeConfig
vars common.SCVariablesPtr
consts historydb.Constants
rw sync.RWMutex
}
// NewUpdater creates a new Updater
func NewUpdater(hdb *historydb.HistoryDB, config *historydb.NodeConfig, vars *common.SCVariables,
consts *historydb.Constants) *Updater {
u := Updater{
hdb: hdb,
config: *config,
consts: *consts,
state: historydb.StateAPI{
NodePublicConfig: historydb.NodePublicConfig{
ForgeDelay: config.ForgeDelay,
},
},
}
u.SetSCVars(vars.AsPtr())
return &u
}
// Store the State in the HistoryDB
func (u *Updater) Store() error {
u.rw.RLock()
defer u.rw.RUnlock()
return tracerr.Wrap(u.hdb.SetStateInternalAPI(&u.state))
}
// SetSCVars sets the smart contract vars (ony updates those that are not nil)
func (u *Updater) SetSCVars(vars *common.SCVariablesPtr) {
u.rw.Lock()
defer u.rw.Unlock()
if vars.Rollup != nil {
u.vars.Rollup = vars.Rollup
rollupVars := historydb.NewRollupVariablesAPI(u.vars.Rollup)
u.state.Rollup = *rollupVars
}
if vars.Auction != nil {
u.vars.Auction = vars.Auction
auctionVars := historydb.NewAuctionVariablesAPI(u.vars.Auction)
u.state.Auction = *auctionVars
}
if vars.WDelayer != nil {
u.vars.WDelayer = vars.WDelayer
u.state.WithdrawalDelayer = *u.vars.WDelayer
}
}
// UpdateRecommendedFee update Status.RecommendedFee information
func (u *Updater) UpdateRecommendedFee() error {
recommendedFee, err := u.hdb.GetRecommendedFee(u.config.MinFeeUSD)
if err != nil {
return tracerr.Wrap(err)
}
u.rw.Lock()
u.state.RecommendedFee = *recommendedFee
u.rw.Unlock()
return nil
}
// UpdateMetrics update Status.Metrics information
func (u *Updater) UpdateMetrics() error {
u.rw.RLock()
lastBatch := u.state.Network.LastBatch
u.rw.RUnlock()
if lastBatch == nil {
return nil
}
lastBatchNum := lastBatch.BatchNum
metrics, err := u.hdb.GetMetricsInternalAPI(lastBatchNum)
if err != nil {
return tracerr.Wrap(err)
}
u.rw.Lock()
u.state.Metrics = *metrics
u.rw.Unlock()
return nil
}
// UpdateNetworkInfoBlock update Status.Network block related information
func (u *Updater) UpdateNetworkInfoBlock(lastEthBlock, lastSyncBlock common.Block) {
u.rw.Lock()
u.state.Network.LastSyncBlock = lastSyncBlock.Num
u.state.Network.LastEthBlock = lastEthBlock.Num
u.rw.Unlock()
}
// UpdateNetworkInfo update Status.Network information
func (u *Updater) UpdateNetworkInfo(
lastEthBlock, lastSyncBlock common.Block,
lastBatchNum common.BatchNum, currentSlot int64,
) error {
// Get last batch in API format
lastBatch, err := u.hdb.GetBatchInternalAPI(lastBatchNum)
if tracerr.Unwrap(err) == sql.ErrNoRows {
lastBatch = nil
} else if err != nil {
return tracerr.Wrap(err)
}
u.rw.RLock()
auctionVars := u.vars.Auction
u.rw.RUnlock()
// Get next forgers
lastClosedSlot := currentSlot + int64(auctionVars.ClosedAuctionSlots)
nextForgers, err := u.hdb.GetNextForgersInternalAPI(auctionVars, &u.consts.Auction,
lastSyncBlock, currentSlot, lastClosedSlot)
if tracerr.Unwrap(err) == sql.ErrNoRows {
nextForgers = nil
} else if err != nil {
return tracerr.Wrap(err)
}
bucketUpdates, err := u.hdb.GetBucketUpdatesInternalAPI()
if err == sql.ErrNoRows {
bucketUpdates = nil
} else if err != nil {
return tracerr.Wrap(err)
}
u.rw.Lock()
// Update NodeInfo struct
for i, bucketParams := range u.state.Rollup.Buckets {
for _, bucketUpdate := range bucketUpdates {
if bucketUpdate.NumBucket == i {
bucketParams.Withdrawals = bucketUpdate.Withdrawals
u.state.Rollup.Buckets[i] = bucketParams
break
}
}
}
u.state.Network.LastSyncBlock = lastSyncBlock.Num
u.state.Network.LastEthBlock = lastEthBlock.Num
u.state.Network.LastBatch = lastBatch
u.state.Network.CurrentSlot = currentSlot
u.state.Network.NextForgers = nextForgers
u.rw.Unlock()
return nil
}

+ 11
- 33
synchronizer/synchronizer.go

@ -183,28 +183,6 @@ type StartBlockNums struct {
WDelayer int64
}
// SCVariables joins all the smart contract variables in a single struct
type SCVariables struct {
Rollup common.RollupVariables `validate:"required"`
Auction common.AuctionVariables `validate:"required"`
WDelayer common.WDelayerVariables `validate:"required"`
}
// SCVariablesPtr joins all the smart contract variables as pointers in a single
// struct
type SCVariablesPtr struct {
Rollup *common.RollupVariables `validate:"required"`
Auction *common.AuctionVariables `validate:"required"`
WDelayer *common.WDelayerVariables `validate:"required"`
}
// SCConsts joins all the smart contract constants in a single struct
type SCConsts struct {
Rollup common.RollupConstants
Auction common.AuctionConstants
WDelayer common.WDelayerConstants
}
// Config is the Synchronizer configuration
type Config struct {
StatsRefreshPeriod time.Duration
@ -214,14 +192,14 @@ type Config struct {
// Synchronizer implements the Synchronizer type
type Synchronizer struct {
ethClient eth.ClientInterface
consts SCConsts
consts common.SCConsts
historyDB *historydb.HistoryDB
l2DB *l2db.L2DB
stateDB *statedb.StateDB
cfg Config
initVars SCVariables
initVars common.SCVariables
startBlockNum int64
vars SCVariables
vars common.SCVariables
stats *StatsHolder
resetStateFailed bool
}
@ -244,7 +222,7 @@ func NewSynchronizer(ethClient eth.ClientInterface, historyDB *historydb.History
return nil, tracerr.Wrap(fmt.Errorf("NewSynchronizer ethClient.WDelayerConstants(): %w",
err))
}
consts := SCConsts{
consts := common.SCConsts{
Rollup: *rollupConstants,
Auction: *auctionConstants,
WDelayer: *wDelayerConstants,
@ -310,11 +288,11 @@ func (s *Synchronizer) WDelayerConstants() *common.WDelayerConstants {
}
// SCVars returns a copy of the Smart Contract Variables
func (s *Synchronizer) SCVars() SCVariablesPtr {
return SCVariablesPtr{
Rollup: s.vars.Rollup.Copy(),
Auction: s.vars.Auction.Copy(),
WDelayer: s.vars.WDelayer.Copy(),
func (s *Synchronizer) SCVars() *common.SCVariables {
return &common.SCVariables{
Rollup: *s.vars.Rollup.Copy(),
Auction: *s.vars.Auction.Copy(),
WDelayer: *s.vars.WDelayer.Copy(),
}
}
@ -727,7 +705,7 @@ func (s *Synchronizer) reorg(uncleBlock *common.Block) (int64, error) {
}
func getInitialVariables(ethClient eth.ClientInterface,
consts *SCConsts) (*SCVariables, *StartBlockNums, error) {
consts *common.SCConsts) (*common.SCVariables, *StartBlockNums, error) {
rollupInit, rollupInitBlock, err := ethClient.RollupEventInit()
if err != nil {
return nil, nil, tracerr.Wrap(fmt.Errorf("RollupEventInit: %w", err))
@ -743,7 +721,7 @@ func getInitialVariables(ethClient eth.ClientInterface,
rollupVars := rollupInit.RollupVariables()
auctionVars := auctionInit.AuctionVariables(consts.Auction.InitialMinimalBidding)
wDelayerVars := wDelayerInit.WDelayerVariables()
return &SCVariables{
return &common.SCVariables{
Rollup: *rollupVars,
Auction: *auctionVars,
WDelayer: *wDelayerVars,

+ 12
- 12
synchronizer/synchronizer_test.go

@ -378,9 +378,9 @@ func TestSyncGeneral(t *testing.T) {
assert.Equal(t, int64(1), stats.Eth.LastBlock.Num)
assert.Equal(t, int64(1), stats.Sync.LastBlock.Num)
vars := s.SCVars()
assert.Equal(t, clientSetup.RollupVariables, vars.Rollup)
assert.Equal(t, clientSetup.AuctionVariables, vars.Auction)
assert.Equal(t, clientSetup.WDelayerVariables, vars.WDelayer)
assert.Equal(t, *clientSetup.RollupVariables, vars.Rollup)
assert.Equal(t, *clientSetup.AuctionVariables, vars.Auction)
assert.Equal(t, *clientSetup.WDelayerVariables, vars.WDelayer)
dbBlocks, err := s.historyDB.GetAllBlocks()
require.NoError(t, err)
@ -541,9 +541,9 @@ func TestSyncGeneral(t *testing.T) {
assert.Equal(t, int64(4), stats.Eth.LastBlock.Num)
assert.Equal(t, int64(4), stats.Sync.LastBlock.Num)
vars = s.SCVars()
assert.Equal(t, clientSetup.RollupVariables, vars.Rollup)
assert.Equal(t, clientSetup.AuctionVariables, vars.Auction)
assert.Equal(t, clientSetup.WDelayerVariables, vars.WDelayer)
assert.Equal(t, *clientSetup.RollupVariables, vars.Rollup)
assert.Equal(t, *clientSetup.AuctionVariables, vars.Auction)
assert.Equal(t, *clientSetup.WDelayerVariables, vars.WDelayer)
dbExits, err := s.historyDB.GetAllExits()
require.NoError(t, err)
@ -673,9 +673,9 @@ func TestSyncGeneral(t *testing.T) {
assert.Equal(t, false, stats.Synced())
assert.Equal(t, int64(6), stats.Eth.LastBlock.Num)
vars = s.SCVars()
assert.Equal(t, clientSetup.RollupVariables, vars.Rollup)
assert.Equal(t, clientSetup.AuctionVariables, vars.Auction)
assert.Equal(t, clientSetup.WDelayerVariables, vars.WDelayer)
assert.Equal(t, *clientSetup.RollupVariables, vars.Rollup)
assert.Equal(t, *clientSetup.AuctionVariables, vars.Auction)
assert.Equal(t, *clientSetup.WDelayerVariables, vars.WDelayer)
// At this point, the DB only has data up to block 1
dbBlock, err := s.historyDB.GetLastBlock()
@ -712,9 +712,9 @@ func TestSyncGeneral(t *testing.T) {
}
vars = s.SCVars()
assert.Equal(t, clientSetup.RollupVariables, vars.Rollup)
assert.Equal(t, clientSetup.AuctionVariables, vars.Auction)
assert.Equal(t, clientSetup.WDelayerVariables, vars.WDelayer)
assert.Equal(t, *clientSetup.RollupVariables, vars.Rollup)
assert.Equal(t, *clientSetup.AuctionVariables, vars.Auction)
assert.Equal(t, *clientSetup.WDelayerVariables, vars.WDelayer)
}
dbBlock, err = s.historyDB.GetLastBlock()

Loading…
Cancel
Save