Browse Source

Add debug info in batch debug file

- Debug Batch File
    - Add details about why a batch is scheduled to be L1Batch or regular
    - Add timestamps in time and blockNumber of important events
    - Use a different naming for the files (batchNum + timestamp)
- Coordinator:
    - To make decisions and store debug info, always use the current block that
      is being mined, instead of the last mined block
feature/sql-semaphore1
Eduard S 3 years ago
parent
commit
97908447b9
5 changed files with 89 additions and 39 deletions
  1. +1
    -1
      cli/node/cfg.buidler.toml
  2. +42
    -7
      coordinator/batch.go
  3. +33
    -19
      coordinator/coordinator.go
  4. +12
    -11
      coordinator/coordinator_test.go
  5. +1
    -1
      test/ethclient.go

+ 1
- 1
cli/node/cfg.buidler.toml

@ -80,7 +80,7 @@ NLevels = 32
ReceiptTimeout = "60s"
ReceiptLoopInterval = "500ms"
CheckLoopInterval = "500ms"
Attempts = 8
Attempts = 4
AttemptsDelay = "200ms"
CallGasLimit = 300000
GasPriceDiv = 100

+ 42
- 7
coordinator/batch.go

@ -6,6 +6,7 @@ import (
"io/ioutil"
"math/big"
"path"
"time"
"github.com/ethereum/go-ethereum/core/types"
"github.com/hermeznetwork/hermez-node/common"
@ -32,6 +33,39 @@ const (
StatusFailed Status = "failed"
)
// Debug information related to the Batch
type Debug struct {
// StartTimestamp of is the time of batch start
StartTimestamp time.Time
// SendTimestamp the time of batch sent to ethereum
SendTimestamp time.Time
// Status of the Batch
Status Status
// StartBlockNum is the blockNum when the Batch was started
StartBlockNum int64
// MineBlockNum is the blockNum in which the batch was mined
MineBlockNum int64
// SendBlockNum is the blockNum when the batch was sent to ethereum
SendBlockNum int64
// LastScheduledL1BatchBlockNum is the blockNum when the last L1Batch
// was scheduled
LastScheduledL1BatchBlockNum int64
// LastL1BatchBlock is the blockNum in which the last L1Batch was
// synced
LastL1BatchBlock int64
// LastL1BatchBlockDelta is the number of blocks after the last L1Batch
LastL1BatchBlockDelta int64
// L1BatchBlockScheduleDeadline is the number of blocks after the last
// L1Batch after which an L1Batch will be scheduled
L1BatchBlockScheduleDeadline int64
// StartToMineBlocksDelay is the number of blocks that happen between
// scheduling a batch and having it mined
StartToMineBlocksDelay int64
// StartToSendDelay is the delay between starting a batch and sending
// it to ethereum, in seconds
StartToSendDelay float64
}
// BatchInfo contans the Batch information
type BatchInfo struct {
BatchNum common.BatchNum
@ -48,23 +82,24 @@ type BatchInfo struct {
CoordIdxs []common.Idx
ForgeBatchArgs *eth.RollupForgeBatchArgs
// FeesInfo
Status Status
EthTx *types.Transaction
Receipt *types.Receipt
Debug Debug
}
// DebugStore is a debug function to store the BatchInfo as a json text file in
// storePath
// storePath. The filename contains the batchNumber followed by a timestamp of
// batch start.
func (b *BatchInfo) DebugStore(storePath string) error {
batchJSON, err := json.MarshalIndent(b, "", " ")
if err != nil {
return tracerr.Wrap(err)
}
oldStateRoot := "null"
if b.ZKInputs != nil && b.ZKInputs.OldStateRoot != nil {
oldStateRoot = b.ZKInputs.OldStateRoot.String()
}
filename := fmt.Sprintf("%010d-%s.json", b.BatchNum, oldStateRoot)
// nolint reason: hardcoded 1_000_000 is the number of nanoseconds in a
// millisecond
//nolint:gomnd
filename := fmt.Sprintf("%08d-%v.%v.json", b.BatchNum,
b.Debug.StartTimestamp.Unix(), b.Debug.StartTimestamp.Nanosecond()/1_000_000)
// nolint reason: 0640 allows rw to owner and r to group
//nolint:gosec
return ioutil.WriteFile(path.Join(storePath, filename), batchJSON, 0640)

+ 33
- 19
coordinator/coordinator.go

@ -203,7 +203,7 @@ func (c *Coordinator) canForge(stats *synchronizer.Stats) bool {
if !stats.Sync.Auction.CurrentSlot.ForgerCommitment &&
c.consts.Auction.RelativeBlock(stats.Eth.LastBlock.Num+1) >= int64(c.vars.Auction.SlotDeadline) {
log.Debugw("Coordinator: anyone can forge in the current slot (slotDeadline passed)",
"block", stats.Eth.LastBlock.Num)
"block", stats.Eth.LastBlock.Num+1)
anyoneForge = true
}
if stats.Sync.Auction.CurrentSlot.Forger == c.cfg.ForgerAddress || anyoneForge {
@ -219,7 +219,7 @@ func (c *Coordinator) syncStats(ctx context.Context, stats *synchronizer.Stats)
if c.pipeline == nil {
if canForge {
log.Infow("Coordinator: forging state begin", "block",
stats.Eth.LastBlock.Num, "batch", stats.Sync.LastBatch)
stats.Eth.LastBlock.Num+1, "batch", stats.Sync.LastBatch)
batchNum := common.BatchNum(stats.Sync.LastBatch)
var err error
if c.pipeline, err = c.newPipeline(ctx); err != nil {
@ -234,7 +234,7 @@ func (c *Coordinator) syncStats(ctx context.Context, stats *synchronizer.Stats)
}
} else {
if !canForge {
log.Infow("Coordinator: forging state end", "block", stats.Eth.LastBlock.Num)
log.Infow("Coordinator: forging state end", "block", stats.Eth.LastBlock.Num+1)
c.pipeline.Stop(c.ctx)
c.pipeline = nil
}
@ -443,6 +443,11 @@ func (t *TxManager) SetLastBlock(lastBlock int64) {
}
func (t *TxManager) rollupForgeBatch(ctx context.Context, batchInfo *BatchInfo) error {
batchInfo.Debug.Status = StatusSent
batchInfo.Debug.SendBlockNum = t.lastBlock + 1
batchInfo.Debug.SendTimestamp = time.Now()
batchInfo.Debug.StartToSendDelay = batchInfo.Debug.SendTimestamp.Sub(
batchInfo.Debug.StartTimestamp).Seconds()
var ethTx *types.Transaction
var err error
for attempt := 0; attempt < t.cfg.EthClientAttempts; attempt++ {
@ -450,11 +455,11 @@ func (t *TxManager) rollupForgeBatch(ctx context.Context, batchInfo *BatchInfo)
if err != nil {
if strings.Contains(err.Error(), common.AuctionErrMsgCannotForge) {
log.Debugw("TxManager ethClient.RollupForgeBatch", "err", err,
"block", t.lastBlock)
"block", t.lastBlock+1)
return tracerr.Wrap(err)
}
log.Errorw("TxManager ethClient.RollupForgeBatch",
"attempt", attempt, "err", err, "block", t.lastBlock,
"attempt", attempt, "err", err, "block", t.lastBlock+1,
"batchNum", batchInfo.BatchNum)
} else {
break
@ -469,7 +474,6 @@ func (t *TxManager) rollupForgeBatch(ctx context.Context, batchInfo *BatchInfo)
return tracerr.Wrap(fmt.Errorf("reached max attempts for ethClient.RollupForgeBatch: %w", err))
}
batchInfo.EthTx = ethTx
batchInfo.Status = StatusSent
log.Infow("TxManager ethClient.RollupForgeBatch", "batch", batchInfo.BatchNum, "tx", ethTx.Hash().Hex())
t.cfg.debugBatchStore(batchInfo)
if err := t.l2DB.DoneForging(common.TxIDsFromL2Txs(batchInfo.L2Txs), batchInfo.BatchNum); err != nil {
@ -511,12 +515,15 @@ func (t *TxManager) handleReceipt(batchInfo *BatchInfo) (*int64, error) {
receipt := batchInfo.Receipt
if receipt != nil {
if receipt.Status == types.ReceiptStatusFailed {
batchInfo.Status = StatusFailed
batchInfo.Debug.Status = StatusFailed
t.cfg.debugBatchStore(batchInfo)
log.Errorw("TxManager receipt status is failed", "receipt", receipt)
return nil, tracerr.Wrap(fmt.Errorf("ethereum transaction receipt statis is failed"))
} else if receipt.Status == types.ReceiptStatusSuccessful {
batchInfo.Status = StatusMined
batchInfo.Debug.Status = StatusMined
batchInfo.Debug.MineBlockNum = receipt.BlockNumber.Int64()
batchInfo.Debug.StartToMineBlocksDelay = batchInfo.Debug.MineBlockNum -
batchInfo.Debug.StartBlockNum
t.cfg.debugBatchStore(batchInfo)
if batchInfo.BatchNum > t.lastConfirmedBatch {
t.lastConfirmedBatch = batchInfo.BatchNum
@ -795,7 +802,9 @@ func (p *Pipeline) Stop(ctx context.Context) {
p.cancel()
p.wg.Wait()
for _, prover := range p.provers {
if err := prover.Cancel(ctx); err != nil {
if err := prover.Cancel(ctx); ctx.Err() != nil {
continue
} else if err != nil {
log.Errorw("prover.Cancel", "err", err)
}
}
@ -827,6 +836,8 @@ func (p *Pipeline) forgeBatch(batchNum common.BatchNum) (*BatchInfo, error) {
}
batchInfo := BatchInfo{BatchNum: batchNum} // to accumulate metadata of the batch
batchInfo.Debug.StartTimestamp = time.Now()
batchInfo.Debug.StartBlockNum = p.stats.Eth.LastBlock.Num + 1
selectionCfg := &txselector.SelectionConfig{
MaxL1UserTxs: common.RollupConstMaxL1UserTx,
@ -840,9 +851,9 @@ func (p *Pipeline) forgeBatch(batchNum common.BatchNum) (*BatchInfo, error) {
var coordIdxs []common.Idx
// 1. Decide if we forge L2Tx or L1+L2Tx
if p.shouldL1L2Batch() {
if p.shouldL1L2Batch(&batchInfo) {
batchInfo.L1Batch = true
p.lastScheduledL1BatchBlockNum = p.stats.Eth.LastBlock.Num
p.lastScheduledL1BatchBlockNum = p.stats.Eth.LastBlock.Num + 1
// 2a: L1+L2 txs
p.lastForgeL1TxsNum++
l1UserTxs, err := p.historyDB.GetUnforgedL1UserTxs(p.lastForgeL1TxsNum)
@ -902,7 +913,7 @@ func (p *Pipeline) forgeBatch(batchNum common.BatchNum) (*BatchInfo, error) {
// 5. Save metadata from BatchBuilder output for BatchNum
batchInfo.ZKInputs = zkInputs
batchInfo.Status = StatusForged
batchInfo.Debug.Status = StatusForged
p.cfg.debugBatchStore(&batchInfo)
return &batchInfo, nil
@ -917,25 +928,28 @@ func (p *Pipeline) waitServerProof(ctx context.Context, batchInfo *BatchInfo) er
batchInfo.Proof = proof
batchInfo.PublicInputs = pubInputs
batchInfo.ForgeBatchArgs = prepareForgeBatchArgs(batchInfo)
batchInfo.Status = StatusProof
batchInfo.Debug.Status = StatusProof
p.cfg.debugBatchStore(batchInfo)
return nil
}
func (p *Pipeline) shouldL1L2Batch() bool {
func (p *Pipeline) shouldL1L2Batch(batchInfo *BatchInfo) bool {
// Take the lastL1BatchBlockNum as the biggest between the last
// scheduled one, and the synchronized one.
lastL1BatchBlockNum := p.lastScheduledL1BatchBlockNum
if p.stats.Sync.LastL1BatchBlock > lastL1BatchBlockNum {
lastL1BatchBlockNum = p.stats.Sync.LastL1BatchBlock
}
// Set Debug information
batchInfo.Debug.LastScheduledL1BatchBlockNum = p.lastScheduledL1BatchBlockNum
batchInfo.Debug.LastL1BatchBlock = p.stats.Sync.LastL1BatchBlock
batchInfo.Debug.LastL1BatchBlockDelta = p.stats.Eth.LastBlock.Num + 1 - lastL1BatchBlockNum
batchInfo.Debug.L1BatchBlockScheduleDeadline =
int64(float64(p.vars.Rollup.ForgeL1L2BatchTimeout-1) * p.cfg.L1BatchTimeoutPerc)
// Return true if we have passed the l1BatchTimeoutPerc portion of the
// range before the l1batch timeout.
if p.stats.Eth.LastBlock.Num-lastL1BatchBlockNum >=
int64(float64(p.vars.Rollup.ForgeL1L2BatchTimeout-1)*p.cfg.L1BatchTimeoutPerc) {
return true
}
return false
return p.stats.Eth.LastBlock.Num+1-lastL1BatchBlockNum >=
int64(float64(p.vars.Rollup.ForgeL1L2BatchTimeout-1)*p.cfg.L1BatchTimeoutPerc)
}
func prepareForgeBatchArgs(batchInfo *BatchInfo) *eth.RollupForgeBatchArgs {

+ 12
- 11
coordinator/coordinator_test.go

@ -265,7 +265,6 @@ func TestCoordinatorFlow(t *testing.T) {
canForge, err := ethClient.AuctionCanForge(forger, blockNum+1)
require.NoError(t, err)
if canForge {
// fmt.Println("DBG canForge")
stats.Sync.Auction.CurrentSlot.Forger = forger
}
// Copy stateDB to synchronizer if there was a new batch
@ -451,11 +450,13 @@ func TestPipelineShouldL1L2Batch(t *testing.T) {
// Check that the parameters are the ones we expect and use in this test
require.Equal(t, 0.5, pipeline.cfg.L1BatchTimeoutPerc)
require.Equal(t, int64(9), ethClientSetup.RollupVariables.ForgeL1L2BatchTimeout)
require.Equal(t, int64(10), ethClientSetup.RollupVariables.ForgeL1L2BatchTimeout)
l1BatchTimeoutPerc := pipeline.cfg.L1BatchTimeoutPerc
l1BatchTimeout := ethClientSetup.RollupVariables.ForgeL1L2BatchTimeout
startBlock := int64(100)
// Empty batchInfo to pass to shouldL1L2Batch() which sets debug information
batchInfo := BatchInfo{}
//
// No scheduled L1Batch
@ -466,21 +467,21 @@ func TestPipelineShouldL1L2Batch(t *testing.T) {
stats.Sync.LastBlock = stats.Eth.LastBlock
stats.Sync.LastL1BatchBlock = 0
pipeline.stats = stats
assert.Equal(t, true, pipeline.shouldL1L2Batch())
assert.Equal(t, true, pipeline.shouldL1L2Batch(&batchInfo))
stats.Sync.LastL1BatchBlock = startBlock
// We are are one block before the timeout range * 0.5
stats.Eth.LastBlock.Num = startBlock + int64(float64(l1BatchTimeout)*l1BatchTimeoutPerc) - 1
stats.Eth.LastBlock.Num = startBlock - 1 + int64(float64(l1BatchTimeout-1)*l1BatchTimeoutPerc) - 1
stats.Sync.LastBlock = stats.Eth.LastBlock
pipeline.stats = stats
assert.Equal(t, false, pipeline.shouldL1L2Batch())
assert.Equal(t, false, pipeline.shouldL1L2Batch(&batchInfo))
// We are are at timeout range * 0.5
stats.Eth.LastBlock.Num = startBlock + int64(float64(l1BatchTimeout)*l1BatchTimeoutPerc)
stats.Eth.LastBlock.Num = startBlock - 1 + int64(float64(l1BatchTimeout-1)*l1BatchTimeoutPerc)
stats.Sync.LastBlock = stats.Eth.LastBlock
pipeline.stats = stats
assert.Equal(t, true, pipeline.shouldL1L2Batch())
assert.Equal(t, true, pipeline.shouldL1L2Batch(&batchInfo))
//
// Scheduled L1Batch
@ -489,16 +490,16 @@ func TestPipelineShouldL1L2Batch(t *testing.T) {
stats.Sync.LastL1BatchBlock = startBlock - 10
// We are are one block before the timeout range * 0.5
stats.Eth.LastBlock.Num = startBlock + int64(float64(l1BatchTimeout)*l1BatchTimeoutPerc) - 1
stats.Eth.LastBlock.Num = startBlock - 1 + int64(float64(l1BatchTimeout-1)*l1BatchTimeoutPerc) - 1
stats.Sync.LastBlock = stats.Eth.LastBlock
pipeline.stats = stats
assert.Equal(t, false, pipeline.shouldL1L2Batch())
assert.Equal(t, false, pipeline.shouldL1L2Batch(&batchInfo))
// We are are at timeout range * 0.5
stats.Eth.LastBlock.Num = startBlock + int64(float64(l1BatchTimeout)*l1BatchTimeoutPerc)
stats.Eth.LastBlock.Num = startBlock - 1 + int64(float64(l1BatchTimeout-1)*l1BatchTimeoutPerc)
stats.Sync.LastBlock = stats.Eth.LastBlock
pipeline.stats = stats
assert.Equal(t, true, pipeline.shouldL1L2Batch())
assert.Equal(t, true, pipeline.shouldL1L2Batch(&batchInfo))
}
// ethAddTokens adds the tokens from the blocks to the blockchain

+ 1
- 1
test/ethclient.go

@ -313,7 +313,7 @@ func NewClientSetupExample() *ClientSetup {
}
rollupVariables := &common.RollupVariables{
FeeAddToken: big.NewInt(11),
ForgeL1L2BatchTimeout: 9,
ForgeL1L2BatchTimeout: 10,
WithdrawalDelay: 80,
Buckets: buckets,
}

Loading…
Cancel
Save