diff --git a/.github/workflows/full_test.yml b/.github/workflows/full_test.yml index add285e1b..8bb987af7 100644 --- a/.github/workflows/full_test.yml +++ b/.github/workflows/full_test.yml @@ -59,11 +59,14 @@ jobs: if: github.event_name != 'push' || github.ref_name != 'master' || github.ref_type != 'branch' run: go test -timeout 999s -v ./... - - name: Integration Tests - run: | - if [ -d "tests" ]; then - cd tests - ./run_tests.sh - else - echo "The 'tests' folder does not exist." - fi + - name: Integration Tests - brick + run: cd tests && ./run_tests.sh brick + + - name: Integration Tests - sbp + run: cd tests && ./run_tests.sh sbp + + - name: Integration Tests - dpos + run: cd tests && ./run_tests.sh dpos + + - name: Integration Tests - raft + run: cd tests && ./run_tests.sh raft diff --git a/chain/chainhandle.go b/chain/chainhandle.go index 23567d4fd..70024bffe 100644 --- a/chain/chainhandle.go +++ b/chain/chainhandle.go @@ -8,6 +8,7 @@ package chain import ( "bytes" "container/list" + "context" "encoding/json" "errors" "fmt" @@ -26,12 +27,12 @@ import ( var ( ErrorNoAncestor = errors.New("not found ancestor") - ErrBlockOrphan = errors.New("block is ohphan, so not connected in chain") + ErrBlockOrphan = errors.New("block is orphan, so not connected in chain") ErrBlockCachedErrLRU = errors.New("block is in errored blocks cache") ErrStateNoMarker = errors.New("statedb marker of block is not exists") errBlockStale = errors.New("produced block becomes stale") - errBlockInvalidFork = errors.New("invalid fork occured") + errBlockInvalidFork = errors.New("invalid fork occurred") errBlockTimestamp = errors.New("invalid timestamp") InAddBlock = make(chan struct{}, 1) @@ -384,7 +385,7 @@ func (cp *chainProcessor) reorganize() error { return nil } -func (cs *ChainService) addBlockInternal(newBlock *types.Block, usedBstate *state.BlockState, peerID types.PeerID) (err error, cache bool) { +func (cs *ChainService) addBlockInternal(newBlock *types.Block, usedBState *state.BlockState, peerID types.PeerID) (err error, cache bool) { if !cs.VerifyTimestamp(newBlock) { return &ErrBlock{ err: errBlockTimestamp, @@ -405,12 +406,12 @@ func (cs *ChainService) addBlockInternal(newBlock *types.Block, usedBstate *stat } // The newly produced block becomes stale because the more block(s) are - // connected to the blockchain so that the best block is cha/nged. In this + // connected to the blockchain so that the best block is changed. In this // case, newBlock is rejected because it is unlikely that newBlock belongs - // to the main branch. Warning: the condition 'usedBstate != nil' is used + // to the main branch. Warning: the condition 'usedBState != nil' is used // to check whether newBlock is produced by the current node itself. Later, // more explicit condition may be needed instead of this. - if usedBstate != nil && newBlock.PrevID() != bestBlock.ID() { + if usedBState != nil && newBlock.PrevID() != bestBlock.ID() { return &ErrBlock{ err: errBlockStale, block: &types.BlockInfo{ @@ -425,7 +426,7 @@ func (cs *ChainService) addBlockInternal(newBlock *types.Block, usedBstate *stat if cs.IsForkEnable() { return nil } - if usedBstate != nil { + if usedBState != nil { return nil } @@ -457,7 +458,7 @@ func (cs *ChainService) addBlockInternal(newBlock *types.Block, usedBstate *stat // handle orphan if cs.isOrphan(newBlock) { - if usedBstate != nil { + if usedBState != nil { return fmt.Errorf("block received from BP can not be orphan"), false } err := cs.handleOrphan(newBlock, bestBlock, peerID) @@ -476,7 +477,7 @@ func (cs *ChainService) addBlockInternal(newBlock *types.Block, usedBstate *stat <-InAddBlock }() - cp, err := newChainProcessor(newBlock, usedBstate, cs) + cp, err := newChainProcessor(newBlock, usedBState, cs) if err != nil { return err, true } @@ -496,7 +497,7 @@ func (cs *ChainService) addBlockInternal(newBlock *types.Block, usedBstate *stat return nil, true } -func (cs *ChainService) addBlock(newBlock *types.Block, usedBstate *state.BlockState, peerID types.PeerID) error { +func (cs *ChainService) addBlock(newBlock *types.Block, usedBState *state.BlockState, peerID types.PeerID) error { hashID := types.ToHashID(newBlock.BlockHash()) if cs.errBlocks.Contains(hashID) { @@ -511,7 +512,7 @@ func (cs *ChainService) addBlock(newBlock *types.Block, usedBstate *state.BlockS } var needCache bool - err, needCache = cs.addBlockInternal(newBlock, usedBstate, peerID) + err, needCache = cs.addBlockInternal(newBlock, usedBState, peerID) if err != nil { if needCache { evicted := cs.errBlocks.Add(hashID, newBlock) @@ -562,7 +563,7 @@ type blockExecutor struct { execTx TxExecFn txs []*types.Tx validatePost ValidatePostFn - coinbaseAcccount []byte + coinbaseAccount []byte commitOnly bool verifyOnly bool validateSignWait ValidateSignWaitFn @@ -577,9 +578,9 @@ func newBlockExecutor(cs *ChainService, bState *state.BlockState, block *types.B commitOnly := false // The DPoS block factory executes transactions during block generation. In - // such a case it send block with block state so that bState != nil. On the + // such a case it sends block with block state so that bState != nil. On the // contrary, the block propagated from the network is not half-executed. - // Hence we need a new block state and tx executor (execTx). + // Hence, we need a new block state and tx executor (execTx). if bState == nil { if err := cs.validator.ValidateBlock(block); err != nil { return nil, err @@ -590,7 +591,8 @@ func newBlockExecutor(cs *ChainService, bState *state.BlockState, block *types.B state.SetPrevBlockHash(block.GetHeader().GetPrevBlockHash()), ) bi = types.NewBlockHeaderInfo(block) - exec = NewTxExecutor(cs.ChainConsensus, cs.cdb, bi, contract.ChainService) + // FIXME currently the verify only function is allowed long execution time, + exec = NewTxExecutor(context.Background(), cs.ChainConsensus, cs.cdb, bi, contract.ChainService) validateSignWait = func() error { return cs.validator.WaitVerifyDone() @@ -601,15 +603,15 @@ func newBlockExecutor(cs *ChainService, bState *state.BlockState, block *types.B // executed by the block factory. commitOnly = true } - bState.SetGasPrice(system.GetGasPriceFromState(bState)) + bState.SetGasPrice(system.GetGasPrice()) bState.Receipts().SetHardFork(cs.cfg.Hardfork, block.BlockNo()) return &blockExecutor{ - BlockState: bState, - sdb: cs.sdb, - execTx: exec, - txs: block.GetBody().GetTxs(), - coinbaseAcccount: block.GetHeader().GetCoinbaseAccount(), + BlockState: bState, + sdb: cs.sdb, + execTx: exec, + txs: block.GetBody().GetTxs(), + coinbaseAccount: block.GetHeader().GetCoinbaseAccount(), validatePost: func() error { return cs.validator.ValidatePost(bState.GetRoot(), bState.Receipts(), block) }, @@ -621,10 +623,10 @@ func newBlockExecutor(cs *ChainService, bState *state.BlockState, block *types.B } // NewTxExecutor returns a new TxExecFn. -func NewTxExecutor(ccc consensus.ChainConsensusCluster, cdb contract.ChainAccessor, bi *types.BlockHeaderInfo, preloadService int) TxExecFn { +func NewTxExecutor(execCtx context.Context, ccc consensus.ChainConsensusCluster, cdb contract.ChainAccessor, bi *types.BlockHeaderInfo, preloadService int) TxExecFn { return func(bState *state.BlockState, tx types.Transaction) error { if bState == nil { - logger.Error().Msg("bstate is nil in txexec") + logger.Error().Msg("bstate is nil in txExec") return ErrGatherChain } if bi.ForkVersion < 0 { @@ -633,7 +635,7 @@ func NewTxExecutor(ccc consensus.ChainConsensusCluster, cdb contract.ChainAccess } blockSnap := bState.Snapshot() - err := executeTx(ccc, cdb, bState, tx, bi, preloadService) + err := executeTx(execCtx, ccc, cdb, bState, tx, bi, preloadService) if err != nil { logger.Error().Err(err).Str("hash", enc.ToString(tx.GetHash())).Msg("tx failed") if err2 := bState.Rollback(blockSnap); err2 != nil { @@ -674,8 +676,8 @@ func (e *blockExecutor) execute() error { } } - //TODO check result of verifing txs - if err := SendBlockReward(e.BlockState, e.coinbaseAcccount); err != nil { + //TODO check result of verifying txs + if err := SendBlockReward(e.BlockState, e.coinbaseAccount); err != nil { return err } @@ -689,6 +691,7 @@ func (e *blockExecutor) execute() error { } if err := e.validatePost(); err != nil { + // TODO write verbose tx result if debug log is enabled return err } @@ -880,14 +883,7 @@ func resetAccount(account *state.V, fee *big.Int, nonce *uint64) error { return account.PutState() } -func executeTx( - ccc consensus.ChainConsensusCluster, - cdb contract.ChainAccessor, - bs *state.BlockState, - tx types.Transaction, - bi *types.BlockHeaderInfo, - preloadService int, -) error { +func executeTx(execCtx context.Context, ccc consensus.ChainConsensusCluster, cdb contract.ChainAccessor, bs *state.BlockState, tx types.Transaction, bi *types.BlockHeaderInfo, preloadService int) error { var ( txBody = tx.GetBody() isQuirkTx = types.IsQuirkTx(tx.GetHash()) @@ -947,7 +943,7 @@ func executeTx( var events []*types.Event switch txBody.Type { case types.TxType_NORMAL, types.TxType_REDEPLOY, types.TxType_TRANSFER, types.TxType_CALL, types.TxType_DEPLOY: - rv, events, txFee, err = contract.Execute(bs, cdb, tx.GetTx(), sender, receiver, bi, preloadService, false) + rv, events, txFee, err = contract.Execute(execCtx, bs, cdb, tx.GetTx(), sender, receiver, bi, preloadService, false) sender.SubBalance(txFee) case types.TxType_GOVERNANCE: txFee = new(big.Int).SetUint64(0) @@ -979,7 +975,7 @@ func executeTx( } return types.ErrNotAllowedFeeDelegation } - rv, events, txFee, err = contract.Execute(bs, cdb, tx.GetTx(), sender, receiver, bi, preloadService, true) + rv, events, txFee, err = contract.Execute(execCtx, bs, cdb, tx.GetTx(), sender, receiver, bi, preloadService, true) receiver.SubBalance(txFee) } diff --git a/chain/chainhandle_test.go b/chain/chainhandle_test.go index 531d59ded..a71e1c1bc 100644 --- a/chain/chainhandle_test.go +++ b/chain/chainhandle_test.go @@ -13,7 +13,6 @@ import ( "github.com/aergoio/aergo-lib/db" "github.com/aergoio/aergo/v2/account/key" "github.com/aergoio/aergo/v2/contract" - "github.com/aergoio/aergo/v2/contract/system" "github.com/aergoio/aergo/v2/internal/common" "github.com/aergoio/aergo/v2/state" "github.com/aergoio/aergo/v2/types" @@ -37,7 +36,6 @@ func initTest(t *testing.T, testmode bool) { t.Fatalf("failed init : %s", err.Error()) } types.InitGovernance("dpos", true) - system.InitGovernance("dpos") } @@ -78,33 +76,33 @@ func TestErrorInExecuteTx(t *testing.T) { tx := &types.Tx{} - err := executeTx(nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) + err := executeTx(nil, nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) assert.EqualError(t, err, types.ErrTxFormatInvalid.Error(), "execute empty tx") tx.Body = &types.TxBody{} - err = executeTx(nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) + err = executeTx(nil, nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) assert.EqualError(t, err, types.ErrTxInvalidChainIdHash.Error(), "execute empty tx body") tx.Body.ChainIdHash = common.Hasher(chainID) tx.Body.Account = makeTestAddress(t) tx.Body.Recipient = makeTestAddress(t) - err = executeTx(nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) + err = executeTx(nil, nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) assert.EqualError(t, err, types.ErrTxHasInvalidHash.Error(), "execute tx body with account") signTestAddress(t, tx) - err = executeTx(nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) + err = executeTx(nil, nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) assert.EqualError(t, err, types.ErrTxNonceTooLow.Error(), "execute tx body with account") tx.Body.Nonce = 1 tx.Body.Amount = new(big.Int).Add(types.StakingMinimum, types.StakingMinimum).Bytes() signTestAddress(t, tx) - err = executeTx(nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) + err = executeTx(nil, nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) assert.EqualError(t, err, types.ErrInsufficientBalance.Error(), "execute tx body with nonce") tx.Body.Amount = types.MaxAER.Bytes() signTestAddress(t, tx) - err = executeTx(nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) + err = executeTx(nil, nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) assert.EqualError(t, err, types.ErrInsufficientBalance.Error(), "execute tx body with nonce") } @@ -120,13 +118,13 @@ func TestBasicExecuteTx(t *testing.T) { tx.Body.Recipient = makeTestAddress(t) tx.Body.Nonce = 1 signTestAddress(t, tx) - err := executeTx(nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) + err := executeTx(nil, nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) assert.NoError(t, err, "execute amount 0") tx.Body.Nonce = 2 tx.Body.Amount = new(big.Int).SetUint64(1000).Bytes() signTestAddress(t, tx) - err = executeTx(nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) + err = executeTx(nil, nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) assert.NoError(t, err, "execute amount 1000") tx.Body.Nonce = 3 @@ -136,6 +134,6 @@ func TestBasicExecuteTx(t *testing.T) { tx.Body.Type = types.TxType_GOVERNANCE tx.Body.Payload = []byte(`{"Name":"v1stake"}`) signTestAddress(t, tx) - err = executeTx(nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) + err = executeTx(nil, nil, nil, bs, types.NewTransaction(tx), newTestBlockInfo(chainID), contract.ChainService) assert.NoError(t, err, "execute governance type") } diff --git a/chain/chainservice.go b/chain/chainservice.go index f867b9d91..e29eec028 100644 --- a/chain/chainservice.go +++ b/chain/chainservice.go @@ -26,7 +26,6 @@ import ( "github.com/aergoio/aergo/v2/fee" "github.com/aergoio/aergo/v2/internal/enc" "github.com/aergoio/aergo/v2/message" - "github.com/aergoio/aergo/v2/p2p/p2putil" "github.com/aergoio/aergo/v2/pkg/component" "github.com/aergoio/aergo/v2/state" "github.com/aergoio/aergo/v2/types" @@ -292,7 +291,6 @@ func NewChainService(cfg *cfg.Config) *ChainService { // For a strict governance transaction validation. types.InitGovernance(cs.ConsensusType(), cs.IsPublic()) - system.InitGovernance(cs.ConsensusType()) //reset parameter of aergo.system systemState, err := cs.SDB().GetSystemAccountState() @@ -635,7 +633,7 @@ func (cm *ChainManager) Receive(context actor.Context) { block := msg.Block logger.Debug().Str("hash", block.ID()).Str("prev", block.PrevID()).Uint64("bestno", cm.cdb.getBestBlockNo()). - Uint64("no", block.GetHeader().GetBlockNo()).Str("peer", p2putil.ShortForm(msg.PeerID)).Bool("syncer", msg.IsSync).Msg("add block chainservice") + Uint64("no", block.GetHeader().GetBlockNo()).Stringer("peer", types.LogPeerShort(msg.PeerID)).Bool("syncer", msg.IsSync).Msg("add block chainservice") var bstate *state.BlockState if msg.Bstate != nil { diff --git a/consensus/chain/block.go b/consensus/chain/block.go index cbde03dfb..d81aa4649 100644 --- a/consensus/chain/block.go +++ b/consensus/chain/block.go @@ -1,6 +1,7 @@ package chain import ( + "context" "errors" "fmt" "time" @@ -8,7 +9,6 @@ import ( "github.com/aergoio/aergo/v2/chain" "github.com/aergoio/aergo/v2/internal/enc" "github.com/aergoio/aergo/v2/message" - "github.com/aergoio/aergo/v2/p2p/p2putil" "github.com/aergoio/aergo/v2/pkg/component" "github.com/aergoio/aergo/v2/state" "github.com/aergoio/aergo/v2/types" @@ -22,7 +22,7 @@ var ( errBlockSizeLimit = errors.New("the transactions included exceeded the block size limit") ) -// ErrTimeout can be used to indicatefor any kind of timeout. +// ErrTimeout can be used to indicate for any kind of timeout. type ErrTimeout struct { Kind string Timeout int64 @@ -71,6 +71,7 @@ type BlockGenerator struct { rejected *RejTxInfo noTTE bool // disable eviction by timeout if true + ctx context.Context // block generation context hs component.ICompSyncRequester bi *types.BlockHeaderInfo txOp TxOp @@ -79,11 +80,13 @@ type BlockGenerator struct { maxBlockBodySize uint32 } -func NewBlockGenerator(hs component.ICompSyncRequester, bi *types.BlockHeaderInfo, bState *state.BlockState, - txOp TxOp, skipEmpty bool) *BlockGenerator { +func NewBlockGenerator(hs component.ICompSyncRequester, ctx context.Context, bi *types.BlockHeaderInfo, bState *state.BlockState, txOp TxOp, skipEmpty bool) *BlockGenerator { + if ctx == nil { + ctx = context.Background() + } return &BlockGenerator{ - bState: bState, - + bState: bState, + ctx: ctx, hs: hs, bi: bi, txOp: txOp, @@ -207,7 +210,7 @@ func ConnectBlock(hs component.ICompSyncRequester, block *types.Block, blockStat } func SyncChain(hs *component.ComponentHub, targetHash []byte, targetNo types.BlockNo, peerID types.PeerID) error { - logger.Info().Str("peer", p2putil.ShortForm(peerID)).Uint64("no", targetNo). + logger.Info().Stringer("peer", types.LogPeerShort(peerID)).Uint64("no", targetNo). Str("hash", enc.ToString(targetHash)).Msg("request to sync for consensus") notiC := make(chan error) @@ -225,7 +228,7 @@ func SyncChain(hs *component.ComponentHub, targetHash []byte, targetNo types.Blo } } - logger.Info().Str("peer", p2putil.ShortForm(peerID)).Msg("succeeded to sync for consensus") + logger.Info().Stringer("peer", types.LogPeerShort(peerID)).Msg("succeeded to sync for consensus") // TODO check best block is equal to target Hash/no return nil } diff --git a/consensus/chain/tx.go b/consensus/chain/tx.go index ebdfd3501..9d8412c68 100644 --- a/consensus/chain/tx.go +++ b/consensus/chain/tx.go @@ -6,13 +6,13 @@ package chain import ( + "context" "errors" "time" "github.com/aergoio/aergo-lib/log" "github.com/aergoio/aergo/v2/chain" "github.com/aergoio/aergo/v2/contract" - "github.com/aergoio/aergo/v2/internal/enc" "github.com/aergoio/aergo/v2/message" "github.com/aergoio/aergo/v2/pkg/component" "github.com/aergoio/aergo/v2/state" @@ -136,8 +136,28 @@ func (g *BlockGenerator) GatherTXs() ([]types.Transaction, error) { contract.CloseDatabase() }() + // block generation timeout check. this function works like BlockFactory#checkBpTimeout() + checkBGTimeout := NewCompTxOp( + TxOpFn(func(bState *state.BlockState, txIn types.Transaction) error { + select { + case <-g.ctx.Done(): + // TODO use function Cause() for precise control, later. cause can be used in go1.20 and later + causeErr := g.ctx.Err() + //causeErr := context.Cause(g.ctx) + switch causeErr { + case context.Canceled: // Only quitting of Aergo triggers Canceled error for now. + return ErrQuit + default: + return ErrTimeout{Kind: "block"} + } + default: + return nil + } + }), + ) + if nCand > 0 { - op := NewCompTxOp(g.txOp) + op := NewCompTxOp(checkBGTimeout, g.txOp) var preloadTx *types.Tx for i, tx := range txIn { @@ -153,15 +173,11 @@ func (g *BlockGenerator) GatherTXs() ([]types.Transaction, error) { //don't include tx that error is occurred if e, ok := err.(ErrTimeout); ok { - if logger.IsDebugEnabled() { - logger.Debug().Msg("stop gathering tx due to time limit") - } + logger.Debug().Msg("finishing gathering tx due to time limit") err = e break } else if cause, ok := err.(*contract.VmTimeoutError); ok { - if logger.IsDebugEnabled() { - logger.Debug().Msg("stop gathering tx due to time limit") - } + logger.Debug().Msg("stop gathering tx and cancel last tx due to time limit") // Mark the rejected TX by timeout. The marked TX will be // forced to be the first TX of the next block. By doing this, // the TX may have a chance to use the maximum block execution @@ -181,9 +197,8 @@ func (g *BlockGenerator) GatherTXs() ([]types.Transaction, error) { } break } else if err != nil { - if logger.IsDebugEnabled() { - logger.Debug().Err(err).Int("idx", i).Str("hash", enc.ToString(tx.GetHash())).Msg("skip error tx") - } + logger.Debug().Err(err).Int("idx", i).Stringer("hash", types.LogBase58(tx.GetHash())).Msg("skip error tx") + //FIXME handling system error (panic?) // ex) gas error/nonce error skip, but other system error panic continue diff --git a/consensus/consensus.go b/consensus/consensus.go index 656c446cc..8d77a4b12 100644 --- a/consensus/consensus.go +++ b/consensus/consensus.go @@ -69,6 +69,8 @@ type Consensus interface { ChainConsensus ConsensusAccessor Ticker() *time.Ticker + // QueueJob queues block generation job. + // It waits until next block generation time is reached in raft consensus and sbp. QueueJob(now time.Time, jq chan<- interface{}) BlockFactory() BlockFactory QuitChan() chan interface{} diff --git a/consensus/impl/dpos/blockfactory.go b/consensus/impl/dpos/blockfactory.go index 10dfb9d41..f8ee9ca7b 100644 --- a/consensus/impl/dpos/blockfactory.go +++ b/consensus/impl/dpos/blockfactory.go @@ -7,6 +7,7 @@ package dpos import ( "bytes" + "context" "fmt" "runtime" "runtime/debug" @@ -35,10 +36,10 @@ type txExec struct { execTx bc.TxExecFn } -func newTxExec(cdb contract.ChainAccessor, bi *types.BlockHeaderInfo) chain.TxOp { +func newTxExec(execCtx context.Context, cdb contract.ChainAccessor, bi *types.BlockHeaderInfo) chain.TxOp { // Block hash not determined yet return &txExec{ - execTx: bc.NewTxExecutor(nil, cdb, bi, contract.BlockFactory), + execTx: bc.NewTxExecutor(execCtx, nil, cdb, bi, contract.BlockFactory), } } @@ -51,7 +52,7 @@ func (te *txExec) Apply(bState *state.BlockState, tx types.Transaction) error { type BlockFactory struct { *component.ComponentHub jobQueue chan interface{} - workerQueue chan *bpInfo + workerQueue chan bfWork bpTimeoutC chan struct{} quit <-chan interface{} maxBlockBodySize uint32 @@ -61,6 +62,9 @@ type BlockFactory struct { sdb *state.ChainStateDB bv types.BlockVersionner + ctx context.Context + ctxCancelFunc context.CancelFunc + recentRejectedTx *chain.RejTxInfo noTTE bool } @@ -76,7 +80,7 @@ func NewBlockFactory( bf := &BlockFactory{ ComponentHub: hub, jobQueue: make(chan interface{}, slotQueueMax), - workerQueue: make(chan *bpInfo), + workerQueue: make(chan bfWork), bpTimeoutC: make(chan struct{}, 1), maxBlockBodySize: chain.MaxBlockBodySize(), quit: quitC, @@ -92,10 +96,21 @@ func NewBlockFactory( return bf.checkBpTimeout() }), ) - contract.SetBPTimeout(bf.bpTimeoutC) + bf.initContext() return bf } +func (bf *BlockFactory) initContext() { + // TODO change context to WithCancelCause later for more precise control + bf.ctx, bf.ctxCancelFunc = context.WithCancel(context.Background()) + go func() { + select { + case <-bf.quit: + bf.ctxCancelFunc() + } + }() +} + func (bf *BlockFactory) setStateDB(sdb *state.ChainStateDB) { bf.sdb = sdb.Clone() } @@ -124,13 +139,14 @@ func (bf *BlockFactory) controller() { return err } - timeLeft := bpi.slot.RemainingTimeMS() - if timeLeft <= 0 { - return chain.ErrTimeout{Kind: "slot", Timeout: timeLeft} + timeLeftMS := bpi.slot.GetBpTimeout() + if timeLeftMS <= 0 { + return chain.ErrTimeout{Kind: "slot", Timeout: timeLeftMS} } + bfContext, _ := context.WithTimeout(bf.ctx, time.Duration(timeLeftMS)*time.Millisecond) select { - case bf.workerQueue <- bpi: + case bf.workerQueue <- bfWork{execCtx: bfContext, bpi: bpi}: default: logger.Error().Msgf( "skip block production for the slot %v (best block: %v) due to a pending job", @@ -139,14 +155,6 @@ func (bf *BlockFactory) controller() { return nil } - notifyBpTimeout := func(bpi *bpInfo) { - timeout := bpi.slot.GetBpTimeout() - time.Sleep(time.Duration(timeout) * time.Millisecond) - // TODO: skip when the triggered block has already been generated! - bf.bpTimeoutC <- struct{}{} - logger.Debug().Int64("timeout", timeout).Msg("block production timeout signaled") - } - for { select { case info := <-bf.jobQueue: @@ -163,8 +171,6 @@ func (bf *BlockFactory) controller() { continue } - notifyBpTimeout(bpi) - case <-bf.quit: return } @@ -182,9 +188,10 @@ func (bf *BlockFactory) worker() { for { select { - case bpi := <-bf.workerQueue: + case bfw := <-bf.workerQueue: retry: - block, blockState, err := bf.generateBlock(bpi, lpbNo) + bpi := bfw.bpi + block, blockState, err := bf.generateBlock(bfw.execCtx, bpi, lpbNo) if err == chain.ErrQuit { return } @@ -217,7 +224,7 @@ func (bf *BlockFactory) worker() { } } -func (bf *BlockFactory) generateBlock(bpi *bpInfo, lpbNo types.BlockNo) (block *types.Block, bs *state.BlockState, err error) { +func (bf *BlockFactory) generateBlock(execCtx context.Context, bpi *bpInfo, lpbNo types.BlockNo) (block *types.Block, bs *state.BlockState, err error) { defer func() { if panicMsg := recover(); panicMsg != nil { block = nil @@ -232,11 +239,11 @@ func (bf *BlockFactory) generateBlock(bpi *bpInfo, lpbNo types.BlockNo) (block * bpi.bestBlock.GetHeader().GetBlocksRootHash(), state.SetPrevBlockHash(bpi.bestBlock.BlockHash()), ) - bs.SetGasPrice(system.GetGasPriceFromState(bs)) + bs.SetGasPrice(system.GetGasPrice()) bs.Receipts().SetHardFork(bf.bv, bi.No) bGen := chain.NewBlockGenerator( - bf, bi, bs, chain.NewCompTxOp(bf.txOp, newTxExec(bpi.ChainDB, bi)), false). + bf, execCtx, bi, bs, newTxExec(execCtx, bpi.ChainDB, bi), false). WithDeco(bf.deco()). SetNoTTE(bf.noTTE) diff --git a/consensus/impl/dpos/blockfactory_test.go b/consensus/impl/dpos/blockfactory_test.go new file mode 100644 index 000000000..c94e7bf56 --- /dev/null +++ b/consensus/impl/dpos/blockfactory_test.go @@ -0,0 +1,54 @@ +package dpos + +import ( + "context" + "github.com/stretchr/testify/assert" + "testing" + "time" +) + +func TestBlockFactory_context(t *testing.T) { + quitC := make(chan interface{}) + factory := &BlockFactory{quit: quitC} + factory.initContext() + + timeoutCtx, _ := context.WithTimeout(factory.ctx, time.Millisecond) + cancelCtx, cancelFunc := context.WithTimeout(factory.ctx, time.Minute) + quitCtx, _ := context.WithTimeout(factory.ctx, time.Minute) + time.Sleep(time.Millisecond * time.Duration(2)) + + // first channel is done by deadline + select { + case <-timeoutCtx.Done(): + err := timeoutCtx.Err() + assert.Equal(t, context.DeadlineExceeded, err, "err by deadline") + //assert.Equal(t, err, context.Cause(timeoutCtx), "cause and err is differ") + default: + assert.Fail(t, "deadline expected, but not") + } + + // second channel is canceled by self cancel func + cancelFunc() + select { + case <-cancelCtx.Done(): + err := cancelCtx.Err() + assert.Equalf(t, context.Canceled, err, "err by cancel") + //assert.Equal(t, err, context.Cause(cancelCtx), "cause and err is differ") + default: + assert.Fail(t, "cancel expected, but not") + } + + // third channel is canceled by parent with cause + close(quitC) + <-factory.ctx.Done() + assert.Equal(t, context.Canceled, factory.ctx.Err(), "factory err ErrQuit expected") + //assert.Equal(t, chain.ErrQuit, context.Cause(factory.ctx), "factory cause ErrQuit expected") + select { + case <-quitCtx.Done(): + err := quitCtx.Err() + assert.Equalf(t, context.Canceled, err, "err by quit") + //assert.Equal(t, chain.ErrQuit, context.Cause(quitCtx), "cause ErrQuit expected") + default: + assert.Fail(t, "errQuit expected, but not") + } +} diff --git a/consensus/impl/dpos/dpos.go b/consensus/impl/dpos/dpos.go index 331b186e5..a9b85c218 100644 --- a/consensus/impl/dpos/dpos.go +++ b/consensus/impl/dpos/dpos.go @@ -6,6 +6,7 @@ package dpos import ( + "context" "encoding/binary" "encoding/json" "fmt" @@ -79,6 +80,11 @@ func (bi *bpInfo) updateBestBlock() *types.Block { return block } +type bfWork struct { + execCtx context.Context + bpi *bpInfo +} + // GetName returns the name of the consensus. func GetName() string { return consensus.ConsensusName[consensus.ConsensusDPOS] diff --git a/consensus/impl/dpos/slot/slot.go b/consensus/impl/dpos/slot/slot.go index b56cc1060..f80719f8d 100644 --- a/consensus/impl/dpos/slot/slot.go +++ b/consensus/impl/dpos/slot/slot.go @@ -107,7 +107,7 @@ func (s *Slot) IsFor(bpIdx bp.Index, bpCount uint16) bool { return s.NextBpIndex(bpCount) == int64(bpIdx) } -// GetBpTimeout returns the time available for block production. +// GetBpTimeout returns the time available for block production in milliseconds. func (s *Slot) GetBpTimeout() int64 { rTime := s.RemainingTimeMS() @@ -136,7 +136,7 @@ func (s *Slot) NextBpIndex(bpCount uint16) int64 { // BpMaxTime returns the max time limit for block production in nsec. func BpMaxTime() time.Duration { - return time.Duration(bpMaxTimeLimitMs) * 1000 + return time.Duration(bpMaxTimeLimitMs) * 1000000 } func msToPrevIndex(ms int64) int64 { diff --git a/consensus/impl/dpos/status.go b/consensus/impl/dpos/status.go index 8fa303a02..b6e8f739c 100644 --- a/consensus/impl/dpos/status.go +++ b/consensus/impl/dpos/status.go @@ -6,6 +6,7 @@ import ( "github.com/aergoio/aergo/v2/consensus" "github.com/aergoio/aergo/v2/consensus/impl/dpos/bp" + "github.com/aergoio/aergo/v2/contract/system" "github.com/aergoio/aergo/v2/state" "github.com/aergoio/aergo/v2/types" ) @@ -83,6 +84,10 @@ func (s *Status) Update(block *types.Block) { } bps, _ = s.bps.AddSnapshot(block.BlockNo()) + + // if a system param was changed, apply its new value + system.CommitParams(true) + } else { // Rollback resulting from a reorganization: The code below assumes // that there is no block-by-block rollback; it assumes that the @@ -109,6 +114,11 @@ func (s *Status) Update(block *types.Block) { } else { logger.Debug().Uint64("from block no", block.BlockNo()).Msg("VPR reloaded") } + + // if a system param was changed, discard its new value + // this is mainly for block revert case + // the params are reloaded from db on block reorganization + system.CommitParams(false) } s.libState.gc(bps) diff --git a/consensus/impl/raftv2/blockfactory.go b/consensus/impl/raftv2/blockfactory.go index dc9afb111..55b0baf67 100644 --- a/consensus/impl/raftv2/blockfactory.go +++ b/consensus/impl/raftv2/blockfactory.go @@ -2,6 +2,7 @@ package raftv2 import ( "bytes" + "context" "encoding/binary" "encoding/json" "errors" @@ -48,10 +49,10 @@ type txExec struct { execTx bc.TxExecFn } -func newTxExec(ccc consensus.ChainConsensusCluster, cdb consensus.ChainDB, bi *types.BlockHeaderInfo) chain.TxOp { +func newTxExec(execCtx context.Context, ccc consensus.ChainConsensusCluster, cdb consensus.ChainDB, bi *types.BlockHeaderInfo) chain.TxOp { // Block hash not determined yet return &txExec{ - execTx: bc.NewTxExecutor(ccc, cdb, bi, contract.BlockFactory), + execTx: bc.NewTxExecutor(execCtx, ccc, cdb, bi, contract.BlockFactory), } } @@ -62,7 +63,8 @@ func (te *txExec) Apply(bState *state.BlockState, tx types.Transaction) error { type Work struct { *types.Block - term uint64 + term uint64 + execCtx context.Context } func (work *Work) GetTimeout() time.Duration { @@ -122,7 +124,7 @@ type BlockFactory struct { workerQueue chan *Work jobQueue chan interface{} - bpTimeoutC chan struct{} + bpTimeoutC chan struct{} // FIXME change to context based logic like dpos factory quit chan interface{} ready leaderReady @@ -139,6 +141,9 @@ type BlockFactory struct { raftServer *raftServer bv types.BlockVersionner + + ctx context.Context + ctxCancelFunc context.CancelFunc } // GetName returns the name of the consensus. @@ -192,12 +197,22 @@ func New(cfg *config.Config, hub *component.ComponentHub, cdb consensus.ChainWAL return bf.checkBpTimeout() }), ) - - contract.SetBPTimeout(bf.bpTimeoutC) + bf.initContext() return bf, nil } +func (bf *BlockFactory) initContext() { + // TODO change context to WithCancelCause later for more precise control + bf.ctx, bf.ctxCancelFunc = context.WithCancel(context.Background()) + go func() { + select { + case <-bf.quit: + bf.ctxCancelFunc() + } + }() +} + func (bf *BlockFactory) newRaftServer(cfg *config.Config) error { if err := bf.InitCluster(cfg); err != nil { return err @@ -222,7 +237,7 @@ func (bf *BlockFactory) Ticker() *time.Ticker { return time.NewTicker(BlockFactoryTickMs) } -// QueueJob send a block triggering information to jq. +// QueueJob send a block triggering information to jq, and hold to wait func (bf *BlockFactory) QueueJob(now time.Time, jq chan<- interface{}) { bf.jobLock.Lock() defer bf.jobLock.Unlock() @@ -264,6 +279,7 @@ func (bf *BlockFactory) QueueJob(now time.Time, jq chan<- interface{}) { logger.Debug().Str("work", work.ToString()).Str("prev", prevToString(prev)).Msg("new work generated") jq <- work + time.Sleep(BlockIntervalMs) } } @@ -361,6 +377,9 @@ func (bf *BlockFactory) controller() { return err } + bfContext, _ := context.WithTimeout(bf.ctx, work.GetTimeout()) + work.execCtx = bfContext + select { case bf.workerQueue <- work: default: @@ -370,13 +389,6 @@ func (bf *BlockFactory) controller() { return nil } - notifyBpTimeout := func(work *Work) { - timeout := work.GetTimeout() - time.Sleep(timeout) - bf.bpTimeoutC <- struct{}{} - logger.Debug().Int64("timeout(ms)", timeout.Nanoseconds()/int64(time.Millisecond)).Msg("block production timeout signaled") - } - for { select { case info := <-bf.jobQueue: @@ -393,8 +405,6 @@ func (bf *BlockFactory) controller() { continue } - notifyBpTimeout(work) - case <-bf.quit: return } @@ -508,15 +518,15 @@ func (bf *BlockFactory) generateBlock(work *Work) (*types.Block, *state.BlockSta } bi := types.NewBlockHeaderInfoFromPrevBlock(bestBlock, time.Now().UnixNano(), bf.bv) - txOp := chain.NewCompTxOp(bf.txOp, newTxExec(bf, bf.ChainWAL, bi)) + txOp := chain.NewCompTxOp(bf.txOp, newTxExec(work.execCtx, bf, bf.ChainWAL, bi)) blockState := bf.sdb.NewBlockState( bestBlock.GetHeader().GetBlocksRootHash(), state.SetPrevBlockHash(bestBlock.BlockHash()), ) - blockState.SetGasPrice(system.GetGasPriceFromState(blockState)) + blockState.SetGasPrice(system.GetGasPrice()) blockState.Receipts().SetHardFork(bf.bv, bi.No) - block, err := chain.NewBlockGenerator(bf, bi, blockState, txOp, RaftSkipEmptyBlock).GenerateBlock() + block, err := chain.NewBlockGenerator(bf, work.execCtx, bi, blockState, txOp, RaftSkipEmptyBlock).GenerateBlock() if err == chain.ErrBlockEmpty { //need reset previous work return nil, nil, chain.ErrBlockEmpty diff --git a/consensus/impl/raftv2/cluster.go b/consensus/impl/raftv2/cluster.go index 8c8662aae..2cf27b7cb 100644 --- a/consensus/impl/raftv2/cluster.go +++ b/consensus/impl/raftv2/cluster.go @@ -696,7 +696,7 @@ func MaxUint64(x, y uint64) uint64 { } if state.Get() != types.RUNNING { - logger.Debug().Str("peer", p2putil.ShortForm(peerID)).Msg("peer is not running") + logger.Debug().Stringer("peer", types.LogPeerShort(peerID)).Msg("peer is not running") continue } diff --git a/consensus/impl/raftv2/snapshot.go b/consensus/impl/raftv2/snapshot.go index fd4a95d40..3688a3fbb 100644 --- a/consensus/impl/raftv2/snapshot.go +++ b/consensus/impl/raftv2/snapshot.go @@ -10,7 +10,6 @@ import ( "github.com/aergoio/aergo/v2/consensus" "github.com/aergoio/aergo/v2/consensus/chain" "github.com/aergoio/aergo/v2/p2p/p2pcommon" - "github.com/aergoio/aergo/v2/p2p/p2putil" "github.com/aergoio/aergo/v2/pkg/component" "github.com/aergoio/aergo/v2/types" "github.com/aergoio/etcd/raft/raftpb" @@ -182,12 +181,12 @@ func (chainsnap *ChainSnapshotter) requestSync(snap *consensus.ChainSnapshot) er break } - logger.Debug().Str("peer", p2putil.ShortForm(peerID)).Str("leader", EtcdIDToString(leader)).Msg("peer is not alive") + logger.Debug().Stringer("peer", types.LogPeerShort(peerID)).Str("leader", EtcdIDToString(leader)).Msg("peer is not alive") time.Sleep(DfltTimeWaitPeerLive) } - logger.Debug().Str("peer", p2putil.ShortForm(peerID)).Str("leader", EtcdIDToString(leader)).Msg("target peer to sync") + logger.Debug().Stringer("peer", types.LogPeerShort(peerID)).Str("leader", EtcdIDToString(leader)).Msg("target peer to sync") return peerID, err } diff --git a/consensus/impl/sbp/sbp.go b/consensus/impl/sbp/sbp.go index 48e6b5994..a8ef44cc0 100644 --- a/consensus/impl/sbp/sbp.go +++ b/consensus/impl/sbp/sbp.go @@ -1,6 +1,7 @@ package sbp import ( + "context" "runtime" "time" @@ -34,7 +35,8 @@ type txExec struct { func newTxExec(cdb consensus.ChainDB, bi *types.BlockHeaderInfo) chain.TxOp { // Block hash not determined yet return &txExec{ - execTx: bc.NewTxExecutor(nil, contract.ChainAccessor(cdb), bi, contract.BlockFactory), + // FIXME block creation timeout check will not work in SBP unless the context is changed to context.WithTimeout() + execTx: bc.NewTxExecutor(context.Background(), nil, contract.ChainAccessor(cdb), bi, contract.BlockFactory), } } @@ -43,7 +45,7 @@ func (te *txExec) Apply(bState *state.BlockState, tx types.Transaction) error { return err } -// SimpleBlockFactory implments a simple block factory which generate block each cfg.Consensus.BlockInterval. +// SimpleBlockFactory implements a simple block factory which generate block each cfg.Consensus.BlockInterval. // // This can be used for testing purpose. type SimpleBlockFactory struct { @@ -116,6 +118,7 @@ func (s *SimpleBlockFactory) QueueJob(now time.Time, jq chan<- interface{}) { } s.prevBlock = b jq <- b + time.Sleep(s.blockInterval) } } @@ -187,11 +190,11 @@ func (s *SimpleBlockFactory) Start() { prevBlock.GetHeader().GetBlocksRootHash(), state.SetPrevBlockHash(prevBlock.BlockHash()), ) - blockState.SetGasPrice(system.GetGasPriceFromState(blockState)) + blockState.SetGasPrice(system.GetGasPrice()) blockState.Receipts().SetHardFork(s.bv, bi.No) txOp := chain.NewCompTxOp(s.txOp, newTxExec(s.ChainDB, bi)) - block, err := chain.NewBlockGenerator(s, bi, blockState, txOp, false).GenerateBlock() + block, err := chain.NewBlockGenerator(s, context.Background(), bi, blockState, txOp, false).GenerateBlock() if err == chain.ErrQuit { return } else if err != nil { diff --git a/contract/contract.go b/contract/contract.go index f77b0bd5f..8c2b24df0 100644 --- a/contract/contract.go +++ b/contract/contract.go @@ -3,6 +3,7 @@ package contract import "C" import ( "bytes" + "context" "fmt" "math/big" "regexp" @@ -64,15 +65,8 @@ func SetPreloadTx(tx *types.Tx, service int) { preloaders[service].requestedTx = tx } -func Execute( - bs *state.BlockState, - cdb ChainAccessor, - tx *types.Tx, - sender, receiver *state.V, - bi *types.BlockHeaderInfo, - preloadService int, - isFeeDelegation bool, -) (rv string, events []*types.Event, usedFee *big.Int, err error) { +// Execute executes a normal transaction which is possibly executing smart contract. +func Execute(execCtx context.Context, bs *state.BlockState, cdb ChainAccessor, tx *types.Tx, sender, receiver *state.V, bi *types.BlockHeaderInfo, preloadService int, isFeeDelegation bool) (rv string, events []*types.Event, usedFee *big.Int, err error) { txBody := tx.GetBody() @@ -187,9 +181,7 @@ func Execute( rv, events, ctrFee, err = PreCall(ex, bs, sender, contractState, receiver.RP(), gasLimit) } else { // create a new context - ctx := NewVmContext(bs, cdb, sender, receiver, contractState, sender.ID(), - tx.GetHash(), bi, "", true, false, receiver.RP(), - preloadService, txBody.GetAmountBigInt(), gasLimit, isFeeDelegation) + ctx := NewVmContext(execCtx, bs, cdb, sender, receiver, contractState, sender.ID(), tx.GetHash(), bi, "", true, false, receiver.RP(), preloadService, txBody.GetAmountBigInt(), gasLimit, isFeeDelegation) // execute the transaction if receiver.IsDeploy() { @@ -323,10 +315,8 @@ func preloadWorker() { } // create a new context - ctx := NewVmContext(bs, nil, nil, receiver, contractState, txBody.GetAccount(), - tx.GetHash(), request.bi, "", false, false, receiver.RP(), - request.preloadService, txBody.GetAmountBigInt(), txBody.GetGasLimit(), - txBody.Type == types.TxType_FEEDELEGATION) + // FIXME need valid context + ctx := NewVmContext(context.Background(), bs, nil, nil, receiver, contractState, txBody.GetAccount(), tx.GetHash(), request.bi, "", false, false, receiver.RP(), request.preloadService, txBody.GetAmountBigInt(), txBody.GetGasLimit(), txBody.Type == types.TxType_FEEDELEGATION) // load a new executor ex, err := PreloadExecutor(bs, contractState, txBody.Payload, receiver.ID(), ctx) @@ -371,10 +361,6 @@ func useGas(version int32) bool { return version >= 2 && PubNet } -func SetBPTimeout(timeout <-chan struct{}) { - bpTimeout = timeout -} - func GasUsed(txFee, gasPrice *big.Int, txType types.TxType, version int32) uint64 { if fee.IsZeroFee() || txType == types.TxType_GOVERNANCE || version < 2 { return 0 diff --git a/contract/errors.go b/contract/errors.go index 3695fe103..9a9641484 100644 --- a/contract/errors.go +++ b/contract/errors.go @@ -53,7 +53,7 @@ func (e *VmSystemError) System() bool { type VmTimeoutError struct{} func (e *VmTimeoutError) Error() string { - return "contract timeout" + return "contract timeout during vm execution" } func (e *VmTimeoutError) System() bool { diff --git a/contract/name/execute.go b/contract/name/execute.go index 68739e853..6dc00e739 100644 --- a/contract/name/execute.go +++ b/contract/name/execute.go @@ -14,23 +14,18 @@ import ( func ExecuteNameTx(bs *state.BlockState, scs *state.ContractState, txBody *types.TxBody, sender, receiver *state.V, blockInfo *types.BlockHeaderInfo) ([]*types.Event, error) { - systemContractState, err := bs.StateDB.GetSystemAccountState() - - ci, err := ValidateNameTx(txBody, sender, scs, systemContractState) + ci, err := ValidateNameTx(txBody, sender, scs) if err != nil { return nil, err } - var events []*types.Event - var nameState *state.V owner := getOwner(scs, []byte(types.AergoName), false) if owner != nil { if bytes.Equal(sender.ID(), owner) { nameState = sender } else { - nameState, err = bs.GetAccountStateV(owner) - if err != nil { + if nameState, err = bs.GetAccountStateV(owner); err != nil { return nil, err } } @@ -38,17 +33,18 @@ func ExecuteNameTx(bs *state.BlockState, scs *state.ContractState, txBody *types nameState = receiver } + var events []*types.Event switch ci.Name { case types.NameCreate: - if err = CreateName(scs, txBody, sender, nameState, - ci.Args[0].(string)); err != nil { + nameArg := ci.Args[0].(string) + if err = CreateName(scs, txBody, sender, nameState, nameArg); err != nil { return nil, err } jsonArgs := "" if blockInfo.ForkVersion < 2 { - jsonArgs = `{"name":"` + ci.Args[0].(string) + `"}` + jsonArgs = `{"name":"` + nameArg + `"}` } else { - jsonArgs = `["` + ci.Args[0].(string) + `"]` + jsonArgs = `["` + nameArg + `"]` } events = append(events, &types.Event{ ContractAddress: receiver.ID(), @@ -57,16 +53,16 @@ func ExecuteNameTx(bs *state.BlockState, scs *state.ContractState, txBody *types JsonArgs: jsonArgs, }) case types.NameUpdate: - if err = UpdateName(bs, scs, txBody, sender, nameState, - ci.Args[0].(string), ci.Args[1].(string)); err != nil { + nameArg := ci.Args[0].(string) + toArg := ci.Args[1].(string) + if err = UpdateName(bs, scs, txBody, sender, nameState, nameArg, toArg); err != nil { return nil, err } jsonArgs := "" if blockInfo.ForkVersion < 2 { - jsonArgs = `{"name":"` + ci.Args[0].(string) + - `","to":"` + ci.Args[1].(string) + `"}` + jsonArgs = `{"name":"` + nameArg + `","to":"` + toArg + `"}` } else { - jsonArgs = `["` + ci.Args[0].(string) + `","` + ci.Args[1].(string) + `"]` + jsonArgs = `["` + nameArg + `","` + toArg + `"]` } events = append(events, &types.Event{ ContractAddress: receiver.ID(), @@ -75,7 +71,8 @@ func ExecuteNameTx(bs *state.BlockState, scs *state.ContractState, txBody *types JsonArgs: jsonArgs, }) case types.SetContractOwner: - ownerState, err := SetContractOwner(bs, scs, ci.Args[0].(string), nameState) + ownerArg := ci.Args[0].(string) + ownerState, err := SetContractOwner(bs, scs, ownerArg, nameState) if err != nil { return nil, err } @@ -87,9 +84,7 @@ func ExecuteNameTx(bs *state.BlockState, scs *state.ContractState, txBody *types return events, nil } -func ValidateNameTx(tx *types.TxBody, sender *state.V, - scs, systemcs *state.ContractState) (*types.CallInfo, error) { - +func ValidateNameTx(tx *types.TxBody, sender *state.V, scs *state.ContractState) (*types.CallInfo, error) { if sender != nil && sender.Balance().Cmp(tx.GetAmountBigInt()) < 0 { return nil, types.ErrInsufficientBalance } @@ -99,30 +94,25 @@ func ValidateNameTx(tx *types.TxBody, sender *state.V, return nil, err } - name := ci.Args[0].(string) - + nameArg := ci.Args[0].(string) switch ci.Name { case types.NameCreate: - namePrice := system.GetNamePriceFromState(systemcs) - if namePrice.Cmp(tx.GetAmountBigInt()) > 0 { + if system.GetNamePrice().Cmp(tx.GetAmountBigInt()) > 0 { return nil, types.ErrTooSmallAmount } - owner := getOwner(scs, []byte(name), false) - if owner != nil { - return nil, fmt.Errorf("aleady occupied %s", string(name)) + if owner := getOwner(scs, []byte(nameArg), false); owner != nil { + return nil, fmt.Errorf("aleady occupied %s", string(nameArg)) } case types.NameUpdate: - namePrice := system.GetNamePriceFromState(systemcs) - if namePrice.Cmp(tx.GetAmountBigInt()) > 0 { + if system.GetNamePrice().Cmp(tx.GetAmountBigInt()) > 0 { return nil, types.ErrTooSmallAmount } - if (!bytes.Equal(tx.Account, []byte(name))) && - (!bytes.Equal(tx.Account, getOwner(scs, []byte(name), false))) { - return nil, fmt.Errorf("owner not matched : %s", name) + if (!bytes.Equal(tx.Account, []byte(nameArg))) && + (!bytes.Equal(tx.Account, getOwner(scs, []byte(nameArg), false))) { + return nil, fmt.Errorf("owner not matched : %s", nameArg) } case types.SetContractOwner: - owner := getOwner(scs, []byte(types.AergoName), false) - if owner != nil { + if owner := getOwner(scs, []byte(types.AergoName), false); owner != nil { return nil, fmt.Errorf("owner aleady set to %s", types.EncodeAddress(owner)) } default: @@ -135,8 +125,6 @@ func ValidateNameTx(tx *types.TxBody, sender *state.V, func SetContractOwner(bs *state.BlockState, scs *state.ContractState, address string, nameState *state.V) (*state.V, error) { - name := []byte(types.AergoName) - rawaddr, err := types.DecodeAddress(address) if err != nil { return nil, err @@ -150,6 +138,7 @@ func SetContractOwner(bs *state.BlockState, scs *state.ContractState, ownerState.AddBalance(nameState.Balance()) nameState.SubBalance(nameState.Balance()) + name := []byte(types.AergoName) if err = registerOwner(scs, name, rawaddr, name); err != nil { return nil, err } diff --git a/contract/name/name.go b/contract/name/name.go index 39e7efebb..9fc8335cf 100644 --- a/contract/name/name.go +++ b/contract/name/name.go @@ -37,12 +37,13 @@ func createName(scs *state.ContractState, name []byte, owner []byte) error { // UpdateName is avaliable after bid implement func UpdateName(bs *state.BlockState, scs *state.ContractState, tx *types.TxBody, sender, receiver *state.V, name, to string) error { - amount := tx.GetAmountBigInt() if len(getAddress(scs, []byte(name))) <= types.NameLength { return fmt.Errorf("%s is not created yet", string(name)) } destination, _ := types.DecodeAddress(to) destination = GetAddress(scs, destination) + + amount := tx.GetAmountBigInt() sender.SubBalance(amount) receiver.AddBalance(amount) contract, err := bs.StateDB.OpenContractStateAccount(types.ToAccountID(destination)) @@ -88,7 +89,7 @@ func Resolve(bs *state.BlockState, name []byte, legacy bool) ([]byte, error) { } func openContract(bs *state.BlockState) (*state.ContractState, error) { - v, err := bs.GetAccountStateV([]byte("aergo.name")) + v, err := bs.GetAccountStateV([]byte(types.AergoName)) if err != nil { return nil, err } @@ -101,8 +102,7 @@ func openContract(bs *state.BlockState) (*state.ContractState, error) { // GetAddress is resolve name for mempool func GetAddress(scs *state.ContractState, name []byte) []byte { - if len(name) == types.AddressLength || - types.IsSpecialAccount(name) { + if len(name) == types.AddressLength || types.IsSpecialAccount(name) { return name } return getAddress(scs, name) @@ -110,8 +110,7 @@ func GetAddress(scs *state.ContractState, name []byte) []byte { // GetAddressLegacy is resolve name for mempool by buggy logic, leaved for backward compatibility func GetAddressLegacy(scs *state.ContractState, name []byte) []byte { - if len(name) == types.AddressLength || - strings.Contains(string(name), ".") { + if len(name) == types.AddressLength || strings.Contains(string(name), ".") { return name } return getAddress(scs, name) diff --git a/contract/name/name_test.go b/contract/name/name_test.go index 2748dcdeb..4c6a5d274 100644 --- a/contract/name/name_test.go +++ b/contract/name/name_test.go @@ -45,13 +45,12 @@ func TestName(t *testing.T) { receiver, _ := sdb.GetStateDB().GetAccountStateV(tx.Recipient) bs := sdb.NewBlockState(sdb.GetRoot()) scs := openContractState(t, bs) - systemcs := openSystemContractState(t, bs) err := CreateName(scs, tx, sender, receiver, name) assert.NoError(t, err, "create name") scs = nextBlockContractState(t, bs, scs) - _, err = ValidateNameTx(tx, sender, scs, systemcs) + _, err = ValidateNameTx(tx, sender, scs) assert.Error(t, err, "same name") ret := getAddress(scs, []byte(name)) diff --git a/contract/system/execute_test.go b/contract/system/execute_test.go index 4d3fe2907..6f1f8acff 100644 --- a/contract/system/execute_test.go +++ b/contract/system/execute_test.go @@ -720,6 +720,8 @@ func TestProposalExecute2(t *testing.T) { blockInfo.No++ blockInfo.ForkVersion = config.AllEnabledHardforkConfig.Version(blockInfo.No) + // BP Count + votingTx := &types.Tx{ Body: &types.TxBody{ Account: sender.ID(), @@ -747,6 +749,8 @@ func TestProposalExecute2(t *testing.T) { internalVoteResult, err := loadVoteResult(scs, GenProposalKey(bpCount.ID())) assert.Equal(t, new(big.Int).Mul(balance2, big.NewInt(3)), internalVoteResult.GetTotal(), "check result total") + // Staking Min + votingTx = &types.Tx{ Body: &types.TxBody{ Account: sender.ID(), @@ -765,6 +769,10 @@ func TestProposalExecute2(t *testing.T) { _, err = ExecuteSystemTx(scs, votingTx.GetBody(), sender3, receiver, blockInfo) assert.NoError(t, err, "could not execute system tx") + // Gas Price + + origGasPrice := GetGasPrice() + votingTx = &types.Tx{ Body: &types.TxBody{ Account: sender.ID(), @@ -782,8 +790,15 @@ func TestProposalExecute2(t *testing.T) { votingTx.Body.Payload = []byte(`{"Name":"v1voteDAO", "Args":["GASPRICE", "1004"]}`) _, err = ExecuteSystemTx(scs, votingTx.GetBody(), sender3, receiver, blockInfo) assert.NoError(t, err, "could not execute system tx") - gasPrice := GetGasPrice() - assert.Equal(t, balance0_5, gasPrice, "result of gas price voting") + + // check the value for the current block + assert.Equal(t, origGasPrice, GetGasPrice(), "result of gas price voting") + // check the value for the next block + assert.Equal(t, balance0_5, GetNextBlockParam("GASPRICE"), "result of gas price voting") + // commit the new value + CommitParams(true) + // check the value for the current block + assert.Equal(t, balance0_5, GetGasPrice(), "result of gas price voting") blockInfo.No += StakingDelay unstakingTx := &types.Tx{ @@ -815,6 +830,8 @@ func TestProposalExecute2(t *testing.T) { _, err = ExecuteSystemTx(scs, unstakingTx.GetBody(), sender, receiver, blockInfo) assert.NoError(t, err, "could not execute system tx") + oldNamePrice := GetNamePrice() + votingTx.Body.Account = sender2.ID() votingTx.Body.Payload = []byte(`{"Name":"v1voteDAO", "Args":["NAMEPRICE", "1004"]}`) _, err = ExecuteSystemTx(scs, votingTx.GetBody(), sender2, receiver, blockInfo) @@ -830,8 +847,15 @@ func TestProposalExecute2(t *testing.T) { internalVoteResult, err = loadVoteResult(scs, GenProposalKey(namePrice.ID())) assert.Equal(t, new(big.Int).Mul(balance2, big.NewInt(2)), internalVoteResult.GetTotal(), "check result total") assert.Equal(t, "1004", string(voteResult.Votes[0].Candidate), "1st place") - currentNamePrice := GetNamePrice() - assert.Equal(t, "1004", currentNamePrice.String(), "current name price") + + // check the value for the current block + assert.Equal(t, oldNamePrice, GetNamePrice(), "check name price") + // check the value for the next block + assert.Equal(t, big.NewInt(1004), GetNextBlockParam("NAMEPRICE"), "check name price") + // commit the new value + CommitParams(true) + // check the value for the current block + assert.Equal(t, big.NewInt(1004), GetNamePrice(), "check name price") /* blockInfo += StakingDelay diff --git a/contract/system/param.go b/contract/system/param.go index a4fe2ba1a..b9375640b 100644 --- a/contract/system/param.go +++ b/contract/system/param.go @@ -3,12 +3,56 @@ package system import ( "math/big" "strings" + "sync" "github.com/aergoio/aergo/v2/state" "github.com/aergoio/aergo/v2/types" ) -type parameters map[string]*big.Int +type parameters struct { + mutex sync.RWMutex + params map[string]*big.Int +} + +func (p *parameters) setParam(proposalID string, value *big.Int) { + p.mutex.Lock() + defer p.mutex.Unlock() + + p.params[proposalID] = value +} + +// save the new value for the param, to be active on the next block +func (p *parameters) setNextBlockParam(proposalID string, value *big.Int) { + p.mutex.Lock() + defer p.mutex.Unlock() + + p.params[nextBlockParamKey(proposalID)] = value +} + +func (p *parameters) delNextBlockParam(proposalID string) { + p.mutex.Lock() + defer p.mutex.Unlock() + + delete(p.params, nextBlockParamKey(proposalID)) +} + +func (p *parameters) getNextBlockParam(proposalID string) *big.Int { + p.mutex.Lock() + defer p.mutex.Unlock() + + return p.params[nextBlockParamKey(proposalID)] +} + +func (p *parameters) getParam(proposalID string) *big.Int { + p.mutex.Lock() + defer p.mutex.Unlock() + + return p.params[proposalID] +} + +func nextBlockParamKey(id string) string { + return id + "next" +} const ( RESET = -1 @@ -26,7 +70,10 @@ const ( ) var ( - systemParams parameters + systemParams *parameters = ¶meters{ + mutex: sync.RWMutex{}, + params: map[string]*big.Int{}, + } //DefaultParams is for aergo v1 compatibility DefaultParams = map[string]*big.Int{ @@ -36,16 +83,30 @@ var ( } ) +func genParamKey(id string) []byte { + return []byte("param\\" + strings.ToUpper(id)) +} + +// This is also called on chain reorganization func InitSystemParams(g dataGetter, bpCount int) { + // discard any new params computed for the next block + CommitParams(false) + // (re)load param values from database initDefaultBpCount(bpCount) - systemParams = loadParam(g) + systemParams = loadParams(g) } -func genParamKey(id string) []byte { - return []byte("param\\" + strings.ToUpper(id)) +// This function must be called before all the aergosvr +// services start. +func initDefaultBpCount(count int) { + // Ensure that it is not modified after it is initialized. + if DefaultParams[bpCount.ID()] == nil { + DefaultParams[bpCount.ID()] = big.NewInt(int64(count)) + } } -func loadParam(g dataGetter) parameters { +// load the params from the database or use the default values +func loadParams(g dataGetter) *parameters { ret := map[string]*big.Int{} for i := sysParamIndex(0); i < sysParamMax; i++ { id := i.ID() @@ -53,35 +114,68 @@ func loadParam(g dataGetter) parameters { if err != nil { panic("could not load blockchain parameter") } - if data == nil { + if data != nil { + ret[id] = new(big.Int).SetBytes(data) + } else { ret[id] = DefaultParams[id] - continue } - ret[id] = new(big.Int).SetBytes(data) } - return ret + return ¶meters{ + mutex: sync.RWMutex{}, + params: ret, + } } -func (p parameters) getLastParam(proposalID string) *big.Int { - if val, ok := p[proposalID]; ok { - return val +func updateParam(s dataSetter, id string, value *big.Int) error { + // save the param to the database (in a db txn, commit when the block is connected) + if err := s.SetData(genParamKey(id), value.Bytes()); err != nil { + return err } - return DefaultParams[proposalID] + // save the new value for the param, only active on the next block + systemParams.setNextBlockParam(id, value) + return nil } -func (p parameters) setLastParam(proposalID string, value *big.Int) *big.Int { - p[proposalID] = value - return value +// if a system param was changed, apply or discard its new value +func CommitParams(apply bool) { + for i := sysParamIndex(0); i < sysParamMax; i++ { + id := i.ID() + // check if the param has a new value + if param := systemParams.getNextBlockParam(id); param != nil { + if apply { + // set the new value for the current block + systemParams.setParam(id, param) + } + // delete the new value + systemParams.delNextBlockParam(id) + } + } } -func updateParam(s dataSetter, id string, value *big.Int) (*big.Int, error) { - if err := s.SetData(genParamKey(id), value.Bytes()); err != nil { - return nil, err +// get the param value for the next block +func GetNextBlockParam(proposalID string) *big.Int { + // check the value for the next block + if val := systemParams.getNextBlockParam(proposalID); val != nil { + return val } - ret := systemParams.setLastParam(id, value) - return ret, nil + // check the value for the current block + if val := systemParams.getParam(proposalID); val != nil { + return val + } + // default value + return DefaultParams[proposalID] +} + +// get the param value for the current block +func GetParam(proposalID string) *big.Int { + if val := systemParams.getParam(proposalID); val != nil { + return val + } + return DefaultParams[proposalID] } +// these 4 functions are reading the param value for the current block + func GetStakingMinimum() *big.Int { return GetParam(stakingMin.ID()) } @@ -94,6 +188,12 @@ func GetNamePrice() *big.Int { return GetParam(namePrice.ID()) } +func GetBpCount() int { + return int(GetParam(bpCount.ID()).Uint64()) +} + +// these functions are reading the param value directly from the state + func GetNamePriceFromState(scs *state.ContractState) *big.Int { return getParamFromState(scs, namePrice) } diff --git a/contract/system/param_test.go b/contract/system/param_test.go index e9812c92e..f23c601ab 100644 --- a/contract/system/param_test.go +++ b/contract/system/param_test.go @@ -8,17 +8,17 @@ import ( func TestValidateDefaultParams(t *testing.T) { // Staking minimum amount ( 10,000 aergo ) - stakingMin, ok := DefaultParams[stakingMin.ID()] - assert.Truef(t, ok, "stakingMin is not valid. check contract/system/param.go") + stakingMin := DefaultParams[stakingMin.ID()] + assert.NotNilf(t, stakingMin, "stakingMin is not valid. check contract/system/param.go") assert.Equalf(t, "10000000000000000000000", stakingMin.String(), "StakingMinimum is not valid. check contract/system/param.go") // gas price ( 50 gaer ) - gasPrice, ok := DefaultParams[gasPrice.ID()] - assert.Truef(t, ok, "gasPrice is not valid. check contract/system/param.go") + gasPrice := DefaultParams[gasPrice.ID()] + assert.NotNilf(t, gasPrice, "gasPrice is not valid. check contract/system/param.go") assert.Equalf(t, "50000000000", gasPrice.String(), "GasPrice is not valid. check contract/system/param.go") // Proposal price ( 1 aergo ) namePrice := DefaultParams[namePrice.ID()] - assert.Truef(t, ok, "namePrice is not valid. check contract/system/param.go") + assert.NotNilf(t, namePrice, "namePrice is not valid. check contract/system/param.go") assert.Equalf(t, "1000000000000000000", namePrice.String(), "ProposalPrice is not valid. check contract/system/param.go") } diff --git a/contract/system/proposal_test.go b/contract/system/proposal_test.go index e462508d9..77365d114 100644 --- a/contract/system/proposal_test.go +++ b/contract/system/proposal_test.go @@ -123,6 +123,14 @@ func TestProposalBPCount(t *testing.T) { _, err = ExecuteSystemTx(scs, validCandiTx.GetBody(), sender2, receiver, blockInfo) assert.NoError(t, err, "valid") + + // check the value for the current block + assert.Equal(t, 3, GetBpCount(), "check bp") + // check the value for the next block + assert.Equal(t, big.NewInt(13), GetNextBlockParam("BPCOUNT"), "check bp") + // commit the new value + CommitParams(true) + // check the value for the current block assert.Equal(t, 13, GetBpCount(), "check bp") } @@ -203,8 +211,20 @@ func TestFailProposals(t *testing.T) { _, err = ExecuteSystemTx(scs, validCandiTx.GetBody(), sender2, receiver, blockInfo) assert.NoError(t, err, "valid") + + // check the value for the current block + assert.Equal(t, 3, GetBpCount(), "check bp") + // check the value for the next block + assert.Equal(t, big.NewInt(13), GetNextBlockParam("BPCOUNT"), "check bp") + // commit the new value + CommitParams(true) + // check the value for the current block assert.Equal(t, 13, GetBpCount(), "check bp") + // gas price + + oldGasPrice := GetGasPrice() + invalidCandiTx.Body.Payload = []byte(`{"Name":"v1voteDAO", "Args":["gasprice", "500000000000000000000000001"]}`) _, err = ExecuteSystemTx(scs, invalidCandiTx.GetBody(), sender, receiver, blockInfo) assert.Error(t, err, "invalid range") @@ -221,5 +241,13 @@ func TestFailProposals(t *testing.T) { validCandiTx.Body.Payload = []byte(`{"Name":"v1voteDAO", "Args":["gasprice", "101"]}`) _, err = ExecuteSystemTx(scs, validCandiTx.GetBody(), sender2, receiver, blockInfo) assert.NoError(t, err, "valid") + + // check the value for the current block + assert.Equal(t, oldGasPrice, GetGasPrice(), "check gas price") + // check the value for the next block + assert.Equal(t, big.NewInt(101), GetNextBlockParam("GASPRICE"), "check gas price") + // commit the new value + CommitParams(true) + // check the value for the current block assert.Equal(t, big.NewInt(101), GetGasPrice(), "check gas price") } diff --git a/contract/system/staking.go b/contract/system/staking.go index fec058c84..7b3956db7 100644 --- a/contract/system/staking.go +++ b/contract/system/staking.go @@ -13,8 +13,6 @@ import ( "github.com/aergoio/aergo/v2/types" ) -var consensusType string - var ( stakingKey = []byte("staking") stakingTotalKey = []byte("stakingtotal") @@ -25,10 +23,6 @@ var ( const StakingDelay = 60 * 60 * 24 //block interval //const StakingDelay = 5 -func InitGovernance(consensus string) { - consensusType = consensus -} - type stakeCmd struct { *SystemContext amount *big.Int @@ -61,23 +55,18 @@ func (c *stakeCmd) run() (*types.Event, error) { } sender.SubBalance(amount) receiver.AddBalance(amount) + + jsonArgs := "" if c.SystemContext.BlockInfo.ForkVersion < 2 { - return &types.Event{ - ContractAddress: receiver.ID(), - EventIdx: 0, - EventName: "stake", - JsonArgs: `{"who":"` + - types.EncodeAddress(sender.ID()) + - `", "amount":"` + amount.String() + `"}`, - }, nil + jsonArgs = `{"who":"` + types.EncodeAddress(sender.ID()) + `", "amount":"` + amount.String() + `"}` + } else { + jsonArgs = `["` + types.EncodeAddress(sender.ID()) + `", {"_bignum":"` + amount.String() + `"}]` } return &types.Event{ ContractAddress: receiver.ID(), EventIdx: 0, EventName: "stake", - JsonArgs: `["` + - types.EncodeAddress(sender.ID()) + - `", {"_bignum":"` + amount.String() + `"}]`, + JsonArgs: jsonArgs, }, nil } @@ -114,23 +103,18 @@ func (c *unstakeCmd) run() (*types.Event, error) { } sender.AddBalance(balanceAdjustment) receiver.SubBalance(balanceAdjustment) + + jsonArgs := "" if c.SystemContext.BlockInfo.ForkVersion < 2 { - return &types.Event{ - ContractAddress: receiver.ID(), - EventIdx: 0, - EventName: "unstake", - JsonArgs: `{"who":"` + - types.EncodeAddress(sender.ID()) + - `", "amount":"` + balanceAdjustment.String() + `"}`, - }, nil + jsonArgs = `{"who":"` + types.EncodeAddress(sender.ID()) + `", "amount":"` + balanceAdjustment.String() + `"}` + } else { + jsonArgs = `["` + types.EncodeAddress(sender.ID()) + `", {"_bignum":"` + balanceAdjustment.String() + `"}]` } return &types.Event{ ContractAddress: receiver.ID(), EventIdx: 0, EventName: "unstake", - JsonArgs: `["` + - types.EncodeAddress(sender.ID()) + - `", {"_bignum":"` + balanceAdjustment.String() + `"}]`, + JsonArgs: jsonArgs, }, nil } diff --git a/contract/system/validation.go b/contract/system/validation.go index cf3fcd37c..33573835c 100644 --- a/contract/system/validation.go +++ b/contract/system/validation.go @@ -130,7 +130,7 @@ func validateForStaking(account []byte, txBody *types.TxBody, scs *state.Contrac return nil, types.ErrLessTimeHasPassed } toBe := new(big.Int).Add(staked.GetAmountBigInt(), txBody.GetAmountBigInt()) - stakingMin := GetStakingMinimumFromState(scs) + stakingMin := GetStakingMinimum() if stakingMin.Cmp(toBe) > 0 { return nil, types.ErrTooSmallAmount } @@ -164,7 +164,7 @@ func validateForUnstaking(account []byte, txBody *types.TxBody, scs *state.Contr return nil, types.ErrLessTimeHasPassed } toBe := new(big.Int).Sub(staked.GetAmountBigInt(), txBody.GetAmountBigInt()) - stakingMin := GetStakingMinimumFromState(scs) + stakingMin := GetStakingMinimum() if toBe.Cmp(big.NewInt(0)) != 0 && stakingMin.Cmp(toBe) > 0 { return nil, types.ErrTooSmallAmount } diff --git a/contract/system/vote.go b/contract/system/vote.go index 5d2ee9187..d859a982e 100644 --- a/contract/system/vote.go +++ b/contract/system/vote.go @@ -27,8 +27,6 @@ const ( var ( votingCatalog []types.VotingIssue - lastBpCount int - voteKey = []byte("vote") totalKey = []byte("total") sortKey = []byte("sort") @@ -94,7 +92,6 @@ func (c *vprCmd) subVote(v *types.Vote) error { votingPowerRank.sub(c.Sender.AccountID(), c.Sender.ID(), v.GetAmountBigInt()) // Hotfix - reproduce vpr calculation for block 138015125 // When block is reverted, votingPowerRank is not reverted and calculated three times. - // TODO : implement commit, revert, reorg for governance variables. if c.BlockInfo.No == 138015125 && c.Sender.AccountID().String() == "36t2u7Q31HmEbkkYZng7DHNm3xepxHKUfgGrAXNA8pMW" { for i := 0; i < 2; i++ { votingPowerRank.sub(c.Sender.AccountID(), c.Sender.ID(), v.GetAmountBigInt()) @@ -107,7 +104,6 @@ func (c *vprCmd) addVote(v *types.Vote) error { votingPowerRank.add(c.Sender.AccountID(), c.Sender.ID(), v.GetAmountBigInt()) // Hotfix - reproduce vpr calculation for block 138015125 // When block is reverted, votingPowerRank is not reverted and calculated three times. - // TODO : implement commit, revert, reorg for governance variables. if c.BlockInfo.No == 138015125 && c.Sender.AccountID().String() == "36t2u7Q31HmEbkkYZng7DHNm3xepxHKUfgGrAXNA8pMW" { for i := 0; i < 2; i++ { votingPowerRank.add(c.Sender.AccountID(), c.Sender.ID(), v.GetAmountBigInt()) @@ -198,23 +194,18 @@ func (c *voteCmd) run() (*types.Event, error) { if err := c.updateVoteResult(); err != nil { return nil, err } + + jsonArgs := "" if c.SystemContext.BlockInfo.ForkVersion < 2 { - return &types.Event{ - ContractAddress: c.Receiver.ID(), - EventIdx: 0, - EventName: c.op.ID(), - JsonArgs: `{"who":"` + - types.EncodeAddress(c.txBody.Account) + - `", "vote":` + string(c.args) + `}`, - }, nil + jsonArgs = `{"who":"` + types.EncodeAddress(c.txBody.Account) + `", "vote":` + string(c.args) + `}` + } else { + jsonArgs = `["` + types.EncodeAddress(c.txBody.Account) + `", ` + string(c.args) + `]` } return &types.Event{ ContractAddress: c.Receiver.ID(), EventIdx: 0, EventName: c.op.ID(), - JsonArgs: `["` + - types.EncodeAddress(c.txBody.Account) + - `", ` + string(c.args) + `]`, + JsonArgs: jsonArgs, }, nil } @@ -359,21 +350,6 @@ func GetVoteResult(ar AccountStateReader, id []byte, n int) (*types.VoteList, er return getVoteResult(scs, id, n) } -// initDefaultBpCount sets lastBpCount to bpCount. -// -// Caution: This function must be called only once before all the aergosvr -// services start. -func initDefaultBpCount(count int) { - // Ensure that it is not modified after it is initialized. - if DefaultParams[bpCount.ID()] == nil { - DefaultParams[bpCount.ID()] = big.NewInt(int64(count)) - } -} - -func GetBpCount() int { - return int(GetParam(bpCount.ID()).Uint64()) -} - // GetRankers returns the IDs of the top n rankers. func GetRankers(ar AccountStateReader) ([]string, error) { n := GetBpCount() @@ -390,10 +366,6 @@ func GetRankers(ar AccountStateReader) ([]string, error) { return bps, nil } -func GetParam(proposalID string) *big.Int { - return systemParams.getLastParam(proposalID) -} - func serializeVoteList(vl *types.VoteList, ex bool) []byte { var data []byte for _, v := range vl.GetVotes() { diff --git a/contract/system/vote_test.go b/contract/system/vote_test.go index 57b0562f4..8eff82f3b 100644 --- a/contract/system/vote_test.go +++ b/contract/system/vote_test.go @@ -35,7 +35,6 @@ func initTest(t *testing.T) (*state.ContractState, *state.V, *state.V) { t.Fatalf("failed init : %s", err.Error()) } // Need to pass the - InitGovernance("dpos") const testSender = "AmPNYHyzyh9zweLwDyuoiUuTVCdrdksxkRWDjVJS76WQLExa2Jr4" scs, err := bs.GetSystemAccountState() diff --git a/contract/system/voteresult.go b/contract/system/voteresult.go index 33d9aded0..53656739d 100644 --- a/contract/system/voteresult.go +++ b/contract/system/voteresult.go @@ -117,7 +117,7 @@ func (vr *VoteResult) Sync() error { if !ok { return fmt.Errorf("abnormal winner is in vote %s", string(vr.key)) } - if _, err := updateParam(vr.scs, string(vr.key), value); err != nil { + if err := updateParam(vr.scs, string(vr.key), value); err != nil { return err } } diff --git a/contract/vm.go b/contract/vm.go index 992496850..5417501ef 100644 --- a/contract/vm.go +++ b/contract/vm.go @@ -21,6 +21,7 @@ package contract import "C" import ( "bytes" + "context" "encoding/hex" "encoding/json" "errors" @@ -52,6 +53,8 @@ const ( maxCallDepth = 64 checkFeeDelegationFn = "check_delegation" constructor = "constructor" + + vmTimeoutErrMsg = "contract timeout during vm execution" ) var ( @@ -83,6 +86,9 @@ type contractInfo struct { amount *big.Int } +// vmContext contains context datas during execution of smart contract. +// It has both call infos which are immutable, and real time statuses +// which are mutable during execution type vmContext struct { curContract *contractInfo bs *state.BlockState @@ -93,7 +99,7 @@ type vmContext struct { node string confirmed bool isQuery bool - nestedView int32 + nestedView int32 // indicates which parent called the contract in view (read-only mode) isFeeDelegation bool service C.int callState map[types.AccountID]*callState @@ -106,6 +112,7 @@ type vmContext struct { traceFile *os.File gasLimit uint64 remainedGas uint64 + execCtx context.Context } type recoveryEntry struct { @@ -171,9 +178,7 @@ func getTraceFile(blkno uint64, tx []byte) *os.File { return f } -func NewVmContext(blockState *state.BlockState, cdb ChainAccessor, sender, reciever *state.V, - contractState *state.ContractState, senderID []byte, txHash []byte, bi *types.BlockHeaderInfo, node string, confirmed bool, - query bool, rp uint64, service int, amount *big.Int, gasLimit uint64, feeDelegation bool) *vmContext { +func NewVmContext(execCtx context.Context, blockState *state.BlockState, cdb ChainAccessor, sender, reciever *state.V, contractState *state.ContractState, senderID, txHash []byte, bi *types.BlockHeaderInfo, node string, confirmed, query bool, rp uint64, service int, amount *big.Int, gasLimit uint64, feeDelegation bool) *vmContext { cs := &callState{ctrState: contractState, curState: reciever.State()} @@ -191,6 +196,7 @@ func NewVmContext(blockState *state.BlockState, cdb ChainAccessor, sender, recie gasLimit: gasLimit, remainedGas: gasLimit, isFeeDelegation: feeDelegation, + execCtx: execCtx, } ctx.callState = make(map[types.AccountID]*callState) ctx.callState[reciever.AccountID()] = cs @@ -223,6 +229,7 @@ func NewVmContextQuery( confirmed: true, blockInfo: types.NewBlockHeaderInfo(bb), isQuery: true, + execCtx: context.Background(), // FIXME query also should cancel if query is too long } ctx.callState = make(map[types.AccountID]*callState) @@ -235,7 +242,7 @@ func (ctx *vmContext) IsGasSystem() bool { } // get the remaining gas from the given LState -func (ctx *vmContext) getRemainingGas(L *LState) { +func (ctx *vmContext) refreshRemainingGas(L *LState) { if ctx.IsGasSystem() { ctx.remainedGas = uint64(C.lua_gasget(L)) } @@ -376,7 +383,7 @@ func newExecutor( if ctx.IsGasSystem() { ce.setGas() defer func() { - ce.getRemainingGas() + ce.refreshRemainingGas() if ctrLgr.IsDebugEnabled() { ctrLgr.Debug().Uint64("gas used", ce.ctx.usedGas()).Str("lua vm", "loaded").Msg("gas information") } @@ -569,7 +576,7 @@ func (ce *executor) call(instLimit C.int, target *LState) C.int { if ce.err != nil { return 0 } - defer ce.getRemainingGas() + defer ce.refreshRemainingGas() if ce.isView == true { ce.ctx.nestedView++ defer func() { @@ -600,27 +607,28 @@ func (ce *executor) call(instLimit C.int, target *LState) C.int { } ce.processArgs() if ce.err != nil { - ctrLgr.Debug().Err(ce.err).Str("contract", - types.EncodeAddress(ce.ctx.curContract.contractId)).Msg("invalid argument") + ctrLgr.Debug().Err(ce.err).Stringer("contract", + types.LogAddr(ce.ctx.curContract.contractId)).Msg("invalid argument") return 0 } ce.setCountHook(instLimit) - nret := C.int(0) - if cErrMsg := C.vm_pcall(ce.L, ce.numArgs, &nret); cErrMsg != nil { + nRet := C.int(0) + cErrMsg := C.vm_pcall(ce.L, ce.numArgs, &nRet) + if cErrMsg != nil { errMsg := C.GoString(cErrMsg) if C.luaL_hassyserror(ce.L) != C.int(0) { ce.err = newVmSystemError(errors.New(errMsg)) } else { - if C.luaL_hasuncatchablerror(ce.L) != C.int(0) && - C.ERR_BF_TIMEOUT == errMsg { + isUncatchable := C.luaL_hasuncatchablerror(ce.L) != C.int(0) + if isUncatchable && (errMsg == C.ERR_BF_TIMEOUT || errMsg == vmTimeoutErrMsg) { ce.err = &VmTimeoutError{} } else { ce.err = errors.New(errMsg) } } - ctrLgr.Debug().Err(ce.err).Str( + ctrLgr.Debug().Err(ce.err).Stringer( "contract", - types.EncodeAddress(ce.ctx.curContract.contractId), + types.LogAddr(ce.ctx.curContract.contractId), ).Msg("contract is failed") if target != nil { if C.luaL_hasuncatchablerror(ce.L) != C.int(0) { @@ -634,19 +642,19 @@ func (ce *executor) call(instLimit C.int, target *LState) C.int { } if target == nil { var errRet C.int - retMsg := C.GoString(C.vm_get_json_ret(ce.L, nret, &errRet)) + retMsg := C.GoString(C.vm_get_json_ret(ce.L, nRet, &errRet)) if errRet == 1 { ce.err = errors.New(retMsg) } else { ce.jsonRet = retMsg } } else { - if cErrMsg := C.vm_copy_result(ce.L, target, nret); cErrMsg != nil { - errMsg := C.GoString(cErrMsg) + if c2ErrMsg := C.vm_copy_result(ce.L, target, nRet); c2ErrMsg != nil { + errMsg := C.GoString(c2ErrMsg) ce.err = errors.New(errMsg) - ctrLgr.Debug().Err(ce.err).Str( + ctrLgr.Debug().Err(ce.err).Stringer( "contract", - types.EncodeAddress(ce.ctx.curContract.contractId), + types.LogAddr(ce.ctx.curContract.contractId), ).Msg("failed to move results") } } @@ -663,7 +671,7 @@ func (ce *executor) call(instLimit C.int, target *LState) C.int { _, _ = ce.ctx.traceFile.WriteString(fmt.Sprintf("contract %s used fee: %s\n", address, ce.ctx.usedFee().String())) } - return nret + return nRet } func (ce *executor) commitCalledContract() error { @@ -783,8 +791,8 @@ func (ce *executor) close() { } } -func (ce *executor) getRemainingGas() { - ce.ctx.getRemainingGas(ce.L) +func (ce *executor) refreshRemainingGas() { + ce.ctx.refreshRemainingGas(ce.L) } func (ce *executor) gas() uint64 { @@ -839,8 +847,11 @@ func Call( ce := newExecutor(contract, contractAddress, ctx, &ci, ctx.curContract.amount, false, false, contractState) defer ce.close() + startTime := time.Now() // execute the contract call ce.call(callMaxInstLimit, nil) + vmExecTime := time.Now().Sub(startTime).Microseconds() + vmLogger.Trace().Int64("execµs", vmExecTime).Stringer("txHash", types.LogBase58(ce.ctx.txHash)).Msg("tx execute time in vm") // check if there is an error err = ce.err diff --git a/contract/vm_callback.go b/contract/vm_callback.go index b91e45075..5997b0840 100644 --- a/contract/vm_callback.go +++ b/contract/vm_callback.go @@ -29,6 +29,7 @@ import ( "encoding/hex" "errors" "fmt" + "github.com/aergoio/aergo-lib/log" "index/suffixarray" "math/big" "regexp" @@ -50,6 +51,7 @@ import ( var ( mulAergo, mulGaer, zeroBig *big.Int creatorMetaKey = []byte("Creator") + vmLogger = log.NewLogger("contract.vm") ) const ( @@ -283,7 +285,7 @@ func luaCallContract(L *LState, service C.int, contractId *C.char, fname *C.char } // get the remaining gas from the parent LState - ctx.getRemainingGas(L) + ctx.refreshRemainingGas(L) // create a new executor with the remaining gas on the child LState ce := newExecutor(callee, cid, ctx, &ci, amountBig, false, false, cs.ctrState) defer func() { @@ -341,7 +343,13 @@ func luaCallContract(L *LState, service C.int, contractId *C.char, fname *C.char if ctx.traceFile != nil { _, _ = ctx.traceFile.WriteString(fmt.Sprintf("recovery snapshot: %d\n", seq)) } - return -1, C.CString("[Contract.LuaCallContract] call err: " + ce.err.Error()) + switch ceErr := ce.err.(type) { + case *VmTimeoutError: + return -1, C.CString(ceErr.Error()) + default: + return -1, C.CString("[Contract.LuaCallContract] call err: " + ceErr.Error()) + + } } if seq == 1 { @@ -402,7 +410,7 @@ func luaDelegateCallContract(L *LState, service C.int, contractId *C.char, } // get the remaining gas from the parent LState - ctx.getRemainingGas(L) + ctx.refreshRemainingGas(L) // create a new executor with the remaining gas on the child LState ce := newExecutor(contract, cid, ctx, &ci, zeroBig, false, false, contractState) defer func() { @@ -525,7 +533,7 @@ func luaSendAmount(L *LState, service C.int, contractId *C.char, amount *C.char) } // get the remaining gas from the parent LState - ctx.getRemainingGas(L) + ctx.refreshRemainingGas(L) // create a new executor with the remaining gas on the child LState ce := newExecutor(code, cid, ctx, &ci, amountBig, false, false, cs.ctrState) defer func() { @@ -1233,7 +1241,7 @@ func luaDeployContract( } // get the remaining gas from the parent LState - ctx.getRemainingGas(L) + ctx.refreshRemainingGas(L) // create a new executor with the remaining gas on the child LState ce := newExecutor(runCode, newContract.ID(), ctx, &ci, amountBig, true, false, contractState) defer func() { @@ -1492,6 +1500,8 @@ func luaCheckView(service C.int) C.int { return C.int(ctx.nestedView) } +// luaCheckTimeout checks whether the block creation timeout occurred. +// //export luaCheckTimeout func luaCheckTimeout(service C.int) C.int { @@ -1511,27 +1521,13 @@ func luaCheckTimeout(service C.int) C.int { return 0 } + ctx := contexts[service] select { - case <-bpTimeout: + case <-ctx.execCtx.Done(): return 1 default: return 0 } - - // Temporarily disable timeout check to prevent contract timeout raised from chain service - // if service < BlockFactory { - // service = service + MaxVmService - // } - // if service != BlockFactory { - // return 0 - // } - // select { - // case <-bpTimeout: - // return 1 - // default: - // return 0 - // } - //return 0 } //export luaIsFeeDelegation diff --git a/contract/vm_dummy/test_files/gas_per_function.lua b/contract/vm_dummy/test_files/gas_per_function.lua index 3b27a8192..bd1bf0be4 100644 --- a/contract/vm_dummy/test_files/gas_per_function.lua +++ b/contract/vm_dummy/test_files/gas_per_function.lua @@ -1286,9 +1286,9 @@ function run_test(function_name, ...) end -function deposit() +function default() -- do nothing, only receive native aergo tokens end abi.register(run_test) -abi.payable(deposit) +abi.payable(default) diff --git a/contract/vm_dummy/test_files/type_random.lua b/contract/vm_dummy/test_files/type_random.lua index e7fe68929..47f63d23c 100644 --- a/contract/vm_dummy/test_files/type_random.lua +++ b/contract/vm_dummy/test_files/type_random.lua @@ -1,5 +1,16 @@ function random(...) - return system.random(...) + return system.random(...) end -abi.register(random) +function get_numbers(count) + local list = {} + + for i = 1, count do + local num = system.random(1, 100) + table.insert(list, num) + end + + return list +end + +abi.register(random, get_numbers) diff --git a/contract/vm_dummy/test_files/type_random_caller.lua b/contract/vm_dummy/test_files/type_random_caller.lua new file mode 100644 index 000000000..9e87c9c6d --- /dev/null +++ b/contract/vm_dummy/test_files/type_random_caller.lua @@ -0,0 +1,35 @@ +-- check if numbers generated by a called contract are +-- the same if called on the same transaction +function check_if_equal(address) + + local list1 = contract.call(address, 'get_numbers', 10) + local list2 = contract.call(address, 'get_numbers', 10) + local list3 = contract.call(address, 'get_numbers', 10) + + if lists_are_equal(list1, list2) then + return true + end + if lists_are_equal(list2, list3) then + return true + end + + return false +end + +function lists_are_equal(list1, list2) + -- check if lengths are different + if #list1 ~= #list2 then + return false + end + + -- compare each element + for i = 1, #list1 do + if list1[i] ~= list2[i] then + return false + end + end + + return true +end + +abi.register(check_if_equal) diff --git a/contract/vm_dummy/vm_dummy.go b/contract/vm_dummy/vm_dummy.go index 5de79b039..c1fbce3c6 100644 --- a/contract/vm_dummy/vm_dummy.go +++ b/contract/vm_dummy/vm_dummy.go @@ -2,6 +2,7 @@ package vm_dummy // helper functions import ( + "context" "errors" "fmt" "math" @@ -31,6 +32,9 @@ var ( const ( lStateMaxSize = 10 * 7 + + dummyBlockIntervalSec = 1 + dummyBlockExecTimeMs = (dummyBlockIntervalSec * 1000) >> 2 ) func init() { @@ -98,6 +102,7 @@ func LoadDummyChain(opts ...DummyChainOptions) (*DummyChain, error) { sdb: state.NewChainStateDB(), tmpDir: dataPath, gasPrice: types.NewAmount(1, types.Aer), + timeout: dummyBlockExecTimeMs, } defer func() { if err != nil { @@ -129,7 +134,6 @@ func LoadDummyChain(opts ...DummyChainOptions) (*DummyChain, error) { // To pass the governance tests. types.InitGovernance("dpos", true) - system.InitGovernance("dpos") // To pass dao parameters test scs, err := bc.sdb.GetStateDB().GetSystemAccountState() @@ -240,7 +244,7 @@ func (bc *DummyChain) GetBestBlock() (*types.Block, error) { } type LuaTxTester interface { - run(bs *state.BlockState, bc *DummyChain, bi *types.BlockHeaderInfo, receiptTx db.Transaction) error + run(execCtx context.Context, bs *state.BlockState, bc *DummyChain, bi *types.BlockHeaderInfo, receiptTx db.Transaction) error Hash() []byte okMsg() string } @@ -273,7 +277,7 @@ func (l *luaTxAccount) okMsg() string { return "SUCCESS" } -func (l *luaTxAccount) run(bs *state.BlockState, bc *DummyChain, bi *types.BlockHeaderInfo, receiptTx db.Transaction) error { +func (l *luaTxAccount) run(execCtx context.Context, bs *state.BlockState, bc *DummyChain, bi *types.BlockHeaderInfo, receiptTx db.Transaction) error { id := types.ToAccountID(l.name) accountState, err := bs.GetAccountState(id) if err != nil { @@ -311,7 +315,7 @@ func (l *luaTxSend) okMsg() string { return "SUCCESS" } -func (l *luaTxSend) run(bs *state.BlockState, bc *DummyChain, bi *types.BlockHeaderInfo, receiptTx db.Transaction) error { +func (l *luaTxSend) run(execCtx context.Context, bs *state.BlockState, bc *DummyChain, bi *types.BlockHeaderInfo, receiptTx db.Transaction) error { senderID := types.ToAccountID(l.sender) receiverID := types.ToAccountID(l.receiver) @@ -522,7 +526,7 @@ func contractFrame(l luaTxContract, bs *state.BlockState, cdb contract.ChainAcce } -func (l *luaTxDeploy) run(bs *state.BlockState, bc *DummyChain, bi *types.BlockHeaderInfo, receiptTx db.Transaction) error { +func (l *luaTxDeploy) run(execCtx context.Context, bs *state.BlockState, bc *DummyChain, bi *types.BlockHeaderInfo, receiptTx db.Transaction) error { if l.cErr != nil { return l.cErr } @@ -530,8 +534,7 @@ func (l *luaTxDeploy) run(bs *state.BlockState, bc *DummyChain, bi *types.BlockH func(sender, contractV *state.V, contractId types.AccountID, eContractState *state.ContractState) (string, []*types.Event, *big.Int, error) { contractV.State().SqlRecoveryPoint = 1 - ctx := contract.NewVmContext(bs, nil, sender, contractV, eContractState, sender.ID(), l.Hash(), bi, "", true, - false, contractV.State().SqlRecoveryPoint, contract.BlockFactory, l.amount(), math.MaxUint64, false) + ctx := contract.NewVmContext(execCtx, bs, nil, sender, contractV, eContractState, sender.ID(), l.Hash(), bi, "", true, false, contractV.State().SqlRecoveryPoint, contract.BlockFactory, l.amount(), math.MaxUint64, false) rv, events, ctrFee, err := contract.Create(eContractState, l.payload(), l.recipient(), ctx) if err != nil { @@ -587,11 +590,10 @@ func (l *luaTxCall) Fail(expectedErr string) *luaTxCall { return l } -func (l *luaTxCall) run(bs *state.BlockState, bc *DummyChain, bi *types.BlockHeaderInfo, receiptTx db.Transaction) error { +func (l *luaTxCall) run(execCtx context.Context, bs *state.BlockState, bc *DummyChain, bi *types.BlockHeaderInfo, receiptTx db.Transaction) error { err := contractFrame(l, bs, bc, receiptTx, func(sender, contractV *state.V, contractId types.AccountID, eContractState *state.ContractState) (string, []*types.Event, *big.Int, error) { - ctx := contract.NewVmContext(bs, bc, sender, contractV, eContractState, sender.ID(), l.Hash(), bi, "", true, - false, contractV.State().SqlRecoveryPoint, contract.BlockFactory, l.amount(), math.MaxUint64, l.feeDelegate) + ctx := contract.NewVmContext(execCtx, bs, bc, sender, contractV, eContractState, sender.ID(), l.Hash(), bi, "", true, false, contractV.State().SqlRecoveryPoint, contract.BlockFactory, l.amount(), math.MaxUint64, l.feeDelegate) rv, events, ctrFee, err := contract.Call(eContractState, l.payload(), l.recipient(), ctx) if err != nil { @@ -626,16 +628,11 @@ func (bc *DummyChain) ConnectBlock(txs ...LuaTxTester) error { defer tx.Commit() defer contract.CloseDatabase() - timeout := make(chan struct{}) - go func() { - if bc.timeout != 0 { - <-time.Tick(time.Duration(bc.timeout) * time.Millisecond) - timeout <- struct{}{} - } - }() - contract.SetBPTimeout(timeout) + //timeout := make(chan struct{}) + blockContext, _ := context.WithTimeout(context.Background(), time.Duration(bc.timeout)*time.Millisecond) + //contract.SetBPTimeout(timeout) for _, x := range txs { - if err := x.run(blockState, bc, types.NewBlockHeaderInfo(bc.cBlock), tx); err != nil { + if err := x.run(blockContext, blockState, bc, types.NewBlockHeaderInfo(bc.cBlock), tx); err != nil { return err } } diff --git a/contract/vm_dummy/vm_dummy_pub_test.go b/contract/vm_dummy/vm_dummy_pub_test.go index a1cdd8ac5..1b42ee665 100644 --- a/contract/vm_dummy/vm_dummy_pub_test.go +++ b/contract/vm_dummy/vm_dummy_pub_test.go @@ -82,9 +82,9 @@ func TestGasPerFunction(t *testing.T) { // transfer funds to the contracts err = bc.ConnectBlock( - NewLuaTxCall("user", "contract_v2", uint64(10e18), `{"Name":"deposit"}`), - NewLuaTxCall("user", "contract_v3", uint64(10e18), `{"Name":"deposit"}`), - NewLuaTxCall("user", "contract_v4", uint64(10e18), `{"Name":"deposit"}`), + NewLuaTxCall("user", "contract_v2", uint64(10e18), `{"Name":"default"}`), + NewLuaTxCall("user", "contract_v3", uint64(10e18), `{"Name":"default"}`), + NewLuaTxCall("user", "contract_v4", uint64(10e18), `{"Name":"default"}`), ) assert.NoError(t, err, "sending funds to contracts") diff --git a/contract/vm_dummy/vm_dummy_test.go b/contract/vm_dummy/vm_dummy_test.go index 831953571..b4a9bd4af 100644 --- a/contract/vm_dummy/vm_dummy_test.go +++ b/contract/vm_dummy/vm_dummy_test.go @@ -20,7 +20,7 @@ import ( ) const min_version int32 = 2 -const max_version int32 = 3 +const max_version int32 = 4 func TestMaxCallDepth(t *testing.T) { //code := readLuaCode(t, "maxcalldepth_1.lua") @@ -1030,6 +1030,8 @@ func TestUpdateSize(t *testing.T) { } func TestTimeoutCnt(t *testing.T) { + // FIXME delete skip after gas limit patch + t.Skip("disabled until gas limit check is added") code := readLuaCode(t, "timeout_1.lua") code2 := readLuaCode(t, "timeout_2.lua") @@ -2255,14 +2257,19 @@ func checkRandomIntValue(v string, min, max int) error { } func TestTypeRandom(t *testing.T) { - code := readLuaCode(t, "type_random.lua") + code1 := readLuaCode(t, "type_random.lua") + code2 := readLuaCode(t, "type_random_caller.lua") for version := min_version; version <= max_version; version++ { bc, err := LoadDummyChain(SetHardForkVersion(version)) require.NoErrorf(t, err, "failed to create dummy chain") defer bc.Release() - err = bc.ConnectBlock(NewLuaTxAccount("user1", 1, types.Aergo), NewLuaTxDeploy("user1", "random", 0, code)) + err = bc.ConnectBlock( + NewLuaTxAccount("user1", 1, types.Aergo), + NewLuaTxDeploy("user1", "random", 0, code1), + NewLuaTxDeploy("user1", "caller", 0, code2), + ) require.NoErrorf(t, err, "failed to deploy") err = bc.ConnectBlock(NewLuaTxCall("user1", "random", 0, `{"Name": "random", "Args":[]}`).Fail("1 or 2 arguments required")) @@ -2299,6 +2306,12 @@ func TestTypeRandom(t *testing.T) { err = bc.Query("random", `{"Name": "random", "Args":[3,1]}`, "system.random: the maximum value must be greater than the minimum value", "") require.NoErrorf(t, err, "failed to query") + tx = NewLuaTxCall("user1", "caller", 0, `{"Name": "check_if_equal", "Args":["`+nameToAddress("random")+`"]}`) + err = bc.ConnectBlock(tx) + require.NoErrorf(t, err, "failed to call tx") + receipt = bc.GetReceipt(tx.Hash()) + assert.Equalf(t, `false`, receipt.GetRet(), "random numbers are the same on the same transaction") + } } diff --git a/mempool/mempool.go b/mempool/mempool.go index 64bc9219f..9a4577301 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -676,15 +676,11 @@ func (mp *MemPool) validateTx(tx types.Transaction, account types.Address) error return err } case types.AergoName: - systemcs, err := mp.stateDB.GetSystemAccountState() - if err != nil { - return err - } sender, err := mp.stateDB.GetAccountStateV(account) if err != nil { return err } - if _, err := name.ValidateNameTx(tx.GetBody(), sender, scs, systemcs); err != nil { + if _, err := name.ValidateNameTx(tx.GetBody(), sender, scs); err != nil { return err } case types.AergoEnterprise: diff --git a/p2p/actorwork.go b/p2p/actorwork.go index f55736eee..84343c4bc 100644 --- a/p2p/actorwork.go +++ b/p2p/actorwork.go @@ -28,7 +28,7 @@ const ( func (p2ps *P2P) GetAddresses(peerID types.PeerID, size uint32) bool { remotePeer, ok := p2ps.pm.GetPeer(peerID) if !ok { - p2ps.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("Message addressRequest to Unknown peer, check if a bug") + p2ps.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("Message addressRequest to Unknown peer, check if a bug") return false } @@ -43,7 +43,7 @@ func (p2ps *P2P) GetAddresses(peerID types.PeerID, size uint32) bool { func (p2ps *P2P) GetBlockHeaders(msg *message.GetBlockHeaders) bool { remotePeer, exists := p2ps.pm.GetPeer(msg.ToWhom) if !exists { - p2ps.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(msg.ToWhom)).Msg("Request to invalid peer") + p2ps.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(msg.ToWhom)).Msg("Request to invalid peer") return false } @@ -60,14 +60,14 @@ func (p2ps *P2P) GetBlockHeaders(msg *message.GetBlockHeaders) bool { func (p2ps *P2P) GetBlocks(peerID types.PeerID, blockHashes []message.BlockHash) bool { remotePeer, exists := p2ps.pm.GetPeer(peerID) if !exists { - p2ps.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Str(p2putil.LogProtoID, p2pcommon.GetBlocksRequest.String()).Msg("Message to Unknown peer, check if a bug") + p2ps.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Stringer(p2putil.LogProtoID, p2pcommon.GetBlocksRequest).Msg("Message to Unknown peer, check if a bug") return false } if len(blockHashes) == 0 { - p2ps.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Str(p2putil.LogProtoID, p2pcommon.GetBlocksRequest.String()).Msg("meaningless GetBlocks request with zero hash") + p2ps.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Stringer(p2putil.LogProtoID, p2pcommon.GetBlocksRequest).Msg("meaningless GetBlocks request with zero hash") return false } - p2ps.Debug().Str(p2putil.LogPeerName, remotePeer.Name()).Int(p2putil.LogBlkCount, len(blockHashes)).Str("first_hash", enc.ToString(blockHashes[0])).Msg("Sending Get block request") + p2ps.Debug().Str(p2putil.LogPeerName, remotePeer.Name()).Int(p2putil.LogBlkCount, len(blockHashes)).Stringer("first_hash", types.LogBase58(blockHashes[0])).Msg("Sending Get block request") hashes := make([][]byte, len(blockHashes)) for i, hash := range blockHashes { @@ -86,7 +86,7 @@ func (p2ps *P2P) GetBlocksChunk(context actor.Context, msg *message.GetBlockChun blockHashes := msg.Hashes remotePeer, exists := p2ps.pm.GetPeer(peerID) if !exists { - p2ps.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Str(p2putil.LogProtoID, p2pcommon.GetBlocksRequest.String()).Msg("Message to Unknown peer, check if a bug") + p2ps.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Stringer(p2putil.LogProtoID, p2pcommon.GetBlocksRequest).Msg("Message to Unknown peer, check if a bug") context.Respond(&message.GetBlockChunksRsp{Seq: msg.Seq, ToWhom: peerID, Err: fmt.Errorf("invalid peer")}) return } @@ -100,7 +100,7 @@ func (p2ps *P2P) GetBlockHashes(context actor.Context, msg *message.GetHashes) { remotePeer, exists := p2ps.pm.GetPeer(peerID) if !exists { - p2ps.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Str(p2putil.LogProtoID, p2pcommon.GetHashesRequest.String()).Msg("Invalid peerID") + p2ps.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Stringer(p2putil.LogProtoID, p2pcommon.GetHashesRequest).Msg("Invalid peerID") context.Respond(&message.GetHashesRsp{Seq: msg.Seq, Hashes: nil, PrevInfo: msg.PrevInfo, Count: 0, Err: message.PeerNotFoundError}) return } @@ -114,7 +114,7 @@ func (p2ps *P2P) GetBlockHashByNo(context actor.Context, msg *message.GetHashByN remotePeer, exists := p2ps.pm.GetPeer(peerID) if !exists { - p2ps.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Str(p2putil.LogProtoID, p2pcommon.GetHashByNoRequest.String()).Msg("Invalid peerID") + p2ps.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Stringer(p2putil.LogProtoID, p2pcommon.GetHashByNoRequest).Msg("Invalid peerID") context.Respond(&message.GetHashByNoRsp{Seq: msg.Seq, Err: message.PeerNotFoundError}) return } @@ -145,7 +145,7 @@ func (p2ps *P2P) NotifyNewBlock(blockNotice message.NotifyNewBlock) bool { return true } -// NotifyNewBlock send notice message of new block to a peer +// NotifyBlockProduced send notice message of new block to a peer func (p2ps *P2P) NotifyBlockProduced(blockNotice message.NotifyNewBlock) bool { // TODO fill producerID, but actually there is no way go find producer, for now. req := &types.BlockProducedNotice{ProducerID: nil, BlockNo: blockNotice.BlockNo, Block: blockNotice.Block} @@ -170,7 +170,7 @@ func (p2ps *P2P) NotifyBlockProduced(blockNotice message.NotifyNewBlock) bool { func (p2ps *P2P) GetTXs(peerID types.PeerID, txHashes []message.TXHash) bool { remotePeer, ok := p2ps.pm.GetPeer(peerID) if !ok { - p2ps.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("Invalid peer. check for bug") + p2ps.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("Invalid peer. check for bug") return false } if len(txHashes) == 0 { @@ -222,7 +222,7 @@ func (p2ps *P2P) GetSyncAncestor(context actor.Context, msg *message.GetSyncAnce peerID := msg.ToWhom remotePeer, exists := p2ps.pm.GetPeer(peerID) if !exists { - p2ps.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("invalid peer id") + p2ps.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("invalid peer id") context.Respond(&message.GetSyncAncestorRsp{Seq: msg.Seq, Ancestor: nil}) return } diff --git a/p2p/certmanager.go b/p2p/certmanager.go index fc93908ef..4712b7fb0 100644 --- a/p2p/certmanager.go +++ b/p2p/certmanager.go @@ -81,7 +81,7 @@ type bpCertificateManager struct { func (cm *bpCertificateManager) CreateCertificate(remoteMeta p2pcommon.PeerMeta) (*p2pcommon.AgentCertificateV1, error) { if !types.IsSamePeerID(cm.settings.AgentID, remoteMeta.ID) { // this agent is not in charge of that bp id. - cm.logger.Info().Str("agentID", p2putil.ShortForm(remoteMeta.ID)).Msg("failed to issue certificate, since peer is not registered agent") + cm.logger.Info().Stringer("agentID", types.LogPeerShort(remoteMeta.ID)).Msg("failed to issue certificate, since peer is not registered agent") return nil, p2pcommon.ErrInvalidRole } @@ -161,12 +161,12 @@ func (cm *agentCertificateManager) AddCertificate(cert *p2pcommon.AgentCertifica defer cm.mutex.Unlock() if !p2putil.ContainsID(cm.self.ProducerIDs, cert.BPID) { // this agent is not in charge of that bp id. - cm.logger.Info().Str("bpID", p2putil.ShortForm(cert.BPID)).Msg("drop issued certificate, since issuer is not my managed producer") + cm.logger.Info().Stringer("bpID", types.LogPeerShort(cert.BPID)).Msg("drop issued certificate, since issuer is not my managed producer") return } if !types.IsSamePeerID(cm.self.ID, cert.AgentID) { // this certificate is not my certificate - cm.logger.Info().Str("bpID", p2putil.ShortForm(cert.BPID)).Str("agentID", p2putil.ShortForm(cert.AgentID)).Msg("drop issued certificate, since agent id is not me") + cm.logger.Info().Stringer("bpID", types.LogPeerShort(cert.BPID)).Stringer("agentID", types.LogPeerShort(cert.AgentID)).Msg("drop issued certificate, since agent id is not me") return } diff --git a/p2p/handshakev2.go b/p2p/handshakev2.go index 713dd2f68..59e673619 100644 --- a/p2p/handshakev2.go +++ b/p2p/handshakev2.go @@ -72,7 +72,7 @@ func (h *InboundWireHandshaker) handleInboundPeer(ctx context.Context, rwc io.Re if bestVer == p2pcommon.P2PVersionUnknown { return h.writeErrAndReturn(fmt.Errorf("no matched p2p version for %v", hsReq.Versions), p2pcommon.HSCodeNoMatchedVersion, rwc) } else { - h.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Str("version", bestVer.String()).Msg("Responding best p2p version") + h.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Str("version", bestVer.String()).Msg("Responding best p2p version") resp := p2pcommon.HSHeadResp{Magic: hsReq.Magic, RespCode: bestVer.Uint32()} err = h.writeWireHSResponse(resp, rwc) select { @@ -138,7 +138,7 @@ func (h *OutboundWireHandshaker) handleOutboundPeer(ctx context.Context, rwc io. return nil, fmt.Errorf("remote peer failed: %v", respHeader.RespCode) } bestVersion := p2pcommon.P2PVersion(respHeader.RespCode) - h.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Str("version", bestVersion.String()).Msg("Responded best p2p version") + h.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Str("version", bestVersion.String()).Msg("Responded best p2p version") // continue to handshake with VersionedHandshaker innerHS, err := h.verM.GetVersionedHandshaker(bestVersion, h.peerID, rwc) if err != nil { diff --git a/p2p/metric/metricsman.go b/p2p/metric/metricsman.go index c8c228a3a..b7b1faf13 100644 --- a/p2p/metric/metricsman.go +++ b/p2p/metric/metricsman.go @@ -80,7 +80,7 @@ func (mm *metricsManager) NewMetric(pid types.PeerID, manNum uint32) *PeerMetric mm.mutex.Lock() defer mm.mutex.Unlock() if old, found := mm.metricsMap[pid]; found { - mm.logger.Warn().Str("peer_id", p2putil.ShortForm(pid)).Uint32("oldNum", old.seq).Uint32("newNum", manNum).Msg("metric for to add peer is already exist. replacing new metric") + mm.logger.Warn().Stringer("peer_id", types.LogPeerShort(pid)).Uint32("oldNum", old.seq).Uint32("newNum", manNum).Msg("metric for to add peer is already exist. replacing new metric") } peerMetric := &PeerMetric{mm: mm, PeerID: pid, seq: manNum, InMetric: NewExponentMetric5(mm.interval), OutMetric: NewExponentMetric5(mm.interval), Since: time.Now()} mm.metricsMap[pid] = peerMetric @@ -91,11 +91,11 @@ func (mm *metricsManager) Remove(pid types.PeerID, manNum uint32) *PeerMetric { mm.mutex.Lock() defer mm.mutex.Unlock() if metric, found := mm.metricsMap[pid]; !found { - mm.logger.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(pid)).Msg("metric for to remove peer is not exist.") + mm.logger.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(pid)).Msg("metric for to remove peer is not exist.") return nil } else { if metric.seq != manNum { - mm.logger.Warn().Uint32("exist_num", metric.seq).Uint32("man_num", manNum).Str(p2putil.LogPeerID, p2putil.ShortForm(pid)).Msg("ignore remove. different manage number") + mm.logger.Warn().Uint32("exist_num", metric.seq).Uint32("man_num", manNum).Stringer(p2putil.LogPeerID, types.LogPeerShort(pid)).Msg("ignore remove. different manage number") } atomic.AddInt64(&mm.deadTotalIn, metric.totalIn) atomic.AddInt64(&mm.deadTotalOut, metric.totalOut) diff --git a/p2p/msgorder.go b/p2p/msgorder.go index 77286cabd..5a62a9b84 100644 --- a/p2p/msgorder.go +++ b/p2p/msgorder.go @@ -70,7 +70,7 @@ func (pr *pbRequestOrder) SendTo(pi p2pcommon.RemotePeer) error { p.reqMutex.Unlock() err := p.rw.WriteMsg(pr.message) if err != nil { - p.logger.Warn().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, pr.GetProtocolID().String()).Str(p2putil.LogMsgID, pr.GetMsgID().String()).Err(err).Msg("fail to SendTo") + p.logger.Warn().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, pr.GetProtocolID()).Str(p2putil.LogMsgID, pr.GetMsgID().String()).Err(err).Msg("fail to SendTo") p.reqMutex.Lock() delete(p.requests, pr.message.ID()) p.reqMutex.Unlock() @@ -78,7 +78,7 @@ func (pr *pbRequestOrder) SendTo(pi p2pcommon.RemotePeer) error { } if pr.trace { - p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, pr.GetProtocolID().String()). + p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, pr.GetProtocolID()). Str(p2putil.LogMsgID, pr.GetMsgID().String()).Msg("Send request message") } return nil @@ -92,11 +92,11 @@ func (pr *pbResponseOrder) SendTo(pi p2pcommon.RemotePeer) error { p := pi.(*remotePeerImpl) err := p.rw.WriteMsg(pr.message) if err != nil { - p.logger.Warn().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, pr.GetProtocolID().String()).Str(p2putil.LogMsgID, pr.GetMsgID().String()).Err(err).Msg("fail to SendTo") + p.logger.Warn().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, pr.GetProtocolID()).Str(p2putil.LogMsgID, pr.GetMsgID().String()).Err(err).Msg("fail to SendTo") return err } if pr.trace { - p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, pr.GetProtocolID().String()). + p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, pr.GetProtocolID()). Str(p2putil.LogMsgID, pr.GetMsgID().String()).Str(p2putil.LogOrgReqID, pr.message.OriginalID().String()).Msg("Send response message") } @@ -128,7 +128,7 @@ func (pr *pbBlkNoticeOrder) SendTo(pi p2pcommon.RemotePeer) error { if skipNotice || passedTime < MinNewBlkNoticeInterval { p.skipCnt++ if p.skipCnt&0x03ff == 0 && pr.trace { - p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, pr.GetProtocolID().String()).Int32("skip_cnt", p.skipCnt).Msg("Skipped NewBlockNotice ") + p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, pr.GetProtocolID()).Int32("skip_cnt", p.skipCnt).Msg("Skipped NewBlockNotice ") } return nil @@ -142,12 +142,12 @@ func (pr *pbBlkNoticeOrder) SendTo(pi p2pcommon.RemotePeer) error { } err := p.rw.WriteMsg(pr.message) if err != nil { - p.logger.Warn().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, pr.GetProtocolID().String()).Str(p2putil.LogMsgID, pr.GetMsgID().String()).Err(err).Msg("fail to SendTo") + p.logger.Warn().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, pr.GetProtocolID()).Str(p2putil.LogMsgID, pr.GetMsgID().String()).Err(err).Msg("fail to SendTo") return err } p.lastBlkNoticeTime = time.Now() if p.skipCnt > 100 && pr.trace { - p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, pr.GetProtocolID().String()).Int32("skip_cnt", p.skipCnt).Msg("Send NewBlockNotice after long skip") + p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, pr.GetProtocolID()).Int32("skip_cnt", p.skipCnt).Msg("Send NewBlockNotice after long skip") } p.skipCnt = 0 return nil @@ -164,11 +164,11 @@ func (pr *pbBpNoticeOrder) SendTo(pi p2pcommon.RemotePeer) error { p.blkHashCache.ContainsOrAdd(blkHash, cachePlaceHolder) err := p.rw.WriteMsg(pr.message) if err != nil { - p.logger.Warn().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, pr.GetProtocolID().String()).Str(p2putil.LogMsgID, pr.GetMsgID().String()).Err(err).Msg("fail to SendTo") + p.logger.Warn().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, pr.GetProtocolID()).Str(p2putil.LogMsgID, pr.GetMsgID().String()).Err(err).Msg("fail to SendTo") return err } if pr.trace { - p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, pr.GetProtocolID().String()). + p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, pr.GetProtocolID()). Str(p2putil.LogMsgID, pr.GetMsgID().String()).Str(p2putil.LogBlkHash, enc.ToString(pr.block.Hash)).Msg("Notify block produced") } return nil @@ -184,11 +184,11 @@ func (pr *pbTxNoticeOrder) SendTo(pi p2pcommon.RemotePeer) error { err := p.rw.WriteMsg(pr.message) if err != nil { - p.logger.Warn().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, pr.GetProtocolID().String()).Str(p2putil.LogMsgID, pr.GetMsgID().String()).Err(err).Msg("fail to SendTo") + p.logger.Warn().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, pr.GetProtocolID()).Str(p2putil.LogMsgID, pr.GetMsgID().String()).Err(err).Msg("fail to SendTo") return err } if p.logger.IsDebugEnabled() && pr.trace { - p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, pr.GetProtocolID().String()). + p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, pr.GetProtocolID()). Str(p2putil.LogMsgID, pr.GetMsgID().String()).Int("hash_cnt", len(pr.txHashes)).Array("hashes", types.NewLogTxIDsMarshaller(pr.txHashes, 10)).Msg("Sent tx notice") } return nil @@ -231,12 +231,12 @@ func (pr *pbTossOrder) SendTo(pi p2pcommon.RemotePeer) error { p := pi.(*remotePeerImpl) err := p.rw.WriteMsg(pr.message) if err != nil { - p.logger.Warn().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, pr.GetProtocolID().String()).Str(p2putil.LogMsgID, pr.GetMsgID().String()).Err(err).Msg("fail to toss") + p.logger.Warn().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, pr.GetProtocolID()).Str(p2putil.LogMsgID, pr.GetMsgID().String()).Err(err).Msg("fail to toss") return err } if pr.trace { - p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, pr.GetProtocolID().String()). + p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, pr.GetProtocolID()). Str(p2putil.LogMsgID, pr.GetMsgID().String()).Msg("toss message") } return nil diff --git a/p2p/p2p.go b/p2p/p2p.go index d2d8708b3..901efc617 100644 --- a/p2p/p2p.go +++ b/p2p/p2p.go @@ -521,7 +521,7 @@ func (p2ps *P2P) initLocalSettings(conf *config.P2PConfig) { if err != nil { panic("invalid agentID " + conf.Agent + " : " + err.Error()) } - p2ps.Logger.Info().Str("fullID", pid.String()).Str("agentID", p2putil.ShortForm(pid)).Msg("found agent setting. use peer as agent if connected") + p2ps.Logger.Info().Str("fullID", pid.String()).Stringer("agentID", types.LogPeerShort(pid)).Msg("found agent setting. use peer as agent if connected") p2ps.localSettings.AgentID = pid } else { p2ps.Logger.Debug().Msg("no agent was set. local peer is standalone producer.") diff --git a/p2p/peerfinder.go b/p2p/peerfinder.go index ba3a8f5e6..a35ccfd0c 100644 --- a/p2p/peerfinder.go +++ b/p2p/peerfinder.go @@ -75,7 +75,7 @@ func (dp *dynamicPeerFinder) OnPeerDisconnect(peer p2pcommon.RemotePeer) { } func (dp *dynamicPeerFinder) OnPeerConnect(pid types.PeerID) { - dp.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(pid)).Msg("check and remove peerID in pool") + dp.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(pid)).Msg("check and remove peerID in pool") if stat := dp.qStats[pid]; stat == nil { // first query will be sent quickly dp.qStats[pid] = &queryStat{pid: pid, nextTurn: time.Now().Add(p2pcommon.PeerFirstInterval)} diff --git a/p2p/peermanager.go b/p2p/peermanager.go index 60db0043d..bfe018777 100644 --- a/p2p/peermanager.go +++ b/p2p/peermanager.go @@ -203,7 +203,7 @@ func (pm *peerManager) initDesignatedPeerList() { pm.logger.Warn().Err(err).Str("str", addrStr).Msg("invalid NPAddPeer address") continue } - pm.logger.Info().Str(p2putil.LogFullID, peerMeta.ID.Pretty()).Str(p2putil.LogPeerID, p2putil.ShortForm(peerMeta.ID)).Str("addr", peerMeta.Addresses[0].String()).Msg("Adding Designated peer") + pm.logger.Info().Str(p2putil.LogFullID, peerMeta.ID.Pretty()).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerMeta.ID)).Str("addr", peerMeta.Addresses[0].String()).Msg("Adding Designated peer") pm.designatedPeers[peerMeta.ID] = peerMeta } } @@ -328,14 +328,14 @@ func (pm *peerManager) tryRegister(hsResult connPeerResult) p2pcommon.RemotePeer peerID := meta.ID preExistPeer, ok := pm.remotePeers[peerID] if ok { - pm.logger.Info().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("Peer add collision. Outbound connection of higher hash will survive.") + pm.logger.Info().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("Peer add collision. Outbound connection of higher hash will survive.") iAmLowerOrEqual := p2putil.ComparePeerID(pm.is.SelfNodeID(), meta.ID) <= 0 if iAmLowerOrEqual == remote.Connection.Outbound { - pm.logger.Info().Str("local_peer_id", p2putil.ShortForm(pm.is.SelfNodeID())).Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Bool("outbound", remote.Connection.Outbound).Msg("Close connection and keep earlier handshake connection.") + pm.logger.Info().Stringer("local_peer_id", types.LogPeerShort(pm.is.SelfNodeID())).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Bool("outbound", remote.Connection.Outbound).Msg("Close connection and keep earlier handshake connection.") return nil } else { - pm.logger.Info().Str("local_peer_id", p2putil.ShortForm(pm.is.SelfNodeID())).Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Bool("outbound", remote.Connection.Outbound).Msg("Keep connection and close earlier handshake connection.") - pm.logger.Info().Str("local_peer_id", p2putil.ShortForm(pm.is.SelfNodeID())).Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Bool("outbound", remote.Connection.Outbound).Msg("Keep connection and close earlier handshake connection.") + pm.logger.Info().Stringer("local_peer_id", types.LogPeerShort(pm.is.SelfNodeID())).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Bool("outbound", remote.Connection.Outbound).Msg("Keep connection and close earlier handshake connection.") + pm.logger.Info().Stringer("local_peer_id", types.LogPeerShort(pm.is.SelfNodeID())).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Bool("outbound", remote.Connection.Outbound).Msg("Keep connection and close earlier handshake connection.") // stopping lower valued connection preExistPeer.Stop() } @@ -418,14 +418,14 @@ func (pm *peerManager) removePeer(peer p2pcommon.RemotePeer) bool { return false } if target.ManageNumber() != peer.ManageNumber() { - pm.logger.Debug().Uint32("remove_num", peer.ManageNumber()).Uint32("exist_num", target.ManageNumber()).Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("remove peer is requested but already removed and other instance is on") + pm.logger.Debug().Uint32("remove_num", peer.ManageNumber()).Uint32("exist_num", target.ManageNumber()).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("remove peer is requested but already removed and other instance is on") return false } if target.State() == types.RUNNING { - pm.logger.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("remove peer is requested but peer is still running") + pm.logger.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("remove peer is requested but peer is still running") } pm.deletePeer(peer) - pm.logger.Info().Uint32("manage_num", peer.ManageNumber()).Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("removed peer in peermanager") + pm.logger.Info().Uint32("manage_num", peer.ManageNumber()).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("removed peer in peermanager") pm.mutex.Lock() defer pm.mutex.Unlock() diff --git a/p2p/raftsupport/concclusterreceiver.go b/p2p/raftsupport/concclusterreceiver.go index b97dc17ac..aaee3b0c1 100644 --- a/p2p/raftsupport/concclusterreceiver.go +++ b/p2p/raftsupport/concclusterreceiver.go @@ -148,17 +148,17 @@ func (r *ConcurrentClusterInfoReceiver) handleInWaiting(peer p2pcommon.RemotePee // remote peer response malformed data. body, ok := msgBody.(*types.GetClusterInfoResponse) if !ok { - r.logger.Debug().Str(p2putil.LogPeerName, peer.Name()).Str(p2putil.LogMsgID, msg.ID().String()).Msg("get cluster invalid response data") + r.logger.Debug().Str(p2putil.LogPeerName, peer.Name()).Stringer(p2putil.LogMsgID, msg.ID()).Msg("get cluster invalid response data") return } else if len(body.MbrAttrs) == 0 || body.Error != "" { - r.logger.Debug().Str(p2putil.LogPeerName, peer.Name()).Str(p2putil.LogMsgID, msg.ID().String()).Err(errors.New(body.Error)).Msg("get cluster response empty member") + r.logger.Debug().Str(p2putil.LogPeerName, peer.Name()).Stringer(p2putil.LogMsgID, msg.ID()).Err(errors.New(body.Error)).Msg("get cluster response empty member") return } - r.logger.Debug().Str(p2putil.LogPeerName, peer.Name()).Str(p2putil.LogMsgID, msg.ID().String()).Object("resp", body).Msg("received get cluster response") + r.logger.Debug().Str(p2putil.LogPeerName, peer.Name()).Stringer(p2putil.LogMsgID, msg.ID()).Object("resp", body).Msg("received get cluster response") // return the result if len(body.Error) != 0 { - r.logger.Debug().Str(p2putil.LogPeerName, peer.Name()).Str(p2putil.LogMsgID, msg.ID().String()).Err(errors.New(body.Error)).Msg("get cluster response error") + r.logger.Debug().Str(p2putil.LogPeerName, peer.Name()).Stringer(p2putil.LogMsgID, msg.ID()).Err(errors.New(body.Error)).Msg("get cluster response error") return } r.succResps[peer.ID()] = body @@ -206,7 +206,7 @@ func (r *ConcurrentClusterInfoReceiver) calculate(err error) *message.GetCluster } } if bestRsp != nil { - r.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(bestPid)).Object("resp", bestRsp).Msg("chosed best response") + r.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(bestPid)).Object("resp", bestRsp).Msg("chosed best response") rsp.ClusterID = bestRsp.GetClusterID() rsp.ChainID = bestRsp.GetChainID() rsp.Members = bestRsp.GetMbrAttrs() diff --git a/p2p/raftsupport/rafttransport.go b/p2p/raftsupport/rafttransport.go index 0faf2f17b..d11e32605 100644 --- a/p2p/raftsupport/rafttransport.go +++ b/p2p/raftsupport/rafttransport.go @@ -105,7 +105,7 @@ func (t *AergoRaftTransport) Send(msgs []raftpb.Message) { peer.SendMessage(t.mf.NewRaftMsgOrder(m.Type, &m)) continue } else { - t.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(member.GetPeerID())).Msg("peer is unreachable") + t.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(member.GetPeerID())).Msg("peer is unreachable") t.raftAcc.ReportUnreachable(member.GetPeerID()) continue } @@ -148,17 +148,17 @@ func (t *AergoRaftTransport) AddRemote(id rtypes.ID, urls []string) { func (t *AergoRaftTransport) AddPeer(id rtypes.ID, peerID types.PeerID, urls []string) { t.mutex.Lock() defer t.mutex.Unlock() - t.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Str("id", id.String()).Msg("Adding member peer") + t.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Str("id", id.String()).Msg("Adding member peer") member := t.raftAcc.GetMemberByID(uint64(id)) if member == nil { - t.logger.Info().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Str("id", id.String()).Msg("can't find member") + t.logger.Info().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Str("id", id.String()).Msg("can't find member") return } st, exist := t.statuses[id] if exist { if _, exist := t.pm.GetPeer(member.GetPeerID()); exist { - t.logger.Info().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Str("id", id.String()).Msg("peer already exists") + t.logger.Info().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Str("id", id.String()).Msg("peer already exists") st.activate() return } else { @@ -200,7 +200,7 @@ func (t *AergoRaftTransport) RemovePeer(id rtypes.ID) { if peer, exist := t.pm.GetPeer(st.pid); exist { peer.Stop() } - t.logger.Info().Str(p2putil.LogPeerID, p2putil.ShortForm(st.pid)).Uint64("raftID", uint64(id)).Msg("removed raft peer") + t.logger.Info().Stringer(p2putil.LogPeerID, types.LogPeerShort(st.pid)).Uint64("raftID", uint64(id)).Msg("removed raft peer") } func (t *AergoRaftTransport) RemoveAllPeers() { @@ -247,7 +247,7 @@ func (t *AergoRaftTransport) OnRaftSnapshot(s network.Stream) { peer, found := t.pm.GetPeer(peerID) if !found { addr := s.Conn().RemoteMultiaddr() - t.logger.Info().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Str("multiaddr", addr.String()).Msg("snapshot stream from leader node") + t.logger.Info().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Str("multiaddr", addr.String()).Msg("snapshot stream from leader node") hsresp.RespCode = p2pcommon.HSCodeAuthFail s.Write(hsresp.Marshal()) s.Close() diff --git a/p2p/raftsupport/status.go b/p2p/raftsupport/status.go index 2def4f43f..a03f1d6f5 100644 --- a/p2p/raftsupport/status.go +++ b/p2p/raftsupport/status.go @@ -34,11 +34,11 @@ func (s *rPeerStatus) activate() { s.mu.Lock() defer s.mu.Unlock() if !s.active { - s.logger.Info().Str(p2putil.LogPeerID, p2putil.ShortForm(s.pid)).Str("raftID", s.id.String()).Msgf("peer became active") + s.logger.Info().Stringer(p2putil.LogPeerID, types.LogPeerShort(s.pid)).Str("raftID", s.id.String()).Msgf("peer became active") s.active = true s.since = time.Now() } else { - s.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(s.pid)).Str("raftID", s.id.String()).Msgf("activate called to already active peer") + s.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(s.pid)).Str("raftID", s.id.String()).Msgf("activate called to already active peer") } } @@ -47,12 +47,12 @@ func (s *rPeerStatus) deactivate(cause string) { defer s.mu.Unlock() if s.active { - s.logger.Info().Str("cause", cause).Str(p2putil.LogPeerID, p2putil.ShortForm(s.pid)).Str("raftID", s.id.String()).Msgf("peer became inactive") + s.logger.Info().Str("cause", cause).Stringer(p2putil.LogPeerID, types.LogPeerShort(s.pid)).Str("raftID", s.id.String()).Msgf("peer became inactive") s.active = false s.since = time.Time{} } else { - s.logger.Debug().Str("cause", cause).Str(p2putil.LogPeerID, p2putil.ShortForm(s.pid)).Str("raftID", s.id.String()).Msgf("deactivate called to already inactive peer") + s.logger.Debug().Str("cause", cause).Stringer(p2putil.LogPeerID, types.LogPeerShort(s.pid)).Str("raftID", s.id.String()).Msgf("deactivate called to already inactive peer") } } diff --git a/p2p/remotepeer.go b/p2p/remotepeer.go index 6d2cf2ae8..fc34c3fa7 100644 --- a/p2p/remotepeer.go +++ b/p2p/remotepeer.go @@ -284,19 +284,19 @@ func (p *remotePeerImpl) handleMsg(msg p2pcommon.Message) (err error) { subProto := msg.Subprotocol() defer func() { if r := recover(); r != nil { - p.logger.Error().Str(p2putil.LogProtoID, subProto.String()).Str("callStack", string(debug.Stack())).Interface("panic", r).Msg("There were panic in handler.") + p.logger.Error().Stringer(p2putil.LogProtoID, subProto).Str("callStack", string(debug.Stack())).Interface("panic", r).Msg("There were panic in handler.") err = fmt.Errorf("internal error") } }() if p.State() > types.RUNNING { - p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogMsgID, msg.ID().String()).Str(p2putil.LogProtoID, subProto.String()).Str("current_state", p.State().String()).Msg("peer is not running. silently drop input message") + p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogMsgID, msg.ID()).Stringer(p2putil.LogProtoID, subProto).Str("current_state", p.State().String()).Msg("peer is not running. silently drop input message") return nil } handler, found := p.handlers[subProto] if !found { - p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogMsgID, msg.ID().String()).Str(p2putil.LogProtoID, subProto.String()).Msg("invalid protocol") + p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogMsgID, msg.ID()).Stringer(p2putil.LogProtoID, subProto).Msg("invalid protocol") return fmt.Errorf("invalid protocol %s", subProto) } @@ -304,7 +304,7 @@ func (p *remotePeerImpl) handleMsg(msg p2pcommon.Message) (err error) { payload, err := handler.ParsePayload(msg.Payload()) if err != nil { - p.logger.Warn().Err(err).Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogMsgID, msg.ID().String()).Str(p2putil.LogProtoID, subProto.String()).Msg("invalid message data") + p.logger.Warn().Err(err).Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogMsgID, msg.ID()).Stringer(p2putil.LogProtoID, subProto).Msg("invalid message data") return fmt.Errorf("invalid message data") } //err = p.signer.verifyMsg(msg, p.remoteInfo.ID) @@ -314,7 +314,7 @@ func (p *remotePeerImpl) handleMsg(msg p2pcommon.Message) (err error) { //} err = handler.CheckAuth(msg, payload) if err != nil { - p.logger.Warn().Err(err).Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogMsgID, msg.ID().String()).Str(p2putil.LogProtoID, subProto.String()).Msg("Failed to authenticate message") + p.logger.Warn().Err(err).Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogMsgID, msg.ID()).Stringer(p2putil.LogProtoID, subProto).Msg("Failed to authenticate message") return fmt.Errorf("Failed to authenticate message.") } @@ -335,7 +335,7 @@ func (p *remotePeerImpl) Stop() { func (p *remotePeerImpl) SendMessage(msg p2pcommon.MsgOrder) { if p.State() > types.RUNNING { - p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, msg.GetProtocolID().String()). + p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, msg.GetProtocolID()). Str(p2putil.LogMsgID, msg.GetMsgID().String()).Str("current_state", p.State().String()).Msg("Cancel sending message, since peer is not running state") return } @@ -343,7 +343,7 @@ func (p *remotePeerImpl) SendMessage(msg p2pcommon.MsgOrder) { case p.writeBuf <- msg: // it's OK default: - p.logger.Info().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, msg.GetProtocolID().String()). + p.logger.Info().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, msg.GetProtocolID()). Str(p2putil.LogMsgID, msg.GetMsgID().String()).Msg("Remote peer is busy or down") // TODO find more elegant way to handled flooding queue. in lots of cases, pending for dropped tx notice or newBlock notice (not blockProduced notice) are not critical in lots of cases. p.Stop() @@ -352,7 +352,7 @@ func (p *remotePeerImpl) SendMessage(msg p2pcommon.MsgOrder) { func (p *remotePeerImpl) TrySendMessage(msg p2pcommon.MsgOrder) bool { if p.State() > types.RUNNING { - p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, msg.GetProtocolID().String()). + p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, msg.GetProtocolID()). Str(p2putil.LogMsgID, msg.GetMsgID().String()).Str("current_state", p.State().String()).Msg("Cancel sending message, since peer is not running state") return false } @@ -367,7 +367,7 @@ func (p *remotePeerImpl) TrySendMessage(msg p2pcommon.MsgOrder) bool { func (p *remotePeerImpl) SendAndWaitMessage(msg p2pcommon.MsgOrder, timeout time.Duration) error { if p.State() > types.RUNNING { - p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, msg.GetProtocolID().String()). + p.logger.Debug().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, msg.GetProtocolID()). Str(p2putil.LogMsgID, msg.GetMsgID().String()).Str("current_state", p.State().String()).Msg("Cancel sending message, since peer is not running state") return fmt.Errorf("not running") } @@ -375,7 +375,7 @@ func (p *remotePeerImpl) SendAndWaitMessage(msg p2pcommon.MsgOrder, timeout time case p.writeBuf <- msg: return nil case <-time.NewTimer(timeout).C: - p.logger.Info().Str(p2putil.LogPeerName, p.Name()).Str(p2putil.LogProtoID, msg.GetProtocolID().String()). + p.logger.Info().Str(p2putil.LogPeerName, p.Name()).Stringer(p2putil.LogProtoID, msg.GetProtocolID()). Str(p2putil.LogMsgID, msg.GetMsgID().String()).Msg("Remote peer is busy or down") // TODO find more elegant way to handled flooding queue. in lots of cases, pending for dropped tx notice or newBlock notice (not blockProduced notice) are not critical in lots of cases. p.Stop() @@ -551,7 +551,7 @@ func (p *remotePeerImpl) addCert(cert *p2pcommon.AgentCertificateV1) { } } newCerts = append(newCerts, cert) - p.logger.Info().Str(p2putil.LogPeerName, p.Name()).Str("bpID", p2putil.ShortForm(cert.BPID)).Time("cTime", cert.CreateTime).Time("eTime", cert.ExpireTime).Msg("agent certificate is added to certificate list of remote peer") + p.logger.Info().Str(p2putil.LogPeerName, p.Name()).Stringer("bpID", types.LogPeerShort(cert.BPID)).Time("cTime", cert.CreateTime).Time("eTime", cert.ExpireTime).Msg("agent certificate is added to certificate list of remote peer") p.remoteInfo.Certificates = newCerts if len(newCerts) > 0 && p.AcceptedRole() == types.PeerRole_Watcher { p.logger.Info().Str(p2putil.LogPeerName, p.Name()).Msg("peer has certificates now. peer is promoted to Agent") @@ -566,7 +566,7 @@ func (p *remotePeerImpl) cleanupCerts() { if cert.IsValidInTime(now, p2pcommon.TimeErrorTolerance) { certs2 = append(certs2, cert) } else { - p.logger.Info().Str(p2putil.LogPeerName, p.Name()).Str("issuer", p2putil.ShortForm(cert.BPID)).Msg("Certificate is expired") + p.logger.Info().Str(p2putil.LogPeerName, p.Name()).Stringer("issuer", types.LogPeerShort(cert.BPID)).Msg("Certificate is expired") } } p.remoteInfo.Certificates = certs2 diff --git a/p2p/rolemanager.go b/p2p/rolemanager.go index 31e3c591d..a9d31b4b5 100644 --- a/p2p/rolemanager.go +++ b/p2p/rolemanager.go @@ -44,12 +44,12 @@ func (rm *RaftRoleManager) UpdateBP(toAdd []types.PeerID, toRemove []types.PeerI for _, pid := range toRemove { delete(rm.raftBP, pid) changes = append(changes, p2pcommon.RoleModifier{ID: pid, Role: types.PeerRole_Watcher}) - rm.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(pid)).Msg("raftBP removed") + rm.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(pid)).Msg("raftBP removed") } for _, pid := range toAdd { rm.raftBP[pid] = true changes = append(changes, p2pcommon.RoleModifier{ID: pid, Role: types.PeerRole_Producer}) - rm.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(pid)).Msg("raftBP added") + rm.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(pid)).Msg("raftBP added") } rm.is.PeerManager().UpdatePeerRole(changes) } diff --git a/p2p/subproto/advice.go b/p2p/subproto/advice.go index be74adb85..6c8363713 100644 --- a/p2p/subproto/advice.go +++ b/p2p/subproto/advice.go @@ -32,7 +32,7 @@ func (a *LogHandleTimeAdvice) PreHandle() { func (a *LogHandleTimeAdvice) PostHandle(msg p2pcommon.Message, msgBody p2pcommon.MessageBody) { a.logger.WithLevel(a.level). Int64("elapsed", time.Since(a.timestamp).Nanoseconds()/1000). - Str(p2putil.LogProtoID, msg.Subprotocol().String()). - Str(p2putil.LogMsgID, msg.ID().String()). + Stringer(p2putil.LogProtoID, msg.Subprotocol()). + Stringer(p2putil.LogMsgID, msg.ID()). Msg("handle takes") } diff --git a/p2p/subproto/bp.go b/p2p/subproto/bp.go index 2bbda3973..5cf90f2f1 100644 --- a/p2p/subproto/bp.go +++ b/p2p/subproto/bp.go @@ -117,7 +117,7 @@ func (h *toAgentBPNoticeHandler) Handle(msg p2pcommon.Message, msgBody p2pcommon } if !checkBPNoticeSender(bpID, remotePeer) { - h.logger.Debug().Err(err).Str(p2putil.LogPeerName, remotePeer.Name()).Str("bpID", p2putil.ShortForm(bpID)).Str("blockID", block.BlockID().String()).Msg("peer is not access right to send bp notice") + h.logger.Debug().Err(err).Str(p2putil.LogPeerName, remotePeer.Name()).Stringer("bpID", types.LogPeerShort(bpID)).Str("blockID", block.BlockID().String()).Msg("peer is not access right to send bp notice") return } diff --git a/p2p/subproto/cert.go b/p2p/subproto/cert.go index b094c182b..7bd7f63da 100644 --- a/p2p/subproto/cert.go +++ b/p2p/subproto/cert.go @@ -124,13 +124,13 @@ func (h *certRenewedNoticeHandler) Handle(msg p2pcommon.Message, msgBody p2pcomm if !p2putil.ContainsID(p.Meta().ProducerIDs, cert.BPID) { // TODO add penalty // this agent is not in charge of that bp id. - h.logger.Info().Str(p2putil.LogPeerName, p.Name()).Str("bpID", p2putil.ShortForm(cert.BPID)).Msg("drop renewed certificate, since issuer is not managed producer of remote peer") + h.logger.Info().Str(p2putil.LogPeerName, p.Name()).Stringer("bpID", types.LogPeerShort(cert.BPID)).Msg("drop renewed certificate, since issuer is not managed producer of remote peer") return } if !types.IsSamePeerID(p.ID(), cert.AgentID) { // TODO add penalty // this certificate is not my certificate - h.logger.Info().Str(p2putil.LogPeerName, p.Name()).Str("bpID", p2putil.ShortForm(cert.BPID)).Str("agentID", p2putil.ShortForm(cert.AgentID)).Msg("drop renewed certificate, since agent id is not the remote peer") + h.logger.Info().Str(p2putil.LogPeerName, p.Name()).Stringer("bpID", types.LogPeerShort(cert.BPID)).Stringer("agentID", types.LogPeerShort(cert.AgentID)).Msg("drop renewed certificate, since agent id is not the remote peer") return } diff --git a/p2p/subproto/getblock.go b/p2p/subproto/getblock.go index 765b127e2..4d96e1aee 100644 --- a/p2p/subproto/getblock.go +++ b/p2p/subproto/getblock.go @@ -51,7 +51,7 @@ func (bh *blockRequestHandler) Handle(msg p2pcommon.Message, msgBody p2pcommon.M if bh.issue() { go bh.handleBlkReq(msg, data) } else { - bh.logger.Info().Str(p2putil.LogProtoID, bh.protocol.String()).Str(p2putil.LogMsgID, msg.ID().String()).Str(p2putil.LogPeerName, remotePeer.Name()).Msg("return error for busy") + bh.logger.Info().Stringer(p2putil.LogProtoID, bh.protocol).Stringer(p2putil.LogMsgID, msg.ID()).Str(p2putil.LogPeerName, remotePeer.Name()).Msg("return error for busy") resp := &types.GetBlockResponse{ Status: types.ResultStatus_RESOURCE_EXHAUSTED, Blocks: nil, HasNext: false} @@ -139,7 +139,7 @@ func (bh *blockRequestHandler) handleBlkReq(msg p2pcommon.Message, data *types.G } } -// newBlockRespHandler creates handler for GetBlockResponse +// NewBlockRespHandler creates handler for GetBlockResponse func NewBlockRespHandler(pm p2pcommon.PeerManager, peer p2pcommon.RemotePeer, logger *log.Logger, actor p2pcommon.ActorService, sm p2pcommon.SyncManager) *blockResponseHandler { bh := &blockResponseHandler{BaseMsgHandler{protocol: p2pcommon.GetBlocksResponse, pm: pm, sm: sm, peer: peer, actor: actor, logger: logger}} return bh diff --git a/p2p/subproto/ping.go b/p2p/subproto/ping.go index 410a49e10..6027e9288 100644 --- a/p2p/subproto/ping.go +++ b/p2p/subproto/ping.go @@ -52,7 +52,7 @@ func (ph *pingRequestHandler) Handle(msg p2pcommon.Message, msgBody p2pcommon.Me } // generate response message - ph.logger.Debug().Str(p2putil.LogPeerName, remotePeer.Name()).Str(p2putil.LogMsgID, msg.ID().String()).Msg("Sending ping response") + ph.logger.Debug().Str(p2putil.LogPeerName, remotePeer.Name()).Stringer(p2putil.LogMsgID, msg.ID()).Msg("Sending ping response") resp := &types.Pong{} remotePeer.SendMessage(remotePeer.MF().NewMsgResponseOrder(msg.ID(), p2pcommon.PingResponse, resp)) } diff --git a/p2p/subproto/tx.go b/p2p/subproto/tx.go index 97e1a91db..d928f8e77 100644 --- a/p2p/subproto/tx.go +++ b/p2p/subproto/tx.go @@ -52,7 +52,7 @@ func (th *txRequestHandler) Handle(msg p2pcommon.Message, msgBody p2pcommon.Mess p2putil.DebugLogReceive(th.logger, th.protocol, msg.ID().String(), remotePeer, body) if err := th.sm.HandleGetTxReq(remotePeer, msg.ID(), body); err != nil { - th.logger.Info().Str(p2putil.LogPeerName, remotePeer.Name()).Str(p2putil.LogMsgID, msg.ID().String()).Err(err).Msg("return err for concurrent get tx request") + th.logger.Info().Str(p2putil.LogPeerName, remotePeer.Name()).Stringer(p2putil.LogMsgID, msg.ID()).Err(err).Msg("return err for concurrent get tx request") resp := &types.GetTransactionsResponse{ Status: types.ResultStatus_RESOURCE_EXHAUSTED, Hashes: nil, @@ -77,7 +77,7 @@ func (th *txResponseHandler) Handle(msg p2pcommon.Message, msgBody p2pcommon.Mes p2putil.DebugLogReceiveResponse(th.logger, th.protocol, msg.ID().String(), msg.OriginalID().String(), th.peer, data) if !remotePeer.GetReceiver(msg.OriginalID())(msg, data) { - th.logger.Warn().Str(p2putil.LogMsgID, msg.ID().String()).Msg("unknown getTX response") + th.logger.Warn().Stringer(p2putil.LogMsgID, msg.ID()).Msg("unknown getTX response") remotePeer.ConsumeRequest(msg.OriginalID()) } } diff --git a/p2p/syncmanager.go b/p2p/syncmanager.go index 9ecd6a785..c5d481501 100644 --- a/p2p/syncmanager.go +++ b/p2p/syncmanager.go @@ -10,7 +10,6 @@ import ( "github.com/aergoio/aergo-lib/log" "github.com/aergoio/aergo/v2/chain" - "github.com/aergoio/aergo/v2/internal/enc" "github.com/aergoio/aergo/v2/message" "github.com/aergoio/aergo/v2/p2p/p2pcommon" "github.com/aergoio/aergo/v2/p2p/p2putil" @@ -89,7 +88,7 @@ func (sm *syncManager) HandleNewBlockNotice(peer p2pcommon.RemotePeer, data *typ // request block info if selfnode does not have block already foundBlock, _ := sm.actor.GetChainAccessor().GetBlock(data.BlockHash) if foundBlock == nil { - sm.logger.Debug().Str(p2putil.LogBlkHash, enc.ToString(data.BlockHash)).Str(p2putil.LogPeerName, peer.Name()).Msg("new block notice of unknown hash. request back to notifier") + sm.logger.Debug().Stringer(p2putil.LogBlkHash, types.LogBase58(data.BlockHash)).Str(p2putil.LogPeerName, peer.Name()).Msg("new block notice of unknown hash. request back to notifier") sm.actor.SendRequest(message.P2PSvc, &message.GetBlockInfos{ToWhom: peerID, Hashes: []message.BlockHash{message.BlockHash(data.BlockHash)}}) } diff --git a/p2p/synctx.go b/p2p/synctx.go index b3e5be415..a16878d86 100644 --- a/p2p/synctx.go +++ b/p2p/synctx.go @@ -191,7 +191,7 @@ func (tm *syncTxManager) HandleNewTxNotice(peer p2pcommon.RemotePeer, txIDs []ty tm.toNoticeIdQueue.PushBack(&queryQueue{peerID: peerID, txIDs: toQueue}) } - tm.logger.Trace().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Int("newCnt", len(newComer)).Int("queCnt", len(queued)).Int("dupCnt", len(duplicated)).Array("newComer", types.NewLogTxIDsMarshaller(newComer, 10)).Array("duplicated", types.NewLogTxIDsMarshaller(duplicated, 10)).Array("queued", types.NewLogTxIDsMarshaller(queued, 10)).Int("frontCacheSize", len(tm.frontCache)).Msg("push txs, to query next time") + tm.logger.Trace().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Int("newCnt", len(newComer)).Int("queCnt", len(queued)).Int("dupCnt", len(duplicated)).Array("newComer", types.NewLogTxIDsMarshaller(newComer, 10)).Array("duplicated", types.NewLogTxIDsMarshaller(duplicated, 10)).Array("queued", types.NewLogTxIDsMarshaller(queued, 10)).Int("frontCacheSize", len(tm.frontCache)).Msg("push txs, to query next time") } } @@ -220,7 +220,7 @@ func (tm *syncTxManager) retryGetTx(peerID types.PeerID, hashes [][]byte) { for i, hash := range hashes { txIDs[i] = types.ToTxID(hash) } - tm.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Array("txIDs", types.NewLogTxIDsMarshaller(txIDs, 10)).Msg("push txs that are failed to get by server busy") + tm.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Array("txIDs", types.NewLogTxIDsMarshaller(txIDs, 10)).Msg("push txs that are failed to get by server busy") tm.pushBackToFrontCache(peerID, txIDs) } } @@ -464,7 +464,7 @@ func (tm *syncTxManager) refineFrontCache() { tm.sendGetTx(peer, ids) } else { // peer probably disconnected. - tm.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Array("hashes", types.NewLogTxIDsMarshaller(ids, 10)).Msg("syncManager failed to send get tx, since peer is disconnected just before") + tm.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Array("hashes", types.NewLogTxIDsMarshaller(ids, 10)).Msg("syncManager failed to send get tx, since peer is disconnected just before") toRetry := make([]types.TxID, len(ids)) copy(toRetry, ids) tm.burnFailedTxFrontCache(peerID, toRetry) diff --git a/p2p/transport/networktransport.go b/p2p/transport/networktransport.go index f8ea38070..7208c4811 100644 --- a/p2p/transport/networktransport.go +++ b/p2p/transport/networktransport.go @@ -117,12 +117,12 @@ func (sl *networkTransport) AddStreamHandler(pid core.ProtocolID, handler networ func (sl *networkTransport) GetOrCreateStreamWithTTL(meta p2pcommon.PeerMeta, ttl time.Duration, protocolIDs ...core.ProtocolID) (core.Stream, error) { var peerAddr = meta.Addresses[0] var peerID = meta.ID - sl.logger.Debug().Str("peerAddr", peerAddr.String()).Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("connecting to peer") + sl.logger.Debug().Str("peerAddr", peerAddr.String()).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("connecting to peer") sl.Peerstore().AddAddr(peerID, peerAddr, ttl) ctx := context.Background() s, err := sl.NewStream(ctx, peerID, protocolIDs...) if err != nil { - sl.logger.Info().Err(err).Str("addr", peerAddr.String()).Str(p2putil.LogPeerID, p2putil.ShortForm(meta.ID)).Str("p2p_proto", p2putil.ProtocolIDsToString(protocolIDs)).Msg("Error while get stream") + sl.logger.Info().Err(err).Str("addr", peerAddr.String()).Stringer(p2putil.LogPeerID, types.LogPeerShort(meta.ID)).Str("p2p_proto", p2putil.ProtocolIDsToString(protocolIDs)).Msg("Error while get stream") return nil, err } return s, nil @@ -178,7 +178,7 @@ func (sl *networkTransport) startListener() { panic(err.Error()) } sl.Host = newHost - sl.logger.Info().Str(p2putil.LogFullID, sl.ID().Pretty()).Str(p2putil.LogPeerID, p2putil.ShortForm(sl.ID())).Str("addr[0]", listens[0].String()).Msg("Set self node's pid, and listening for connections") + sl.logger.Info().Str(p2putil.LogFullID, sl.ID().Pretty()).Stringer(p2putil.LogPeerID, types.LogPeerShort(sl.ID())).Str("addr[0]", listens[0].String()).Msg("Set self node's pid, and listening for connections") } func (sl *networkTransport) Stop() error { diff --git a/p2p/v030/v030handshake.go b/p2p/v030/v030handshake.go index c353d73a1..eab08ef72 100644 --- a/p2p/v030/v030handshake.go +++ b/p2p/v030/v030handshake.go @@ -46,7 +46,7 @@ func NewV030VersionedHS(pm p2pcommon.PeerManager, actor p2pcommon.ActorService, // handshakeOutboundPeer start handshake with outbound peer func (h *V030Handshaker) DoForOutbound(ctx context.Context) (*p2pcommon.HandshakeResult, error) { - h.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Msg("Starting versioned handshake for outbound peer connection") + h.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Msg("Starting versioned handshake for outbound peer connection") bestBlock, err := h.actor.GetChainAccessor().GetBestBlock() if err != nil { return nil, err @@ -81,13 +81,13 @@ func (h *V030Handshaker) sendLocalStatus(ctx context.Context, hostStatus *types. var err error container := createMessage(p2pcommon.StatusRequest, p2pcommon.NewMsgID(), hostStatus) if container == nil { - h.logger.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Msg("failed to create p2p message") + h.logger.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Msg("failed to create p2p message") h.sendGoAway("internal error") // h.logger.Warn().Str(LogPeerID, ShortForm(peerID)).Err(err).Msg("failed to create p2p message") return fmt.Errorf("failed to craete container message") } if err = h.msgRW.WriteMsg(container); err != nil { - h.logger.Info().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Err(err).Msg("failed to write local status ") + h.logger.Info().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Err(err).Msg("failed to write local status ") return err } select { @@ -117,7 +117,7 @@ func (h *V030Handshaker) receiveRemoteStatus(ctx context.Context) (*types.Status if data.Subprotocol() == p2pcommon.GoAway { return h.handleGoAway(h.peerID, data) } else { - h.logger.Info().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Str("expected", p2pcommon.StatusRequest.String()).Str("actual", data.Subprotocol().String()).Msg("unexpected message type") + h.logger.Info().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Str("expected", p2pcommon.StatusRequest.String()).Str("actual", data.Subprotocol().String()).Msg("unexpected message type") h.sendGoAway("unexpected message type") return nil, fmt.Errorf("unexpected message type") } @@ -175,7 +175,7 @@ func (h *V030Handshaker) checkRemoteStatus(remotePeerStatus *types.Status) error rMeta := p2pcommon.NewMetaFromStatus(remotePeerStatus) if rMeta.ID != h.peerID { - h.logger.Debug().Str("received_peer_id", rMeta.ID.Pretty()).Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Msg("Inconsistent peerID") + h.logger.Debug().Str("received_peer_id", rMeta.ID.Pretty()).Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Msg("Inconsistent peerID") h.sendGoAway("Inconsistent peerID") return fmt.Errorf("Inconsistent peerID") } @@ -186,7 +186,7 @@ func (h *V030Handshaker) checkRemoteStatus(remotePeerStatus *types.Status) error // DoForInbound is handle handshake from inbound peer func (h *V030Handshaker) DoForInbound(ctx context.Context) (*p2pcommon.HandshakeResult, error) { - h.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Msg("Starting versioned handshake for inbound peer connection") + h.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Msg("Starting versioned handshake for inbound peer connection") // inbound: receive, check and send remotePeerStatus, err := h.receiveRemoteStatus(ctx) @@ -219,7 +219,7 @@ func (h *V030Handshaker) DoForInbound(ctx context.Context) (*p2pcommon.Handshake func (h *V030Handshaker) handleGoAway(peerID types.PeerID, data p2pcommon.Message) (*types.Status, error) { goAway := &types.GoAwayNotice{} if err := p2putil.UnmarshalMessageBody(data.Payload(), goAway); err != nil { - h.logger.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Err(err).Msg("Remote peer sent goAway but failed to decode internal message") + h.logger.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Err(err).Msg("Remote peer sent goAway but failed to decode internal message") return nil, err } return nil, fmt.Errorf("remote peer refuse handshake: %s", goAway.GetMessage()) diff --git a/p2p/v030/v032handshake.go b/p2p/v030/v032handshake.go index f92ab94bf..803bffc7f 100644 --- a/p2p/v030/v032handshake.go +++ b/p2p/v030/v032handshake.go @@ -54,7 +54,7 @@ func (h *V032Handshaker) checkRemoteStatus(remotePeerStatus *types.Status) error } func (h *V032Handshaker) DoForOutbound(ctx context.Context) (*p2pcommon.HandshakeResult, error) { - h.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Msg("Starting versioned handshake for outbound peer connection") + h.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Msg("Starting versioned handshake for outbound peer connection") bestBlock, err := h.actor.GetChainAccessor().GetBestBlock() if err != nil { @@ -88,7 +88,7 @@ func (h *V032Handshaker) DoForOutbound(ctx context.Context) (*p2pcommon.Handshak } func (h *V032Handshaker) DoForInbound(ctx context.Context) (*p2pcommon.HandshakeResult, error) { - h.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Msg("Starting versioned handshake for inbound peer connection") + h.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Msg("Starting versioned handshake for inbound peer connection") // inbound: receive, check and send remotePeerStatus, err := h.receiveRemoteStatus(ctx) diff --git a/p2p/v030/v033handshake.go b/p2p/v030/v033handshake.go index 5c7e7d24c..ab5399fdf 100644 --- a/p2p/v030/v033handshake.go +++ b/p2p/v030/v033handshake.go @@ -66,7 +66,7 @@ func (h *V033Handshaker) checkRemoteStatus(remotePeerStatus *types.Status) error rMeta := p2pcommon.NewMetaFromStatus(remotePeerStatus) if rMeta.ID != h.peerID { - h.logger.Debug().Str("received_peer_id", rMeta.ID.Pretty()).Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Msg("Inconsistent peerID") + h.logger.Debug().Str("received_peer_id", rMeta.ID.Pretty()).Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Msg("Inconsistent peerID") h.sendGoAway("Inconsistent peerID") return fmt.Errorf("inconsistent peerID") } @@ -83,7 +83,7 @@ func (h *V033Handshaker) checkRemoteStatus(remotePeerStatus *types.Status) error } func (h *V033Handshaker) DoForOutbound(ctx context.Context) (*p2pcommon.HandshakeResult, error) { - h.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Msg("Starting versioned handshake for outbound peer connection") + h.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Msg("Starting versioned handshake for outbound peer connection") // find my best block bestBlock, err := h.actor.GetChainAccessor().GetBestBlock() @@ -119,7 +119,7 @@ func (h *V033Handshaker) DoForOutbound(ctx context.Context) (*p2pcommon.Handshak } func (h *V033Handshaker) DoForInbound(ctx context.Context) (*p2pcommon.HandshakeResult, error) { - h.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Msg("Starting versioned handshake for inbound peer connection") + h.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Msg("Starting versioned handshake for inbound peer connection") // inbound: receive, check and send remotePeerStatus, err := h.receiveRemoteStatus(ctx) diff --git a/p2p/v200/v200handshake.go b/p2p/v200/v200handshake.go index e4a98ae21..9f35813ef 100644 --- a/p2p/v200/v200handshake.go +++ b/p2p/v200/v200handshake.go @@ -64,7 +64,7 @@ func NewV200VersionedHS(is p2pcommon.InternalService, log *log.Logger, vm p2pcom // handshakeOutboundPeer start handshake with outbound peer func (h *V200Handshaker) DoForOutbound(ctx context.Context) (*p2pcommon.HandshakeResult, error) { - h.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Msg("Starting versioned handshake for outbound peer connection") + h.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Msg("Starting versioned handshake for outbound peer connection") // find my best block bestBlock, err := h.is.GetChainAccessor().GetBestBlock() @@ -102,13 +102,13 @@ func (h *V200Handshaker) sendLocalStatus(ctx context.Context, hostStatus *types. var err error container := createMessage(p2pcommon.StatusRequest, p2pcommon.NewMsgID(), hostStatus) if container == nil { - h.logger.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Msg("failed to create p2p message") + h.logger.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Msg("failed to create p2p message") h.sendGoAway("internal error") // h.logger.Warn().Str(LogPeerID, ShortForm(peerID)).Err(err).Msg("failed to create p2p message") return fmt.Errorf("failed to craete container message") } if err = h.msgRW.WriteMsg(container); err != nil { - h.logger.Info().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Err(err).Msg("failed to write local status ") + h.logger.Info().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Err(err).Msg("failed to write local status ") return err } select { @@ -138,7 +138,7 @@ func (h *V200Handshaker) receiveRemoteStatus(ctx context.Context) (*types.Status if data.Subprotocol() == p2pcommon.GoAway { return h.handleGoAway(h.peerID, data) } else { - h.logger.Info().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Str("expected", p2pcommon.StatusRequest.String()).Str("actual", data.Subprotocol().String()).Msg("unexpected message type") + h.logger.Info().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Str("expected", p2pcommon.StatusRequest.String()).Str("actual", data.Subprotocol().String()).Msg("unexpected message type") h.sendGoAway("unexpected message type") return nil, fmt.Errorf("unexpected message type") } @@ -183,7 +183,7 @@ func (h *V200Handshaker) checkRemoteStatus(remotePeerStatus *types.Status) error rMeta := p2pcommon.NewMetaFromStatus(remotePeerStatus) if rMeta.ID != h.peerID { - h.logger.Debug().Str("received_peer_id", rMeta.ID.Pretty()).Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Msg("Inconsistent peerID") + h.logger.Debug().Str("received_peer_id", rMeta.ID.Pretty()).Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Msg("Inconsistent peerID") h.sendGoAway("Inconsistent peerID") return fmt.Errorf("inconsistent peerID") } @@ -207,7 +207,7 @@ func (h *V200Handshaker) checkRemoteStatus(remotePeerStatus *types.Status) error // DoForInbound is handle handshake from inbound peer func (h *V200Handshaker) DoForInbound(ctx context.Context) (*p2pcommon.HandshakeResult, error) { - h.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(h.peerID)).Msg("Starting versioned handshake for inbound peer connection") + h.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(h.peerID)).Msg("Starting versioned handshake for inbound peer connection") // inbound: receive, check and send remotePeerStatus, err := h.receiveRemoteStatus(ctx) @@ -242,7 +242,7 @@ func (h *V200Handshaker) DoForInbound(ctx context.Context) (*p2pcommon.Handshake func (h *V200Handshaker) handleGoAway(peerID types.PeerID, data p2pcommon.Message) (*types.Status, error) { goAway := &types.GoAwayNotice{} if err := p2putil.UnmarshalMessageBody(data.Payload(), goAway); err != nil { - h.logger.Warn().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Err(err).Msg("Remote peer sent goAway but failed to decode internal message") + h.logger.Warn().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Err(err).Msg("Remote peer sent goAway but failed to decode internal message") return nil, err } return nil, fmt.Errorf("remote peer refuse handshake: %s", goAway.GetMessage()) @@ -266,7 +266,7 @@ func (h *V200Handshaker) checkByRole(status *types.Status) error { } func (h *V200Handshaker) checkAgent(status *types.Status) error { - h.logger.Debug().Int("certCnt", len(status.Certificates)).Str(p2putil.LogPeerID, p2putil.ShortForm(h.remoteMeta.ID)).Msg("checking peer as agent") + h.logger.Debug().Int("certCnt", len(status.Certificates)).Stringer(p2putil.LogPeerID, types.LogPeerShort(h.remoteMeta.ID)).Msg("checking peer as agent") // Agent must have at least one block producer if len(h.remoteMeta.ProducerIDs) == 0 { @@ -280,16 +280,16 @@ func (h *V200Handshaker) checkAgent(status *types.Status) error { for i, pCert := range status.Certificates { cert, err := p2putil.CheckAndGetV1(pCert) if err != nil { - h.logger.Info().Err(err).Str(p2putil.LogPeerID, p2putil.ShortForm(h.remoteMeta.ID)).Msg("invalid agent certificate") + h.logger.Info().Err(err).Stringer(p2putil.LogPeerID, types.LogPeerShort(h.remoteMeta.ID)).Msg("invalid agent certificate") return ErrInvalidAgentStatus } // check certificate if !types.IsSamePeerID(cert.AgentID, h.remoteMeta.ID) { - h.logger.Info().Err(err).Str(p2putil.LogPeerID, p2putil.ShortForm(h.remoteMeta.ID)).Msg("certificate is not for this agent") + h.logger.Info().Err(err).Stringer(p2putil.LogPeerID, types.LogPeerShort(h.remoteMeta.ID)).Msg("certificate is not for this agent") return ErrInvalidAgentStatus } if _, exist := producers[cert.BPID]; !exist { - h.logger.Info().Err(err).Str(p2putil.LogPeerID, p2putil.ShortForm(h.remoteMeta.ID)).Str("bpID", p2putil.ShortForm(cert.BPID)).Msg("peer id of certificate not matched") + h.logger.Info().Err(err).Stringer(p2putil.LogPeerID, types.LogPeerShort(h.remoteMeta.ID)).Stringer("bpID", types.LogPeerShort(cert.BPID)).Msg("peer id of certificate not matched") return ErrInvalidAgentStatus } diff --git a/p2p/v200/v200handshake_test.go b/p2p/v200/v200handshake_test.go index 074eff420..072195921 100644 --- a/p2p/v200/v200handshake_test.go +++ b/p2p/v200/v200handshake_test.go @@ -459,7 +459,7 @@ func TestV200Handshaker_checkAgent(t *testing.T) { id, _ := types.IDFromPrivateKey(priv) producerIDs[i] = id certs[i], _ = p2putil.NewAgentCertV1(id, agentID, p2putil.ConvertPKToBTCEC(priv), []string{ipExternal1}, time.Hour*24) - logger.Info().Str("peerID", p2putil.ShortForm(id)).Int("idx", i).Msg("producer id") + logger.Info().Stringer("peerID", types.LogPeerShort(id)).Int("idx", i).Msg("producer id") } pCerts, _ := p2putil.ConvertCertsToProto(certs) wrongCert := *certs[0] diff --git a/p2p/waitpeermanager.go b/p2p/waitpeermanager.go index 3ff02b3f3..242211816 100644 --- a/p2p/waitpeermanager.go +++ b/p2p/waitpeermanager.go @@ -55,7 +55,7 @@ func (dpm *basePeerManager) OnInboundConn(s network.Stream) { addr := s.Conn().RemoteMultiaddr() ip, port, err := types.GetIPPortFromMultiaddr(addr) if err != nil { - dpm.logger.Warn().Err(err).Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("Can't get ip address and port from inbound peer") + dpm.logger.Warn().Err(err).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("Can't get ip address and port from inbound peer") s.Close() } conn := p2pcommon.RemoteConn{Outbound: false, IP: ip, Port: port} @@ -63,7 +63,7 @@ func (dpm *basePeerManager) OnInboundConn(s network.Stream) { dpm.logger.Info().Str(p2putil.LogFullID, peerID.Pretty()).Str("multiaddr", addr.String()).Msg("new inbound peer arrived") if banned, _ := dpm.lm.IsBanned(ip.String(), peerID); banned { - dpm.logger.Info().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Str("multiaddr", addr.String()).Msg("inbound peer is banned by list manager") + dpm.logger.Info().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Str("multiaddr", addr.String()).Msg("inbound peer is banned by list manager") s.Close() return } @@ -71,7 +71,7 @@ func (dpm *basePeerManager) OnInboundConn(s network.Stream) { query := inboundConnEvent{conn: conn, meta: tempMeta, p2pVer: p2pcommon.P2PVersionUnknown, foundC: make(chan bool)} dpm.pm.inboundConnChan <- query if exist := <-query.foundC; exist { - dpm.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("same peer as inbound peer already exists.") + dpm.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("same peer as inbound peer already exists.") s.Close() return } @@ -133,7 +133,7 @@ func (dpm *basePeerManager) connectWaitingPeers(maxJob int) { // dpm.logger.Info().Str(p2putil.LogPeerName, p2putil.ShortMetaForm(wp.Meta)).Msg("Skipping banned peer") // continue //} - dpm.logger.Info().Int("trial", wp.TrialCnt).Str(p2putil.LogPeerID, p2putil.ShortForm(wp.Meta.ID)).Msg("Starting scheduled try to connect peer") + dpm.logger.Info().Int("trial", wp.TrialCnt).Stringer(p2putil.LogPeerID, types.LogPeerShort(wp.Meta.ID)).Msg("Starting scheduled try to connect peer") dpm.workingJobs[wp.Meta.ID] = ConnWork{Meta: wp.Meta, PeerID: wp.Meta.ID, StartTime: time.Now()} go dpm.runTryOutboundConnect(wp) @@ -168,7 +168,7 @@ func (dpm *basePeerManager) runTryOutboundConnect(wp *p2pcommon.WaitingPeer) { meta := wp.Meta s, err := dpm.getStream(meta) if err != nil { - dpm.logger.Info().Err(err).Str(p2putil.LogPeerID, p2putil.ShortForm(meta.ID)).Msg("Failed to get stream.") + dpm.logger.Info().Err(err).Stringer(p2putil.LogPeerID, types.LogPeerShort(meta.ID)).Msg("Failed to get stream.") workResult.Result = err return } @@ -181,7 +181,7 @@ func (dpm *basePeerManager) runTryOutboundConnect(wp *p2pcommon.WaitingPeer) { return //} else { // if meta.IPAddress != completeMeta.IPAddress { - // dpm.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(completeMeta.ID)).Str("before", meta.IPAddress).Str("after", completeMeta.IPAddress).Msg("IP address of remote peer is changed to ") + // dpm.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(completeMeta.ID)).Str("before", meta.IPAddress).Str("after", completeMeta.IPAddress).Msg("IP address of remote peer is changed to ") // } } } @@ -206,7 +206,7 @@ func (dpm *basePeerManager) getStream(meta p2pcommon.PeerMeta) (network.Stream, func (dpm *basePeerManager) tryAddPeer(outbound bool, meta p2pcommon.PeerMeta, s network.Stream, h p2pcommon.HSHandler) (p2pcommon.PeerMeta, bool) { hResult, err := h.Handle(s, defaultHandshakeTTL) if err != nil { - dpm.logger.Debug().Err(err).Bool("outbound", outbound).Str(p2putil.LogPeerID, p2putil.ShortForm(meta.ID)).Msg("Failed to handshake") + dpm.logger.Debug().Err(err).Bool("outbound", outbound).Stringer(p2putil.LogPeerID, types.LogPeerShort(meta.ID)).Msg("Failed to handshake") return meta, false } @@ -240,7 +240,7 @@ func (dpm *basePeerManager) createRemoteInfo(conn network.Conn, r p2pcommon.Hand if len(r.Certificates) > 0 { ri.AcceptedRole = types.PeerRole_Agent } else { - dpm.logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(r.Meta.ID)).Msg("treat peer which claims agent but with no certificates, as Watcher") + dpm.logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(r.Meta.ID)).Msg("treat peer which claims agent but with no certificates, as Watcher") } default: ri.AcceptedRole = r.Meta.Role diff --git a/polaris/client/polarisconnect.go b/polaris/client/polarisconnect.go index b784e1c8c..58c85b729 100644 --- a/polaris/client/polarisconnect.go +++ b/polaris/client/polarisconnect.go @@ -139,9 +139,9 @@ func (pcs *PolarisConnectSvc) queryPeers(msg *message.MapQueryMsg) *message.MapQ addrs, err := pcs.connectAndQuery(meta, msg.BestBlock.Hash, msg.BestBlock.Header.BlockNo) if err != nil { if err == ErrTooLowVersion { - pcs.Logger.Error().Err(err).Str("polarisID", p2putil.ShortForm(meta.ID)).Msg("Polaris responded this aergosvr is too low, check and upgrade aergosvr") + pcs.Logger.Error().Err(err).Stringer("polarisID", types.LogPeerShort(meta.ID)).Msg("Polaris responded this aergosvr is too low, check and upgrade aergosvr") } else { - pcs.Logger.Warn().Err(err).Str("polarisID", p2putil.ShortForm(meta.ID)).Msg("failed to get peer addresses") + pcs.Logger.Warn().Err(err).Stringer("polarisID", types.LogPeerShort(meta.ID)).Msg("failed to get peer addresses") } continue } diff --git a/polaris/server/mapservice.go b/polaris/server/mapservice.go index 5b8050b2e..b1ad0ad0e 100644 --- a/polaris/server/mapservice.go +++ b/polaris/server/mapservice.go @@ -124,10 +124,10 @@ func (pms *PeerMapService) onConnect(s types.Stream) { peerID := s.Conn().RemotePeer() remoteIP, port, err := types.GetIPPortFromMultiaddr(s.Conn().RemoteMultiaddr()) if err != nil { - pms.Logger.Info().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("Invalid address information") + pms.Logger.Info().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("Invalid address information") return } - pms.Logger.Info().Str("addr", s.Conn().RemoteMultiaddr().String()).Str(p2putil.LogFullID, peerID.Pretty()).Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("remote peer connected") + pms.Logger.Info().Str("addr", s.Conn().RemoteMultiaddr().String()).Str(p2putil.LogFullID, peerID.Pretty()).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("remote peer connected") conn := p2pcommon.RemoteConn{IP: remoteIP, Port: port, Outbound: false} tempMeta := p2pcommon.PeerMeta{ID: peerID, Addresses: []types.Multiaddr{s.Conn().RemoteMultiaddr()}} @@ -137,27 +137,27 @@ func (pms *PeerMapService) onConnect(s types.Stream) { // receive input container, query, err := pms.readRequest(remotePeerInfo, rw) if err != nil { - pms.Logger.Info().Err(err).Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("failed to read query") + pms.Logger.Info().Err(err).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("failed to read query") return } // check blacklist if banned, _ := pms.lm.IsBanned(remoteIP.String(), peerID); banned { - pms.Logger.Info().Str("address", remoteIP.String()).Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("close soon banned peer") + pms.Logger.Info().Str("address", remoteIP.String()).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("close soon banned peer") return } resp, err := pms.handleQuery(conn, container, query) if err != nil { - pms.Logger.Info().Err(err).Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("failed to handle query") + pms.Logger.Info().Err(err).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("failed to handle query") return } // response to peer if err = pms.writeResponse(container, remotePeerInfo, resp, rw); err != nil { - pms.Logger.Info().Err(err).Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("failed to write query") + pms.Logger.Info().Err(err).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("failed to write query") return } - pms.Logger.Debug().Str("status", resp.Status.String()).Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Int("peer_cnt", len(resp.Addresses)).Msg("Sent map response") + pms.Logger.Debug().Str("status", resp.Status.String()).Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Int("peer_cnt", len(resp.Addresses)).Msg("Sent map response") // TODO send goodbye message. time.Sleep(time.Second * 3) @@ -321,7 +321,7 @@ func isEqualMeta(m1, m2 p2pcommon.PeerMeta) (eq bool) { func (pms *PeerMapService) unregisterPeer(peerID types.PeerID) { pms.rwmutex.Lock() defer pms.rwmutex.Unlock() - pms.Logger.Info().Str(p2putil.LogPeerID, p2putil.ShortForm(peerID)).Msg("Unregistering bad peer") + pms.Logger.Info().Stringer(p2putil.LogPeerID, types.LogPeerShort(peerID)).Msg("Unregistering bad peer") delete(pms.peerRegistry, peerID) } @@ -551,10 +551,10 @@ func (pms *PeerMapService) checkConnectness(meta p2pcommon.PeerMeta) bool { tempState := &peerState{PeerMapService: pms, meta: meta, addr: meta.ToPeerAddress(), lCheckTime: time.Now(), temporary: true} _, err := tempState.checkConnect(PolarisPingTTL) if err != nil { - pms.Logger.Debug().Err(err).Str(p2putil.LogPeerID, p2putil.ShortForm(meta.ID)).Msg("Ping check was failed.") + pms.Logger.Debug().Err(err).Stringer(p2putil.LogPeerID, types.LogPeerShort(meta.ID)).Msg("Ping check was failed.") return false } else { - pms.Logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(meta.ID)).Msg("Ping check is succeeded.") + pms.Logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(meta.ID)).Msg("Ping check is succeeded.") return true } } diff --git a/polaris/server/peerstate.go b/polaris/server/peerstate.go index 7aa5c3c63..c256c942a 100644 --- a/polaris/server/peerstate.go +++ b/polaris/server/peerstate.go @@ -73,7 +73,7 @@ func (hc *peerState) check(wg *sync.WaitGroup, timeout time.Duration) { } func (hc *peerState) checkConnect(timeout time.Duration) (*types.Ping, error) { - hc.Logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(hc.meta.ID)).Msg("staring up healthcheck") + hc.Logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(hc.meta.ID)).Msg("staring up healthcheck") hc.lCheckTime = time.Now() s, err := hc.nt.GetOrCreateStreamWithTTL(hc.meta, PolarisPingTTL, common.PolarisPingSub) if err != nil { @@ -171,7 +171,7 @@ func (pc *pingChecker) DoCall(done chan<- interface{}) { return } - pc.Logger.Debug().Str(p2putil.LogPeerID, p2putil.ShortForm(pc.meta.ID)).Interface("ping_resp", pingResp).Msg("Healthcheck finished successful") + pc.Logger.Debug().Stringer(p2putil.LogPeerID, types.LogPeerShort(pc.meta.ID)).Interface("ping_resp", pingResp).Msg("Healthcheck finished successful") return } diff --git a/rpc/rpcstream.go b/rpc/rpcstream.go index 85f3ba08b..44f0320ba 100644 --- a/rpc/rpcstream.go +++ b/rpc/rpcstream.go @@ -115,7 +115,7 @@ SEND_LOOP: if err != nil { logger.Warn().Uint32("streamId", s.id).Err(err).Msg("failed to broadcast block metadata stream") } else { - logger.Trace().Uint32("streamId", s.id).Object("hash", types.LogBase58{Bytes: &block.Hash}).Msg("broadcast new block metadata") + logger.Trace().Uint32("streamId", s.id).Stringer("hash", types.LogBase58(block.Hash)).Msg("broadcast new block metadata") } } } diff --git a/rpc/txputter.go b/rpc/txputter.go index 6754ab448..b5521a8a2 100644 --- a/rpc/txputter.go +++ b/rpc/txputter.go @@ -110,7 +110,7 @@ func (m *txPutter) putToNextTx() int { m.rs[m.offset] = &r calculated := tx.CalculateTxHash() if !bytes.Equal(hash, calculated) { - m.logger.Trace().Object("calculated", types.LogBase58{Bytes: &calculated}).Object("in", types.LogBase58{Bytes: &hash}).Msg("tx hash mismatch") + m.logger.Trace().Stringer("calculated", types.LogBase58(calculated)).Stringer("in", types.LogBase58(hash)).Msg("tx hash mismatch") r.Error = types.CommitStatus_TX_INVALID_HASH } else { f := m.hub.RequestFuture(message.MemPoolSvc, diff --git a/syncer/blockfetcher.go b/syncer/blockfetcher.go index e0a57be5a..fbe64018c 100644 --- a/syncer/blockfetcher.go +++ b/syncer/blockfetcher.go @@ -11,7 +11,6 @@ import ( "github.com/aergoio/aergo/v2/internal/enc" "github.com/aergoio/aergo/v2/message" - "github.com/aergoio/aergo/v2/p2p/p2putil" "github.com/aergoio/aergo/v2/pkg/component" "github.com/aergoio/aergo/v2/types" "github.com/rs/zerolog" @@ -554,7 +553,7 @@ func (bf *BlockFetcher) findFinished(msg *message.GetBlockChunksRsp, peerMatch b if task.isPeerMatched(msg.ToWhom) { bf.runningQueue.Remove(e) - logger.Debug().Str("peer", p2putil.ShortForm(msg.ToWhom)).Err(msg.Err).Str("start", enc.ToString(task.hashes[0])).Int("count", task.count).Int("runqueue", bf.runningQueue.Len()).Msg("task finished with error") + logger.Debug().Stringer("peer", types.LogPeerShort(msg.ToWhom)).Err(msg.Err).Str("start", enc.ToString(task.hashes[0])).Int("count", task.count).Int("runqueue", bf.runningQueue.Len()).Msg("task finished with error") return task, nil } } else { @@ -655,7 +654,7 @@ func (ps *PeerSet) addNew(peerID types.PeerID) { ps.pushFree(&SyncPeer{No: peerno, ID: peerID}) ps.total++ - logger.Info().Str("peer", p2putil.ShortForm(peerID)).Int("peerno", peerno).Int("no", ps.total).Msg("new peer added") + logger.Info().Stringer("peer", types.LogPeerShort(peerID)).Int("peerno", peerno).Int("no", ps.total).Msg("new peer added") } /* diff --git a/syncer/blockprocessor.go b/syncer/blockprocessor.go index 71559b55b..b533bc49c 100644 --- a/syncer/blockprocessor.go +++ b/syncer/blockprocessor.go @@ -8,7 +8,6 @@ import ( "github.com/aergoio/aergo/v2/chain" "github.com/aergoio/aergo/v2/internal/enc" "github.com/aergoio/aergo/v2/message" - "github.com/aergoio/aergo/v2/p2p/p2putil" "github.com/aergoio/aergo/v2/pkg/component" "github.com/aergoio/aergo/v2/types" ) @@ -78,13 +77,13 @@ func (bproc *BlockProcessor) isValidResponse(msg interface{}) error { } if blocks == nil || len(blocks) == 0 { - logger.Error().Err(msg.Err).Str("peer", p2putil.ShortForm(msg.ToWhom)).Msg("GetBlockChunksRsp is empty") + logger.Error().Err(msg.Err).Stringer("peer", types.LogPeerShort(msg.ToWhom)).Msg("GetBlockChunksRsp is empty") return &ErrSyncMsg{msg: msg, str: "blocks is empty"} } for _, block := range blocks { if prev != nil && !bytes.Equal(prev, block.GetHeader().GetPrevBlockHash()) { - logger.Error().Str("peer", p2putil.ShortForm(msg.ToWhom)).Msg("GetBlockChunksRsp hashes inconsistent") + logger.Error().Stringer("peer", types.LogPeerShort(msg.ToWhom)).Msg("GetBlockChunksRsp hashes inconsistent") return &ErrSyncMsg{msg: msg, str: "blocks hash not matched"} } @@ -130,12 +129,12 @@ func (bproc *BlockProcessor) GetBlockChunkRsp(msg *message.GetBlockChunksRsp) er bf := bproc.blockFetcher - logger.Debug().Str("peer", p2putil.ShortForm(msg.ToWhom)).Uint64("startNo", msg.Blocks[0].GetHeader().BlockNo).Int("count", len(msg.Blocks)).Msg("received GetBlockChunkRsp") + logger.Debug().Stringer("peer", types.LogPeerShort(msg.ToWhom)).Uint64("startNo", msg.Blocks[0].GetHeader().BlockNo).Int("count", len(msg.Blocks)).Msg("received GetBlockChunkRsp") task, err := bf.findFinished(msg, false) if err != nil { //TODO invalid peer - logger.Error().Str("peer", p2putil.ShortForm(msg.ToWhom)). + logger.Error().Stringer("peer", types.LogPeerShort(msg.ToWhom)). Int("count", len(msg.Blocks)). Str("from", enc.ToString(msg.Blocks[0].GetHash())). Str("to", enc.ToString(msg.Blocks[len(msg.Blocks)-1].GetHash())). @@ -155,12 +154,12 @@ func (bproc *BlockProcessor) GetBlockChunkRsp(msg *message.GetBlockChunksRsp) er func (bproc *BlockProcessor) GetBlockChunkRspError(msg *message.GetBlockChunksRsp, err error) error { bf := bproc.blockFetcher - logger.Error().Err(err).Str("peer", p2putil.ShortForm(msg.ToWhom)).Msg("receive GetBlockChunksRsp with error message") + logger.Error().Err(err).Stringer("peer", types.LogPeerShort(msg.ToWhom)).Msg("receive GetBlockChunksRsp with error message") task, err := bf.findFinished(msg, true) if err != nil { //TODO invalid peer - logger.Error().Err(err).Str("peer", p2putil.ShortForm(msg.ToWhom)).Msg("dropped unknown block error message") + logger.Error().Err(err).Stringer("peer", types.LogPeerShort(msg.ToWhom)).Msg("dropped unknown block error message") return nil } diff --git a/syncer/finder.go b/syncer/finder.go index dce1ead75..70a179a38 100644 --- a/syncer/finder.go +++ b/syncer/finder.go @@ -8,7 +8,6 @@ import ( "github.com/aergoio/aergo/v2/chain" "github.com/aergoio/aergo/v2/internal/enc" "github.com/aergoio/aergo/v2/message" - "github.com/aergoio/aergo/v2/p2p/p2putil" "github.com/aergoio/aergo/v2/pkg/component" "github.com/aergoio/aergo/v2/types" "github.com/pkg/errors" @@ -171,7 +170,7 @@ func (finder *Finder) getAnchors() ([][]byte, error) { func (finder *Finder) getAncestor(anchors [][]byte) (*types.BlockInfo, error) { // send remote Peer - logger.Debug().Str("peer", p2putil.ShortForm(finder.ctx.PeerID)).Msg("send GetAncestor message to peer") + logger.Debug().Stringer("peer", types.LogPeerShort(finder.ctx.PeerID)).Msg("send GetAncestor message to peer") finder.compRequester.TellTo(message.P2PSvc, &message.GetSyncAncestor{Seq: finder.GetSeq(), ToWhom: finder.ctx.PeerID, Hashes: anchors}) timer := time.NewTimer(finder.dfltTimeout) diff --git a/syncer/stubsyncer.go b/syncer/stubsyncer.go index 8c1ff9274..0558f93d3 100644 --- a/syncer/stubsyncer.go +++ b/syncer/stubsyncer.go @@ -11,7 +11,6 @@ import ( "github.com/aergoio/aergo-actor/actor" "github.com/aergoio/aergo/v2/chain" "github.com/aergoio/aergo/v2/message" - "github.com/aergoio/aergo/v2/p2p/p2putil" "github.com/aergoio/aergo/v2/pkg/component" "github.com/aergoio/aergo/v2/types" "github.com/stretchr/testify/assert" @@ -212,9 +211,9 @@ func (syncer *StubSyncer) GetAnchors(msg *message.GetAnchors) { go func() { if syncer.cfg.debugContext.debugFinder { if syncer.stubPeers[0].timeDelaySec > 0 { - logger.Debug().Str("peer", p2putil.ShortForm(types.PeerID(syncer.stubPeers[0].addr.PeerID))).Msg("slow target peer sleep") + logger.Debug().Stringer("peer", types.LogPeerShort(types.PeerID(syncer.stubPeers[0].addr.PeerID))).Msg("slow target peer sleep") time.Sleep(syncer.stubPeers[0].timeDelaySec) - logger.Debug().Str("peer", p2putil.ShortForm(types.PeerID(syncer.stubPeers[0].addr.PeerID))).Msg("slow target peer wakeup") + logger.Debug().Stringer("peer", types.LogPeerShort(types.PeerID(syncer.stubPeers[0].addr.PeerID))).Msg("slow target peer wakeup") } } @@ -274,9 +273,9 @@ func (syncer *StubSyncer) GetBlockChunks(msg *message.GetBlockChunks) { } go func() { if stubPeer.timeDelaySec > 0 { - logger.Debug().Str("peer", p2putil.ShortForm(types.PeerID(stubPeer.addr.PeerID))).Msg("slow peer sleep") + logger.Debug().Stringer("peer", types.LogPeerShort(types.PeerID(stubPeer.addr.PeerID))).Msg("slow peer sleep") time.Sleep(stubPeer.timeDelaySec) - logger.Debug().Str("peer", p2putil.ShortForm(types.PeerID(stubPeer.addr.PeerID))).Msg("slow peer wakeup") + logger.Debug().Stringer("peer", types.LogPeerShort(types.PeerID(stubPeer.addr.PeerID))).Msg("slow peer wakeup") } //send reply @@ -305,7 +304,7 @@ func (syncer *StubSyncer) findStubPeer(peerID types.PeerID) *StubPeer { } } - logger.Error().Str("peer", p2putil.ShortForm(peerID)).Msg("can't find peer") + logger.Error().Stringer("peer", types.LogPeerShort(peerID)).Msg("can't find peer") panic("peer find fail") } diff --git a/syncer/syncerservice.go b/syncer/syncerservice.go index ce489fcf2..7a5624cae 100644 --- a/syncer/syncerservice.go +++ b/syncer/syncerservice.go @@ -12,7 +12,6 @@ import ( "github.com/aergoio/aergo/v2/chain" cfg "github.com/aergoio/aergo/v2/config" "github.com/aergoio/aergo/v2/message" - "github.com/aergoio/aergo/v2/p2p/p2putil" "github.com/aergoio/aergo/v2/pkg/component" "github.com/aergoio/aergo/v2/types" "github.com/pkg/errors" @@ -316,7 +315,7 @@ func (syncer *Syncer) handleSyncStart(msg *message.SyncStart) error { var err error var bestBlock *types.Block - logger.Debug().Uint64("targetNo", msg.TargetNo).Str("peer", p2putil.ShortForm(msg.PeerID)).Msg("syncer requested") + logger.Debug().Uint64("targetNo", msg.TargetNo).Stringer("peer", types.LogPeerShort(msg.PeerID)).Msg("syncer requested") if syncer.isRunning { logger.Debug().Uint64("targetNo", msg.TargetNo).Msg("skipped syncer is running") diff --git a/tests/bp01.id b/tests/bp01.id new file mode 100644 index 000000000..e6b88a555 --- /dev/null +++ b/tests/bp01.id @@ -0,0 +1 @@ +16Uiu2HAmG4PSXYUxkPbNb7qTcEExFpgAwBrm3hB32aJXuvX2f1sd \ No newline at end of file diff --git a/tests/bp01.key b/tests/bp01.key new file mode 100644 index 000000000..8ab66a8aa --- /dev/null +++ b/tests/bp01.key @@ -0,0 +1 @@ + A<��iT�B-:�1J �\�G��1�� T'�� \ No newline at end of file diff --git a/tests/bp01.pub b/tests/bp01.pub new file mode 100644 index 000000000..d42833955 Binary files /dev/null and b/tests/bp01.pub differ diff --git a/tests/bp02.id b/tests/bp02.id new file mode 100644 index 000000000..102b8a198 --- /dev/null +++ b/tests/bp02.id @@ -0,0 +1 @@ +16Uiu2HAmMzncFmnpjigZJRoraToKkABvZimMUAyXf6bdrZeN7mbJ \ No newline at end of file diff --git a/tests/bp02.key b/tests/bp02.key new file mode 100644 index 000000000..9c9f8c626 --- /dev/null +++ b/tests/bp02.key @@ -0,0 +1,2 @@ + �5 +&���d�f�#[k��1~� �Ľ�=�� \ No newline at end of file diff --git a/tests/bp02.pub b/tests/bp02.pub new file mode 100644 index 000000000..f9627f3fc --- /dev/null +++ b/tests/bp02.pub @@ -0,0 +1 @@ +!�˅����:^�p�B�X̾����2/���~� \ No newline at end of file diff --git a/tests/bp03.id b/tests/bp03.id new file mode 100644 index 000000000..ce5e264a5 --- /dev/null +++ b/tests/bp03.id @@ -0,0 +1 @@ +16Uiu2HAmKB7RYXe1uHNYMtkuuM2fEHxsv6P9PZ45ogJw6aZD3y7x \ No newline at end of file diff --git a/tests/bp03.key b/tests/bp03.key new file mode 100644 index 000000000..769f934f5 --- /dev/null +++ b/tests/bp03.key @@ -0,0 +1 @@ + ⧾o�{V�IԖu�Q���7��6$M���p�G� \ No newline at end of file diff --git a/tests/bp03.pub b/tests/bp03.pub new file mode 100644 index 000000000..8704fcc3d --- /dev/null +++ b/tests/bp03.pub @@ -0,0 +1 @@ +!`݋/�|t�i��+�յ͚��B�5mHK4[EC \ No newline at end of file diff --git a/tests/common.sh b/tests/common.sh index d0884c4b4..632e18d1d 100644 --- a/tests/common.sh +++ b/tests/common.sh @@ -1,4 +1,41 @@ +start_nodes() { + + if [ "$consensus" == "sbp" ]; then + # open the aergo node in testmode + ../bin/aergosvr --testmode --home ./aergo-files > logs 2> logs & + pid=$! + else + # open the 3 nodes + ../bin/aergosvr --home ./node1 >> logs1 2>> logs1 & + pid1=$! + ../bin/aergosvr --home ./node2 >> logs2 2>> logs2 & + pid2=$! + ../bin/aergosvr --home ./node3 >> logs3 2>> logs3 & + pid3=$! + fi + + # wait the node(s) to be ready + if [ "$consensus" == "sbp" ]; then + sleep 3 + elif [ "$consensus" == "dpos" ]; then + sleep 5 + elif [ "$consensus" == "raft" ]; then + sleep 2 + fi + +} + +stop_nodes() { + + if [ "$consensus" == "sbp" ]; then + kill $pid + else + kill $pid1 $pid2 $pid3 + fi + +} + get_deploy_args() { contract_file=$1 @@ -27,12 +64,12 @@ get_receipt() { set +e while true; do - output=$(../bin/aergocli receipt get $txhash 2>&1 > receipt.json) + output=$(../bin/aergocli receipt get --port $query_port $txhash 2>&1 > receipt.json) #echo "output: $output" if [[ $output == *"tx not found"* ]]; then - sleep 0.5 + sleep 0.4 counter=$((counter+1)) if [ $counter -gt 10 ]; then echo "Error: tx not found: $txhash" diff --git a/tests/config-node1.toml b/tests/config-node1.toml new file mode 100644 index 000000000..8ffb68f7e --- /dev/null +++ b/tests/config-node1.toml @@ -0,0 +1,48 @@ +# aergo TOML Configuration File (https://github.com/toml-lang/toml) +# base configurations +enableprofile = false + +[rpc] +netserviceaddr = "0.0.0.0" +netserviceport = 7845 +netservicetrace = false +nstls = false +nscert = "" +nskey = "" +nsallowcors = false + +[p2p] +netprotocoladdr = "127.0.0.1" +netprotocolport = 2001 +npbindaddr = "0.0.0.0" +npbindport = 2001 +nptls = false +npcert = "" +npkey = "bp01.key" +npaddpeers = [ +#"/ip4/127.0.0.1/tcp/2001/p2p/16Uiu2HAmG4PSXYUxkPbNb7qTcEExFpgAwBrm3hB32aJXuvX2f1sd", +"/ip4/127.0.0.1/tcp/2002/p2p/16Uiu2HAmMzncFmnpjigZJRoraToKkABvZimMUAyXf6bdrZeN7mbJ", +"/ip4/127.0.0.1/tcp/2003/p2p/16Uiu2HAmKB7RYXe1uHNYMtkuuM2fEHxsv6P9PZ45ogJw6aZD3y7x" +] +npexposeself = false +npdiscoverpeers = false +npusepolaris = false +peerrole = "producer" + +[blockchain] +maxblocksize = 1000000 + +[mempool] +showmetrics = false + +[consensus] +enablebp = true + +[consensus.raft] +newcluster=true +name="bp01" + +[hardfork] +v2 = "0" +v3 = "10000" +v4 = "10000" diff --git a/tests/config-node2.toml b/tests/config-node2.toml new file mode 100644 index 000000000..f68988e37 --- /dev/null +++ b/tests/config-node2.toml @@ -0,0 +1,48 @@ +# aergo TOML Configuration File (https://github.com/toml-lang/toml) +# base configurations +enableprofile = false + +[rpc] +netserviceaddr = "0.0.0.0" +netserviceport = 8845 +netservicetrace = false +nstls = false +nscert = "" +nskey = "" +nsallowcors = false + +[p2p] +netprotocoladdr = "127.0.0.1" +netprotocolport = 2002 +npbindaddr = "0.0.0.0" +npbindport = 2002 +nptls = false +npcert = "" +npkey = "bp02.key" +npaddpeers = [ +"/ip4/127.0.0.1/tcp/2001/p2p/16Uiu2HAmG4PSXYUxkPbNb7qTcEExFpgAwBrm3hB32aJXuvX2f1sd", +#"/ip4/127.0.0.1/tcp/2002/p2p/16Uiu2HAmMzncFmnpjigZJRoraToKkABvZimMUAyXf6bdrZeN7mbJ", +"/ip4/127.0.0.1/tcp/2003/p2p/16Uiu2HAmKB7RYXe1uHNYMtkuuM2fEHxsv6P9PZ45ogJw6aZD3y7x" +] +npexposeself = false +npdiscoverpeers = false +npusepolaris = false +peerrole = "producer" + +[blockchain] +maxblocksize = 1000000 + +[mempool] +showmetrics = false + +[consensus] +enablebp = true + +[consensus.raft] +newcluster=true +name="bp02" + +[hardfork] +v2 = "0" +v3 = "10000" +v4 = "10000" diff --git a/tests/config-node3.toml b/tests/config-node3.toml new file mode 100644 index 000000000..888aca265 --- /dev/null +++ b/tests/config-node3.toml @@ -0,0 +1,48 @@ +# aergo TOML Configuration File (https://github.com/toml-lang/toml) +# base configurations +enableprofile = false + +[rpc] +netserviceaddr = "0.0.0.0" +netserviceport = 9845 +netservicetrace = false +nstls = false +nscert = "" +nskey = "" +nsallowcors = false + +[p2p] +netprotocoladdr = "127.0.0.1" +netprotocolport = 2003 +npbindaddr = "0.0.0.0" +npbindport = 2003 +nptls = false +npcert = "" +npkey = "bp03.key" +npaddpeers = [ +"/ip4/127.0.0.1/tcp/2001/p2p/16Uiu2HAmG4PSXYUxkPbNb7qTcEExFpgAwBrm3hB32aJXuvX2f1sd", +"/ip4/127.0.0.1/tcp/2002/p2p/16Uiu2HAmMzncFmnpjigZJRoraToKkABvZimMUAyXf6bdrZeN7mbJ" +#"/ip4/127.0.0.1/tcp/2003/p2p/16Uiu2HAmKB7RYXe1uHNYMtkuuM2fEHxsv6P9PZ45ogJw6aZD3y7x" +] +npexposeself = false +npdiscoverpeers = false +npusepolaris = false +peerrole = "producer" + +[blockchain] +maxblocksize = 1000000 + +[mempool] +showmetrics = false + +[consensus] +enablebp = true + +[consensus.raft] +newcluster=true +name="bp03" + +[hardfork] +v2 = "0" +v3 = "10000" +v4 = "10000" diff --git a/tests/config.toml b/tests/config-sbp.toml similarity index 100% rename from tests/config.toml rename to tests/config-sbp.toml diff --git a/tests/genesis-dpos.json b/tests/genesis-dpos.json new file mode 100644 index 000000000..da860252f --- /dev/null +++ b/tests/genesis-dpos.json @@ -0,0 +1,17 @@ +{ + "chain_id":{ + "magic":"test.chain", + "public":true, + "mainnet":false, + "consensus":"dpos" + }, + "timestamp": 1559883600000000000, + "balance": { + "AmPpcKvToDCUkhT1FJjdbNvR4kNDhLFJGHkSqfjWe3QmHm96qv4R": "1000000000000000000000" + }, + "bps": [ + "16Uiu2HAmG4PSXYUxkPbNb7qTcEExFpgAwBrm3hB32aJXuvX2f1sd", + "16Uiu2HAmMzncFmnpjigZJRoraToKkABvZimMUAyXf6bdrZeN7mbJ", + "16Uiu2HAmKB7RYXe1uHNYMtkuuM2fEHxsv6P9PZ45ogJw6aZD3y7x" + ] +} diff --git a/tests/genesis-raft.json b/tests/genesis-raft.json new file mode 100644 index 000000000..c996b3590 --- /dev/null +++ b/tests/genesis-raft.json @@ -0,0 +1,31 @@ +{ + "chain_id":{ + "magic":"test.chain", + "public":true, + "mainnet":false, + "consensus":"raft" + }, + "timestamp": 1559883600000000000, + "balance": { + "AmPpcKvToDCUkhT1FJjdbNvR4kNDhLFJGHkSqfjWe3QmHm96qv4R": "1000000000000000000000" + }, + "bps": [ + ], + "enterprise_bps": [ + { + "name": "bp01", + "address": "/ip4/127.0.0.1/tcp/2001", + "peerid": "16Uiu2HAmG4PSXYUxkPbNb7qTcEExFpgAwBrm3hB32aJXuvX2f1sd" + }, + { + "name": "bp02", + "address": "/ip4/127.0.0.1/tcp/2002", + "peerid": "16Uiu2HAmMzncFmnpjigZJRoraToKkABvZimMUAyXf6bdrZeN7mbJ" + }, + { + "name": "bp03", + "address": "/ip4/127.0.0.1/tcp/2003", + "peerid": "16Uiu2HAmKB7RYXe1uHNYMtkuuM2fEHxsv6P9PZ45ogJw6aZD3y7x" + } + ] +} diff --git a/tests/run_tests.sh b/tests/run_tests.sh index bb5defb65..dfb7dd55c 100755 --- a/tests/run_tests.sh +++ b/tests/run_tests.sh @@ -1,23 +1,72 @@ # stop on errors set -e +source common.sh -# run the brick test -./test-brick.sh +arg=$1 +if [ "$arg" != "sbp" ] && [ "$arg" != "dpos" ] && [ "$arg" != "raft" ] && [ "$arg" != "brick" ]; then + echo "Usage: $0 [brick|sbp|dpos|raft]" + exit 1 +fi +echo "Running integration tests for $arg" + +if [ "$arg" == "brick" ]; then + # run the brick test + ../bin/brick -V test.brick + exit 0 +fi + +consensus=$arg + +if [ "$consensus" == "sbp" ]; then + # delete and recreate the aergo folder + rm -rf ./aergo-files + mkdir aergo-files + # copy the config file + cp config-sbp.toml ./aergo-files/config.toml + # delete the old logs + rm -f logs +else + # delete and recreate the aergo folder + rm -rf node1 + rm -rf node2 + rm -rf node3 + mkdir node1 + mkdir node2 + mkdir node3 + # copy the config files + cp config-node1.toml node1/config.toml + cp config-node2.toml node2/config.toml + cp config-node3.toml node3/config.toml + # delete the old logs + rm -f logs1 logs2 logs3 + # create the genesis block + echo "creating genesis block..." + ../bin/aergosvr init --genesis ./genesis-$consensus.json --home ./node1 + ../bin/aergosvr init --genesis ./genesis-$consensus.json --home ./node2 + ../bin/aergosvr init --genesis ./genesis-$consensus.json --home ./node3 +fi -# delete and recreate the aergo folder -rm -rf ./aergo-files -mkdir aergo-files -# copy the config file -cp config.toml ./aergo-files/ +# define the config files according to the consensus +if [ "$consensus" == "sbp" ]; then + config_files=("./aergo-files/config.toml") +elif [ "$consensus" == "dpos" ]; then + config_files=("./node1/config.toml" "./node2/config.toml" "./node3/config.toml") +elif [ "$consensus" == "raft" ]; then + config_files=("./node1/config.toml" "./node2/config.toml" "./node3/config.toml") +fi + +# define which port used for queries +if [ "$consensus" == "sbp" ]; then + query_port="7845" +else + query_port="9845" +fi -# open the aergo server in testmode to create the config file echo "" -echo "starting the aergo server..." -../bin/aergosvr --testmode --home ./aergo-files > logs 2> logs & -pid=$! -# wait it to be ready -sleep 2 +echo "starting nodes..." +start_nodes +# get the current hardfork version version=$(../bin/aergocli blockchain | jq .ChainInfo.Chainid.Version | sed 's/"//g') if [ $version -ne 2 ]; then echo "Wrong hardfork version!" @@ -36,17 +85,16 @@ function set_version() { block_no=$(../bin/aergocli blockchain | jq .Height | sed 's/"//g') # increment 2 numbers block_no=$((block_no+2)) - # terminate the server process - kill $pid - # save the hardfork config on the config file - echo "updating the config file..." - sed -i "s/^v${version} = \"10000\"$/v${version} = \"${block_no}\"/" ./aergo-files/config.toml + # terminate the server process(es) + stop_nodes + # save the hardfork config on the config file(s) + echo "updating the config file(s)..." + for config_file in "${config_files[@]}"; do + sed -i "s/^v$version = \"10000\"$/v$version = \"${block_no}\"/" $config_file + done # restart the aergo server - echo "restarting the aergo server..." - ../bin/aergosvr --testmode --home ./aergo-files > logs 2> logs & - pid=$! - # wait it to be ready - sleep 3 + echo "restarting the aergo nodes..." + start_nodes # check if it worked new_version=$(../bin/aergocli blockchain | jq .ChainInfo.Chainid.Version | sed 's/"//g') if [ $new_version -ne $version ]; then @@ -78,6 +126,10 @@ function check() { fi } +# make these variables accessible to the called scripts +export consensus +export query_port + # create the account used on tests echo "creating user account..." ../bin/aergocli account import --keystore . --if 47zh1byk8MqWkQo5y8dvbrex99ZMdgZqfydar7w2QQgQqc7YrmFsBuMeF1uHWa5TwA1ZwQ7V6 --password bmttest @@ -112,12 +164,13 @@ check ./test-gas-op.sh check ./test-gas-bf.sh check ./test-gas-verify-proof.sh check ./test-gas-per-function-v4.sh +check ./test-contract-deploy.sh # terminate the server process echo "" -echo "closing the aergo server" +echo "closing the aergo nodes" echo "" -kill $pid +stop_nodes # print the summary if [ $num_failed_tests -gt 0 ]; then diff --git a/tests/test-brick.sh b/tests/test-brick.sh deleted file mode 100755 index 155dd32c6..000000000 --- a/tests/test-brick.sh +++ /dev/null @@ -1 +0,0 @@ -../bin/brick -V test.brick diff --git a/tests/test-gas-per-function-v2.sh b/tests/test-gas-per-function-v2.sh index 3fd16a4a8..40091b577 100755 --- a/tests/test-gas-per-function-v2.sh +++ b/tests/test-gas-per-function-v2.sh @@ -16,9 +16,26 @@ address=$(cat receipt.json | jq .contractAddress | sed 's/"//g') assert_equals "$status" "CREATED" +echo "-- transfer funds to the contract --" + +from=AmPpcKvToDCUkhT1FJjdbNvR4kNDhLFJGHkSqfjWe3QmHm96qv4R + +txhash=$(../bin/aergocli --keystore . --password bmttest \ + sendtx --from $from --to $address --amount 5aergo \ + | jq .hash | sed 's/"//g') + +get_receipt $txhash + +status=$(cat receipt.json | jq .status | sed 's/"//g') +ret=$(cat receipt.json | jq .ret | sed 's/"//g') + +assert_equals "$status" "SUCCESS" +assert_equals "$ret" "{}" + + echo "-- get account's nonce --" -account_state=$(../bin/aergocli getstate --address AmPpcKvToDCUkhT1FJjdbNvR4kNDhLFJGHkSqfjWe3QmHm96qv4R) +account_state=$(../bin/aergocli getstate --address $from) nonce=$(echo $account_state | jq .nonce | sed 's/"//g') @@ -141,7 +158,7 @@ add_test "system.getCreator" 135156 add_test "system.getOrigin" 135656 add_test "contract.send" 135716 -add_test "contract.balance" 135797 +#add_test "contract.balance" 135797 add_test "contract.deploy" 158752 add_test "contract.call" 149642 add_test "contract.pcall" 150563 diff --git a/tests/test-gas-per-function-v3.sh b/tests/test-gas-per-function-v3.sh index abe5ad2ee..35c39caff 100755 --- a/tests/test-gas-per-function-v3.sh +++ b/tests/test-gas-per-function-v3.sh @@ -16,9 +16,26 @@ address=$(cat receipt.json | jq .contractAddress | sed 's/"//g') assert_equals "$status" "CREATED" +echo "-- transfer funds to the contract --" + +from=AmPpcKvToDCUkhT1FJjdbNvR4kNDhLFJGHkSqfjWe3QmHm96qv4R + +txhash=$(../bin/aergocli --keystore . --password bmttest \ + sendtx --from $from --to $address --amount 5aergo \ + | jq .hash | sed 's/"//g') + +get_receipt $txhash + +status=$(cat receipt.json | jq .status | sed 's/"//g') +ret=$(cat receipt.json | jq .ret | sed 's/"//g') + +assert_equals "$status" "SUCCESS" +assert_equals "$ret" "{}" + + echo "-- get account's nonce --" -account_state=$(../bin/aergocli getstate --address AmPpcKvToDCUkhT1FJjdbNvR4kNDhLFJGHkSqfjWe3QmHm96qv4R) +account_state=$(../bin/aergocli getstate --address $from) nonce=$(echo $account_state | jq .nonce | sed 's/"//g') @@ -141,7 +158,7 @@ add_test "system.getCreator" 135156 add_test "system.getOrigin" 135656 add_test "contract.send" 135716 -add_test "contract.balance" 135797 +#add_test "contract.balance" 135797 add_test "contract.deploy" 158752 add_test "contract.call" 149642 add_test "contract.pcall" 150563 diff --git a/tests/test-gas-per-function-v4.sh b/tests/test-gas-per-function-v4.sh index 6613ff97a..0d8a95978 100755 --- a/tests/test-gas-per-function-v4.sh +++ b/tests/test-gas-per-function-v4.sh @@ -6,11 +6,7 @@ fork_version=$1 echo "-- deploy --" -../bin/aergoluac --payload ../contract/vm_dummy/test_files/gas_per_function.lua > payload.out - -txhash=$(../bin/aergocli --keystore . --password bmttest \ - contract deploy AmPpcKvToDCUkhT1FJjdbNvR4kNDhLFJGHkSqfjWe3QmHm96qv4R \ - --payload `cat payload.out` | jq .hash | sed 's/"//g') +deploy ../contract/vm_dummy/test_files/gas_per_function.lua get_receipt $txhash @@ -20,9 +16,26 @@ address=$(cat receipt.json | jq .contractAddress | sed 's/"//g') assert_equals "$status" "CREATED" +echo "-- transfer funds to the contract --" + +from=AmPpcKvToDCUkhT1FJjdbNvR4kNDhLFJGHkSqfjWe3QmHm96qv4R + +txhash=$(../bin/aergocli --keystore . --password bmttest \ + sendtx --from $from --to $address --amount 5aergo \ + | jq .hash | sed 's/"//g') + +get_receipt $txhash + +status=$(cat receipt.json | jq .status | sed 's/"//g') +ret=$(cat receipt.json | jq .ret | sed 's/"//g') + +assert_equals "$status" "SUCCESS" +assert_equals "$ret" "{}" + + echo "-- get account's nonce --" -account_state=$(../bin/aergocli getstate --address AmPpcKvToDCUkhT1FJjdbNvR4kNDhLFJGHkSqfjWe3QmHm96qv4R) +account_state=$(../bin/aergocli getstate --address $from) nonce=$(echo $account_state | jq .nonce | sed 's/"//g') @@ -145,7 +158,7 @@ add_test "system.getCreator" 143761 add_test "system.getOrigin" 144261 add_test "contract.send" 144321 -add_test "contract.balance" 144402 +#add_test "contract.balance" 144402 add_test "contract.deploy" 168092 add_test "contract.call" 159738 add_test "contract.pcall" 160659 diff --git a/tests/timeout/README b/tests/timeout/README new file mode 100644 index 000000000..8479c7613 --- /dev/null +++ b/tests/timeout/README @@ -0,0 +1,8 @@ +This files is contract sources for testing timeout. + +The timeout test is not automated yet, and you should test it manually. + +1. build your own chain (make local private network) +2. compile and deploy both callercontract.lua and longloop.lua +3. call upgradeto of `callercontract` contract with param `longloop` contract +4. call `justLoop` function on `longloop` , `invoke` and `directInvoke` on `callercontract` and watch log of aergo server or debug with IDE . \ No newline at end of file diff --git a/tests/timeout/callercontract.lua b/tests/timeout/callercontract.lua new file mode 100644 index 000000000..0e48f455e --- /dev/null +++ b/tests/timeout/callercontract.lua @@ -0,0 +1,104 @@ +state.var { + _version = state.value(), + _implementation = state.value(), +} + +-- A internal type check function +-- @type internal +-- @param x variable to check +-- @param t (string) expected type +local function _typecheck(x, t) + if (x and t == 'address') then + assert(type(x) == 'string', string.format("%s must be string type", t)) + -- check address length + assert(52 == #x, string.format("invalid address length: %s (%s)", x, #x)) + -- check character + local invalidChar = string.match(x, '[^123456789ABCDEFGHJKLMNPQRSTUVWXYZabcdefghijkmnopqrstuvwxyz]') + assert(nil == invalidChar, string.format("invalid address format: %s contains invalid char %s", x, invalidChar or 'nil')) + else + -- check default lua types + assert(type(x) == t, string.format("invalid type: %s != %s", type(x), t or 'nil')) + end +end + +function _onlyProxyOwner() + assert(system.getCreator() == system.getSender(), string.format("only proxy owner. Owner: %s | sender: %s", system.getCreator(), system.getSender())) +end + +-- Tells the address of the proxy owner +-- @type query +-- @return (address) The address of the proxy owner +function proxyOwner() + return system.getCreator() +end +-- Allows the upgradeability owner to upgrade the current version of the proxy. +-- @type call +-- @param version (string) Representing the version name of the new implementation to be set. +-- @param implementation (contractAddress) Representing the address of the new implementation to be set. +-- @event Upgraded(version, implementation) +function upgradeTo(version, implementation) + _onlyProxyOwner() + _upgradeTo(version, implementation) + contract.delegatecall(_implementation:get(), "init") +end +function _upgradeTo(version, implementation) + _typecheck(version, 'string') + _typecheck(implementation, 'address') + assert(system.isContract(implementation), string.format("[upgradeTo] invalid address format: %s", implementation)) + assert(_implementation:get() ~= implementation, string.format("[upgradeTo] same implementation address %s", implementation)) + _version:set(version) + _implementation:set(implementation) + contract.event("Upgraded", version, implementation) +end + +-- Tells the version name of the current implementation +-- @type query +-- @return (string) Representing the name of the current version +function version() + return _version:get() +end +-- Tells the address of the implementation where every call will be delegated. +-- @type query +-- @return (address) of the implementation to which it will be delegated +function implementation() + return _implementation:get() +end +function default() +end +function invoke(callName, ...) + assert(nil ~= _implementation:get(), "implementation is nil") + if nil == callName then + callName = "default" + end + return contract.delegatecall(_implementation:get(), callName, ...) +end + +-- Used to clearly show that it is a query-only +function query(callName, ...) + assert(nil ~= _implementation:get(), "implementation is nil") + assert(nil ~= callName, "callName is nil") + return contract.delegatecall(_implementation:get(), callName, ...) +end + +function directInvoke(callName, ...) + assert(nil ~= _implementation:get(), "implementation is nil") + if nil == callName then + callName = "default" + end + return contract.call(_implementation:get(), callName, ...) +end + +-- Used to clearly show that it is a query-only +function directQuery(callName, ...) + assert(nil ~= _implementation:get(), "implementation is nil") + assert(nil ~= callName, "callName is nil") + return contract.call(_implementation:get(), callName, ...) +end + +function check_delegation(fname, ...) + return contract.delegatecall(_implementation:get(), "checkDelegation", ...) +end + +abi.register(upgradeTo, invoke, directInvoke) +abi.register_view(proxyOwner, version, implementation, query, directQuery) +abi.fee_delegation(invoke, directInvoke) diff --git a/tests/timeout/longloop.lua b/tests/timeout/longloop.lua new file mode 100644 index 000000000..f2db441e4 --- /dev/null +++ b/tests/timeout/longloop.lua @@ -0,0 +1,32 @@ +-- state dbs +state.var { + _t = state.value(), -- 의미없는 값 +} + +function init() + _t:set(0) +end + +function justLoop(count) + local t = 0 + while t < count do + t = t + 1 + _t:set(t) + end + return t +end + +function t() + return _t:get() +end + +function catch(count) + return pcall(justLoop, count) +end + +function contract_catch(count) + return contract.pcall(justLoop, count) +end + +abi.register(init, justLoop, catch, contract_catch) +abi.register_view(t) diff --git a/tests/timeout/proxy.lua b/tests/timeout/proxy.lua new file mode 100644 index 000000000..bca9995ba --- /dev/null +++ b/tests/timeout/proxy.lua @@ -0,0 +1,130 @@ +-- A internal type check function +-- @type internal +-- @param x variable to check +-- @param t (string) expected type +local function _typecheck(x, t) + if (x and t == 'address') then + assert(type(x) == 'string', string.format("%s must be string type", t)) + -- check address length + assert(52 == #x, string.format("invalid address length: %s (%s)", x, #x)) + -- check character + local invalidChar = string.match(x, '[^123456789ABCDEFGHJKLMNPQRSTUVWXYZabcdefghijkmnopqrstuvwxyz]') + assert(nil == invalidChar, string.format("invalid address format: %s contains invalid char %s", x, invalidChar or 'nil')) + else + -- check default lua types + assert(type(x) == t, string.format("invalid type: %s != %s", type(x), t or 'nil')) + end +end +state.var { + _version = state.value(), + _implementation = state.value(), + _payableList = state.map(), + _payableArray = state.value() +} +function _onlyProxyOwner() + assert(system.getCreator() == system.getSender(), string.format("only proxy owner. Owner: %s | sender: %s", system.getCreator(), system.getSender())) +end +-- Tells the address of the proxy owner +-- @type query +-- @return (address) The address of the proxy owner +function proxyOwner() + return system.getCreator() +end +-- Allows the upgradeability owner to upgrade the current version of the proxy. +-- @type call +-- @param version (string) Representing the version name of the new implementation to be set. +-- @param implementation (contractAddress) Representing the address of the new implementation to be set. +-- @event UpgradedPayableList() +-- @event Upgraded(version, implementation) +function upgradeTo(version, implementation, payableList) + _onlyProxyOwner() + _upgradePayableList(payableList) + _upgradeTo(version, implementation) + contract.delegatecall(_implementation:get(), "init") +end +function _upgradeTo(version, implementation) + _typecheck(version, 'string') + _typecheck(implementation, 'address') + assert(system.isContract(implementation), string.format("[upgradeTo] invalid address format: %s", implementation)) + assert(_implementation:get() ~= implementation, string.format("[upgradeTo] same implementation address %s", implementation)) + _version:set(version) + _implementation:set(implementation) + contract.event("Upgraded", version, implementation) +end +function _deletePayableList(oldList) + if nil ~= _payableArray:get() then + for i, payable in ipairs(oldList) do + _payableList:delete(payable) + end + end +end +function _setPayableList(newList) + _payableArray:set(newList) + if nil ~= newList then + for i, payable in ipairs(newList) do + _payableList[payable] = true + end + end +end +function _upgradePayableList(newList) + assert(nil == newList or 'table' == type(newList), "invalid payable list") + local oldList = _payableArray:get() + _deletePayableList(oldList) + _setPayableList(newList) + -- temporary commit + -- contract.event("UpgradedPayableList", oldList, newList) + contract.event("UpgradedPayableList") +end +function upgradePayableList(newList) + _onlyProxyOwner() + _upgradePayableList(newList) +end +function payableList() + return _payableArray:get() +end +function _isPayable(name) + return _payableList[name] +end +function refund(addr, amount) + _onlyProxyOwner() + _typecheck(addr, 'address') + contract.send(addr, amount) +end +-- Tells the version name of the current implementation +-- @type query +-- @return (string) Representing the name of the current version +function version() + return _version:get() +end +-- Tells the address of the implementation where every call will be delegated. +-- @type query +-- @return (address) of the implementation to which it will be delegated +function implementation() + return _implementation:get() +end +function default() +end +function invoke(callName, ...) + assert(nil ~= _implementation:get(), "implementation is nil") + if nil == callName then + callName = "default" + end + if not bignum.iszero(system.getAmount()) then + assert(_isPayable(callName), string.format("[%s] is not payable.", callName)) + end + return contract.delegatecall(_implementation:get(), callName, ...) +end +-- Used to clearly show that it is a query-only +function query(callName, ...) + assert(nil ~= _implementation:get(), "implementation is nil") + assert(nil ~= callName, "callName is nil") + return contract.delegatecall(_implementation:get(), callName, ...) +end + +function check_delegation(fname, ...) + return contract.delegatecall(_implementation:get(), "checkDelegation", ...) +end +abi.register(upgradeTo, upgradePayableList, refund) +abi.register_view(proxyOwner, version, implementation, payableList, query) +abi.payable(default,invoke) +abi.fee_delegation(invoke) diff --git a/types/logging.go b/types/logging.go index 4c21ce63d..329b24c40 100644 --- a/types/logging.go +++ b/types/logging.go @@ -7,7 +7,6 @@ package types import ( "fmt" - "github.com/aergoio/aergo/v2/internal/enc" "github.com/rs/zerolog" ) @@ -60,10 +59,28 @@ func marshalTrx(tr Transaction, a *zerolog.Array) { } } -type LogBase58 struct { - Bytes *[]byte +// LogBase58 is thin wrapper which show base58 encoded string of byte array +type LogBase58 []byte + +func (t LogBase58) String() string { + return enc.ToString(t) +} + +// LogAddr is thin wrapper which show base58 encoded form of wallet or smart contract +type LogAddr Address + +func (t LogAddr) String() string { + return EncodeAddress(t) } -func (t LogBase58) MarshalZerologObject(e *zerolog.Event) { - e.Str("b58", enc.ToString(*t.Bytes)) +type LogPeerShort PeerID + +func (t LogPeerShort) String() string { + // basically this function is same as function p2putils.ShortForm() + pretty := PeerID(t).Pretty() + if len(pretty) > 10 { + return fmt.Sprintf("%s*%s", pretty[:2], pretty[len(pretty)-6:]) + } else { + return pretty + } } diff --git a/types/logging_test.go b/types/logging_test.go new file mode 100644 index 000000000..7c1bfb964 --- /dev/null +++ b/types/logging_test.go @@ -0,0 +1,97 @@ +package types + +import ( + "github.com/aergoio/aergo-lib/log" + "github.com/aergoio/aergo/v2/internal/enc" + "github.com/rs/zerolog" + "testing" +) + +func BenchmarkLogMemAllocationCompared(b *testing.B) { + type fields struct { + Bytes *[]byte + } + logger := log.NewLogger("benchmark.logger") + + for i := 0; i < b.N; i++ { + sampleBytes := "sample" + logger.Warn().Int("idx", i).Str("var", sampleBytes).Msg("bench log") + } +} + +func BenchmarkLogMemAllocation(b *testing.B) { + type fields struct { + Bytes *[]byte + } + logger := log.NewLogger("benchmark.logger") + + for i := 0; i < b.N; i++ { + sampleBytes := []byte("sample") + logger.Warn().Int("idx", i).Str("var", EncodeB58(sampleBytes)).Msg("bench log") + } +} +func BenchmarkLogMemAllocationD(b *testing.B) { + type fields struct { + Bytes *[]byte + } + logger := log.NewLogger("benchmark.logger") + + for i := 0; i < b.N; i++ { + sampleBytes := []byte("sample") + logger.Debug().Int("idx", i).Str("var", EncodeB58(sampleBytes)).Msg("bench log") + } +} + +func BenchmarkLogMemAllocationRun(b *testing.B) { + type fields struct { + Bytes *[]byte + } + logger := log.NewLogger("benchmark.logger") + + for i := 0; i < b.N; i++ { + sampleBytes := []byte("sample") + logger.Warn().Int("idx", i).Stringer("var", LogBase58(sampleBytes)).Msg("bench log") + } +} + +func BenchmarkLogMemAllocationRunD(b *testing.B) { + type fields struct { + Bytes *[]byte + } + logger := log.NewLogger("benchmark.logger") + + for i := 0; i < b.N; i++ { + sampleBytes := []byte("sample") + logger.Debug().Int("idx", i).Stringer("var", LogBase58(sampleBytes)).Msg("bench log") + } +} + +type LogB58Wrapper []byte + +func (t LogB58Wrapper) MarshalZerologObject(e *zerolog.Event) { + e.Str("b58", enc.ToString(t)) +} + +func BenchmarkLogMemAllocationWrapper(b *testing.B) { + type fields struct { + Bytes *[]byte + } + logger := log.NewLogger("benchmark.logger") + + for i := 0; i < b.N; i++ { + sampleBytes := []byte("sample") + logger.Warn().Int("idx", i).Object("var", LogB58Wrapper(sampleBytes)).Msg("bench log") + } +} + +func BenchmarkLogMemAllocationWrapperD(b *testing.B) { + type fields struct { + Bytes *[]byte + } + logger := log.NewLogger("benchmark.logger") + + for i := 0; i < b.N; i++ { + sampleBytes := []byte("sample") + logger.Debug().Int("idx", i).Object("var", LogB58Wrapper(sampleBytes)).Msg("bench log") + } +}