Browse Source

Merge pull request #470 from hermeznetwork/feature/debug1

Add debug info in batch debug file
feature/sql-semaphore1
arnau 3 years ago
committed by GitHub
parent
commit
b0e925fd71
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
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" ReceiptTimeout = "60s"
ReceiptLoopInterval = "500ms" ReceiptLoopInterval = "500ms"
CheckLoopInterval = "500ms" CheckLoopInterval = "500ms"
Attempts = 8
Attempts = 4
AttemptsDelay = "200ms" AttemptsDelay = "200ms"
CallGasLimit = 300000 CallGasLimit = 300000
GasPriceDiv = 100 GasPriceDiv = 100

+ 42
- 7
coordinator/batch.go

@ -6,6 +6,7 @@ import (
"io/ioutil" "io/ioutil"
"math/big" "math/big"
"path" "path"
"time"
"github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/types"
"github.com/hermeznetwork/hermez-node/common" "github.com/hermeznetwork/hermez-node/common"
@ -32,6 +33,39 @@ const (
StatusFailed Status = "failed" 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 // BatchInfo contans the Batch information
type BatchInfo struct { type BatchInfo struct {
BatchNum common.BatchNum BatchNum common.BatchNum
@ -48,23 +82,24 @@ type BatchInfo struct {
CoordIdxs []common.Idx CoordIdxs []common.Idx
ForgeBatchArgs *eth.RollupForgeBatchArgs ForgeBatchArgs *eth.RollupForgeBatchArgs
// FeesInfo // FeesInfo
Status Status
EthTx *types.Transaction EthTx *types.Transaction
Receipt *types.Receipt Receipt *types.Receipt
Debug Debug
} }
// DebugStore is a debug function to store the BatchInfo as a json text file in // 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 { func (b *BatchInfo) DebugStore(storePath string) error {
batchJSON, err := json.MarshalIndent(b, "", " ") batchJSON, err := json.MarshalIndent(b, "", " ")
if err != nil { if err != nil {
return tracerr.Wrap(err) 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 reason: 0640 allows rw to owner and r to group
//nolint:gosec //nolint:gosec
return ioutil.WriteFile(path.Join(storePath, filename), batchJSON, 0640) 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 && if !stats.Sync.Auction.CurrentSlot.ForgerCommitment &&
c.consts.Auction.RelativeBlock(stats.Eth.LastBlock.Num+1) >= int64(c.vars.Auction.SlotDeadline) { 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)", 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 anyoneForge = true
} }
if stats.Sync.Auction.CurrentSlot.Forger == c.cfg.ForgerAddress || anyoneForge { 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 c.pipeline == nil {
if canForge { if canForge {
log.Infow("Coordinator: forging state begin", "block", 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) batchNum := common.BatchNum(stats.Sync.LastBatch)
var err error var err error
if c.pipeline, err = c.newPipeline(ctx); err != nil { if c.pipeline, err = c.newPipeline(ctx); err != nil {
@ -234,7 +234,7 @@ func (c *Coordinator) syncStats(ctx context.Context, stats *synchronizer.Stats)
} }
} else { } else {
if !canForge { 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.Stop(c.ctx)
c.pipeline = nil c.pipeline = nil
} }
@ -443,6 +443,11 @@ func (t *TxManager) SetLastBlock(lastBlock int64) {
} }
func (t *TxManager) rollupForgeBatch(ctx context.Context, batchInfo *BatchInfo) error { 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 ethTx *types.Transaction
var err error var err error
for attempt := 0; attempt < t.cfg.EthClientAttempts; attempt++ { 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 err != nil {
if strings.Contains(err.Error(), common.AuctionErrMsgCannotForge) { if strings.Contains(err.Error(), common.AuctionErrMsgCannotForge) {
log.Debugw("TxManager ethClient.RollupForgeBatch", "err", err, log.Debugw("TxManager ethClient.RollupForgeBatch", "err", err,
"block", t.lastBlock)
"block", t.lastBlock+1)
return tracerr.Wrap(err) return tracerr.Wrap(err)
} }
log.Errorw("TxManager ethClient.RollupForgeBatch", log.Errorw("TxManager ethClient.RollupForgeBatch",
"attempt", attempt, "err", err, "block", t.lastBlock,
"attempt", attempt, "err", err, "block", t.lastBlock+1,
"batchNum", batchInfo.BatchNum) "batchNum", batchInfo.BatchNum)
} else { } else {
break 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)) return tracerr.Wrap(fmt.Errorf("reached max attempts for ethClient.RollupForgeBatch: %w", err))
} }
batchInfo.EthTx = ethTx batchInfo.EthTx = ethTx
batchInfo.Status = StatusSent
log.Infow("TxManager ethClient.RollupForgeBatch", "batch", batchInfo.BatchNum, "tx", ethTx.Hash().Hex()) log.Infow("TxManager ethClient.RollupForgeBatch", "batch", batchInfo.BatchNum, "tx", ethTx.Hash().Hex())
t.cfg.debugBatchStore(batchInfo) t.cfg.debugBatchStore(batchInfo)
if err := t.l2DB.DoneForging(common.TxIDsFromL2Txs(batchInfo.L2Txs), batchInfo.BatchNum); err != nil { 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 receipt := batchInfo.Receipt
if receipt != nil { if receipt != nil {
if receipt.Status == types.ReceiptStatusFailed { if receipt.Status == types.ReceiptStatusFailed {
batchInfo.Status = StatusFailed
batchInfo.Debug.Status = StatusFailed
t.cfg.debugBatchStore(batchInfo) t.cfg.debugBatchStore(batchInfo)
log.Errorw("TxManager receipt status is failed", "receipt", receipt) log.Errorw("TxManager receipt status is failed", "receipt", receipt)
return nil, tracerr.Wrap(fmt.Errorf("ethereum transaction receipt statis is failed")) return nil, tracerr.Wrap(fmt.Errorf("ethereum transaction receipt statis is failed"))
} else if receipt.Status == types.ReceiptStatusSuccessful { } 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) t.cfg.debugBatchStore(batchInfo)
if batchInfo.BatchNum > t.lastConfirmedBatch { if batchInfo.BatchNum > t.lastConfirmedBatch {
t.lastConfirmedBatch = batchInfo.BatchNum t.lastConfirmedBatch = batchInfo.BatchNum
@ -795,7 +802,9 @@ func (p *Pipeline) Stop(ctx context.Context) {
p.cancel() p.cancel()
p.wg.Wait() p.wg.Wait()
for _, prover := range p.provers { 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) 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 := 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{ selectionCfg := &txselector.SelectionConfig{
MaxL1UserTxs: common.RollupConstMaxL1UserTx, MaxL1UserTxs: common.RollupConstMaxL1UserTx,
@ -840,9 +851,9 @@ func (p *Pipeline) forgeBatch(batchNum common.BatchNum) (*BatchInfo, error) {
var coordIdxs []common.Idx var coordIdxs []common.Idx
// 1. Decide if we forge L2Tx or L1+L2Tx // 1. Decide if we forge L2Tx or L1+L2Tx
if p.shouldL1L2Batch() {
if p.shouldL1L2Batch(&batchInfo) {
batchInfo.L1Batch = true batchInfo.L1Batch = true
p.lastScheduledL1BatchBlockNum = p.stats.Eth.LastBlock.Num
p.lastScheduledL1BatchBlockNum = p.stats.Eth.LastBlock.Num + 1
// 2a: L1+L2 txs // 2a: L1+L2 txs
p.lastForgeL1TxsNum++ p.lastForgeL1TxsNum++
l1UserTxs, err := p.historyDB.GetUnforgedL1UserTxs(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 // 5. Save metadata from BatchBuilder output for BatchNum
batchInfo.ZKInputs = zkInputs batchInfo.ZKInputs = zkInputs
batchInfo.Status = StatusForged
batchInfo.Debug.Status = StatusForged
p.cfg.debugBatchStore(&batchInfo) p.cfg.debugBatchStore(&batchInfo)
return &batchInfo, nil return &batchInfo, nil
@ -917,25 +928,28 @@ func (p *Pipeline) waitServerProof(ctx context.Context, batchInfo *BatchInfo) er
batchInfo.Proof = proof batchInfo.Proof = proof
batchInfo.PublicInputs = pubInputs batchInfo.PublicInputs = pubInputs
batchInfo.ForgeBatchArgs = prepareForgeBatchArgs(batchInfo) batchInfo.ForgeBatchArgs = prepareForgeBatchArgs(batchInfo)
batchInfo.Status = StatusProof
batchInfo.Debug.Status = StatusProof
p.cfg.debugBatchStore(batchInfo) p.cfg.debugBatchStore(batchInfo)
return nil return nil
} }
func (p *Pipeline) shouldL1L2Batch() bool {
func (p *Pipeline) shouldL1L2Batch(batchInfo *BatchInfo) bool {
// Take the lastL1BatchBlockNum as the biggest between the last // Take the lastL1BatchBlockNum as the biggest between the last
// scheduled one, and the synchronized one. // scheduled one, and the synchronized one.
lastL1BatchBlockNum := p.lastScheduledL1BatchBlockNum lastL1BatchBlockNum := p.lastScheduledL1BatchBlockNum
if p.stats.Sync.LastL1BatchBlock > lastL1BatchBlockNum { if p.stats.Sync.LastL1BatchBlock > lastL1BatchBlockNum {
lastL1BatchBlockNum = p.stats.Sync.LastL1BatchBlock 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 // Return true if we have passed the l1BatchTimeoutPerc portion of the
// range before the l1batch timeout. // 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 { 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) canForge, err := ethClient.AuctionCanForge(forger, blockNum+1)
require.NoError(t, err) require.NoError(t, err)
if canForge { if canForge {
// fmt.Println("DBG canForge")
stats.Sync.Auction.CurrentSlot.Forger = forger stats.Sync.Auction.CurrentSlot.Forger = forger
} }
// Copy stateDB to synchronizer if there was a new batch // 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 // 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, 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 l1BatchTimeoutPerc := pipeline.cfg.L1BatchTimeoutPerc
l1BatchTimeout := ethClientSetup.RollupVariables.ForgeL1L2BatchTimeout l1BatchTimeout := ethClientSetup.RollupVariables.ForgeL1L2BatchTimeout
startBlock := int64(100) startBlock := int64(100)
// Empty batchInfo to pass to shouldL1L2Batch() which sets debug information
batchInfo := BatchInfo{}
// //
// No scheduled L1Batch // No scheduled L1Batch
@ -466,21 +467,21 @@ func TestPipelineShouldL1L2Batch(t *testing.T) {
stats.Sync.LastBlock = stats.Eth.LastBlock stats.Sync.LastBlock = stats.Eth.LastBlock
stats.Sync.LastL1BatchBlock = 0 stats.Sync.LastL1BatchBlock = 0
pipeline.stats = stats pipeline.stats = stats
assert.Equal(t, true, pipeline.shouldL1L2Batch())
assert.Equal(t, true, pipeline.shouldL1L2Batch(&batchInfo))
stats.Sync.LastL1BatchBlock = startBlock stats.Sync.LastL1BatchBlock = startBlock
// We are are one block before the timeout range * 0.5 // 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 stats.Sync.LastBlock = stats.Eth.LastBlock
pipeline.stats = stats pipeline.stats = stats
assert.Equal(t, false, pipeline.shouldL1L2Batch())
assert.Equal(t, false, pipeline.shouldL1L2Batch(&batchInfo))
// We are are at timeout range * 0.5 // 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 stats.Sync.LastBlock = stats.Eth.LastBlock
pipeline.stats = stats pipeline.stats = stats
assert.Equal(t, true, pipeline.shouldL1L2Batch())
assert.Equal(t, true, pipeline.shouldL1L2Batch(&batchInfo))
// //
// Scheduled L1Batch // Scheduled L1Batch
@ -489,16 +490,16 @@ func TestPipelineShouldL1L2Batch(t *testing.T) {
stats.Sync.LastL1BatchBlock = startBlock - 10 stats.Sync.LastL1BatchBlock = startBlock - 10
// We are are one block before the timeout range * 0.5 // 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 stats.Sync.LastBlock = stats.Eth.LastBlock
pipeline.stats = stats pipeline.stats = stats
assert.Equal(t, false, pipeline.shouldL1L2Batch())
assert.Equal(t, false, pipeline.shouldL1L2Batch(&batchInfo))
// We are are at timeout range * 0.5 // 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 stats.Sync.LastBlock = stats.Eth.LastBlock
pipeline.stats = stats 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 // 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{ rollupVariables := &common.RollupVariables{
FeeAddToken: big.NewInt(11), FeeAddToken: big.NewInt(11),
ForgeL1L2BatchTimeout: 9,
ForgeL1L2BatchTimeout: 10,
WithdrawalDelay: 80, WithdrawalDelay: 80,
Buckets: buckets, Buckets: buckets,
} }

Loading…
Cancel
Save