Skip to content

Commit

Permalink
Merge branch 'topic/hardfork-v4' into topic/normal-txn-do-not-call-co…
Browse files Browse the repository at this point in the history
…ntract
  • Loading branch information
kroggen committed Oct 17, 2023
2 parents 0bc2cc8 + 07b830e commit 09a876d
Show file tree
Hide file tree
Showing 103 changed files with 1,510 additions and 562 deletions.
19 changes: 11 additions & 8 deletions .github/workflows/full_test.yml
Original file line number Diff line number Diff line change
Expand Up @@ -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
68 changes: 32 additions & 36 deletions chain/chainhandle.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ package chain
import (
"bytes"
"container/list"
"context"
"encoding/json"
"errors"
"fmt"
Expand All @@ -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)
Expand Down Expand Up @@ -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,
Expand All @@ -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{
Expand All @@ -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
}

Expand Down Expand Up @@ -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)
Expand All @@ -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
}
Expand All @@ -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) {
Expand All @@ -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)
Expand Down Expand Up @@ -562,7 +563,7 @@ type blockExecutor struct {
execTx TxExecFn
txs []*types.Tx
validatePost ValidatePostFn
coinbaseAcccount []byte
coinbaseAccount []byte
commitOnly bool
verifyOnly bool
validateSignWait ValidateSignWaitFn
Expand All @@ -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
Expand All @@ -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()
Expand All @@ -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)
},
Expand All @@ -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 {
Expand All @@ -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 {
Expand Down Expand Up @@ -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
}

Expand All @@ -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
}

Expand Down Expand Up @@ -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())
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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)
}

Expand Down
20 changes: 9 additions & 11 deletions chain/chainhandle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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")

}

Expand Down Expand Up @@ -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")
}

Expand All @@ -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
Expand All @@ -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")
}
4 changes: 1 addition & 3 deletions chain/chainservice.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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()
Expand Down Expand Up @@ -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 {
Expand Down

0 comments on commit 09a876d

Please sign in to comment.