Skip to content

Commit

Permalink
[consensus] remove config.EVMNetworkID() (#3491)
Browse files Browse the repository at this point in the history
* [consensus] remove config.EVMNetworkID()
  • Loading branch information
saitofun authored Jul 4, 2022
1 parent 7382ab7 commit c75411c
Show file tree
Hide file tree
Showing 12 changed files with 54 additions and 36 deletions.
3 changes: 3 additions & 0 deletions blockchain/block/block_deserializer.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,9 @@ func NewDeserializer(evmNetworkID uint32) *Deserializer {
}
}

// EvmNetworkID get evm network ID
func (bd *Deserializer) EvmNetworkID() uint32 { return bd.evmNetworkID }

// SetEvmNetworkID sets the evm network ID for web3 actions
func (bd *Deserializer) SetEvmNetworkID(id uint32) *Deserializer {
bd.evmNetworkID = id
Expand Down
1 change: 1 addition & 0 deletions consensus/consensus.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,7 @@ func NewConsensus(
SetPriKey(cfg.ProducerPrivateKey()).
SetConfig(cfg).
SetChainManager(bc).
SetBlockDeserializer(block.NewDeserializer(bc.EvmNetworkID())).
SetClock(clock).
SetBroadcast(ops.broadcastHandler).
SetDelegatesByEpochFunc(func(epochNum uint64) ([]string, error) {
Expand Down
6 changes: 2 additions & 4 deletions consensus/scheme/rolldpos/blockproposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,6 @@ package rolldpos
import (
"github.com/iotexproject/go-pkgs/hash"
"github.com/iotexproject/iotex-core/blockchain/block"
"github.com/iotexproject/iotex-core/config"
"github.com/iotexproject/iotex-core/endorsement"
"github.com/iotexproject/iotex-core/pkg/util/byteutil"
"github.com/iotexproject/iotex-proto/golang/iotextypes"
Expand Down Expand Up @@ -62,9 +61,8 @@ func (bp *blockProposal) ProposerAddress() string {
return bp.block.ProducerAddress()
}

func (bp *blockProposal) LoadProto(msg *iotextypes.BlockProposal) error {
// TODO: pass the correct EVM network ID at time of newConsensus() or from ctx
blk, err := block.NewDeserializer(config.EVMNetworkID()).FromBlockProto(msg.Block)
func (bp *blockProposal) LoadProto(msg *iotextypes.BlockProposal, deserializer *block.Deserializer) error {
blk, err := deserializer.FromBlockProto(msg.Block)
if err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion consensus/scheme/rolldpos/blockproposal_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ func TestNewBlockProposal(t *testing.T) {
require.NoError(err)

bp3 := newBlockProposal(nil, nil)
require.NoError(bp3.LoadProto(pro))
require.NoError(bp3.LoadProto(pro, block.NewDeserializer(0)))
pro3, err := bp3.Proto()
require.NoError(err)
require.EqualValues(pro, pro3)
Expand Down
5 changes: 3 additions & 2 deletions consensus/scheme/rolldpos/endorsedconsensusmessage.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@
package rolldpos

import (
"github.com/iotexproject/iotex-core/blockchain/block"
"github.com/pkg/errors"

"github.com/iotexproject/iotex-core/endorsement"
Expand Down Expand Up @@ -79,7 +80,7 @@ func (ecm *EndorsedConsensusMessage) Proto() (*iotextypes.ConsensusMessage, erro
}

// LoadProto creates an endorsement message from protobuf message
func (ecm *EndorsedConsensusMessage) LoadProto(msg *iotextypes.ConsensusMessage) error {
func (ecm *EndorsedConsensusMessage) LoadProto(msg *iotextypes.ConsensusMessage, deserializer *block.Deserializer) error {
switch {
case msg.GetVote() != nil:
vote := &ConsensusVote{}
Expand All @@ -89,7 +90,7 @@ func (ecm *EndorsedConsensusMessage) LoadProto(msg *iotextypes.ConsensusMessage)
ecm.message = vote
case msg.GetBlockProposal() != nil:
proposal := &blockProposal{}
if err := proposal.LoadProto(msg.GetBlockProposal()); err != nil {
if err := proposal.LoadProto(msg.GetBlockProposal(), deserializer); err != nil {
return err
}
ecm.message = proposal
Expand Down
3 changes: 2 additions & 1 deletion consensus/scheme/rolldpos/endorsedconsensusmessage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import (
"testing"
"time"

"github.com/iotexproject/iotex-core/blockchain/block"
"github.com/stretchr/testify/require"

"github.com/iotexproject/iotex-core/endorsement"
Expand All @@ -33,7 +34,7 @@ func TestEndorsedConsensusMessage(t *testing.T) {
pb, err := endorsedMessage.Proto()
require.NoError(err)
cem := &EndorsedConsensusMessage{}
require.NoError(cem.LoadProto(pb))
require.NoError(cem.LoadProto(pb, block.NewDeserializer(0)))
require.Equal(uint64(10), cem.Height())
cvote, ok := cem.Document().(*ConsensusVote)
require.True(ok)
Expand Down
17 changes: 7 additions & 10 deletions consensus/scheme/rolldpos/endorsementmanager.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"google.golang.org/protobuf/proto"

"github.com/iotexproject/iotex-core/blockchain/block"
"github.com/iotexproject/iotex-core/config"
"github.com/iotexproject/iotex-core/consensus/scheme/rolldpos/endorsementpb"
"github.com/iotexproject/iotex-core/db"
"github.com/iotexproject/iotex-core/endorsement"
Expand Down Expand Up @@ -121,13 +120,12 @@ func newBlockEndorsementCollection(blk *block.Block) *blockEndorsementCollection
}
}

func (bc *blockEndorsementCollection) fromProto(blockPro *endorsementpb.BlockEndorsementCollection) error {
func (bc *blockEndorsementCollection) fromProto(blockPro *endorsementpb.BlockEndorsementCollection, deserializer *block.Deserializer) error {
bc.endorsers = make(map[string]*endorserEndorsementCollection)
if blockPro.Blk == nil {
bc.blk = nil
} else {
// TODO: pass the correct EVM network ID at time of newConsensus() or from ctx
blk, err := block.NewDeserializer(config.EVMNetworkID()).FromBlockProto(blockPro.Blk)
blk, err := deserializer.FromBlockProto(blockPro.Blk)
if err != nil {
return err
}
Expand Down Expand Up @@ -226,7 +224,7 @@ type endorsementManager struct {
cachedMintedBlk *block.Block
}

func newEndorsementManager(eManagerDB db.KVStore) (*endorsementManager, error) {
func newEndorsementManager(eManagerDB db.KVStore, deserializer *block.Deserializer) (*endorsementManager, error) {
if eManagerDB == nil {
return &endorsementManager{
eManagerDB: nil,
Expand All @@ -243,7 +241,7 @@ func newEndorsementManager(eManagerDB db.KVStore) (*endorsementManager, error) {
if err = proto.Unmarshal(bytes, managerProto); err != nil {
return nil, err
}
if err = manager.fromProto(managerProto); err != nil {
if err = manager.fromProto(managerProto, deserializer); err != nil {
return nil, err
}
manager.eManagerDB = eManagerDB
Expand Down Expand Up @@ -281,18 +279,17 @@ func (m *endorsementManager) SetIsMarjorityFunc(isMajorityFunc EndorsedByMajorit
m.isMajorityFunc = isMajorityFunc
}

func (m *endorsementManager) fromProto(managerPro *endorsementpb.EndorsementManager) error {
func (m *endorsementManager) fromProto(managerPro *endorsementpb.EndorsementManager, deserializer *block.Deserializer) error {
m.collections = make(map[string]*blockEndorsementCollection)
for i, block := range managerPro.BlockEndorsements {
bc := &blockEndorsementCollection{}
if err := bc.fromProto(block); err != nil {
if err := bc.fromProto(block, deserializer); err != nil {
return err
}
m.collections[managerPro.BlkHash[i]] = bc
}
if managerPro.CachedMintedBlk != nil {
// TODO: pass the correct EVM network ID at time of newConsensus() or from ctx
blk, err := block.NewDeserializer(config.EVMNetworkID()).FromBlockProto(managerPro.CachedMintedBlk)
blk, err := deserializer.FromBlockProto(managerPro.CachedMintedBlk)
if err != nil {
return err
}
Expand Down
10 changes: 6 additions & 4 deletions consensus/scheme/rolldpos/endorsementmanager_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import (
"time"

"github.com/golang/mock/gomock"
"github.com/iotexproject/iotex-core/blockchain/block"
"github.com/stretchr/testify/require"

"github.com/iotexproject/iotex-core/endorsement"
Expand Down Expand Up @@ -95,7 +96,8 @@ func TestBlockEndorsementCollection(t *testing.T) {

func TestEndorsementManager(t *testing.T) {
require := require.New(t)
em, err := newEndorsementManager(nil)

em, err := newEndorsementManager(nil, block.NewDeserializer(0))
require.NoError(err)
require.NotNil(em)
require.Equal(0, em.Size())
Expand Down Expand Up @@ -178,7 +180,7 @@ func TestEndorsementManager(t *testing.T) {

func TestEndorsementManagerProto(t *testing.T) {
require := require.New(t)
em, err := newEndorsementManager(nil)
em, err := newEndorsementManager(nil, block.NewDeserializer(0))
require.NoError(err)
require.NotNil(em)

Expand All @@ -203,9 +205,9 @@ func TestEndorsementManagerProto(t *testing.T) {
//test converting emanager pb
emProto, err := em.toProto()
require.NoError(err)
em2, err := newEndorsementManager(nil)
em2, err := newEndorsementManager(nil, block.NewDeserializer(0))
require.NoError(err)
require.NoError(em2.fromProto(emProto))
require.NoError(em2.fromProto(emProto, block.NewDeserializer(0)))

require.Equal(len(em.collections), len(em2.collections))
encoded := encodeToString(cv.BlockHash())
Expand Down
20 changes: 14 additions & 6 deletions consensus/scheme/rolldpos/rolldpos.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ func (r *RollDPoS) HandleConsensusMsg(msg *iotextypes.ConsensusMessage) error {
return nil
}
endorsedMessage := &EndorsedConsensusMessage{}
if err := endorsedMessage.LoadProto(msg); err != nil {
if err := endorsedMessage.LoadProto(msg, r.ctx.blockDeserializer); err != nil {
return errors.Wrapf(err, "failed to decode endorsed consensus message")
}
if !endorsement.VerifyEndorsedDocument(endorsedMessage) {
Expand Down Expand Up @@ -232,11 +232,12 @@ func (r *RollDPoS) Active() bool {
type Builder struct {
cfg config.Config
// TODO: we should use keystore in the future
encodedAddr string
priKey crypto.PrivateKey
chain ChainManager
broadcastHandler scheme.Broadcast
clock clock.Clock
encodedAddr string
priKey crypto.PrivateKey
chain ChainManager
blockDeserializer *block.Deserializer
broadcastHandler scheme.Broadcast
clock clock.Clock
// TODO: explorer dependency deleted at #1085, need to add api params
rp *rolldpos.Protocol
delegatesByEpochFunc DelegatesByEpochFunc
Expand Down Expand Up @@ -271,6 +272,12 @@ func (b *Builder) SetChainManager(chain ChainManager) *Builder {
return b
}

// SetBlockDeserializer set block deserializer
func (b *Builder) SetBlockDeserializer(deserializer *block.Deserializer) *Builder {
b.blockDeserializer = deserializer
return b
}

// SetBroadcast sets the broadcast callback
func (b *Builder) SetBroadcast(broadcastHandler scheme.Broadcast) *Builder {
b.broadcastHandler = broadcastHandler
Expand Down Expand Up @@ -316,6 +323,7 @@ func (b *Builder) Build() (*RollDPoS, error) {
b.cfg.Consensus.RollDPoS.ToleratedOvertime,
b.cfg.Genesis.TimeBasedRotation,
b.chain,
b.blockDeserializer,
b.rp,
b.broadcastHandler,
b.delegatesByEpochFunc,
Expand Down
6 changes: 5 additions & 1 deletion consensus/scheme/rolldpos/rolldposctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"github.com/facebookgo/clock"
fsm "github.com/iotexproject/go-fsm"
"github.com/iotexproject/go-pkgs/crypto"
"github.com/iotexproject/iotex-core/blockchain/block"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"go.uber.org/zap"
Expand Down Expand Up @@ -75,6 +76,7 @@ type rollDPoSCtx struct {

// TODO: explorer dependency deleted at #1085, need to add api params here
chain ChainManager
blockDeserializer *block.Deserializer
broadcastHandler scheme.Broadcast
roundCalc *roundCalculator
eManagerDB db.KVStore
Expand All @@ -95,6 +97,7 @@ func newRollDPoSCtx(
toleratedOvertime time.Duration,
timeBasedRotation bool,
chain ChainManager,
blockDeserializer *block.Deserializer,
rp *rolldpos.Protocol,
broadcastHandler scheme.Broadcast,
delegatesByEpochFunc DelegatesByEpochFunc,
Expand Down Expand Up @@ -142,6 +145,7 @@ func newRollDPoSCtx(
encodedAddr: encodedAddr,
priKey: priKey,
chain: chain,
blockDeserializer: blockDeserializer,
broadcastHandler: broadcastHandler,
clock: clock,
roundCalc: roundCalc,
Expand All @@ -156,7 +160,7 @@ func (ctx *rollDPoSCtx) Start(c context.Context) (err error) {
if err := ctx.eManagerDB.Start(c); err != nil {
return errors.Wrap(err, "Error when starting the collectionDB")
}
eManager, err = newEndorsementManager(ctx.eManagerDB)
eManager, err = newEndorsementManager(ctx.eManagerDB, ctx.blockDeserializer)
}
ctx.round, err = ctx.roundCalc.NewRoundWithToleration(0, ctx.BlockInterval(0), ctx.clock.Now(), eManager, ctx.toleratedOvertime)

Expand Down
15 changes: 9 additions & 6 deletions consensus/scheme/rolldpos/rolldposctx_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,12 +39,12 @@ func TestRollDPoSCtx(t *testing.T) {
b, _, _, _, _ := makeChain(t)

t.Run("case 1:panic because of chain is nil", func(t *testing.T) {
_, err := newRollDPoSCtx(consensusfsm.NewConsensusConfig(cfg), dbConfig, true, time.Second, true, nil, nil, nil, dummyCandidatesByHeightFunc, "", nil, nil, 0)
_, err := newRollDPoSCtx(consensusfsm.NewConsensusConfig(cfg), dbConfig, true, time.Second, true, nil, block.NewDeserializer(0), nil, nil, dummyCandidatesByHeightFunc, "", nil, nil, 0)
require.Error(err)
})

t.Run("case 2:panic because of rp is nil", func(t *testing.T) {
_, err := newRollDPoSCtx(consensusfsm.NewConsensusConfig(cfg), dbConfig, true, time.Second, true, b, nil, nil, dummyCandidatesByHeightFunc, "", nil, nil, 0)
_, err := newRollDPoSCtx(consensusfsm.NewConsensusConfig(cfg), dbConfig, true, time.Second, true, b, block.NewDeserializer(0), nil, nil, dummyCandidatesByHeightFunc, "", nil, nil, 0)
require.Error(err)
})

Expand All @@ -54,7 +54,7 @@ func TestRollDPoSCtx(t *testing.T) {
config.Default.Genesis.NumSubEpochs,
)
t.Run("case 3:panic because of clock is nil", func(t *testing.T) {
_, err := newRollDPoSCtx(consensusfsm.NewConsensusConfig(cfg), dbConfig, true, time.Second, true, b, rp, nil, dummyCandidatesByHeightFunc, "", nil, nil, 0)
_, err := newRollDPoSCtx(consensusfsm.NewConsensusConfig(cfg), dbConfig, true, time.Second, true, b, block.NewDeserializer(0), rp, nil, dummyCandidatesByHeightFunc, "", nil, nil, 0)
require.Error(err)
})

Expand All @@ -64,19 +64,19 @@ func TestRollDPoSCtx(t *testing.T) {
cfg.Consensus.RollDPoS.FSM.AcceptLockEndorsementTTL = time.Second
cfg.Consensus.RollDPoS.FSM.CommitTTL = time.Second
t.Run("case 4:panic because of fsm time bigger than block interval", func(t *testing.T) {
_, err := newRollDPoSCtx(consensusfsm.NewConsensusConfig(cfg), dbConfig, true, time.Second, true, b, rp, nil, dummyCandidatesByHeightFunc, "", nil, c, 0)
_, err := newRollDPoSCtx(consensusfsm.NewConsensusConfig(cfg), dbConfig, true, time.Second, true, b, block.NewDeserializer(0), rp, nil, dummyCandidatesByHeightFunc, "", nil, c, 0)
require.Error(err)
})

cfg.Genesis.Blockchain.BlockInterval = time.Second * 20
t.Run("case 5:panic because of nil CandidatesByHeight function", func(t *testing.T) {
_, err := newRollDPoSCtx(consensusfsm.NewConsensusConfig(cfg), dbConfig, true, time.Second, true, b, rp, nil, nil, "", nil, c, 0)
_, err := newRollDPoSCtx(consensusfsm.NewConsensusConfig(cfg), dbConfig, true, time.Second, true, b, block.NewDeserializer(0), rp, nil, nil, "", nil, c, 0)
require.Error(err)
})

t.Run("case 6:normal", func(t *testing.T) {
bh := config.Default.Genesis.BeringBlockHeight
rctx, err := newRollDPoSCtx(consensusfsm.NewConsensusConfig(cfg), dbConfig, true, time.Second, true, b, rp, nil, dummyCandidatesByHeightFunc, "", nil, c, bh)
rctx, err := newRollDPoSCtx(consensusfsm.NewConsensusConfig(cfg), dbConfig, true, time.Second, true, b, block.NewDeserializer(0), rp, nil, dummyCandidatesByHeightFunc, "", nil, c, bh)
require.NoError(err)
require.Equal(bh, rctx.roundCalc.beringHeight)
require.NotNil(rctx)
Expand All @@ -96,6 +96,7 @@ func TestCheckVoteEndorser(t *testing.T) {
time.Second,
true,
b,
block.NewDeserializer(0),
rp,
nil,
func(epochnum uint64) ([]string, error) {
Expand Down Expand Up @@ -168,6 +169,7 @@ func TestCheckBlockProposer(t *testing.T) {
time.Second,
true,
b,
block.NewDeserializer(0),
rp,
nil,
func(epochnum uint64) ([]string, error) {
Expand Down Expand Up @@ -279,6 +281,7 @@ func TestNotProducingMultipleBlocks(t *testing.T) {
time.Second,
true,
b,
block.NewDeserializer(0),
rp,
nil,
func(epochnum uint64) ([]string, error) {
Expand Down
2 changes: 1 addition & 1 deletion consensus/scheme/rolldpos/roundcalculator.go
Original file line number Diff line number Diff line change
Expand Up @@ -224,7 +224,7 @@ func (c *roundCalculator) newRound(
}
}
if eManager == nil {
if eManager, err = newEndorsementManager(nil); err != nil {
if eManager, err = newEndorsementManager(nil, nil); err != nil {
return nil, err
}
}
Expand Down

0 comments on commit c75411c

Please sign in to comment.