Skip to content

Commit

Permalink
run consensus with several validators
Browse files Browse the repository at this point in the history
Right now it's implemented as a test.
  • Loading branch information
olegrok committed Jan 28, 2025
1 parent fb2357b commit db4cb79
Show file tree
Hide file tree
Showing 9 changed files with 170 additions and 28 deletions.
40 changes: 23 additions & 17 deletions nil/internal/consensus/ibft/ibft.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import (
"github.com/NilFoundation/nil/nil/go-ibft/messages"
protoIBFT "github.com/NilFoundation/nil/nil/go-ibft/messages/proto"
"github.com/NilFoundation/nil/nil/internal/collate"
"github.com/NilFoundation/nil/nil/internal/config"
"github.com/NilFoundation/nil/nil/internal/db"
"github.com/NilFoundation/nil/nil/internal/execution"
"github.com/NilFoundation/nil/nil/internal/network"
Expand All @@ -25,18 +26,20 @@ type ConsensusParams struct {
Scheduler *collate.Scheduler
NetManager *network.Manager
PrivateKey *ecdsa.PrivateKey
Validators []config.ValidatorInfo
}

type backendIBFT struct {
ctx context.Context
db db.DB
consensus *core.IBFT
shardId types.ShardId
scheduler *collate.Scheduler
logger zerolog.Logger
nm *network.Manager
transport transport
signer *Signer
ctx context.Context
db db.DB
consensus *core.IBFT
shardId types.ShardId
scheduler *collate.Scheduler
logger zerolog.Logger
nm *network.Manager
transport transport
signer *Signer
validators []config.ValidatorInfo
}

var _ core.Backend = &backendIBFT{}
Expand Down Expand Up @@ -94,20 +97,23 @@ func NewConsensus(cfg *ConsensusParams) *backendIBFT {
}

backend := &backendIBFT{
db: cfg.Db,
shardId: cfg.ShardId,
scheduler: cfg.Scheduler,
logger: logger,
nm: cfg.NetManager,
signer: NewSigner(cfg.PrivateKey),
db: cfg.Db,
shardId: cfg.ShardId,
scheduler: cfg.Scheduler,
logger: logger,
nm: cfg.NetManager,
signer: NewSigner(cfg.PrivateKey),
validators: cfg.Validators,
}
backend.consensus = core.NewIBFT(l, backend, backend)
return backend
}

func (i *backendIBFT) GetVotingPowers(height uint64) (map[string]*big.Int, error) {
result := make(map[string]*big.Int)
result[string(i.ID())] = big.NewInt(1)
result := make(map[string]*big.Int, len(i.validators))
for _, v := range i.validators {
result[string(v.PublicKey[:])] = big.NewInt(1)
}
return result, nil
}

Expand Down
4 changes: 2 additions & 2 deletions nil/internal/consensus/ibft/signer.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,10 +33,10 @@ func (s *Signer) Sign(data []byte) (types.Signature, error) {
}

func (s *Signer) Verify(data []byte, signature types.Signature) bool {
return s.verifyWithKey(s.rawPublicKey, data, signature)
return s.VerifyWithKey(s.rawPublicKey, data, signature)
}

func (s *Signer) verifyWithKey(publicKey []byte, data []byte, signature types.Signature) bool {
func (s *Signer) VerifyWithKey(publicKey []byte, data []byte, signature types.Signature) bool {
return len(signature) >= 64 && crypto.VerifySignature(publicKey, getHash(data), signature[:64])
}

Expand Down
10 changes: 10 additions & 0 deletions nil/internal/consensus/ibft/validators.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,10 @@
package ibft

import (
"github.com/NilFoundation/nil/nil/internal/config"
)

func (i *backendIBFT) calcProposer(height, round uint64) config.ValidatorInfo {
index := (height + round) % uint64(len(i.validators))
return i.validators[index]
}
27 changes: 23 additions & 4 deletions nil/internal/consensus/ibft/verifier.go
Original file line number Diff line number Diff line change
Expand Up @@ -2,10 +2,12 @@ package ibft

import (
"bytes"
"slices"

"github.com/NilFoundation/nil/nil/common/logging"
"github.com/NilFoundation/nil/nil/go-ibft/messages"
protoIBFT "github.com/NilFoundation/nil/nil/go-ibft/messages/proto"
"github.com/NilFoundation/nil/nil/internal/config"
)

func (i *backendIBFT) IsValidProposal(rawProposal []byte) bool {
Expand All @@ -24,8 +26,24 @@ func (i *backendIBFT) IsValidValidator(msg *protoIBFT.IbftMessage) bool {
return false
}

if !i.signer.Verify(msgNoSig, msg.Signature) {
event := i.logger.Error().Stringer(logging.FieldType, msg.GetType())
index := slices.IndexFunc(i.validators, func(v config.ValidatorInfo) bool {
return bytes.Equal(v.PublicKey[:], msg.From)
})
if index == -1 {
event := i.logger.Error().
Hex("key", msg.From)

if view := msg.GetView(); view != nil {
event = event.Uint64(logging.FieldHeight, view.Height).
Uint64(logging.FieldRound, view.Round)
}
event.Msg("Key not found in validators list")
return false
}

validator := i.validators[index]
if !i.signer.VerifyWithKey(validator.PublicKey[:], msgNoSig, msg.Signature) {
event := i.logger.Error()
if view := msg.GetView(); view != nil {
event = event.Uint64(logging.FieldHeight, view.Height).
Uint64(logging.FieldRound, view.Round)
Expand All @@ -38,7 +56,8 @@ func (i *backendIBFT) IsValidValidator(msg *protoIBFT.IbftMessage) bool {
}

func (i *backendIBFT) IsProposer(id []byte, height, round uint64) bool {
return true
proposer := i.calcProposer(height, round)
return bytes.Equal(proposer.PublicKey[:], id)
}

func (i *backendIBFT) IsValidProposalHash(proposal *protoIBFT.Proposal, hash []byte) bool {
Expand Down Expand Up @@ -66,7 +85,7 @@ func (i *backendIBFT) IsValidProposalHash(proposal *protoIBFT.Proposal, hash []b
}

func (i *backendIBFT) IsValidCommittedSeal(
proposalHash []byte,
_ []byte,
committedSeal *messages.CommittedSeal,
) bool {
return true
Expand Down
11 changes: 11 additions & 0 deletions nil/internal/keys/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,3 +165,14 @@ func (v *ValidatorKeysManager) GetKeys() ([]*ecdsa.PrivateKey, error) {
func (v *ValidatorKeysManager) GetKeysPath() string {
return v.validatorKeysPath
}

func (v *ValidatorKeysManager) GetPublicKeys() ([]hexutil.Bytes, error) {
if !v.init {
return nil, errKeysNotInitialized
}
keys := make([]hexutil.Bytes, 0, len(v.keys))
for _, key := range v.keys {
keys = append(keys, gethcrypto.CompressPubkey(&key.PublicKey))
}
return keys, nil
}
2 changes: 2 additions & 0 deletions nil/services/nilservice/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,8 @@ func NewDefaultConfig() *Config {
GracefulShutdown: true,
Topology: collate.TrivialShardTopologyId,

Validators: make(map[types.ShardId][]config.ValidatorInfo),

Network: network.NewDefaultConfig(),
Telemetry: telemetry.NewDefaultConfig(),
Replay: NewDefaultReplayConfig(),
Expand Down
1 change: 1 addition & 0 deletions nil/services/nilservice/service.go
Original file line number Diff line number Diff line change
Expand Up @@ -494,6 +494,7 @@ func createShards(
Scheduler: collator,
NetManager: networkManager,
PrivateKey: pKey,
Validators: cfg.Validators[shardId],
})
if err := consensus.Init(ctx); err != nil {
return nil, nil, err
Expand Down
47 changes: 47 additions & 0 deletions nil/tests/consensus/consensus_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
package tests

import (
"testing"
"time"

"github.com/NilFoundation/nil/nil/internal/types"
"github.com/NilFoundation/nil/nil/services/nilservice"
"github.com/NilFoundation/nil/nil/tests"
"github.com/stretchr/testify/suite"
)

type SuiteConsensus struct {
tests.ShardedSuite
}

func (s *SuiteConsensus) SetupTest() {
nShards := uint32(3)

s.StartShardAllValidators(&nilservice.Config{
NShards: nShards,
CollatorTickPeriodMs: 200,
}, 10625)
}

func (s *SuiteConsensus) TearDownTest() {
s.Cancel()
}

func (s *SuiteConsensus) TestConsensus() {
for shardId, shard := range s.Shards {
block, err := shard.Client.GetBlock(s.Context, types.ShardId(shardId), "latest", true)
s.Require().NoError(err)

s.Require().Eventually(func() bool {
newBlock, err := shard.Client.GetBlock(s.Context, types.ShardId(shardId), "latest", true)
s.Require().NoError(err)
return newBlock != nil && newBlock.Number > block.Number
}, 20*time.Second, 1*time.Second)
}
}

func TestConsensus(t *testing.T) {
t.Parallel()

suite.Run(t, new(SuiteConsensus))
}
56 changes: 51 additions & 5 deletions nil/tests/sharded_suite.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,8 +69,8 @@ func (s *ShardedSuite) Cancel() {
}
}

func (s *ShardedSuite) createOneShardOneValidatorCfg(
shardId types.ShardId, cfg *nilservice.Config, netCfg *network.Config, keyManagers map[types.ShardId]*keys.ValidatorKeysManager,
func createOneShardOneValidatorCfg(
s *ShardedSuite, shardId types.ShardId, cfg *nilservice.Config, netCfg *network.Config, keyManagers map[types.ShardId]*keys.ValidatorKeysManager,
) *nilservice.Config {
validatorKeysPath := keyManagers[shardId].GetKeysPath()

Expand Down Expand Up @@ -99,7 +99,47 @@ func (s *ShardedSuite) createOneShardOneValidatorCfg(
}
}

func (s *ShardedSuite) start(cfg *nilservice.Config, port int) {
func createShardAllValidatorsCfg(
s *ShardedSuite, shardId types.ShardId, cfg *nilservice.Config, netCfg *network.Config, keyManagers map[types.ShardId]*keys.ValidatorKeysManager,
) *nilservice.Config {
myShards := slices.Collect(common.Range(0, uint(cfg.NShards)))

validatorKeysPath := keyManagers[shardId].GetKeysPath()
validators := make(map[types.ShardId][]config.ValidatorInfo, cfg.NShards)

// Order of validators is important and should be the same for all instances
for kmId := range cfg.NShards {
// Array of public keys for each shard (index is shardId)
pkeys, err := keyManagers[types.ShardId(kmId)].GetPublicKeys()
s.Require().NoError(err)

for i, pkey := range pkeys {
id := types.ShardId(i)
validators[id] = append(validators[id], config.ValidatorInfo{
PublicKey: [33]byte(pkey),
})
}
}

return &nilservice.Config{
NShards: cfg.NShards,
MyShards: myShards,
SplitShards: true,
HttpUrl: s.Shards[shardId].RpcUrl,
Topology: cfg.Topology,
CollatorTickPeriodMs: cfg.CollatorTickPeriodMs,
GasBasePrice: cfg.GasBasePrice,
Network: netCfg,
ZeroStateYaml: cfg.ZeroStateYaml,
ValidatorKeysPath: validatorKeysPath,
Validators: validators,
}
}

func (s *ShardedSuite) start(
cfg *nilservice.Config, port int,
shardCfgGen func(*ShardedSuite, types.ShardId, *nilservice.Config, *network.Config, map[types.ShardId]*keys.ValidatorKeysManager) *nilservice.Config,
) {
s.T().Helper()
s.Context, s.ctxCancel = context.WithCancel(context.Background())

Expand Down Expand Up @@ -137,7 +177,7 @@ func (s *ShardedSuite) start(cfg *nilservice.Config, port int) {

PatchConfigWithTestDefaults(cfg)
for i := range types.ShardId(cfg.NShards) {
shardConfig := s.createOneShardOneValidatorCfg(i, cfg, networkConfigs[i], keysManagers)
shardConfig := shardCfgGen(s, i, cfg, networkConfigs[i], keysManagers)

node, err := nilservice.CreateNode(s.Context, fmt.Sprintf("shard-%d", i), shardConfig, s.Shards[i].Db, nil)
s.Require().NoError(err)
Expand All @@ -162,7 +202,13 @@ func (s *ShardedSuite) start(cfg *nilservice.Config, port int) {
func (s *ShardedSuite) Start(cfg *nilservice.Config, port int) {
s.T().Helper()

s.start(cfg, port)
s.start(cfg, port, createOneShardOneValidatorCfg)
}

func (s *ShardedSuite) StartShardAllValidators(cfg *nilservice.Config, port int) {
s.T().Helper()

s.start(cfg, port, createShardAllValidatorsCfg)
}

func (s *ShardedSuite) connectToShards(nm *network.Manager) {
Expand Down

0 comments on commit db4cb79

Please sign in to comment.