diff --git a/cmd/node/config/config.toml b/cmd/node/config/config.toml index 4da40dfde30..fd0f1f6e6e1 100644 --- a/cmd/node/config/config.toml +++ b/cmd/node/config/config.toml @@ -40,6 +40,11 @@ # Make sure that this is greater than the unbonding period! SetGuardianEpochsDelay = 2 # TODO: for mainnet should be 20, 2 is just for testing + # ChainParametersByEpoch defines chain operation configurable values that can be modified based on epochs + ChainParametersByEpoch = [ + { EnableEpoch = 0, RoundDuration = 6000, ShardConsensusGroupSize = 7, ShardMinNumNodes = 10, MetachainConsensusGroupSize = 10, MetachainMinNumNodes = 10, Hysteresis = 0.2, Adaptivity = false } + ] + [HardwareRequirements] CPUFlags = ["SSE4", "SSE42"] diff --git a/cmd/node/config/nodesSetup.json b/cmd/node/config/nodesSetup.json index 741d9009ad8..daa5fd1b98a 100644 --- a/cmd/node/config/nodesSetup.json +++ b/cmd/node/config/nodesSetup.json @@ -1,12 +1,5 @@ { "startTime": 0, - "roundDuration": 6000, - "consensusGroupSize": 7, - "minNodesPerShard": 10, - "metaChainConsensusGroupSize": 10, - "metaChainMinNodes": 10, - "hysteresis": 0.2, - "adaptivity": false, "initialNodes": [ { "info": "multikey - group1 - legacy delegation", diff --git a/cmd/node/main.go b/cmd/node/main.go index 5a812bc2f45..c75dd40a393 100644 --- a/cmd/node/main.go +++ b/cmd/node/main.go @@ -238,6 +238,14 @@ func readConfigs(ctx *cli.Context, log logger.Logger) (*config.Configs, error) { } log.Debug("config", "file", configurationPaths.RoundActivation) + var nodesSetup config.NodesConfig + configurationPaths.Nodes = ctx.GlobalString(nodesFile.Name) + err = core.LoadJsonFile(&nodesSetup, configurationPaths.Nodes) + if err != nil { + return nil, err + } + log.Debug("config", "file", configurationPaths.Nodes) + if ctx.IsSet(port.Name) { mainP2PConfig.Node.Port = ctx.GlobalString(port.Name) } @@ -267,6 +275,7 @@ func readConfigs(ctx *cli.Context, log logger.Logger) (*config.Configs, error) { ConfigurationPathsHolder: configurationPaths, EpochConfig: epochConfig, RoundConfig: roundConfig, + NodesConfig: &nodesSetup, }, nil } diff --git a/common/chainparametersnotifier/chainParametersNotifier.go b/common/chainparametersnotifier/chainParametersNotifier.go new file mode 100644 index 00000000000..1a3baf2b5ff --- /dev/null +++ b/common/chainparametersnotifier/chainParametersNotifier.go @@ -0,0 +1,97 @@ +package chainparametersnotifier + +import ( + "sync" + + "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/config" + logger "github.com/multiversx/mx-chain-logger-go" +) + +var log = logger.GetOrCreate("common/chainparameters") + +type chainParametersNotifier struct { + mutData sync.RWMutex + wasInitialized bool + currentChainParameters config.ChainParametersByEpochConfig + mutHandler sync.RWMutex + handlers []common.ChainParametersSubscriptionHandler +} + +// NewChainParametersNotifier creates a new instance of a chainParametersNotifier component +func NewChainParametersNotifier() *chainParametersNotifier { + return &chainParametersNotifier{ + wasInitialized: false, + handlers: make([]common.ChainParametersSubscriptionHandler, 0), + } +} + +// UpdateCurrentChainParameters should be called whenever new chain parameters become active on the network +func (cpn *chainParametersNotifier) UpdateCurrentChainParameters(params config.ChainParametersByEpochConfig) { + cpn.mutData.Lock() + shouldSkipParams := cpn.wasInitialized && cpn.currentChainParameters.EnableEpoch == params.EnableEpoch + if shouldSkipParams { + cpn.mutData.Unlock() + + return + } + cpn.wasInitialized = true + cpn.currentChainParameters = params + cpn.mutData.Unlock() + + cpn.mutHandler.RLock() + handlersCopy := make([]common.ChainParametersSubscriptionHandler, len(cpn.handlers)) + copy(handlersCopy, cpn.handlers) + cpn.mutHandler.RUnlock() + + log.Debug("chainParametersNotifier.UpdateCurrentChainParameters", + "enable epoch", params.EnableEpoch, + "shard consensus group size", params.ShardConsensusGroupSize, + "shard min number of nodes", params.ShardMinNumNodes, + "meta consensus group size", params.MetachainConsensusGroupSize, + "meta min number of nodes", params.MetachainMinNumNodes, + "round duration", params.RoundDuration, + "hysteresis", params.Hysteresis, + "adaptivity", params.Adaptivity, + ) + + for _, handler := range handlersCopy { + handler.ChainParametersChanged(params) + } +} + +// RegisterNotifyHandler will register the provided handler to be called whenever chain parameters have changed +func (cpn *chainParametersNotifier) RegisterNotifyHandler(handler common.ChainParametersSubscriptionHandler) { + if check.IfNil(handler) { + return + } + + cpn.mutHandler.Lock() + cpn.handlers = append(cpn.handlers, handler) + cpn.mutHandler.Unlock() + + cpn.mutData.RLock() + handler.ChainParametersChanged(cpn.currentChainParameters) + cpn.mutData.RUnlock() +} + +// CurrentChainParameters returns the current chain parameters +func (cpn *chainParametersNotifier) CurrentChainParameters() config.ChainParametersByEpochConfig { + cpn.mutData.RLock() + defer cpn.mutData.RUnlock() + + return cpn.currentChainParameters +} + +// UnRegisterAll removes all registered handlers queue +func (cpn *chainParametersNotifier) UnRegisterAll() { + cpn.mutHandler.Lock() + cpn.handlers = make([]common.ChainParametersSubscriptionHandler, 0) + cpn.mutHandler.Unlock() +} + +// IsInterfaceNil returns true if there is no value under the interface +func (cpn *chainParametersNotifier) IsInterfaceNil() bool { + return cpn == nil +} diff --git a/common/chainparametersnotifier/chainParametersNotifier_test.go b/common/chainparametersnotifier/chainParametersNotifier_test.go new file mode 100644 index 00000000000..fa1a30959d4 --- /dev/null +++ b/common/chainparametersnotifier/chainParametersNotifier_test.go @@ -0,0 +1,126 @@ +package chainparametersnotifier + +import ( + "sync" + "testing" + + "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/config" + "github.com/stretchr/testify/require" +) + +func TestNewChainParametersNotifier(t *testing.T) { + t.Parallel() + + notifier := NewChainParametersNotifier() + require.False(t, check.IfNil(notifier)) +} + +func TestChainParametersNotifier_UpdateCurrentChainParameters(t *testing.T) { + t.Parallel() + + notifier := NewChainParametersNotifier() + require.False(t, check.IfNil(notifier)) + + chainParams := config.ChainParametersByEpochConfig{ + EnableEpoch: 7, + Adaptivity: true, + Hysteresis: 0.7, + } + notifier.UpdateCurrentChainParameters(chainParams) + + resultedChainParams := notifier.CurrentChainParameters() + require.NotNil(t, resultedChainParams) + + // update with same epoch but other params - should not change (impossible scenario in production, but easier for tests) + chainParams.Hysteresis = 0.8 + notifier.UpdateCurrentChainParameters(chainParams) + require.Equal(t, float32(0.7), notifier.CurrentChainParameters().Hysteresis) + + chainParams.Hysteresis = 0.8 + chainParams.EnableEpoch = 8 + notifier.UpdateCurrentChainParameters(chainParams) + require.Equal(t, float32(0.8), notifier.CurrentChainParameters().Hysteresis) +} + +func TestChainParametersNotifier_RegisterNotifyHandler(t *testing.T) { + t.Parallel() + + notifier := NewChainParametersNotifier() + require.False(t, check.IfNil(notifier)) + + // register a nil handler - should not panic + notifier.RegisterNotifyHandler(nil) + + testNotifee := &dummyNotifee{} + notifier.RegisterNotifyHandler(testNotifee) + + chainParams := config.ChainParametersByEpochConfig{ + ShardMinNumNodes: 37, + } + notifier.UpdateCurrentChainParameters(chainParams) + + require.Equal(t, chainParams, testNotifee.receivedChainParameters) +} + +func TestChainParametersNotifier_UnRegisterAll(t *testing.T) { + t.Parallel() + + notifier := NewChainParametersNotifier() + require.False(t, check.IfNil(notifier)) + + testNotifee := &dummyNotifee{} + notifier.RegisterNotifyHandler(testNotifee) + notifier.UnRegisterAll() + + chainParams := config.ChainParametersByEpochConfig{ + ShardMinNumNodes: 37, + } + notifier.UpdateCurrentChainParameters(chainParams) + + require.Empty(t, testNotifee.receivedChainParameters) +} + +func TestChainParametersNotifier_ConcurrentOperations(t *testing.T) { + t.Parallel() + + notifier := NewChainParametersNotifier() + + numOperations := 500 + wg := sync.WaitGroup{} + wg.Add(numOperations) + for i := 0; i < numOperations; i++ { + go func(idx int) { + switch idx { + case 0: + notifier.RegisterNotifyHandler(&dummyNotifee{}) + case 1: + _ = notifier.CurrentChainParameters() + case 2: + notifier.UpdateCurrentChainParameters(config.ChainParametersByEpochConfig{}) + case 3: + notifier.UnRegisterAll() + case 4: + _ = notifier.IsInterfaceNil() + } + + wg.Done() + }(i % 5) + } + + wg.Wait() +} + +type dummyNotifee struct { + receivedChainParameters config.ChainParametersByEpochConfig +} + +// ChainParametersChanged - +func (dn *dummyNotifee) ChainParametersChanged(chainParameters config.ChainParametersByEpochConfig) { + dn.receivedChainParameters = chainParameters +} + +// IsInterfaceNil - +func (dn *dummyNotifee) IsInterfaceNil() bool { + return dn == nil +} diff --git a/common/constants.go b/common/constants.go index e2ed1d80a5a..6cd62719af0 100644 --- a/common/constants.go +++ b/common/constants.go @@ -851,6 +851,8 @@ const ( const ( // StorerOrder defines the order of storers to be notified of a start of epoch event StorerOrder = iota + // ChainParametersOrder defines the order in which ChainParameters is notified of a start of epoch event + ChainParametersOrder // NodesCoordinatorOrder defines the order in which NodesCoordinator is notified of a start of epoch event NodesCoordinatorOrder // ConsensusOrder defines the order in which Consensus is notified of a start of epoch event @@ -963,7 +965,7 @@ const PutInStorerMaxTime = time.Second const DefaultUnstakedEpoch = math.MaxUint32 // InvalidMessageBlacklistDuration represents the time to keep a peer in the black list if it sends a message that -// does not follow the protocol: example not useing the same marshaler as the other peers +// does not follow the protocol: example not using the same marshaler as the other peers const InvalidMessageBlacklistDuration = time.Second * 3600 // PublicKeyBlacklistDuration represents the time to keep a public key in the black list if it will degrade its diff --git a/common/forking/genericEpochNotifier_test.go b/common/forking/genericEpochNotifier_test.go index ca78700d2a0..a0a649c098c 100644 --- a/common/forking/genericEpochNotifier_test.go +++ b/common/forking/genericEpochNotifier_test.go @@ -1,11 +1,13 @@ package forking import ( + "sync" "sync/atomic" "testing" "time" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/common/mock" "github.com/multiversx/mx-chain-go/testscommon" "github.com/stretchr/testify/assert" @@ -152,3 +154,33 @@ func TestGenericEpochNotifier_CheckEpochInSyncShouldWork(t *testing.T) { assert.Equal(t, uint32(2), atomic.LoadUint32(&numCalls)) assert.True(t, end.Sub(start) >= handlerWait) } + +func TestGenericEpochNotifier_ConcurrentOperations(t *testing.T) { + t.Parallel() + + notifier := NewGenericEpochNotifier() + + numOperations := 500 + wg := sync.WaitGroup{} + wg.Add(numOperations) + for i := 0; i < numOperations; i++ { + go func(idx int) { + switch idx { + case 0: + notifier.RegisterNotifyHandler(&mock.EpochSubscriberHandlerStub{}) + case 1: + _ = notifier.CurrentEpoch() + case 2: + notifier.CheckEpoch(&block.MetaBlock{Epoch: 5}) + case 3: + notifier.UnRegisterAll() + case 4: + _ = notifier.IsInterfaceNil() + } + + wg.Done() + }(i % 5) + } + + wg.Wait() +} diff --git a/common/interface.go b/common/interface.go index 82b2960d0ce..72a2cba2628 100644 --- a/common/interface.go +++ b/common/interface.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" crypto "github.com/multiversx/mx-chain-crypto-go" + "github.com/multiversx/mx-chain-go/config" ) // TrieIteratorChannels defines the channels that are being used when iterating the trie nodes @@ -372,3 +373,9 @@ type ExecutionOrderGetter interface { Len() int IsInterfaceNil() bool } + +// ChainParametersSubscriptionHandler defines the behavior of a chain parameters subscription handler +type ChainParametersSubscriptionHandler interface { + ChainParametersChanged(chainParameters config.ChainParametersByEpochConfig) + IsInterfaceNil() bool +} diff --git a/config/config.go b/config/config.go index 49ef257c341..8d33b87830f 100644 --- a/config/config.go +++ b/config/config.go @@ -290,6 +290,7 @@ type GeneralSettingsConfig struct { GenesisMaxNumberOfShards uint32 SyncProcessTimeInMillis uint32 SetGuardianEpochsDelay uint32 + ChainParametersByEpoch []ChainParametersByEpochConfig } // HardwareRequirementsConfig will hold the hardware requirements config @@ -591,6 +592,20 @@ type Configs struct { ConfigurationPathsHolder *ConfigurationPathsHolder EpochConfig *EpochConfig RoundConfig *RoundConfig + NodesConfig *NodesConfig +} + +// NodesConfig is the data transfer object used to map the nodes' configuration in regard to the genesis nodes setup +type NodesConfig struct { + StartTime int64 `json:"startTime"` + InitialNodes []*InitialNodeConfig `json:"initialNodes"` +} + +// InitialNodeConfig holds data about a genesis node +type InitialNodeConfig struct { + PubKey string `json:"pubkey"` + Address string `json:"address"` + InitialRating uint32 `json:"initialRating"` } // ConfigurationPathsHolder holds all configuration filenames and configuration paths used to start the node @@ -640,3 +655,15 @@ type PoolsCleanersConfig struct { type RedundancyConfig struct { MaxRoundsOfInactivityAccepted int } + +// ChainParametersByEpochConfig holds chain parameters that are configurable based on epochs +type ChainParametersByEpochConfig struct { + RoundDuration uint64 + Hysteresis float32 + EnableEpoch uint32 + ShardConsensusGroupSize uint32 + ShardMinNumNodes uint32 + MetachainConsensusGroupSize uint32 + MetachainMinNumNodes uint32 + Adaptivity bool +} diff --git a/config/tomlConfig_test.go b/config/tomlConfig_test.go index 767e37e3950..1ad4048fce6 100644 --- a/config/tomlConfig_test.go +++ b/config/tomlConfig_test.go @@ -48,6 +48,20 @@ func TestTomlParser(t *testing.T) { } cfgExpected := Config{ + GeneralSettings: GeneralSettingsConfig{ + ChainParametersByEpoch: []ChainParametersByEpochConfig{ + { + EnableEpoch: 0, + RoundDuration: 4000, + ShardMinNumNodes: 4, + ShardConsensusGroupSize: 3, + MetachainMinNumNodes: 6, + MetachainConsensusGroupSize: 5, + Hysteresis: 0.0, + Adaptivity: false, + }, + }, + }, MiniBlocksStorage: StorageConfig{ Cache: CacheConfig{ Capacity: uint32(txBlockBodyStorageSize), @@ -150,6 +164,10 @@ func TestTomlParser(t *testing.T) { }, } testString := ` +[GeneralSettings] + ChainParametersByEpoch = [ + { EnableEpoch = 0, RoundDuration = 4000, ShardConsensusGroupSize = 3, ShardMinNumNodes = 4, MetachainConsensusGroupSize = 5, MetachainMinNumNodes = 6, Hysteresis = 0.0, Adaptivity = false } + ] [MiniBlocksStorage] [MiniBlocksStorage.Cache] Capacity = ` + strconv.Itoa(txBlockBodyStorageSize) + ` diff --git a/consensus/spos/bls/subroundStartRound.go b/consensus/spos/bls/subroundStartRound.go index 571270dd774..0898e039c8b 100644 --- a/consensus/spos/bls/subroundStartRound.go +++ b/consensus/spos/bls/subroundStartRound.go @@ -342,6 +342,9 @@ func (sr *subroundStartRound) generateNextConsensusGroup(roundIndex int64) error sr.SetConsensusGroup(nextConsensusGroup) + consensusGroupSizeForEpoch := sr.NodesCoordinator().ConsensusGroupSizeForShardAndEpoch(shardId, currentHeader.GetEpoch()) + sr.SetConsensusGroupSize(consensusGroupSizeForEpoch) + return nil } diff --git a/consensus/spos/roundConsensus.go b/consensus/spos/roundConsensus.go index b230e124a15..73e87242b63 100644 --- a/consensus/spos/roundConsensus.go +++ b/consensus/spos/roundConsensus.go @@ -88,8 +88,8 @@ func (rcns *roundConsensus) ConsensusGroupSize() int { } // SetConsensusGroupSize sets the consensus group size -func (rcns *roundConsensus) SetConsensusGroupSize(consensusGroudpSize int) { - rcns.consensusGroupSize = consensusGroudpSize +func (rcns *roundConsensus) SetConsensusGroupSize(consensusGroupSize int) { + rcns.consensusGroupSize = consensusGroupSize } // SelfPubKey returns selfPubKey ID diff --git a/epochStart/bootstrap/disabled/disabledAntiFloodHandler.go b/epochStart/bootstrap/disabled/disabledAntiFloodHandler.go index cc1065b9d98..96d656b9e13 100644 --- a/epochStart/bootstrap/disabled/disabledAntiFloodHandler.go +++ b/epochStart/bootstrap/disabled/disabledAntiFloodHandler.go @@ -29,8 +29,8 @@ func (a *antiFloodHandler) CanProcessMessagesOnTopic(_ core.PeerID, _ string, _ return nil } -// ApplyConsensusSize does nothing -func (a *antiFloodHandler) ApplyConsensusSize(_ int) { +// SetConsensusSizeNotifier does nothing +func (a *antiFloodHandler) SetConsensusSizeNotifier(_ process.ChainParametersSubscriber, _ uint32) { } // SetDebugger returns nil diff --git a/epochStart/bootstrap/disabled/disabledNodesCoordinator.go b/epochStart/bootstrap/disabled/disabledNodesCoordinator.go index f7c1502d0c4..e204aec7cc8 100644 --- a/epochStart/bootstrap/disabled/disabledNodesCoordinator.go +++ b/epochStart/bootstrap/disabled/disabledNodesCoordinator.go @@ -103,8 +103,8 @@ func (n *nodesCoordinator) GetConsensusWhitelistedNodes(_ uint32) (map[string]st return nil, nil } -// ConsensusGroupSize - -func (n *nodesCoordinator) ConsensusGroupSize(uint32) int { +// ConsensusGroupSizeForShardAndEpoch - +func (n *nodesCoordinator) ConsensusGroupSizeForShardAndEpoch(uint32, uint32) int { return 0 } diff --git a/epochStart/bootstrap/process.go b/epochStart/bootstrap/process.go index dce9135e0a3..27fc5011cb5 100644 --- a/epochStart/bootstrap/process.go +++ b/epochStart/bootstrap/process.go @@ -759,19 +759,20 @@ func (e *epochStartBootstrap) processNodesConfig(pubKey []byte) ([]*block.MiniBl shardId = e.genesisShardCoordinator.SelfId() } argsNewValidatorStatusSyncers := ArgsNewSyncValidatorStatus{ - DataPool: e.dataPool, - Marshalizer: e.coreComponentsHolder.InternalMarshalizer(), - RequestHandler: e.requestHandler, - ChanceComputer: e.rater, - GenesisNodesConfig: e.genesisNodesConfig, - NodeShuffler: e.nodeShuffler, - Hasher: e.coreComponentsHolder.Hasher(), - PubKey: pubKey, - ShardIdAsObserver: shardId, - ChanNodeStop: e.coreComponentsHolder.ChanStopNodeProcess(), - NodeTypeProvider: e.coreComponentsHolder.NodeTypeProvider(), - IsFullArchive: e.prefsConfig.FullArchive, - EnableEpochsHandler: e.coreComponentsHolder.EnableEpochsHandler(), + DataPool: e.dataPool, + Marshalizer: e.coreComponentsHolder.InternalMarshalizer(), + RequestHandler: e.requestHandler, + ChanceComputer: e.rater, + GenesisNodesConfig: e.genesisNodesConfig, + ChainParametersHandler: e.coreComponentsHolder.ChainParametersHandler(), + NodeShuffler: e.nodeShuffler, + Hasher: e.coreComponentsHolder.Hasher(), + PubKey: pubKey, + ShardIdAsObserver: shardId, + ChanNodeStop: e.coreComponentsHolder.ChanStopNodeProcess(), + NodeTypeProvider: e.coreComponentsHolder.NodeTypeProvider(), + IsFullArchive: e.prefsConfig.FullArchive, + EnableEpochsHandler: e.coreComponentsHolder.EnableEpochsHandler(), NodesCoordinatorRegistryFactory: e.nodesCoordinatorRegistryFactory, } diff --git a/epochStart/bootstrap/process_test.go b/epochStart/bootstrap/process_test.go index ca3fd78a5b8..d0570d3983a 100644 --- a/epochStart/bootstrap/process_test.go +++ b/epochStart/bootstrap/process_test.go @@ -35,6 +35,7 @@ import ( "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" epochStartMocks "github.com/multiversx/mx-chain-go/testscommon/bootstrapMocks/epochStart" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" @@ -73,6 +74,14 @@ func createPkBytes(numShards uint32) map[uint32][]byte { } func createComponentsForEpochStart() (*mock.CoreComponentsMock, *mock.CryptoComponentsMock) { + chainParams := &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + } return &mock.CoreComponentsMock{ IntMarsh: &mock.MarshalizerMock{}, Marsh: &mock.MarshalizerMock{}, @@ -94,6 +103,7 @@ func createComponentsForEpochStart() (*mock.CoreComponentsMock, *mock.CryptoComp return 0 }, }, + ChainParametersHandlerField: chainParams, }, &mock.CryptoComponentsMock{ PubKey: &cryptoMocks.PublicKeyStub{}, diff --git a/epochStart/bootstrap/storageProcess.go b/epochStart/bootstrap/storageProcess.go index 809b0dfbb8b..a7fff35f193 100644 --- a/epochStart/bootstrap/storageProcess.go +++ b/epochStart/bootstrap/storageProcess.go @@ -404,19 +404,20 @@ func (sesb *storageEpochStartBootstrap) processNodesConfig(pubKey []byte) error shardId = sesb.genesisShardCoordinator.SelfId() } argsNewValidatorStatusSyncers := ArgsNewSyncValidatorStatus{ - DataPool: sesb.dataPool, - Marshalizer: sesb.coreComponentsHolder.InternalMarshalizer(), - RequestHandler: sesb.requestHandler, - ChanceComputer: sesb.rater, - GenesisNodesConfig: sesb.genesisNodesConfig, - NodeShuffler: sesb.nodeShuffler, - Hasher: sesb.coreComponentsHolder.Hasher(), - PubKey: pubKey, - ShardIdAsObserver: shardId, - ChanNodeStop: sesb.coreComponentsHolder.ChanStopNodeProcess(), - NodeTypeProvider: sesb.coreComponentsHolder.NodeTypeProvider(), - IsFullArchive: sesb.prefsConfig.FullArchive, - EnableEpochsHandler: sesb.coreComponentsHolder.EnableEpochsHandler(), + DataPool: sesb.dataPool, + Marshalizer: sesb.coreComponentsHolder.InternalMarshalizer(), + RequestHandler: sesb.requestHandler, + ChanceComputer: sesb.rater, + GenesisNodesConfig: sesb.genesisNodesConfig, + ChainParametersHandler: sesb.coreComponentsHolder.ChainParametersHandler(), + NodeShuffler: sesb.nodeShuffler, + Hasher: sesb.coreComponentsHolder.Hasher(), + PubKey: pubKey, + ShardIdAsObserver: shardId, + ChanNodeStop: sesb.coreComponentsHolder.ChanStopNodeProcess(), + NodeTypeProvider: sesb.coreComponentsHolder.NodeTypeProvider(), + IsFullArchive: sesb.prefsConfig.FullArchive, + EnableEpochsHandler: sesb.coreComponentsHolder.EnableEpochsHandler(), NodesCoordinatorRegistryFactory: sesb.nodesCoordinatorRegistryFactory, } sesb.nodesConfigHandler, err = NewSyncValidatorStatus(argsNewValidatorStatusSyncers) diff --git a/epochStart/bootstrap/syncValidatorStatus.go b/epochStart/bootstrap/syncValidatorStatus.go index 0bcb9308311..3279a6043dc 100644 --- a/epochStart/bootstrap/syncValidatorStatus.go +++ b/epochStart/bootstrap/syncValidatorStatus.go @@ -38,19 +38,20 @@ type syncValidatorStatus struct { // ArgsNewSyncValidatorStatus holds the arguments needed for creating a new validator status process component type ArgsNewSyncValidatorStatus struct { - DataPool dataRetriever.PoolsHolder - Marshalizer marshal.Marshalizer - Hasher hashing.Hasher - RequestHandler process.RequestHandler - ChanceComputer nodesCoordinator.ChanceComputer - GenesisNodesConfig sharding.GenesisNodesSetupHandler - NodeShuffler nodesCoordinator.NodesShuffler - PubKey []byte - ShardIdAsObserver uint32 - ChanNodeStop chan endProcess.ArgEndProcess - NodeTypeProvider NodeTypeProviderHandler - IsFullArchive bool - EnableEpochsHandler common.EnableEpochsHandler + DataPool dataRetriever.PoolsHolder + Marshalizer marshal.Marshalizer + Hasher hashing.Hasher + RequestHandler process.RequestHandler + ChanceComputer nodesCoordinator.ChanceComputer + GenesisNodesConfig sharding.GenesisNodesSetupHandler + ChainParametersHandler process.ChainParametersHandler + NodeShuffler nodesCoordinator.NodesShuffler + PubKey []byte + ShardIdAsObserver uint32 + ChanNodeStop chan endProcess.ArgEndProcess + NodeTypeProvider NodeTypeProviderHandler + IsFullArchive bool + EnableEpochsHandler common.EnableEpochsHandler NodesCoordinatorRegistryFactory nodesCoordinator.NodesCoordinatorRegistryFactory } @@ -112,26 +113,25 @@ func NewSyncValidatorStatus(args ArgsNewSyncValidatorStatus) (*syncValidatorStat s.memDB = disabled.CreateMemUnit() argsNodesCoordinator := nodesCoordinator.ArgNodesCoordinator{ - ShardConsensusGroupSize: int(args.GenesisNodesConfig.GetShardConsensusGroupSize()), - MetaConsensusGroupSize: int(args.GenesisNodesConfig.GetMetaConsensusGroupSize()), - Marshalizer: args.Marshalizer, - Hasher: args.Hasher, - Shuffler: args.NodeShuffler, - EpochStartNotifier: &disabled.EpochStartNotifier{}, - BootStorer: s.memDB, - ShardIDAsObserver: args.ShardIdAsObserver, - NbShards: args.GenesisNodesConfig.NumberOfShards(), - EligibleNodes: eligibleValidators, - WaitingNodes: waitingValidators, - SelfPublicKey: args.PubKey, - ConsensusGroupCache: consensusGroupCache, - ShuffledOutHandler: disabled.NewShuffledOutHandler(), - ChanStopNode: args.ChanNodeStop, - NodeTypeProvider: args.NodeTypeProvider, - IsFullArchive: args.IsFullArchive, - EnableEpochsHandler: args.EnableEpochsHandler, - ValidatorInfoCacher: s.dataPool.CurrentEpochValidatorInfo(), - GenesisNodesSetupHandler: s.genesisNodesConfig, + ChainParametersHandler: args.ChainParametersHandler, + Marshalizer: args.Marshalizer, + Hasher: args.Hasher, + Shuffler: args.NodeShuffler, + EpochStartNotifier: &disabled.EpochStartNotifier{}, + BootStorer: s.memDB, + ShardIDAsObserver: args.ShardIdAsObserver, + NbShards: args.GenesisNodesConfig.NumberOfShards(), + EligibleNodes: eligibleValidators, + WaitingNodes: waitingValidators, + SelfPublicKey: args.PubKey, + ConsensusGroupCache: consensusGroupCache, + ShuffledOutHandler: disabled.NewShuffledOutHandler(), + ChanStopNode: args.ChanNodeStop, + NodeTypeProvider: args.NodeTypeProvider, + IsFullArchive: args.IsFullArchive, + EnableEpochsHandler: args.EnableEpochsHandler, + ValidatorInfoCacher: s.dataPool.CurrentEpochValidatorInfo(), + GenesisNodesSetupHandler: s.genesisNodesConfig, NodesCoordinatorRegistryFactory: args.NodesCoordinatorRegistryFactory, } baseNodesCoordinator, err := nodesCoordinator.NewIndexHashedNodesCoordinator(argsNodesCoordinator) diff --git a/epochStart/bootstrap/syncValidatorStatus_test.go b/epochStart/bootstrap/syncValidatorStatus_test.go index 7cfe6061c77..2579596ed51 100644 --- a/epochStart/bootstrap/syncValidatorStatus_test.go +++ b/epochStart/bootstrap/syncValidatorStatus_test.go @@ -15,6 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" epochStartMocks "github.com/multiversx/mx-chain-go/testscommon/bootstrapMocks/epochStart" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/genesisMocks" @@ -261,10 +262,11 @@ func getSyncValidatorStatusArgs() ArgsNewSyncValidatorStatus { return &vic.ValidatorInfoCacherStub{} }, }, - Marshalizer: &mock.MarshalizerMock{}, - Hasher: &hashingMocks.HasherMock{}, - RequestHandler: &testscommon.RequestHandlerStub{}, - ChanceComputer: &shardingMocks.NodesCoordinatorStub{}, + Marshalizer: &mock.MarshalizerMock{}, + Hasher: &hashingMocks.HasherMock{}, + RequestHandler: &testscommon.RequestHandlerStub{}, + ChanceComputer: &shardingMocks.NodesCoordinatorStub{}, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{}, GenesisNodesConfig: &genesisMocks.NodesSetupStub{ NumberOfShardsCalled: func() uint32 { return 1 diff --git a/epochStart/interface.go b/epochStart/interface.go index 37df49df292..a5cb5881ddf 100644 --- a/epochStart/interface.go +++ b/epochStart/interface.go @@ -116,7 +116,7 @@ type StartOfEpochMetaSyncer interface { // NodesConfigProvider will provide the necessary information for start in epoch economics block creation type NodesConfigProvider interface { - ConsensusGroupSize(shardID uint32) int + ConsensusGroupSizeForShardAndEpoch(shardID uint32, epoch uint32) int IsInterfaceNil() bool } diff --git a/epochStart/metachain/baseRewards.go b/epochStart/metachain/baseRewards.go index b48cd8b7470..7de6e9d08db 100644 --- a/epochStart/metachain/baseRewards.go +++ b/epochStart/metachain/baseRewards.go @@ -451,19 +451,30 @@ func (brc *baseRewardsCreator) addExecutionOrdering(txHashes [][]byte) { } } -func (brc *baseRewardsCreator) fillBaseRewardsPerBlockPerNode(baseRewardsPerNode *big.Int) { +func (brc *baseRewardsCreator) fillBaseRewardsPerBlockPerNode(baseRewardsPerNode *big.Int, epoch uint32) { brc.mapBaseRewardsPerBlockPerValidator = make(map[uint32]*big.Int) for i := uint32(0); i < brc.shardCoordinator.NumberOfShards(); i++ { - consensusSize := big.NewInt(int64(brc.nodesConfigProvider.ConsensusGroupSize(i))) + consensusSize := big.NewInt(int64(brc.getConsensusGroupSizeForShardAndEpoch(i, epoch))) brc.mapBaseRewardsPerBlockPerValidator[i] = big.NewInt(0).Div(baseRewardsPerNode, consensusSize) log.Debug("baseRewardsPerBlockPerValidator", "shardID", i, "value", brc.mapBaseRewardsPerBlockPerValidator[i].String()) } - consensusSize := big.NewInt(int64(brc.nodesConfigProvider.ConsensusGroupSize(core.MetachainShardId))) + consensusSize := big.NewInt(int64(brc.getConsensusGroupSizeForShardAndEpoch(core.MetachainShardId, epoch))) brc.mapBaseRewardsPerBlockPerValidator[core.MetachainShardId] = big.NewInt(0).Div(baseRewardsPerNode, consensusSize) log.Debug("baseRewardsPerBlockPerValidator", "shardID", core.MetachainShardId, "value", brc.mapBaseRewardsPerBlockPerValidator[core.MetachainShardId].String()) } +func (brc *baseRewardsCreator) getConsensusGroupSizeForShardAndEpoch(shardID uint32, epoch uint32) int { + if epoch == 0 { + log.Warn("getConsensusGroupSizeForShardAndEpoch called for epoch 0", "shardID", shardID) + return brc.nodesConfigProvider.ConsensusGroupSizeForShardAndEpoch(shardID, 0) + } + + // use previous epoch for fetching the consensus group size, since the epoch start metablock already contains the new epoch + epochForConsensusSize := epoch - 1 + return brc.nodesConfigProvider.ConsensusGroupSizeForShardAndEpoch(shardID, epochForConsensusSize) +} + func (brc *baseRewardsCreator) verifyCreatedRewardMiniBlocksWithMetaBlock(metaBlock data.HeaderHandler, createdMiniBlocks block.MiniBlockSlice) error { numReceivedRewardsMBs := 0 for _, miniBlockHdr := range metaBlock.GetMiniBlockHeaderHandlers() { diff --git a/epochStart/metachain/baseRewards_test.go b/epochStart/metachain/baseRewards_test.go index 50aeb42e7ad..6907dff9727 100644 --- a/epochStart/metachain/baseRewards_test.go +++ b/epochStart/metachain/baseRewards_test.go @@ -1026,15 +1026,23 @@ func TestBaseRewardsCreator_finalizeMiniBlocksEmptyMbsAreRemoved(t *testing.T) { func TestBaseRewardsCreator_fillBaseRewardsPerBlockPerNode(t *testing.T) { t.Parallel() + // should work for epoch 0 even if this is a bad input + testFillBaseRewardsPerBlockPerNode(t, 0) + + // should work for an epoch higher than 0 + testFillBaseRewardsPerBlockPerNode(t, 1) +} + +func testFillBaseRewardsPerBlockPerNode(t *testing.T, epoch uint32) { args := getBaseRewardsArguments() rwd, err := NewBaseRewardsCreator(args) require.Nil(t, err) require.NotNil(t, rwd) baseRewardsPerNode := big.NewInt(1000000) - rwd.fillBaseRewardsPerBlockPerNode(baseRewardsPerNode) - consensusShard := args.NodesConfigProvider.ConsensusGroupSize(0) - consensusMeta := args.NodesConfigProvider.ConsensusGroupSize(core.MetachainShardId) + rwd.fillBaseRewardsPerBlockPerNode(baseRewardsPerNode, epoch) + consensusShard := args.NodesConfigProvider.ConsensusGroupSizeForShardAndEpoch(0, epoch) + consensusMeta := args.NodesConfigProvider.ConsensusGroupSizeForShardAndEpoch(core.MetachainShardId, epoch) expectedRewardPerNodeInShard := big.NewInt(0).Div(baseRewardsPerNode, big.NewInt(int64(consensusShard))) expectedRewardPerNodeInMeta := big.NewInt(0).Div(baseRewardsPerNode, big.NewInt(int64(consensusMeta))) @@ -1200,7 +1208,7 @@ func getBaseRewardsArguments() BaseRewardsCreatorArgs { DataPool: dataRetrieverMock.NewPoolsHolderMock(), ProtocolSustainabilityAddress: "11", // string hex => 17 decimal NodesConfigProvider: &shardingMocks.NodesCoordinatorStub{ - ConsensusGroupSizeCalled: func(shardID uint32) int { + ConsensusGroupSizeCalled: func(shardID uint32, _ uint32) int { if shardID == core.MetachainShardId { return 400 } diff --git a/epochStart/metachain/rewards.go b/epochStart/metachain/rewards.go index 0b279d56c32..368a5bec809 100644 --- a/epochStart/metachain/rewards.go +++ b/epochStart/metachain/rewards.go @@ -77,7 +77,7 @@ func (rc *rewardsCreator) CreateRewardsMiniBlocks( return nil, err } - rc.fillBaseRewardsPerBlockPerNode(economicsData.GetRewardsPerBlock()) + rc.fillBaseRewardsPerBlockPerNode(economicsData.GetRewardsPerBlock(), metaBlock.GetEpoch()) err = rc.addValidatorRewardsToMiniBlocks(validatorsInfo, metaBlock, miniBlocks, protSustRwdTx) if err != nil { return nil, err diff --git a/epochStart/metachain/rewardsV2.go b/epochStart/metachain/rewardsV2.go index ddfc05abcfe..5fce4c54b31 100644 --- a/epochStart/metachain/rewardsV2.go +++ b/epochStart/metachain/rewardsV2.go @@ -107,7 +107,7 @@ func (rc *rewardsCreatorV2) CreateRewardsMiniBlocks( return nil, err } - nodesRewardInfo, dustFromRewardsPerNode := rc.computeRewardsPerNode(validatorsInfo) + nodesRewardInfo, dustFromRewardsPerNode := rc.computeRewardsPerNode(validatorsInfo, metaBlock.GetEpoch()) log.Debug("arithmetic difference from dust rewards per node", "value", dustFromRewardsPerNode) dust, err := rc.addValidatorRewardsToMiniBlocks(metaBlock, miniBlocks, nodesRewardInfo) @@ -264,6 +264,7 @@ func (rc *rewardsCreatorV2) IsInterfaceNil() bool { func (rc *rewardsCreatorV2) computeRewardsPerNode( validatorsInfo state.ShardValidatorsInfoMapHandler, + epoch uint32, ) (map[uint32][]*nodeRewardsData, *big.Int) { var baseRewardsPerBlock *big.Int @@ -289,7 +290,7 @@ func (rc *rewardsCreatorV2) computeRewardsPerNode( "baseRewards", baseRewards.String(), "topUpRewards", topUpRewards.String()) - rc.fillBaseRewardsPerBlockPerNode(baseRewardsPerBlock) + rc.fillBaseRewardsPerBlockPerNode(baseRewardsPerBlock, epoch) accumulatedDust := big.NewInt(0) dust := rc.computeBaseRewardsPerNode(nodesRewardInfo, baseRewards) diff --git a/epochStart/metachain/rewardsV2_test.go b/epochStart/metachain/rewardsV2_test.go index 7abea51dea3..edbe575ff7c 100644 --- a/epochStart/metachain/rewardsV2_test.go +++ b/epochStart/metachain/rewardsV2_test.go @@ -708,7 +708,7 @@ func TestNewRewardsCreatorV2_computeBaseRewardsPerNode(t *testing.T) { for shardID := range shardMap { rwd.mapBaseRewardsPerBlockPerValidator[shardID] = big.NewInt(0).Set(baseRewardPerBlock) - cnsSize := big.NewInt(0).SetInt64(int64(args.NodesConfigProvider.ConsensusGroupSize(shardID))) + cnsSize := big.NewInt(0).SetInt64(int64(args.NodesConfigProvider.ConsensusGroupSizeForShardAndEpoch(shardID, 0))) rwd.mapBaseRewardsPerBlockPerValidator[shardID].Div(rwd.mapBaseRewardsPerBlockPerValidator[shardID], cnsSize) } @@ -769,7 +769,7 @@ func TestNewRewardsCreatorV2_computeRewardsPerNode(t *testing.T) { require.Nil(t, err) require.NotNil(t, rwd) - nodesRewardInfo, accumulatedDust := rwd.computeRewardsPerNode(vInfo) + nodesRewardInfo, accumulatedDust := rwd.computeRewardsPerNode(vInfo, 0) // dust should be really small, checking against baseRewards/1mil limit := core.GetApproximatePercentageOfValue(rewardsForBlocks, 0.000001) @@ -1063,7 +1063,7 @@ func TestNewRewardsCreatorV35_computeRewardsPer3200NodesWithDifferentTopups(t *t } rwd, _ := NewRewardsCreatorV2(setupResult.RewardsCreatorArgsV2) - nodesRewardInfo, _ = rwd.computeRewardsPerNode(vInfo) + nodesRewardInfo, _ = rwd.computeRewardsPerNode(vInfo, 0) currentRewardInfo := nodesRewardInfo[0][0] @@ -1170,7 +1170,7 @@ func TestNewRewardsCreatorV2_computeRewardsPer3200NodesWithDifferentTopups(t *te } rwd, _ := NewRewardsCreatorV2(setupResult.RewardsCreatorArgsV2) - nodesRewardInfo, _ = rwd.computeRewardsPerNode(vInfo) + nodesRewardInfo, _ = rwd.computeRewardsPerNode(vInfo, 0) currentRewardInfo := nodesRewardInfo[0][0] @@ -1292,7 +1292,7 @@ func computeRewardsAndDust(nbEligiblePerShard uint32, args SetupRewardsResult, t rwd, _ := NewRewardsCreatorV2(args.RewardsCreatorArgsV2) var dust *big.Int - nodesRewardInfo, dust = rwd.computeRewardsPerNode(vInfo) + nodesRewardInfo, dust = rwd.computeRewardsPerNode(vInfo, 0) return nodesRewardInfo, dust } @@ -1873,8 +1873,8 @@ func createDefaultValidatorInfo( proposerFeesPerNode uint32, nbBlocksPerShard uint32, ) state.ShardValidatorsInfoMapHandler { - cGrShard := uint32(nodesConfigProvider.ConsensusGroupSize(0)) - cGrMeta := uint32(nodesConfigProvider.ConsensusGroupSize(core.MetachainShardId)) + cGrShard := uint32(nodesConfigProvider.ConsensusGroupSizeForShardAndEpoch(0, 0)) + cGrMeta := uint32(nodesConfigProvider.ConsensusGroupSizeForShardAndEpoch(core.MetachainShardId, 0)) nbBlocksSelectedNodeInShard := nbBlocksPerShard * cGrShard / eligibleNodesPerShard nbBlocksSelectedNodeInMeta := nbBlocksPerShard * cGrMeta / eligibleNodesPerShard diff --git a/epochStart/metachain/rewards_test.go b/epochStart/metachain/rewards_test.go index b40fe8882e9..da570b1544f 100644 --- a/epochStart/metachain/rewards_test.go +++ b/epochStart/metachain/rewards_test.go @@ -607,7 +607,7 @@ func TestRewardsCreator_addValidatorRewardsToMiniBlocks(t *testing.T) { LeaderSuccess: 1, }) - rwdc.fillBaseRewardsPerBlockPerNode(mb.EpochStart.Economics.RewardsPerBlock) + rwdc.fillBaseRewardsPerBlockPerNode(mb.EpochStart.Economics.RewardsPerBlock, 0) err := rwdc.addValidatorRewardsToMiniBlocks(valInfo, mb, miniBlocks, &rewardTx.RewardTx{}) assert.Nil(t, err) assert.Equal(t, cloneMb, miniBlocks[0]) @@ -618,7 +618,7 @@ func TestRewardsCreator_ProtocolRewardsForValidatorFromMultipleShards(t *testing args := getRewardsArguments() args.NodesConfigProvider = &shardingMocks.NodesCoordinatorStub{ - ConsensusGroupSizeCalled: func(shardID uint32) int { + ConsensusGroupSizeCalled: func(shardID uint32, _ uint32) int { if shardID == core.MetachainShardId { return 400 } @@ -648,15 +648,15 @@ func TestRewardsCreator_ProtocolRewardsForValidatorFromMultipleShards(t *testing LeaderSuccess: 1, }) - rwdc.fillBaseRewardsPerBlockPerNode(mb.EpochStart.Economics.RewardsPerBlock) + rwdc.fillBaseRewardsPerBlockPerNode(mb.EpochStart.Economics.RewardsPerBlock, 0) rwdInfoData := rwdc.computeValidatorInfoPerRewardAddress(valInfo, &rewardTx.RewardTx{}, 0) assert.Equal(t, 1, len(rwdInfoData)) rwdInfo := rwdInfoData[pubkey] assert.Equal(t, rwdInfo.address, pubkey) assert.Equal(t, rwdInfo.accumulatedFees.Cmp(big.NewInt(200)), 0) - protocolRewards := uint64(valInfo.GetShardValidatorsInfoMap()[0][0].GetNumSelectedInSuccessBlocks()) * (mb.EpochStart.Economics.RewardsPerBlock.Uint64() / uint64(args.NodesConfigProvider.ConsensusGroupSize(0))) - protocolRewards += uint64(valInfo.GetShardValidatorsInfoMap()[core.MetachainShardId][0].GetNumSelectedInSuccessBlocks()) * (mb.EpochStart.Economics.RewardsPerBlock.Uint64() / uint64(args.NodesConfigProvider.ConsensusGroupSize(core.MetachainShardId))) + protocolRewards := uint64(valInfo.GetShardValidatorsInfoMap()[0][0].GetNumSelectedInSuccessBlocks()) * (mb.EpochStart.Economics.RewardsPerBlock.Uint64() / uint64(args.NodesConfigProvider.ConsensusGroupSizeForShardAndEpoch(0, 0))) + protocolRewards += uint64(valInfo.GetShardValidatorsInfoMap()[core.MetachainShardId][0].GetNumSelectedInSuccessBlocks()) * (mb.EpochStart.Economics.RewardsPerBlock.Uint64() / uint64(args.NodesConfigProvider.ConsensusGroupSizeForShardAndEpoch(core.MetachainShardId, 0))) assert.Equal(t, rwdInfo.rewardsFromProtocol.Uint64(), protocolRewards) } diff --git a/epochStart/metachain/systemSCs_test.go b/epochStart/metachain/systemSCs_test.go index c97d3cdbdd6..0d1dbe0592a 100644 --- a/epochStart/metachain/systemSCs_test.go +++ b/epochStart/metachain/systemSCs_test.go @@ -963,7 +963,7 @@ func createFullArgumentsForSystemSCProcessing(enableEpochsConfig config.EnableEp StakingDataProvider: stakingSCProvider, AuctionListSelector: als, NodesConfigProvider: &shardingMocks.NodesCoordinatorStub{ - ConsensusGroupSizeCalled: func(shardID uint32) int { + ConsensusGroupSizeCalled: func(shardID uint32, _ uint32) int { if shardID == core.MetachainShardId { return 400 } diff --git a/epochStart/mock/coreComponentsMock.go b/epochStart/mock/coreComponentsMock.go index b2f0003d842..eb8474b74bb 100644 --- a/epochStart/mock/coreComponentsMock.go +++ b/epochStart/mock/coreComponentsMock.go @@ -12,29 +12,32 @@ import ( "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" ) // CoreComponentsMock - type CoreComponentsMock struct { - IntMarsh marshal.Marshalizer - Marsh marshal.Marshalizer - Hash hashing.Hasher - EpochNotifierField process.EpochNotifier - EnableEpochsHandlerField common.EnableEpochsHandler - TxSignHasherField hashing.Hasher - UInt64ByteSliceConv typeConverters.Uint64ByteSliceConverter - AddrPubKeyConv core.PubkeyConverter - ValPubKeyConv core.PubkeyConverter - PathHdl storage.PathManagerHandler - ChainIdCalled func() string - MinTransactionVersionCalled func() uint32 - GenesisNodesSetupCalled func() sharding.GenesisNodesSetupHandler - TxVersionCheckField process.TxVersionCheckerHandler - ChanStopNode chan endProcess.ArgEndProcess - NodeTypeProviderField core.NodeTypeProviderHandler - ProcessStatusHandlerInstance common.ProcessStatusHandler - HardforkTriggerPubKeyField []byte - mutCore sync.RWMutex + IntMarsh marshal.Marshalizer + Marsh marshal.Marshalizer + Hash hashing.Hasher + EpochNotifierField process.EpochNotifier + EnableEpochsHandlerField common.EnableEpochsHandler + TxSignHasherField hashing.Hasher + UInt64ByteSliceConv typeConverters.Uint64ByteSliceConverter + AddrPubKeyConv core.PubkeyConverter + ValPubKeyConv core.PubkeyConverter + PathHdl storage.PathManagerHandler + ChainIdCalled func() string + MinTransactionVersionCalled func() uint32 + GenesisNodesSetupCalled func() sharding.GenesisNodesSetupHandler + TxVersionCheckField process.TxVersionCheckerHandler + ChanStopNode chan endProcess.ArgEndProcess + NodeTypeProviderField core.NodeTypeProviderHandler + ProcessStatusHandlerInstance common.ProcessStatusHandler + HardforkTriggerPubKeyField []byte + ChainParametersHandlerField process.ChainParametersHandler + ChainParametersSubscriberField process.ChainParametersSubscriber + mutCore sync.RWMutex } // ChanStopNodeProcess - @@ -155,6 +158,20 @@ func (ccm *CoreComponentsMock) HardforkTriggerPubKey() []byte { return ccm.HardforkTriggerPubKeyField } +// ChainParametersHandler - +func (ccm *CoreComponentsMock) ChainParametersHandler() process.ChainParametersHandler { + if ccm.ChainParametersHandlerField != nil { + return ccm.ChainParametersHandlerField + } + + return &chainParameters.ChainParametersHolderMock{} +} + +// ChainParametersSubscriber - +func (ccm *CoreComponentsMock) ChainParametersSubscriber() process.ChainParametersSubscriber { + return ccm.ChainParametersSubscriberField +} + // IsInterfaceNil - func (ccm *CoreComponentsMock) IsInterfaceNil() bool { return ccm == nil diff --git a/errors/errors.go b/errors/errors.go index dd475327876..91b03a9780f 100644 --- a/errors/errors.go +++ b/errors/errors.go @@ -475,6 +475,9 @@ var ErrNilESDTDataStorage = errors.New("nil esdt data storage") // ErrNilEnableEpochsHandler signals that a nil enable epochs handler was provided var ErrNilEnableEpochsHandler = errors.New("nil enable epochs handler") +// ErrNilChainParametersHandler signals that a nil chain parameters handler was provided +var ErrNilChainParametersHandler = errors.New("nil chain parameters handler") + // ErrSignerNotSupported signals that a not supported signer was provided var ErrSignerNotSupported = errors.New("signer not supported") diff --git a/factory/bootstrap/shardingFactory.go b/factory/bootstrap/shardingFactory.go index 6662129299b..31ce5930142 100644 --- a/factory/bootstrap/shardingFactory.go +++ b/factory/bootstrap/shardingFactory.go @@ -16,6 +16,7 @@ import ( "github.com/multiversx/mx-chain-go/epochStart" errErd "github.com/multiversx/mx-chain-go/errors" "github.com/multiversx/mx-chain-go/factory" + "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/storage" @@ -114,6 +115,7 @@ func CreateNodesCoordinator( enableEpochsHandler common.EnableEpochsHandler, validatorInfoCacher epochStart.ValidatorInfoCacher, nodesCoordinatorRegistryFactory nodesCoordinator.NodesCoordinatorRegistryFactory, + chainParametersHandler process.ChainParametersHandler, ) (nodesCoordinator.NodesCoordinator, error) { if check.IfNil(nodeShufflerOut) { return nil, errErd.ErrNilShuffleOutCloser @@ -148,8 +150,6 @@ func CreateNodesCoordinator( } nbShards := nodesConfig.NumberOfShards() - shardConsensusGroupSize := int(nodesConfig.GetShardConsensusGroupSize()) - metaConsensusGroupSize := int(nodesConfig.GetMetaConsensusGroupSize()) eligibleNodesInfo, waitingNodesInfo := nodesConfig.InitialNodesInfo() eligibleValidators, errEligibleValidators := nodesCoordinator.NodesInfoToValidators(eligibleNodesInfo) @@ -198,28 +198,27 @@ func CreateNodesCoordinator( } argumentsNodesCoordinator := nodesCoordinator.ArgNodesCoordinator{ - ShardConsensusGroupSize: shardConsensusGroupSize, - MetaConsensusGroupSize: metaConsensusGroupSize, - Marshalizer: marshalizer, - Hasher: hasher, - Shuffler: nodeShuffler, - EpochStartNotifier: epochStartNotifier, - BootStorer: bootStorer, - ShardIDAsObserver: shardIDAsObserver, - NbShards: nbShards, - EligibleNodes: eligibleValidators, - WaitingNodes: waitingValidators, - SelfPublicKey: pubKeyBytes, - ConsensusGroupCache: consensusGroupCache, - ShuffledOutHandler: shuffledOutHandler, - Epoch: currentEpoch, - StartEpoch: startEpoch, - ChanStopNode: chanNodeStop, - NodeTypeProvider: nodeTypeProvider, - IsFullArchive: prefsConfig.FullArchive, - EnableEpochsHandler: enableEpochsHandler, - ValidatorInfoCacher: validatorInfoCacher, - GenesisNodesSetupHandler: nodesConfig, + ChainParametersHandler: chainParametersHandler, + Marshalizer: marshalizer, + Hasher: hasher, + Shuffler: nodeShuffler, + EpochStartNotifier: epochStartNotifier, + BootStorer: bootStorer, + ShardIDAsObserver: shardIDAsObserver, + NbShards: nbShards, + EligibleNodes: eligibleValidators, + WaitingNodes: waitingValidators, + SelfPublicKey: pubKeyBytes, + ConsensusGroupCache: consensusGroupCache, + ShuffledOutHandler: shuffledOutHandler, + Epoch: currentEpoch, + StartEpoch: startEpoch, + ChanStopNode: chanNodeStop, + NodeTypeProvider: nodeTypeProvider, + IsFullArchive: prefsConfig.FullArchive, + EnableEpochsHandler: enableEpochsHandler, + ValidatorInfoCacher: validatorInfoCacher, + GenesisNodesSetupHandler: nodesConfig, NodesCoordinatorRegistryFactory: nodesCoordinatorRegistryFactory, } diff --git a/factory/bootstrap/shardingFactory_test.go b/factory/bootstrap/shardingFactory_test.go index c7a54e077f4..4e6f118ab29 100644 --- a/factory/bootstrap/shardingFactory_test.go +++ b/factory/bootstrap/shardingFactory_test.go @@ -15,6 +15,7 @@ import ( "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapMocks" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/genesisMocks" @@ -210,6 +211,7 @@ func TestCreateNodesCoordinator(t *testing.T) { &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, &validatorInfoCacherMocks.ValidatorInfoCacherStub{}, &shardingMocks.NodesCoordinatorRegistryFactoryMock{}, + &chainParameters.ChainParametersHandlerStub{}, ) require.Equal(t, errErd.ErrNilShuffleOutCloser, err) require.True(t, check.IfNil(nodesC)) @@ -236,6 +238,7 @@ func TestCreateNodesCoordinator(t *testing.T) { &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, &validatorInfoCacherMocks.ValidatorInfoCacherStub{}, &shardingMocks.NodesCoordinatorRegistryFactoryMock{}, + &chainParameters.ChainParametersHandlerStub{}, ) require.Equal(t, errErd.ErrNilGenesisNodesSetupHandler, err) require.True(t, check.IfNil(nodesC)) @@ -262,6 +265,7 @@ func TestCreateNodesCoordinator(t *testing.T) { &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, &validatorInfoCacherMocks.ValidatorInfoCacherStub{}, &shardingMocks.NodesCoordinatorRegistryFactoryMock{}, + &chainParameters.ChainParametersHandlerStub{}, ) require.Equal(t, errErd.ErrNilEpochStartNotifier, err) require.True(t, check.IfNil(nodesC)) @@ -288,6 +292,7 @@ func TestCreateNodesCoordinator(t *testing.T) { &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, &validatorInfoCacherMocks.ValidatorInfoCacherStub{}, &shardingMocks.NodesCoordinatorRegistryFactoryMock{}, + &chainParameters.ChainParametersHandlerStub{}, ) require.Equal(t, errErd.ErrNilPublicKey, err) require.True(t, check.IfNil(nodesC)) @@ -314,6 +319,7 @@ func TestCreateNodesCoordinator(t *testing.T) { &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, &validatorInfoCacherMocks.ValidatorInfoCacherStub{}, &shardingMocks.NodesCoordinatorRegistryFactoryMock{}, + &chainParameters.ChainParametersHandlerStub{}, ) require.Equal(t, errErd.ErrNilBootstrapParamsHandler, err) require.True(t, check.IfNil(nodesC)) @@ -340,6 +346,7 @@ func TestCreateNodesCoordinator(t *testing.T) { &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, &validatorInfoCacherMocks.ValidatorInfoCacherStub{}, &shardingMocks.NodesCoordinatorRegistryFactoryMock{}, + &chainParameters.ChainParametersHandlerStub{}, ) require.Equal(t, nodesCoordinator.ErrNilNodeStopChannel, err) require.True(t, check.IfNil(nodesC)) @@ -368,6 +375,7 @@ func TestCreateNodesCoordinator(t *testing.T) { &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, &validatorInfoCacherMocks.ValidatorInfoCacherStub{}, &shardingMocks.NodesCoordinatorRegistryFactoryMock{}, + &chainParameters.ChainParametersHandlerStub{}, ) require.NotNil(t, err) require.True(t, check.IfNil(nodesC)) @@ -400,6 +408,7 @@ func TestCreateNodesCoordinator(t *testing.T) { &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, &validatorInfoCacherMocks.ValidatorInfoCacherStub{}, &shardingMocks.NodesCoordinatorRegistryFactoryMock{}, + &chainParameters.ChainParametersHandlerStub{}, ) require.True(t, errors.Is(err, expectedErr)) require.True(t, check.IfNil(nodesC)) @@ -432,6 +441,7 @@ func TestCreateNodesCoordinator(t *testing.T) { &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, &validatorInfoCacherMocks.ValidatorInfoCacherStub{}, &shardingMocks.NodesCoordinatorRegistryFactoryMock{}, + &chainParameters.ChainParametersHandlerStub{}, ) require.True(t, errors.Is(err, expectedErr)) require.True(t, check.IfNil(nodesC)) @@ -464,6 +474,7 @@ func TestCreateNodesCoordinator(t *testing.T) { &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, &validatorInfoCacherMocks.ValidatorInfoCacherStub{}, &shardingMocks.NodesCoordinatorRegistryFactoryMock{}, + &chainParameters.ChainParametersHandlerStub{}, ) require.NotNil(t, err) require.True(t, check.IfNil(nodesC)) @@ -496,6 +507,7 @@ func TestCreateNodesCoordinator(t *testing.T) { &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, &validatorInfoCacherMocks.ValidatorInfoCacherStub{}, &shardingMocks.NodesCoordinatorRegistryFactoryMock{}, + &chainParameters.ChainParametersHandlerStub{}, ) require.NotNil(t, err) require.True(t, check.IfNil(nodesC)) @@ -549,6 +561,7 @@ func TestCreateNodesCoordinator(t *testing.T) { &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, &validatorInfoCacherMocks.ValidatorInfoCacherStub{}, &shardingMocks.NodesCoordinatorRegistryFactoryMock{}, + &chainParameters.ChainParametersHandlerStub{}, ) require.NotNil(t, err) require.True(t, check.IfNil(nodesC)) @@ -602,6 +615,7 @@ func TestCreateNodesCoordinator(t *testing.T) { &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, &validatorInfoCacherMocks.ValidatorInfoCacherStub{}, &shardingMocks.NodesCoordinatorRegistryFactoryMock{}, + &chainParameters.ChainParametersHandlerStub{}, ) require.Nil(t, err) require.False(t, check.IfNil(nodesC)) diff --git a/factory/consensus/consensusComponents.go b/factory/consensus/consensusComponents.go index decdb7c85fa..c031744f12d 100644 --- a/factory/consensus/consensusComponents.go +++ b/factory/consensus/consensusComponents.go @@ -25,6 +25,7 @@ import ( "github.com/multiversx/mx-chain-go/process/sync" "github.com/multiversx/mx-chain-go/process/sync/storageBootstrap" "github.com/multiversx/mx-chain-go/sharding" + nodesCoord "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state/syncer" "github.com/multiversx/mx-chain-go/trie/statistics" "github.com/multiversx/mx-chain-go/update" @@ -78,7 +79,6 @@ type consensusComponents struct { worker factory.ConsensusWorker peerBlacklistHandler consensus.PeerBlacklistHandler consensusTopic string - consensusGroupSize int } // NewConsensusComponentsFactory creates an instance of consensusComponentsFactory @@ -112,13 +112,6 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { cc := &consensusComponents{} - consensusGroupSize, err := getConsensusGroupSize(ccf.coreComponents.GenesisNodesSetup(), ccf.processComponents.ShardCoordinator()) - if err != nil { - return nil, err - } - - cc.consensusGroupSize = int(consensusGroupSize) - blockchain := ccf.dataComponents.Blockchain() notInitializedGenesisBlock := len(blockchain.GetGenesisHeaderHash()) == 0 || check.IfNil(blockchain.GetGenesisHeader()) @@ -142,7 +135,12 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { } epoch := ccf.getEpoch() - consensusState, err := ccf.createConsensusState(epoch, cc.consensusGroupSize) + + consensusGroupSize, err := getConsensusGroupSize(ccf.coreComponents.GenesisNodesSetup(), ccf.processComponents.ShardCoordinator(), ccf.processComponents.NodesCoordinator(), epoch) + if err != nil { + return nil, err + } + consensusState, err := ccf.createConsensusState(epoch, consensusGroupSize) if err != nil { return nil, err } @@ -214,10 +212,9 @@ func (ccf *consensusComponentsFactory) Create() (*consensusComponents, error) { cc.worker.StartWorking() ccf.dataComponents.Datapool().Headers().RegisterHandler(cc.worker.ReceivedHeader) - // apply consensus group size on the input antiflooder just before consensus creation topic - ccf.networkComponents.InputAntiFloodHandler().ApplyConsensusSize( - ccf.processComponents.NodesCoordinator().ConsensusGroupSize( - ccf.processComponents.ShardCoordinator().SelfId()), + ccf.networkComponents.InputAntiFloodHandler().SetConsensusSizeNotifier( + ccf.coreComponents.ChainParametersSubscriber(), + ccf.processComponents.ShardCoordinator().SelfId(), ) err = ccf.createConsensusTopic(cc) if err != nil { @@ -752,12 +749,17 @@ func checkArgs(args ConsensusComponentsFactoryArgs) error { return nil } -func getConsensusGroupSize(nodesConfig sharding.GenesisNodesSetupHandler, shardCoordinator sharding.Coordinator) (uint32, error) { +func getConsensusGroupSize(nodesConfig sharding.GenesisNodesSetupHandler, shardCoordinator sharding.Coordinator, nodesCoordinator nodesCoord.NodesCoordinator, epoch uint32) (int, error) { + consensusGroupSize := nodesCoordinator.ConsensusGroupSizeForShardAndEpoch(shardCoordinator.SelfId(), epoch) + if consensusGroupSize > 0 { + return consensusGroupSize, nil + } + if shardCoordinator.SelfId() == core.MetachainShardId { - return nodesConfig.GetMetaConsensusGroupSize(), nil + return int(nodesConfig.GetMetaConsensusGroupSize()), nil } if shardCoordinator.SelfId() < shardCoordinator.NumberOfShards() { - return nodesConfig.GetShardConsensusGroupSize(), nil + return int(nodesConfig.GetShardConsensusGroupSize()), nil } return 0, sharding.ErrShardIdOutOfRange diff --git a/factory/consensus/consensusComponentsHandler.go b/factory/consensus/consensusComponentsHandler.go index 4e7779ab367..7fbaeb49381 100644 --- a/factory/consensus/consensusComponentsHandler.go +++ b/factory/consensus/consensusComponentsHandler.go @@ -101,18 +101,6 @@ func (mcc *managedConsensusComponents) BroadcastMessenger() consensus.BroadcastM return mcc.consensusComponents.broadcastMessenger } -// ConsensusGroupSize returns the consensus group size -func (mcc *managedConsensusComponents) ConsensusGroupSize() (int, error) { - mcc.mutConsensusComponents.RLock() - defer mcc.mutConsensusComponents.RUnlock() - - if mcc.consensusComponents == nil { - return 0, errors.ErrNilConsensusComponentsHolder - } - - return mcc.consensusComponents.consensusGroupSize, nil -} - // CheckSubcomponents verifies all subcomponents func (mcc *managedConsensusComponents) CheckSubcomponents() error { mcc.mutConsensusComponents.RLock() diff --git a/factory/consensus/consensusComponentsHandler_test.go b/factory/consensus/consensusComponentsHandler_test.go index c0a89f8a08e..ded44d2a837 100644 --- a/factory/consensus/consensusComponentsHandler_test.go +++ b/factory/consensus/consensusComponentsHandler_test.go @@ -74,24 +74,6 @@ func TestManagedConsensusComponents_Create(t *testing.T) { }) } -func TestManagedConsensusComponents_ConsensusGroupSize(t *testing.T) { - t.Parallel() - - consensusComponentsFactory, _ := consensusComp.NewConsensusComponentsFactory(createMockConsensusComponentsFactoryArgs()) - managedConsensusComponents, _ := consensusComp.NewManagedConsensusComponents(consensusComponentsFactory) - require.NotNil(t, managedConsensusComponents) - - size, err := managedConsensusComponents.ConsensusGroupSize() - require.Equal(t, errorsMx.ErrNilConsensusComponentsHolder, err) - require.Zero(t, size) - - err = managedConsensusComponents.Create() - require.NoError(t, err) - size, err = managedConsensusComponents.ConsensusGroupSize() - require.NoError(t, err) - require.Equal(t, 2, size) -} - func TestManagedConsensusComponents_CheckSubcomponents(t *testing.T) { t.Parallel() diff --git a/factory/consensus/consensusComponents_test.go b/factory/consensus/consensusComponents_test.go index a7b00e6a347..a8f175c0b52 100644 --- a/factory/consensus/consensusComponents_test.go +++ b/factory/consensus/consensusComponents_test.go @@ -114,7 +114,7 @@ func createMockConsensusComponentsFactoryArgs() consensusComp.ConsensusComponent ProcessComponents: &testsMocks.ProcessComponentsStub{ EpochTrigger: &testsMocks.EpochStartTriggerStub{}, EpochNotifier: &testsMocks.EpochStartNotifierStub{}, - NodesCoord: &shardingMocks.NodesCoordinatorStub{}, + NodesCoord: &shardingMocks.NodesCoordinatorMock{}, NodeRedundancyHandlerInternal: &testsMocks.RedundancyHandlerStub{}, HardforkTriggerField: &testscommon.HardforkTriggerStub{}, ReqHandler: &testscommon.RequestHandlerStub{}, @@ -315,7 +315,7 @@ func TestNewConsensusComponentsFactory(t *testing.T) { args := createMockConsensusComponentsFactoryArgs() args.ProcessComponents = &testsMocks.ProcessComponentsStub{ - NodesCoord: &shardingMocks.NodesCoordinatorStub{}, + NodesCoord: &shardingMocks.NodesCoordinatorMock{}, ShardCoord: nil, } ccf, err := consensusComp.NewConsensusComponentsFactory(args) @@ -328,7 +328,7 @@ func TestNewConsensusComponentsFactory(t *testing.T) { args := createMockConsensusComponentsFactoryArgs() args.ProcessComponents = &testsMocks.ProcessComponentsStub{ - NodesCoord: &shardingMocks.NodesCoordinatorStub{}, + NodesCoord: &shardingMocks.NodesCoordinatorMock{}, ShardCoord: &testscommon.ShardsCoordinatorMock{}, RoundHandlerField: nil, } @@ -342,7 +342,7 @@ func TestNewConsensusComponentsFactory(t *testing.T) { args := createMockConsensusComponentsFactoryArgs() args.ProcessComponents = &testsMocks.ProcessComponentsStub{ - NodesCoord: &shardingMocks.NodesCoordinatorStub{}, + NodesCoord: &shardingMocks.NodesCoordinatorMock{}, ShardCoord: &testscommon.ShardsCoordinatorMock{}, RoundHandlerField: &testscommon.RoundHandlerMock{}, HardforkTriggerField: nil, @@ -497,7 +497,7 @@ func TestConsensusComponentsFactory_Create(t *testing.T) { cnt := 0 processCompStub.ShardCoordinatorCalled = func() sharding.Coordinator { cnt++ - if cnt > 2 { + if cnt > 1 { return nil // createBootstrapper fails } return testscommon.NewMultiShardsCoordinatorMock(2) @@ -519,7 +519,7 @@ func TestConsensusComponentsFactory_Create(t *testing.T) { shardC := testscommon.NewMultiShardsCoordinatorMock(2) processCompStub.ShardCoordinatorCalled = func() sharding.Coordinator { cnt++ - if cnt > 2 { + if cnt > 1 { shardC.SelfIDCalled = func() uint32 { return shardC.NoShards + 1 // createBootstrapper returns ErrShardIdOutOfRange } @@ -534,28 +534,6 @@ func TestConsensusComponentsFactory_Create(t *testing.T) { require.Equal(t, sharding.ErrShardIdOutOfRange, err) require.Nil(t, cc) }) - t.Run("createShardBootstrapper fails due to NewShardStorageBootstrapper failure should error", func(t *testing.T) { - t.Parallel() - - args := createMockConsensusComponentsFactoryArgs() - processCompStub, ok := args.ProcessComponents.(*testsMocks.ProcessComponentsStub) - require.True(t, ok) - cnt := 0 - processCompStub.ShardCoordinatorCalled = func() sharding.Coordinator { - cnt++ - if cnt > 3 { - return nil // NewShardStorageBootstrapper fails - } - return testscommon.NewMultiShardsCoordinatorMock(2) - } - ccf, _ := consensusComp.NewConsensusComponentsFactory(args) - require.NotNil(t, ccf) - - cc, err := ccf.Create() - require.Error(t, err) - require.True(t, strings.Contains(err.Error(), "shard coordinator")) - require.Nil(t, cc) - }) t.Run("createUserAccountsSyncer fails due to missing UserAccountTrie should error", func(t *testing.T) { t.Parallel() @@ -583,30 +561,6 @@ func TestConsensusComponentsFactory_Create(t *testing.T) { require.True(t, strings.Contains(err.Error(), "value is not positive")) require.Nil(t, cc) }) - t.Run("createMetaChainBootstrapper fails due to NewMetaStorageBootstrapper failure should error", func(t *testing.T) { - t.Parallel() - - args := createMockConsensusComponentsFactoryArgs() - processCompStub, ok := args.ProcessComponents.(*testsMocks.ProcessComponentsStub) - require.True(t, ok) - cnt := 0 - processCompStub.ShardCoordinatorCalled = func() sharding.Coordinator { - cnt++ - if cnt > 3 { - return nil // NewShardStorageBootstrapper fails - } - shardC := testscommon.NewMultiShardsCoordinatorMock(2) - shardC.CurrentShard = core.MetachainShardId - return shardC - } - ccf, _ := consensusComp.NewConsensusComponentsFactory(args) - require.NotNil(t, ccf) - - cc, err := ccf.Create() - require.Error(t, err) - require.True(t, strings.Contains(err.Error(), "shard coordinator")) - require.Nil(t, cc) - }) t.Run("createUserAccountsSyncer fails due to missing UserAccountTrie should error", func(t *testing.T) { t.Parallel() @@ -697,27 +651,6 @@ func TestConsensusComponentsFactory_Create(t *testing.T) { require.Equal(t, expectedErr, err) require.Nil(t, cc) }) - t.Run("createConsensusState fails due to nil nodes coordinator should error", func(t *testing.T) { - t.Parallel() - - args := createMockConsensusComponentsFactoryArgs() - processCompStub, ok := args.ProcessComponents.(*testsMocks.ProcessComponentsStub) - require.True(t, ok) - cnt := 0 - processCompStub.NodesCoordinatorCalled = func() nodesCoordinator.NodesCoordinator { - cnt++ - if cnt > 2 { - return nil - } - return &shardingMocks.NodesCoordinatorStub{} - } - ccf, _ := consensusComp.NewConsensusComponentsFactory(args) - require.NotNil(t, ccf) - - cc, err := ccf.Create() - require.Equal(t, errorsMx.ErrNilNodesCoordinator, err) - require.Nil(t, cc) - }) t.Run("createConsensusState fails due to GetConsensusWhitelistedNodes failure should error", func(t *testing.T) { t.Parallel() @@ -725,7 +658,7 @@ func TestConsensusComponentsFactory_Create(t *testing.T) { processCompStub, ok := args.ProcessComponents.(*testsMocks.ProcessComponentsStub) require.True(t, ok) processCompStub.NodesCoordinatorCalled = func() nodesCoordinator.NodesCoordinator { - return &shardingMocks.NodesCoordinatorStub{ + return &shardingMocks.NodesCoordinatorMock{ GetConsensusWhitelistedNodesCalled: func(epoch uint32) (map[string]struct{}, error) { return nil, expectedErr }, diff --git a/factory/core/coreComponents.go b/factory/core/coreComponents.go index 247ee7e05f8..010296f3dd9 100644 --- a/factory/core/coreComponents.go +++ b/factory/core/coreComponents.go @@ -20,6 +20,7 @@ import ( "github.com/multiversx/mx-chain-core-go/marshal" marshalizerFactory "github.com/multiversx/mx-chain-core-go/marshal/factory" "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/common/chainparametersnotifier" "github.com/multiversx/mx-chain-go/common/enablers" commonFactory "github.com/multiversx/mx-chain-go/common/factory" "github.com/multiversx/mx-chain-go/common/forking" @@ -52,7 +53,7 @@ type CoreComponentsFactoryArgs struct { RatingsConfig config.RatingsConfig EconomicsConfig config.EconomicsConfig ImportDbConfig config.ImportDbConfig - NodesFilename string + NodesConfig config.NodesConfig WorkingDirectory string ChanStopNodeProcess chan endProcess.ArgEndProcess } @@ -66,7 +67,7 @@ type coreComponentsFactory struct { ratingsConfig config.RatingsConfig economicsConfig config.EconomicsConfig importDbConfig config.ImportDbConfig - nodesFilename string + nodesSetupConfig config.NodesConfig workingDir string chanStopNodeProcess chan endProcess.ArgEndProcess } @@ -98,6 +99,7 @@ type coreComponents struct { minTransactionVersion uint32 epochNotifier process.EpochNotifier roundNotifier process.RoundNotifier + chainParametersSubscriber process.ChainParametersSubscriber enableRoundsHandler process.EnableRoundsHandler epochStartNotifierWithConfirm factory.EpochStartNotifierWithConfirm chanStopNodeProcess chan endProcess.ArgEndProcess @@ -107,6 +109,7 @@ type coreComponents struct { processStatusHandler common.ProcessStatusHandler hardforkTriggerPubKey []byte enableEpochsHandler common.EnableEpochsHandler + chainParametersHandler process.ChainParametersHandler } // NewCoreComponentsFactory initializes the factory which is responsible to creating core components @@ -121,7 +124,7 @@ func NewCoreComponentsFactory(args CoreComponentsFactoryArgs) (*coreComponentsFa economicsConfig: args.EconomicsConfig, workingDir: args.WorkingDirectory, chanStopNodeProcess: args.ChanStopNodeProcess, - nodesFilename: args.NodesFilename, + nodesSetupConfig: args.NodesConfig, }, nil } @@ -178,8 +181,23 @@ func (ccf *coreComponentsFactory) Create() (*coreComponents, error) { syncer.StartSyncingTime() log.Debug("NTP average clock offset", "value", syncer.ClockOffset()) + epochNotifier := forking.NewGenericEpochNotifier() + epochStartHandlerWithConfirm := notifier.NewEpochStartSubscriptionHandler() + + chainParametersNotifier := chainparametersnotifier.NewChainParametersNotifier() + argsChainParametersHandler := sharding.ArgsChainParametersHolder{ + EpochStartEventNotifier: epochStartHandlerWithConfirm, + ChainParameters: ccf.config.GeneralSettings.ChainParametersByEpoch, + ChainParametersNotifier: chainParametersNotifier, + } + chainParametersHandler, err := sharding.NewChainParametersHolder(argsChainParametersHandler) + if err != nil { + return nil, err + } + genesisNodesConfig, err := sharding.NewNodesSetup( - ccf.nodesFilename, + ccf.nodesSetupConfig, + chainParametersHandler, addressPubkeyConverter, validatorPubkeyConverter, ccf.config.GeneralSettings.GenesisMaxNumberOfShards, @@ -209,8 +227,6 @@ func (ccf *coreComponentsFactory) Create() (*coreComponents, error) { "formatted", startTime.Format("Mon Jan 2 15:04:05 MST 2006"), "seconds", startTime.Unix()) - log.Debug("config", "file", ccf.nodesFilename) - genesisTime := time.Unix(genesisNodesConfig.StartTime, 0) roundHandler, err := round.NewRound( genesisTime, @@ -236,7 +252,6 @@ func (ccf *coreComponentsFactory) Create() (*coreComponents, error) { return nil, err } - epochNotifier := forking.NewGenericEpochNotifier() enableEpochsHandler, err := enablers.NewEnableEpochsHandler(ccf.epochConfig.EnableEpochs, epochNotifier) if err != nil { return nil, err @@ -265,12 +280,10 @@ func (ccf *coreComponentsFactory) Create() (*coreComponents, error) { log.Trace("creating ratings data") ratingDataArgs := rating.RatingsDataArg{ - Config: ccf.ratingsConfig, - ShardConsensusSize: genesisNodesConfig.ConsensusGroupSize, - MetaConsensusSize: genesisNodesConfig.MetaChainConsensusGroupSize, - ShardMinNodes: genesisNodesConfig.MinNodesPerShard, - MetaMinNodes: genesisNodesConfig.MetaChainMinNodes, - RoundDurationMiliseconds: genesisNodesConfig.RoundDuration, + Config: ccf.ratingsConfig, + ChainParametersHolder: chainParametersHandler, + RoundDurationMilliseconds: genesisNodesConfig.RoundDuration, + EpochNotifier: epochNotifier, } ratingsData, err := rating.NewRatingsData(ratingDataArgs) if err != nil { @@ -283,10 +296,6 @@ func (ccf *coreComponentsFactory) Create() (*coreComponents, error) { } argsNodesShuffler := &nodesCoordinator.NodesShufflerArgs{ - NodesShard: genesisNodesConfig.MinNumberOfShardNodes(), - NodesMeta: genesisNodesConfig.MinNumberOfMetaNodes(), - Hysteresis: genesisNodesConfig.GetHysteresis(), - Adaptivity: genesisNodesConfig.GetAdaptivity(), ShuffleBetweenShards: true, MaxNodesEnableConfig: ccf.epochConfig.EnableEpochs.MaxNodesChangeEnableEpoch, EnableEpochsHandler: enableEpochsHandler, @@ -338,8 +347,9 @@ func (ccf *coreComponentsFactory) Create() (*coreComponents, error) { minTransactionVersion: ccf.config.GeneralSettings.MinTransactionVersion, epochNotifier: epochNotifier, roundNotifier: roundNotifier, + chainParametersSubscriber: chainParametersNotifier, enableRoundsHandler: enableRoundsHandler, - epochStartNotifierWithConfirm: notifier.NewEpochStartSubscriptionHandler(), + epochStartNotifierWithConfirm: epochStartHandlerWithConfirm, chanStopNodeProcess: ccf.chanStopNodeProcess, encodedAddressLen: encodedAddressLen, nodeTypeProvider: nodeTypeProvider, @@ -347,6 +357,7 @@ func (ccf *coreComponentsFactory) Create() (*coreComponents, error) { processStatusHandler: statusHandler.NewProcessStatusHandler(), hardforkTriggerPubKey: pubKeyBytes, enableEpochsHandler: enableEpochsHandler, + chainParametersHandler: chainParametersHandler, }, nil } diff --git a/factory/core/coreComponentsHandler.go b/factory/core/coreComponentsHandler.go index b10c378023e..4ce17f82ebd 100644 --- a/factory/core/coreComponentsHandler.go +++ b/factory/core/coreComponentsHandler.go @@ -149,6 +149,9 @@ func (mcc *managedCoreComponents) CheckSubcomponents() error { if check.IfNil(mcc.enableEpochsHandler) { return errors.ErrNilEnableEpochsHandler } + if check.IfNil(mcc.chainParametersHandler) { + return errors.ErrNilChainParametersHandler + } if len(mcc.chainID) == 0 { return errors.ErrInvalidChainID } @@ -485,6 +488,18 @@ func (mcc *managedCoreComponents) RoundNotifier() process.RoundNotifier { return mcc.coreComponents.roundNotifier } +// ChainParametersSubscriber returns the chain parameters subscriber +func (mcc *managedCoreComponents) ChainParametersSubscriber() process.ChainParametersSubscriber { + mcc.mutCoreComponents.RLock() + defer mcc.mutCoreComponents.RUnlock() + + if mcc.coreComponents == nil { + return nil + } + + return mcc.coreComponents.chainParametersSubscriber +} + // EnableRoundsHandler returns the rounds activation handler func (mcc *managedCoreComponents) EnableRoundsHandler() process.EnableRoundsHandler { mcc.mutCoreComponents.RLock() @@ -581,6 +596,18 @@ func (mcc *managedCoreComponents) EnableEpochsHandler() common.EnableEpochsHandl return mcc.coreComponents.enableEpochsHandler } +// ChainParametersHandler returns the chain parameters handler +func (mcc *managedCoreComponents) ChainParametersHandler() process.ChainParametersHandler { + mcc.mutCoreComponents.RLock() + defer mcc.mutCoreComponents.RUnlock() + + if mcc.coreComponents == nil { + return nil + } + + return mcc.coreComponents.chainParametersHandler +} + // IsInterfaceNil returns true if there is no value under the interface func (mcc *managedCoreComponents) IsInterfaceNil() bool { return mcc == nil diff --git a/factory/interface.go b/factory/interface.go index 0f1c237d0d9..4c66676fe48 100644 --- a/factory/interface.go +++ b/factory/interface.go @@ -66,7 +66,7 @@ type P2PAntifloodHandler interface { SetDebugger(debugger process.AntifloodDebugger) error SetPeerValidatorMapper(validatorMapper process.PeerValidatorMapper) error SetTopicsForAll(topics ...string) - ApplyConsensusSize(size int) + SetConsensusSizeNotifier(chainParametersNotifier process.ChainParametersSubscriber, shardID uint32) BlacklistPeer(peer core.PeerID, reason string, duration time.Duration) IsOriginatorEligibleForTopic(pid core.PeerID, topic string) error Close() error @@ -120,6 +120,7 @@ type CoreComponentsHolder interface { GenesisNodesSetup() sharding.GenesisNodesSetupHandler NodesShuffler() nodesCoordinator.NodesShuffler EpochNotifier() process.EpochNotifier + ChainParametersSubscriber() process.ChainParametersSubscriber EnableRoundsHandler() process.EnableRoundsHandler RoundNotifier() process.RoundNotifier EpochStartNotifierWithConfirm() EpochStartNotifierWithConfirm @@ -134,6 +135,7 @@ type CoreComponentsHolder interface { ProcessStatusHandler() common.ProcessStatusHandler HardforkTriggerPubKey() []byte EnableEpochsHandler() common.EnableEpochsHandler + ChainParametersHandler() process.ChainParametersHandler IsInterfaceNil() bool } @@ -421,7 +423,6 @@ type ConsensusComponentsHolder interface { Chronology() consensus.ChronologyHandler ConsensusWorker() ConsensusWorker BroadcastMessenger() consensus.BroadcastMessenger - ConsensusGroupSize() (int, error) Bootstrapper() process.Bootstrapper IsInterfaceNil() bool } diff --git a/factory/mock/coreComponentsMock.go b/factory/mock/coreComponentsMock.go index 0393f44c4a1..0c4a66e17dc 100644 --- a/factory/mock/coreComponentsMock.go +++ b/factory/mock/coreComponentsMock.go @@ -21,41 +21,43 @@ import ( // CoreComponentsMock - type CoreComponentsMock struct { - IntMarsh marshal.Marshalizer - TxMarsh marshal.Marshalizer - VmMarsh marshal.Marshalizer - Hash hashing.Hasher - TxSignHasherField hashing.Hasher - UInt64ByteSliceConv typeConverters.Uint64ByteSliceConverter - AddrPubKeyConv core.PubkeyConverter - ValPubKeyConv core.PubkeyConverter - PathHdl storage.PathManagerHandler - WatchdogTimer core.WatchdogTimer - AlarmSch core.TimersScheduler - NtpSyncTimer ntp.SyncTimer - GenesisBlockTime time.Time - ChainIdCalled func() string - MinTransactionVersionCalled func() uint32 - mutIntMarshalizer sync.RWMutex - RoundHandlerField consensus.RoundHandler - EconomicsHandler process.EconomicsDataHandler - APIEconomicsHandler process.EconomicsDataHandler - RatingsConfig process.RatingsInfoHandler - RatingHandler sharding.PeerAccountListAndRatingHandler - NodesConfig sharding.GenesisNodesSetupHandler - Shuffler nodesCoordinator.NodesShuffler - EpochChangeNotifier process.EpochNotifier - RoundChangeNotifier process.RoundNotifier - EnableRoundsHandlerField process.EnableRoundsHandler - EpochNotifierWithConfirm factory.EpochStartNotifierWithConfirm - TxVersionCheckHandler process.TxVersionCheckerHandler - ChanStopProcess chan endProcess.ArgEndProcess - StartTime time.Time - NodeTypeProviderField core.NodeTypeProviderHandler - WasmVMChangeLockerInternal common.Locker - ProcessStatusHandlerInternal common.ProcessStatusHandler - HardforkTriggerPubKeyField []byte - EnableEpochsHandlerField common.EnableEpochsHandler + IntMarsh marshal.Marshalizer + TxMarsh marshal.Marshalizer + VmMarsh marshal.Marshalizer + Hash hashing.Hasher + TxSignHasherField hashing.Hasher + UInt64ByteSliceConv typeConverters.Uint64ByteSliceConverter + AddrPubKeyConv core.PubkeyConverter + ValPubKeyConv core.PubkeyConverter + PathHdl storage.PathManagerHandler + WatchdogTimer core.WatchdogTimer + AlarmSch core.TimersScheduler + NtpSyncTimer ntp.SyncTimer + GenesisBlockTime time.Time + ChainIdCalled func() string + MinTransactionVersionCalled func() uint32 + mutIntMarshalizer sync.RWMutex + RoundHandlerField consensus.RoundHandler + EconomicsHandler process.EconomicsDataHandler + APIEconomicsHandler process.EconomicsDataHandler + RatingsConfig process.RatingsInfoHandler + RatingHandler sharding.PeerAccountListAndRatingHandler + NodesConfig sharding.GenesisNodesSetupHandler + Shuffler nodesCoordinator.NodesShuffler + EpochChangeNotifier process.EpochNotifier + RoundChangeNotifier process.RoundNotifier + EnableRoundsHandlerField process.EnableRoundsHandler + EpochNotifierWithConfirm factory.EpochStartNotifierWithConfirm + TxVersionCheckHandler process.TxVersionCheckerHandler + ChanStopProcess chan endProcess.ArgEndProcess + StartTime time.Time + NodeTypeProviderField core.NodeTypeProviderHandler + WasmVMChangeLockerInternal common.Locker + ProcessStatusHandlerInternal common.ProcessStatusHandler + HardforkTriggerPubKeyField []byte + EnableEpochsHandlerField common.EnableEpochsHandler + ChainParametersHandlerField process.ChainParametersHandler + ChainParametersSubscriberField process.ChainParametersSubscriber } // InternalMarshalizer - @@ -246,6 +248,16 @@ func (ccm *CoreComponentsMock) EnableEpochsHandler() common.EnableEpochsHandler return ccm.EnableEpochsHandlerField } +// ChainParametersHandler - +func (ccm *CoreComponentsMock) ChainParametersHandler() process.ChainParametersHandler { + return ccm.ChainParametersHandlerField +} + +// ChainParametersSubscriber - +func (ccm *CoreComponentsMock) ChainParametersSubscriber() process.ChainParametersSubscriber { + return ccm.ChainParametersSubscriberField +} + // IsInterfaceNil - func (ccm *CoreComponentsMock) IsInterfaceNil() bool { return ccm == nil diff --git a/factory/mock/p2pAntifloodHandlerStub.go b/factory/mock/p2pAntifloodHandlerStub.go index bda3da406d5..2a6a10fbde8 100644 --- a/factory/mock/p2pAntifloodHandlerStub.go +++ b/factory/mock/p2pAntifloodHandlerStub.go @@ -16,6 +16,7 @@ type P2PAntifloodHandlerStub struct { SetDebuggerCalled func(debugger process.AntifloodDebugger) error BlacklistPeerCalled func(peer core.PeerID, reason string, duration time.Duration) IsOriginatorEligibleForTopicCalled func(pid core.PeerID, topic string) error + SetConsensusSizeNotifierCalled func(subscriber process.ChainParametersSubscriber, shardID uint32) } // CanProcessMessage - @@ -42,10 +43,10 @@ func (p2pahs *P2PAntifloodHandlerStub) CanProcessMessagesOnTopic(peer core.PeerI return p2pahs.CanProcessMessagesOnTopicCalled(peer, topic, numMessages, totalSize, sequence) } -// ApplyConsensusSize - -func (p2pahs *P2PAntifloodHandlerStub) ApplyConsensusSize(size int) { - if p2pahs.ApplyConsensusSizeCalled != nil { - p2pahs.ApplyConsensusSizeCalled(size) +// SetConsensusSizeNotifier - +func (p2pahs *P2PAntifloodHandlerStub) SetConsensusSizeNotifier(subscriber process.ChainParametersSubscriber, shardID uint32) { + if p2pahs.SetConsensusSizeNotifierCalled != nil { + p2pahs.SetConsensusSizeNotifierCalled(subscriber, shardID) } } diff --git a/factory/mock/testdata/nodesSetupMock.json b/factory/mock/testdata/nodesSetupMock.json deleted file mode 100644 index 905496ad7c3..00000000000 --- a/factory/mock/testdata/nodesSetupMock.json +++ /dev/null @@ -1,25 +0,0 @@ -{ - "startTime": 0, - "roundDuration": 4000, - "consensusGroupSize": 1, - "minNodesPerShard": 1, - "metaChainActive" : true, - "metaChainConsensusGroupSize" : 1, - "metaChainMinNodes" : 1, - "hysteresis": 0, - "adaptivity": false, - "initialNodes": [ - { - "pubkey": "227a5a5ec0c58171b7f4ee9ecc304ea7b176fb626741a25c967add76d6cd361d6995929f9b60a96237381091cefb1b061225e5bb930b40494a5ac9d7524fd67dfe478e5ccd80f17b093cff5722025761fb0217c39dbd5ae45e01eb5a3113be93", - "address": "erd1ulhw20j7jvgfgak5p05kv667k5k9f320sgef5ayxkt9784ql0zssrzyhjp" - }, - { - "pubkey": "ef9522d654bc08ebf2725468f41a693aa7f3cf1cb93922cff1c8c81fba78274016010916f4a7e5b0855c430a724a2d0b3acd1fe8e61e37273a17d58faa8c0d3ef6b883a33ec648950469a1e9757b978d9ae662a019068a401cff56eea059fd08", - "address": "erd17c4fs6mz2aa2hcvva2jfxdsrdknu4220496jmswer9njznt22eds0rxlr4" - }, - { - "pubkey": "e91ab494cedd4da346f47aaa1a3e792bea24fb9f6cc40d3546bc4ca36749b8bfb0164e40dbad2195a76ee0fd7fb7da075ecbf1b35a2ac20638d53ea5520644f8c16952225c48304bb202867e2d71d396bff5a5971f345bcfe32c7b6b0ca34c84", - "address": "erd10d2gufxesrp8g409tzxljlaefhs0rsgjle3l7nq38de59txxt8csj54cd3" - } - ] -} diff --git a/factory/mock/testdata/nodesSetupMockInvalidRound.json b/factory/mock/testdata/nodesSetupMockInvalidRound.json deleted file mode 100644 index df96538e573..00000000000 --- a/factory/mock/testdata/nodesSetupMockInvalidRound.json +++ /dev/null @@ -1,25 +0,0 @@ -{ - "startTime": 0, - "roundDuration": 500, - "consensusGroupSize": 1, - "minNodesPerShard": 1, - "metaChainActive": true, - "metaChainConsensusGroupSize": 1, - "metaChainMinNodes": 1, - "hysteresis": 0, - "adaptivity": false, - "initialNodes": [ - { - "pubkey": "227a5a5ec0c58171b7f4ee9ecc304ea7b176fb626741a25c967add76d6cd361d6995929f9b60a96237381091cefb1b061225e5bb930b40494a5ac9d7524fd67dfe478e5ccd80f17b093cff5722025761fb0217c39dbd5ae45e01eb5a3113be93", - "address": "erd1ulhw20j7jvgfgak5p05kv667k5k9f320sgef5ayxkt9784ql0zssrzyhjp" - }, - { - "pubkey": "ef9522d654bc08ebf2725468f41a693aa7f3cf1cb93922cff1c8c81fba78274016010916f4a7e5b0855c430a724a2d0b3acd1fe8e61e37273a17d58faa8c0d3ef6b883a33ec648950469a1e9757b978d9ae662a019068a401cff56eea059fd08", - "address": "erd17c4fs6mz2aa2hcvva2jfxdsrdknu4220496jmswer9njznt22eds0rxlr4" - }, - { - "pubkey": "e91ab494cedd4da346f47aaa1a3e792bea24fb9f6cc40d3546bc4ca36749b8bfb0164e40dbad2195a76ee0fd7fb7da075ecbf1b35a2ac20638d53ea5520644f8c16952225c48304bb202867e2d71d396bff5a5971f345bcfe32c7b6b0ca34c84", - "address": "erd10d2gufxesrp8g409tzxljlaefhs0rsgjle3l7nq38de59txxt8csj54cd3" - } - ] -} diff --git a/integrationTests/factory/componentsHelper.go b/integrationTests/factory/componentsHelper.go index 6ad6c5910bf..3006dd3182c 100644 --- a/integrationTests/factory/componentsHelper.go +++ b/integrationTests/factory/componentsHelper.go @@ -7,6 +7,7 @@ import ( "runtime/pprof" "testing" + "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/p2p" @@ -40,6 +41,8 @@ func CreateDefaultConfig(tb testing.TB) *config.Configs { systemSCConfig, _ := common.LoadSystemSmartContractsConfig(configPathsHolder.SystemSC) epochConfig, _ := common.LoadEpochConfig(configPathsHolder.Epoch) roundConfig, _ := common.LoadRoundConfig(configPathsHolder.RoundActivation) + var nodesConfig config.NodesConfig + _ = core.LoadJsonFile(&nodesConfig, NodesSetupPath) mainP2PConfig.KadDhtPeerDiscovery.Enabled = false prefsConfig.Preferences.DestinationShardAsObserver = "0" @@ -69,10 +72,32 @@ func CreateDefaultConfig(tb testing.TB) *config.Configs { } configs.ConfigurationPathsHolder = configPathsHolder configs.ImportDbConfig = &config.ImportDbConfig{} + configs.NodesConfig = &nodesConfig + + configs.GeneralConfig.GeneralSettings.ChainParametersByEpoch = computeChainParameters(uint32(len(configs.NodesConfig.InitialNodes)), configs.GeneralConfig.GeneralSettings.GenesisMaxNumberOfShards) return configs } +func computeChainParameters(numInitialNodes uint32, numShardsWithoutMeta uint32) []config.ChainParametersByEpochConfig { + numShardsWithMeta := numShardsWithoutMeta + 1 + nodesPerShards := numInitialNodes / numShardsWithMeta + shardCnsGroupSize := nodesPerShards + if shardCnsGroupSize > 1 { + shardCnsGroupSize-- + } + diff := numInitialNodes - nodesPerShards*numShardsWithMeta + return []config.ChainParametersByEpochConfig{ + { + ShardConsensusGroupSize: shardCnsGroupSize, + ShardMinNumNodes: nodesPerShards, + MetachainConsensusGroupSize: nodesPerShards, + MetachainMinNumNodes: nodesPerShards + diff, + RoundDuration: 2000, + }, + } +} + func createConfigurationsPathsHolder() *config.ConfigurationPathsHolder { var concatPath = func(filename string) string { return path.Join(BaseNodeConfigPath, filename) diff --git a/integrationTests/factory/consensusComponents/consensusComponents_test.go b/integrationTests/factory/consensusComponents/consensusComponents_test.go index b68e9dd95cc..07127ca8399 100644 --- a/integrationTests/factory/consensusComponents/consensusComponents_test.go +++ b/integrationTests/factory/consensusComponents/consensusComponents_test.go @@ -78,6 +78,7 @@ func TestConsensusComponents_Close_ShouldWork(t *testing.T) { managedCoreComponents.EnableEpochsHandler(), managedDataComponents.Datapool().CurrentEpochValidatorInfo(), managedBootstrapComponents.NodesCoordinatorRegistryFactory(), + managedCoreComponents.ChainParametersHandler(), ) require.Nil(t, err) managedStatusComponents, err := nr.CreateManagedStatusComponents( diff --git a/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go b/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go index dd0a07ad91f..872be428ab0 100644 --- a/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go +++ b/integrationTests/factory/heartbeatComponents/heartbeatComponents_test.go @@ -78,6 +78,7 @@ func TestHeartbeatComponents_Close_ShouldWork(t *testing.T) { managedCoreComponents.EnableEpochsHandler(), managedDataComponents.Datapool().CurrentEpochValidatorInfo(), managedBootstrapComponents.NodesCoordinatorRegistryFactory(), + managedCoreComponents.ChainParametersHandler(), ) require.Nil(t, err) managedStatusComponents, err := nr.CreateManagedStatusComponents( diff --git a/integrationTests/factory/processComponents/processComponents_test.go b/integrationTests/factory/processComponents/processComponents_test.go index 17860520ea9..329587ac4e7 100644 --- a/integrationTests/factory/processComponents/processComponents_test.go +++ b/integrationTests/factory/processComponents/processComponents_test.go @@ -79,6 +79,7 @@ func TestProcessComponents_Close_ShouldWork(t *testing.T) { managedCoreComponents.EnableEpochsHandler(), managedDataComponents.Datapool().CurrentEpochValidatorInfo(), managedBootstrapComponents.NodesCoordinatorRegistryFactory(), + managedCoreComponents.ChainParametersHandler(), ) require.Nil(t, err) managedStatusComponents, err := nr.CreateManagedStatusComponents( diff --git a/integrationTests/factory/statusComponents/statusComponents_test.go b/integrationTests/factory/statusComponents/statusComponents_test.go index dc5d3575b8c..6103209936b 100644 --- a/integrationTests/factory/statusComponents/statusComponents_test.go +++ b/integrationTests/factory/statusComponents/statusComponents_test.go @@ -79,6 +79,7 @@ func TestStatusComponents_Create_Close_ShouldWork(t *testing.T) { managedCoreComponents.EnableEpochsHandler(), managedDataComponents.Datapool().CurrentEpochValidatorInfo(), managedBootstrapComponents.NodesCoordinatorRegistryFactory(), + managedCoreComponents.ChainParametersHandler(), ) require.Nil(t, err) managedStatusComponents, err := nr.CreateManagedStatusComponents( diff --git a/integrationTests/factory/testdata/nodesSetup.json b/integrationTests/factory/testdata/nodesSetup.json index 239fd9a52f6..2a966c72ce8 100644 --- a/integrationTests/factory/testdata/nodesSetup.json +++ b/integrationTests/factory/testdata/nodesSetup.json @@ -1,12 +1,5 @@ { "startTime": 0, - "roundDuration": 4000, - "consensusGroupSize": 3, - "minNodesPerShard": 3, - "metaChainConsensusGroupSize": 3, - "metaChainMinNodes": 3, - "hysteresis": 0, - "adaptivity": false, "initialNodes": [ { "pubkey": "cbc8c9a6a8d9c874e89eb9366139368ae728bd3eda43f173756537877ba6bca87e01a97b815c9f691df73faa16f66b15603056540aa7252d73fecf05d24cd36b44332a88386788fbdb59d04502e8ecb0132d8ebd3d875be4c83e8b87c55eb901", diff --git a/integrationTests/mock/coreComponentsStub.go b/integrationTests/mock/coreComponentsStub.go index dca3f5a1fa6..b682e0d6ce9 100644 --- a/integrationTests/mock/coreComponentsStub.go +++ b/integrationTests/mock/coreComponentsStub.go @@ -54,6 +54,8 @@ type CoreComponentsStub struct { ProcessStatusHandlerInternal common.ProcessStatusHandler HardforkTriggerPubKeyField []byte EnableEpochsHandlerField common.EnableEpochsHandler + ChainParametersHandlerField process.ChainParametersHandler + ChainParametersSubscriberField process.ChainParametersSubscriber } // Create - @@ -259,6 +261,16 @@ func (ccs *CoreComponentsStub) EnableEpochsHandler() common.EnableEpochsHandler return ccs.EnableEpochsHandlerField } +// ChainParametersHandler - +func (ccs *CoreComponentsStub) ChainParametersHandler() process.ChainParametersHandler { + return ccs.ChainParametersHandlerField +} + +// ChainParametersSubscriber - +func (ccs *CoreComponentsStub) ChainParametersSubscriber() process.ChainParametersSubscriber { + return ccs.ChainParametersSubscriberField +} + // IsInterfaceNil - func (ccs *CoreComponentsStub) IsInterfaceNil() bool { return ccs == nil diff --git a/integrationTests/mock/nilAntifloodHandler.go b/integrationTests/mock/nilAntifloodHandler.go index 868a2167767..fab73b964cd 100644 --- a/integrationTests/mock/nilAntifloodHandler.go +++ b/integrationTests/mock/nilAntifloodHandler.go @@ -13,6 +13,10 @@ import ( type NilAntifloodHandler struct { } +// SetConsensusSizeNotifier - +func (nah *NilAntifloodHandler) SetConsensusSizeNotifier(_ process.ChainParametersSubscriber, _ uint32) { +} + // ResetForTopic won't do anything func (nah *NilAntifloodHandler) ResetForTopic(_ string) { } diff --git a/integrationTests/mock/p2pAntifloodHandlerStub.go b/integrationTests/mock/p2pAntifloodHandlerStub.go index c181d10909d..3a9f89397b5 100644 --- a/integrationTests/mock/p2pAntifloodHandlerStub.go +++ b/integrationTests/mock/p2pAntifloodHandlerStub.go @@ -17,6 +17,7 @@ type P2PAntifloodHandlerStub struct { BlacklistPeerCalled func(peer core.PeerID, reason string, duration time.Duration) IsOriginatorEligibleForTopicCalled func(pid core.PeerID, topic string) error SetPeerValidatorMapperCalled func(validatorMapper process.PeerValidatorMapper) error + SetConsensusSizeNotifierCalled func(chainParametersNotifier process.ChainParametersSubscriber, shardID uint32) } // CanProcessMessage - @@ -50,6 +51,13 @@ func (stub *P2PAntifloodHandlerStub) ApplyConsensusSize(size int) { } } +// SetConsensusSizeNotifier - +func (p2pahs *P2PAntifloodHandlerStub) SetConsensusSizeNotifier(chainParametersNotifier process.ChainParametersSubscriber, shardID uint32) { + if p2pahs.SetConsensusSizeNotifierCalled != nil { + p2pahs.SetConsensusSizeNotifierCalled(chainParametersNotifier, shardID) + } +} + // SetDebugger - func (stub *P2PAntifloodHandlerStub) SetDebugger(debugger process.AntifloodDebugger) error { if stub.SetDebuggerCalled != nil { diff --git a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go index ce933a22666..2ee087799e5 100644 --- a/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go +++ b/integrationTests/multiShard/endOfEpoch/startInEpoch/startInEpoch_test.go @@ -31,6 +31,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/factory" "github.com/multiversx/mx-chain-go/storage/storageunit" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" epochNotifierMock "github.com/multiversx/mx-chain-go/testscommon/epochNotifier" "github.com/multiversx/mx-chain-go/testscommon/genericMocks" "github.com/multiversx/mx-chain-go/testscommon/genesisMocks" @@ -234,6 +235,7 @@ func testNodeStartsInEpoch(t *testing.T, shardID uint32, expectedHighestRound ui coreComponents.NodeTypeProviderField = &nodeTypeProviderMock.NodeTypeProviderStub{} coreComponents.ChanStopNodeProcessField = endProcess.GetDummyEndProcessChannel() coreComponents.HardforkTriggerPubKeyField = []byte("provided hardfork pub key") + coreComponents.ChainParametersHandlerField = &chainParameters.ChainParametersHandlerStub{} nodesCoordinatorRegistryFactory, _ := nodesCoordinator.NewNodesCoordinatorRegistryFactory( &marshallerMock.MarshalizerMock{}, diff --git a/integrationTests/nodesCoordinatorFactory.go b/integrationTests/nodesCoordinatorFactory.go index 28267d44c5a..8154d6df5db 100644 --- a/integrationTests/nodesCoordinatorFactory.go +++ b/integrationTests/nodesCoordinatorFactory.go @@ -7,10 +7,12 @@ import ( "github.com/multiversx/mx-chain-core-go/data/endProcess" "github.com/multiversx/mx-chain-core-go/hashing" "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/integrationTests/mock" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/genesisMocks" "github.com/multiversx/mx-chain-go/testscommon/nodeTypeProviderMock" @@ -46,10 +48,6 @@ func (tpn *IndexHashedNodesCoordinatorFactory) CreateNodesCoordinator(arg ArgInd pubKeyBytes, _ := keys.MainKey.Pk.ToByteArray() nodeShufflerArgs := &nodesCoordinator.NodesShufflerArgs{ - NodesShard: uint32(arg.nodesPerShard), - NodesMeta: uint32(arg.nbMetaNodes), - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, @@ -61,23 +59,33 @@ func (tpn *IndexHashedNodesCoordinatorFactory) CreateNodesCoordinator(arg ArgInd StakingV4Step2EnableEpoch, ) argumentsNodesCoordinator := nodesCoordinator.ArgNodesCoordinator{ - ShardConsensusGroupSize: arg.shardConsensusGroupSize, - MetaConsensusGroupSize: arg.metaConsensusGroupSize, - Marshalizer: TestMarshalizer, - Hasher: arg.hasher, - Shuffler: nodeShuffler, - EpochStartNotifier: arg.epochStartSubscriber, - ShardIDAsObserver: arg.shardId, - NbShards: uint32(arg.nbShards), - EligibleNodes: arg.validatorsMap, - WaitingNodes: arg.waitingMap, - SelfPublicKey: pubKeyBytes, - ConsensusGroupCache: arg.consensusGroupCache, - BootStorer: arg.bootStorer, - ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, - ChanStopNode: endProcess.GetDummyEndProcessChannel(), - NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, - IsFullArchive: false, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardMinNumNodes: uint32(arg.nodesPerShard), + MetachainMinNumNodes: uint32(arg.nbMetaNodes), + Hysteresis: hysteresis, + Adaptivity: adaptivity, + ShardConsensusGroupSize: uint32(arg.shardConsensusGroupSize), + MetachainConsensusGroupSize: uint32(arg.metaConsensusGroupSize), + }, nil + }, + }, + Marshalizer: TestMarshalizer, + Hasher: arg.hasher, + Shuffler: nodeShuffler, + EpochStartNotifier: arg.epochStartSubscriber, + ShardIDAsObserver: arg.shardId, + NbShards: uint32(arg.nbShards), + EligibleNodes: arg.validatorsMap, + WaitingNodes: arg.waitingMap, + SelfPublicKey: pubKeyBytes, + ConsensusGroupCache: arg.consensusGroupCache, + BootStorer: arg.bootStorer, + ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, + ChanStopNode: endProcess.GetDummyEndProcessChannel(), + NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, + IsFullArchive: false, EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{ GetActivationEpochCalled: func(flag core.EnableEpochFlag) uint32 { if flag == common.RefactorPeersMiniBlocksFlag || flag == common.StakingV4Step2Flag { @@ -112,10 +120,6 @@ func (ihncrf *IndexHashedNodesCoordinatorWithRaterFactory) CreateNodesCoordinato pubKeyBytes, _ := keys.MainKey.Pk.ToByteArray() shufflerArgs := &nodesCoordinator.NodesShufflerArgs{ - NodesShard: uint32(arg.nodesPerShard), - NodesMeta: uint32(arg.nbMetaNodes), - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, @@ -127,23 +131,33 @@ func (ihncrf *IndexHashedNodesCoordinatorWithRaterFactory) CreateNodesCoordinato StakingV4Step2EnableEpoch, ) argumentsNodesCoordinator := nodesCoordinator.ArgNodesCoordinator{ - ShardConsensusGroupSize: arg.shardConsensusGroupSize, - MetaConsensusGroupSize: arg.metaConsensusGroupSize, - Marshalizer: TestMarshalizer, - Hasher: arg.hasher, - Shuffler: nodeShuffler, - EpochStartNotifier: arg.epochStartSubscriber, - ShardIDAsObserver: arg.shardId, - NbShards: uint32(arg.nbShards), - EligibleNodes: arg.validatorsMap, - WaitingNodes: arg.waitingMap, - SelfPublicKey: pubKeyBytes, - ConsensusGroupCache: arg.consensusGroupCache, - BootStorer: arg.bootStorer, - ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, - ChanStopNode: endProcess.GetDummyEndProcessChannel(), - NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, - IsFullArchive: false, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardMinNumNodes: uint32(arg.nodesPerShard), + MetachainMinNumNodes: uint32(arg.nbMetaNodes), + Hysteresis: hysteresis, + Adaptivity: adaptivity, + ShardConsensusGroupSize: uint32(arg.shardConsensusGroupSize), + MetachainConsensusGroupSize: uint32(arg.metaConsensusGroupSize), + }, nil + }, + }, + Marshalizer: TestMarshalizer, + Hasher: arg.hasher, + Shuffler: nodeShuffler, + EpochStartNotifier: arg.epochStartSubscriber, + ShardIDAsObserver: arg.shardId, + NbShards: uint32(arg.nbShards), + EligibleNodes: arg.validatorsMap, + WaitingNodes: arg.waitingMap, + SelfPublicKey: pubKeyBytes, + ConsensusGroupCache: arg.consensusGroupCache, + BootStorer: arg.bootStorer, + ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, + ChanStopNode: endProcess.GetDummyEndProcessChannel(), + NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, + IsFullArchive: false, EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{ GetActivationEpochCalled: func(flag core.EnableEpochFlag) uint32 { if flag == common.RefactorPeersMiniBlocksFlag { diff --git a/integrationTests/realcomponents/processorRunner.go b/integrationTests/realcomponents/processorRunner.go index 3f3f4837201..20a33dcffc8 100644 --- a/integrationTests/realcomponents/processorRunner.go +++ b/integrationTests/realcomponents/processorRunner.go @@ -95,7 +95,7 @@ func (pr *ProcessorRunner) createCoreComponents(tb testing.TB) { RatingsConfig: *pr.Config.RatingsConfig, EconomicsConfig: *pr.Config.EconomicsConfig, ImportDbConfig: *pr.Config.ImportDbConfig, - NodesFilename: pr.Config.ConfigurationPathsHolder.Nodes, + NodesConfig: *pr.Config.NodesConfig, WorkingDirectory: pr.Config.FlagsConfig.WorkingDir, ChanStopNodeProcess: make(chan endProcess.ArgEndProcess), } @@ -308,6 +308,7 @@ func (pr *ProcessorRunner) createStatusComponents(tb testing.TB) { pr.CoreComponents.EnableEpochsHandler(), pr.DataComponents.Datapool().CurrentEpochValidatorInfo(), pr.BootstrapComponents.NodesCoordinatorRegistryFactory(), + pr.CoreComponents.ChainParametersHandler(), ) require.Nil(tb, err) diff --git a/integrationTests/testConsensusNode.go b/integrationTests/testConsensusNode.go index 5f5987b11cf..2e297291423 100644 --- a/integrationTests/testConsensusNode.go +++ b/integrationTests/testConsensusNode.go @@ -39,6 +39,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/storage/storageunit" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" @@ -257,7 +258,7 @@ func (tcn *TestConsensusNode) initNode(args ArgsTestConsensusNode) { argsKeysHolder := keysManagement.ArgsManagedPeersHolder{ KeyGenerator: args.KeyGen, P2PKeyGenerator: args.P2PKeyGen, - MaxRoundsOfInactivity: 0, + MaxRoundsOfInactivity: 10, PrefsConfig: config.Preferences{}, P2PKeyConverter: p2pFactory.NewP2PKeyConverter(), } @@ -346,7 +347,6 @@ func (tcn *TestConsensusNode) initNode(args ArgsTestConsensusNode) { node.WithStateComponents(stateComponents), node.WithNetworkComponents(networkComponents), node.WithRoundDuration(args.RoundTime), - node.WithConsensusGroupSize(args.ConsensusSize), node.WithConsensusType(args.ConsensusType), node.WithGenesisTime(time.Unix(args.StartTime, 0)), node.WithValidatorSignatureSize(signatureSize), @@ -368,8 +368,14 @@ func (tcn *TestConsensusNode) initNodesCoordinator( cache storage.Cacher, ) { argumentsNodesCoordinator := nodesCoordinator.ArgNodesCoordinator{ - ShardConsensusGroupSize: consensusSize, - MetaConsensusGroupSize: consensusSize, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(consensusSize), + MetachainConsensusGroupSize: uint32(consensusSize), + }, nil + }, + }, Marshalizer: TestMarshalizer, Hasher: hasher, Shuffler: &shardingMocks.NodeShufflerMock{}, diff --git a/integrationTests/testHeartbeatNode.go b/integrationTests/testHeartbeatNode.go index 1ba488b9e12..b74bfaf01fe 100644 --- a/integrationTests/testHeartbeatNode.go +++ b/integrationTests/testHeartbeatNode.go @@ -48,6 +48,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/storage/storageunit" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" @@ -349,8 +350,14 @@ func CreateNodesWithTestHeartbeatNode( suCache, _ := storageunit.NewCache(cacherCfg) for shardId, validatorList := range validatorsMap { argumentsNodesCoordinator := nodesCoordinator.ArgNodesCoordinator{ - ShardConsensusGroupSize: shardConsensusGroupSize, - MetaConsensusGroupSize: metaConsensusGroupSize, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(shardConsensusGroupSize), + MetachainConsensusGroupSize: uint32(metaConsensusGroupSize), + }, nil + }, + }, Marshalizer: TestMarshalizer, Hasher: TestHasher, ShardIDAsObserver: shardId, @@ -397,8 +404,14 @@ func CreateNodesWithTestHeartbeatNode( } argumentsNodesCoordinator := nodesCoordinator.ArgNodesCoordinator{ - ShardConsensusGroupSize: shardConsensusGroupSize, - MetaConsensusGroupSize: metaConsensusGroupSize, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(shardConsensusGroupSize), + MetachainConsensusGroupSize: uint32(metaConsensusGroupSize), + }, nil + }, + }, Marshalizer: TestMarshalizer, Hasher: TestHasher, ShardIDAsObserver: shardId, diff --git a/integrationTests/testProcessorNode.go b/integrationTests/testProcessorNode.go index 14c7a6a1ba2..919be505f3d 100644 --- a/integrationTests/testProcessorNode.go +++ b/integrationTests/testProcessorNode.go @@ -104,10 +104,12 @@ import ( "github.com/multiversx/mx-chain-go/storage/txcache" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapMocks" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" dataRetrieverMock "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" dblookupextMock "github.com/multiversx/mx-chain-go/testscommon/dblookupext" "github.com/multiversx/mx-chain-go/testscommon/economicsmocks" + "github.com/multiversx/mx-chain-go/testscommon/epochNotifier" testFactory "github.com/multiversx/mx-chain-go/testscommon/factory" "github.com/multiversx/mx-chain-go/testscommon/genesisMocks" "github.com/multiversx/mx-chain-go/testscommon/guardianMocks" @@ -1243,12 +1245,10 @@ func CreateRatingsData() *rating.RatingsData { } ratingDataArgs := rating.RatingsDataArg{ - Config: ratingsConfig, - ShardConsensusSize: 63, - MetaConsensusSize: 400, - ShardMinNodes: 400, - MetaMinNodes: 400, - RoundDurationMiliseconds: 6000, + Config: ratingsConfig, + ChainParametersHolder: &chainParameters.ChainParametersHolderMock{}, + EpochNotifier: &epochNotifier.EpochNotifierStub{}, + RoundDurationMilliseconds: 6000, } ratingsData, _ := rating.NewRatingsData(ratingDataArgs) diff --git a/integrationTests/testProcessorNodeWithCoordinator.go b/integrationTests/testProcessorNodeWithCoordinator.go index 63392658a76..de1171a512f 100644 --- a/integrationTests/testProcessorNodeWithCoordinator.go +++ b/integrationTests/testProcessorNodeWithCoordinator.go @@ -9,10 +9,12 @@ import ( "github.com/multiversx/mx-chain-crypto-go/signing" "github.com/multiversx/mx-chain-crypto-go/signing/ed25519" "github.com/multiversx/mx-chain-crypto-go/signing/mcl" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/integrationTests/mock" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/storage/cache" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/genesisMocks" vic "github.com/multiversx/mx-chain-go/testscommon/validatorInfoCacher" @@ -61,8 +63,14 @@ func CreateProcessorNodesWithNodesCoordinator( for i, v := range validatorList { lruCache, _ := cache.NewLRUCache(10000) argumentsNodesCoordinator := nodesCoordinator.ArgNodesCoordinator{ - ShardConsensusGroupSize: shardConsensusGroupSize, - MetaConsensusGroupSize: metaConsensusGroupSize, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(shardConsensusGroupSize), + MetachainConsensusGroupSize: uint32(metaConsensusGroupSize), + }, nil + }, + }, Marshalizer: TestMarshalizer, Hasher: TestHasher, ShardIDAsObserver: shardId, diff --git a/integrationTests/testProcessorNodeWithMultisigner.go b/integrationTests/testProcessorNodeWithMultisigner.go index 42f08a62b39..80f2a183ad2 100644 --- a/integrationTests/testProcessorNodeWithMultisigner.go +++ b/integrationTests/testProcessorNodeWithMultisigner.go @@ -29,6 +29,7 @@ import ( "github.com/multiversx/mx-chain-go/storage/cache" "github.com/multiversx/mx-chain-go/storage/storageunit" "github.com/multiversx/mx-chain-go/testscommon" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" "github.com/multiversx/mx-chain-go/testscommon/genesisMocks" @@ -399,10 +400,6 @@ func CreateNodesWithNodesCoordinatorAndHeaderSigVerifier( nodesMap := make(map[uint32][]*TestProcessorNode) shufflerArgs := &nodesCoordinator.NodesShufflerArgs{ - NodesShard: uint32(nodesPerShard), - NodesMeta: uint32(nbMetaNodes), - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &enableEpochsHandlerMock.EnableEpochsHandlerStub{}, @@ -423,8 +420,18 @@ func CreateNodesWithNodesCoordinatorAndHeaderSigVerifier( for shardId, validatorList := range validatorsMap { consensusCache, _ := cache.NewLRUCache(10000) argumentsNodesCoordinator := nodesCoordinator.ArgNodesCoordinator{ - ShardConsensusGroupSize: shardConsensusGroupSize, - MetaConsensusGroupSize: metaConsensusGroupSize, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(shardConsensusGroupSize), + ShardMinNumNodes: uint32(nodesPerShard), + MetachainConsensusGroupSize: uint32(metaConsensusGroupSize), + MetachainMinNumNodes: uint32(nbMetaNodes), + Hysteresis: hysteresis, + Adaptivity: adaptivity, + }, nil + }, + }, Marshalizer: TestMarshalizer, Hasher: TestHasher, Shuffler: nodeShuffler, @@ -544,8 +551,14 @@ func CreateNodesWithNodesCoordinatorKeygenAndSingleSigner( bootStorer := CreateMemUnit() lruCache, _ := cache.NewLRUCache(10000) argumentsNodesCoordinator := nodesCoordinator.ArgNodesCoordinator{ - ShardConsensusGroupSize: shardConsensusGroupSize, - MetaConsensusGroupSize: metaConsensusGroupSize, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(shardConsensusGroupSize), + MetachainConsensusGroupSize: uint32(metaConsensusGroupSize), + }, nil + }, + }, Marshalizer: TestMarshalizer, Hasher: TestHasher, Shuffler: nodeShuffler, diff --git a/integrationTests/vm/staking/baseTestMetaProcessor.go b/integrationTests/vm/staking/baseTestMetaProcessor.go index 0ae2b5ed2d8..a1d5a36b82e 100644 --- a/integrationTests/vm/staking/baseTestMetaProcessor.go +++ b/integrationTests/vm/staking/baseTestMetaProcessor.go @@ -263,7 +263,7 @@ func (tmp *TestMetaProcessor) createNewHeader(t *testing.T, round uint64) *block round, currentHash, currentHeader.GetRandSeed(), - tmp.NodesCoordinator.ConsensusGroupSize(core.MetachainShardId), + tmp.NodesCoordinator.ConsensusGroupSizeForShardAndEpoch(core.MetachainShardId, 0), ) return header diff --git a/integrationTests/vm/staking/nodesCoordiantorCreator.go b/integrationTests/vm/staking/nodesCoordiantorCreator.go index 27a54719521..698df48c408 100644 --- a/integrationTests/vm/staking/nodesCoordiantorCreator.go +++ b/integrationTests/vm/staking/nodesCoordiantorCreator.go @@ -13,6 +13,7 @@ import ( "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state/accounts" "github.com/multiversx/mx-chain-go/storage" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" nodesSetupMock "github.com/multiversx/mx-chain-go/testscommon/genesisMocks" "github.com/multiversx/mx-chain-go/testscommon/stakingcommon" "github.com/multiversx/mx-chain-storage-go/lrucache" @@ -39,10 +40,6 @@ func createNodesCoordinator( maxNodesConfig []config.MaxNodesChangeConfig, ) nodesCoordinator.NodesCoordinator { shufflerArgs := &nodesCoordinator.NodesShufflerArgs{ - NodesShard: numOfEligibleNodesPerShard, - NodesMeta: numOfMetaNodes, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: maxNodesConfig, EnableEpochs: config.EnableEpochs{ @@ -51,11 +48,23 @@ func createNodesCoordinator( }, EnableEpochsHandler: coreComponents.EnableEpochsHandler(), } + nodeShuffler, _ := nodesCoordinator.NewHashValidatorsShuffler(shufflerArgs) cache, _ := lrucache.NewCache(10000) argumentsNodesCoordinator := nodesCoordinator.ArgNodesCoordinator{ - ShardConsensusGroupSize: shardConsensusGroupSize, - MetaConsensusGroupSize: metaConsensusGroupSize, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + ChainParametersForEpochCalled: func(epoch uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + RoundDuration: 0, + Hysteresis: hysteresis, + ShardConsensusGroupSize: uint32(shardConsensusGroupSize), + ShardMinNumNodes: numOfEligibleNodesPerShard, + MetachainConsensusGroupSize: uint32(metaConsensusGroupSize), + MetachainMinNumNodes: numOfMetaNodes, + Adaptivity: adaptivity, + }, nil + }, + }, Marshalizer: coreComponents.InternalMarshalizer(), Hasher: coreComponents.Hasher(), ShardIDAsObserver: core.MetachainShardId, diff --git a/node/chainSimulator/chainSimulator.go b/node/chainSimulator/chainSimulator.go index 0d3d4d25e6a..6f4f732e6dd 100644 --- a/node/chainSimulator/chainSimulator.go +++ b/node/chainSimulator/chainSimulator.go @@ -48,6 +48,7 @@ type ArgsChainSimulator struct { NumOfShards uint32 MinNodesPerShard uint32 MetaChainMinNodes uint32 + Hysteresis float32 NumNodesWaitingListShard uint32 NumNodesWaitingListMeta uint32 GenesisTimestamp int64 @@ -120,6 +121,7 @@ func (s *simulator) createChainHandlers(args ArgsBaseChainSimulator) error { ConsensusGroupSize: args.ConsensusGroupSize, MetaChainMinNodes: args.MetaChainMinNodes, MetaChainConsensusGroupSize: args.MetaChainConsensusGroupSize, + Hysteresis: args.Hysteresis, RoundsPerEpoch: args.RoundsPerEpoch, InitialEpoch: args.InitialEpoch, AlterConfigsFunction: args.AlterConfigsFunction, diff --git a/node/chainSimulator/components/coreComponents.go b/node/chainSimulator/components/coreComponents.go index 49a7269d74b..f2bad834ad8 100644 --- a/node/chainSimulator/components/coreComponents.go +++ b/node/chainSimulator/components/coreComponents.go @@ -6,6 +6,7 @@ import ( "time" "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/common/chainparametersnotifier" "github.com/multiversx/mx-chain-go/common/enablers" factoryPubKey "github.com/multiversx/mx-chain-go/common/factory" "github.com/multiversx/mx-chain-go/common/forking" @@ -74,6 +75,8 @@ type coreComponentsHolder struct { processStatusHandler common.ProcessStatusHandler hardforkTriggerPubKey []byte enableEpochsHandler common.EnableEpochsHandler + chainParametersSubscriber process.ChainParametersSubscriber + chainParametersHandler process.ChainParametersHandler } // ArgsCoreComponentsHolder will hold arguments needed for the core components holder @@ -148,7 +151,25 @@ func CreateCoreComponents(args ArgsCoreComponentsHolder) (*coreComponentsHolder, instance.alarmScheduler = &mock.AlarmSchedulerStub{} instance.syncTimer = &testscommon.SyncTimerStub{} - instance.genesisNodesSetup, err = sharding.NewNodesSetup(args.NodesSetupPath, instance.addressPubKeyConverter, instance.validatorPubKeyConverter, args.NumShards) + instance.epochStartNotifierWithConfirm = notifier.NewEpochStartSubscriptionHandler() + instance.chainParametersSubscriber = chainparametersnotifier.NewChainParametersNotifier() + chainParametersNotifier := chainparametersnotifier.NewChainParametersNotifier() + argsChainParametersHandler := sharding.ArgsChainParametersHolder{ + EpochStartEventNotifier: instance.epochStartNotifierWithConfirm, + ChainParameters: args.Config.GeneralSettings.ChainParametersByEpoch, + ChainParametersNotifier: chainParametersNotifier, + } + instance.chainParametersHandler, err = sharding.NewChainParametersHolder(argsChainParametersHandler) + if err != nil { + return nil, err + } + + var nodesSetup config.NodesConfig + err = core.LoadJsonFile(&nodesSetup, args.NodesSetupPath) + if err != nil { + return nil, err + } + instance.genesisNodesSetup, err = sharding.NewNodesSetup(nodesSetup, instance.chainParametersHandler, instance.addressPubKeyConverter, instance.validatorPubKeyConverter, args.NumShards) if err != nil { return nil, err } @@ -182,12 +203,10 @@ func CreateCoreComponents(args ArgsCoreComponentsHolder) (*coreComponentsHolder, instance.apiEconomicsData = instance.economicsData instance.ratingsData, err = rating.NewRatingsData(rating.RatingsDataArg{ - Config: args.RatingConfig, - ShardConsensusSize: args.ConsensusGroupSize, - MetaConsensusSize: args.MetaChainConsensusGroupSize, - ShardMinNodes: args.MinNodesPerShard, - MetaMinNodes: args.MinNodesMeta, - RoundDurationMiliseconds: args.RoundDurationInMs, + EpochNotifier: instance.epochNotifier, + Config: args.RatingConfig, + ChainParametersHolder: instance.chainParametersHandler, + RoundDurationMilliseconds: args.RoundDurationInMs, }) if err != nil { return nil, err @@ -199,10 +218,6 @@ func CreateCoreComponents(args ArgsCoreComponentsHolder) (*coreComponentsHolder, } instance.nodesShuffler, err = nodesCoordinator.NewHashValidatorsShuffler(&nodesCoordinator.NodesShufflerArgs{ - NodesShard: args.MinNodesPerShard, - NodesMeta: args.MinNodesMeta, - Hysteresis: 0, - Adaptivity: false, ShuffleBetweenShards: true, MaxNodesEnableConfig: args.EnableEpochsConfig.MaxNodesChangeEnableEpoch, EnableEpochsHandler: instance.enableEpochsHandler, @@ -218,7 +233,6 @@ func CreateCoreComponents(args ArgsCoreComponentsHolder) (*coreComponentsHolder, return nil, err } - instance.epochStartNotifierWithConfirm = notifier.NewEpochStartSubscriptionHandler() instance.chanStopNodeProcess = args.ChanStopNodeProcess instance.genesisTime = time.Unix(instance.genesisNodesSetup.GetStartTime(), 0) instance.chainID = args.Config.GeneralSettings.ChainID @@ -428,6 +442,16 @@ func (c *coreComponentsHolder) EnableEpochsHandler() common.EnableEpochsHandler return c.enableEpochsHandler } +// ChainParametersSubscriber will return the chain parameters subscriber +func (c *coreComponentsHolder) ChainParametersSubscriber() process.ChainParametersSubscriber { + return c.chainParametersSubscriber +} + +// ChainParametersHandler will return the chain parameters handler +func (c *coreComponentsHolder) ChainParametersHandler() process.ChainParametersHandler { + return c.chainParametersHandler +} + func (c *coreComponentsHolder) collectClosableComponents() { c.closeHandler.AddComponent(c.alarmScheduler) c.closeHandler.AddComponent(c.syncTimer) diff --git a/node/chainSimulator/components/coreComponents_test.go b/node/chainSimulator/components/coreComponents_test.go index 7056d9ae48c..99fb00bbaa4 100644 --- a/node/chainSimulator/components/coreComponents_test.go +++ b/node/chainSimulator/components/coreComponents_test.go @@ -5,6 +5,7 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/data/endProcess" + "github.com/multiversx/mx-chain-go/testscommon/components" "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/config" @@ -30,18 +31,31 @@ func createArgsCoreComponentsHolder() ArgsCoreComponentsHolder { }, AddressPubkeyConverter: config.PubkeyConfig{ Length: 32, - Type: "hex", + Type: "bech32", + Hrp: "erd", }, ValidatorPubkeyConverter: config.PubkeyConfig{ - Length: 128, + Length: 96, Type: "hex", }, GeneralSettings: config.GeneralSettingsConfig{ ChainID: "T", MinTransactionVersion: 1, + ChainParametersByEpoch: []config.ChainParametersByEpochConfig{ + { + EnableEpoch: 0, + RoundDuration: 4000, + ShardConsensusGroupSize: 1, + ShardMinNumNodes: 1, + MetachainConsensusGroupSize: 1, + MetachainMinNumNodes: 1, + Hysteresis: 0, + Adaptivity: false, + }, + }, }, Hardfork: config.HardforkConfig{ - PublicKeyToListenFrom: "41378f754e2c7b2745208c3ed21b151d297acdc84c3aca00b9e292cf28ec2d444771070157ea7760ed83c26f4fed387d0077e00b563a95825dac2cbc349fc0025ccf774e37b0a98ad9724d30e90f8c29b4091ccb738ed9ffc0573df776ee9ea30b3c038b55e532760ea4a8f152f2a52848020e5cee1cc537f2c2323399723081", + PublicKeyToListenFrom: components.DummyPk, }, }, EnableEpochsConfig: config.EnableEpochs{}, @@ -127,7 +141,7 @@ func createArgsCoreComponentsHolder() ArgsCoreComponentsHolder { }, ChanStopNodeProcess: make(chan endProcess.ArgEndProcess), InitialRound: 0, - NodesSetupPath: "../../../sharding/mock/testdata/nodesSetupMock.json", + NodesSetupPath: "../../../cmd/node/config/nodesSetup.json", GasScheduleFilename: "../../../cmd/node/config/gasSchedules/gasScheduleV8.toml", NumShards: 3, WorkingDir: ".", @@ -292,8 +306,8 @@ func TestCoreComponents_GettersSetters(t *testing.T) { require.Equal(t, "T", comp.ChainID()) require.Equal(t, uint32(1), comp.MinTransactionVersion()) require.NotNil(t, comp.TxVersionChecker()) - require.Equal(t, uint32(64), comp.EncodedAddressLen()) - hfPk, _ := hex.DecodeString("41378f754e2c7b2745208c3ed21b151d297acdc84c3aca00b9e292cf28ec2d444771070157ea7760ed83c26f4fed387d0077e00b563a95825dac2cbc349fc0025ccf774e37b0a98ad9724d30e90f8c29b4091ccb738ed9ffc0573df776ee9ea30b3c038b55e532760ea4a8f152f2a52848020e5cee1cc537f2c2323399723081") + require.Equal(t, uint32(62), comp.EncodedAddressLen()) + hfPk, _ := hex.DecodeString(components.DummyPk) require.Equal(t, hfPk, comp.HardforkTriggerPubKey()) require.NotNil(t, comp.NodeTypeProvider()) require.NotNil(t, comp.WasmVMChangeLocker()) diff --git a/node/chainSimulator/components/nodeFacade.go b/node/chainSimulator/components/nodeFacade.go index 934807c0659..0136efefc31 100644 --- a/node/chainSimulator/components/nodeFacade.go +++ b/node/chainSimulator/components/nodeFacade.go @@ -85,7 +85,6 @@ func (node *testOnlyProcessingNode) createFacade(configs config.Configs, apiInte nodePack.WithNetworkComponents(node.NetworkComponentsHolder), nodePack.WithInitialNodesPubKeys(node.CoreComponentsHolder.GenesisNodesSetup().InitialNodesPubKeys()), nodePack.WithRoundDuration(node.CoreComponentsHolder.GenesisNodesSetup().GetRoundDuration()), - nodePack.WithConsensusGroupSize(int(node.CoreComponentsHolder.GenesisNodesSetup().GetShardConsensusGroupSize())), nodePack.WithGenesisTime(node.CoreComponentsHolder.GenesisTime()), nodePack.WithConsensusType(configs.GeneralConfig.Consensus.Type), nodePack.WithRequestedItemsHandler(node.ProcessComponentsHolder.RequestedItemsHandler()), diff --git a/node/chainSimulator/components/processComponents_test.go b/node/chainSimulator/components/processComponents_test.go index a8cb2f053e7..88ec4f1b3db 100644 --- a/node/chainSimulator/components/processComponents_test.go +++ b/node/chainSimulator/components/processComponents_test.go @@ -5,6 +5,7 @@ import ( "sync" "testing" + "github.com/multiversx/mx-chain-core-go/core" coreData "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/endProcess" "github.com/multiversx/mx-chain-core-go/hashing/blake2b" @@ -20,6 +21,7 @@ import ( chainStorage "github.com/multiversx/mx-chain-go/storage" "github.com/multiversx/mx-chain-go/testscommon" "github.com/multiversx/mx-chain-go/testscommon/bootstrapMocks" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/multiversx/mx-chain-go/testscommon/components" "github.com/multiversx/mx-chain-go/testscommon/cryptoMocks" "github.com/multiversx/mx-chain-go/testscommon/dataRetriever" @@ -57,7 +59,9 @@ var ( ) func createArgsProcessComponentsHolder() ArgsProcessComponentsHolder { - nodesSetup, _ := sharding.NewNodesSetup("../../../integrationTests/factory/testdata/nodesSetup.json", addrPubKeyConv, valPubKeyConv, 3) + var nodesConfig config.NodesConfig + _ = core.LoadJsonFile(&nodesConfig, "../../../integrationTests/factory/testdata/nodesSetup.json") + nodesSetup, _ := sharding.NewNodesSetup(nodesConfig, &chainParameters.ChainParametersHolderMock{}, addrPubKeyConv, valPubKeyConv, 3) args := ArgsProcessComponentsHolder{ Config: testscommon.GetGeneralConfig(), diff --git a/node/chainSimulator/components/testOnlyProcessingNode.go b/node/chainSimulator/components/testOnlyProcessingNode.go index efa4c12102c..1a13b61576d 100644 --- a/node/chainSimulator/components/testOnlyProcessingNode.go +++ b/node/chainSimulator/components/testOnlyProcessingNode.go @@ -310,6 +310,7 @@ func (node *testOnlyProcessingNode) createNodesCoordinator(pref config.Preferenc node.CoreComponentsHolder.EnableEpochsHandler(), node.DataPool.CurrentEpochValidatorInfo(), node.BootstrapComponentsHolder.NodesCoordinatorRegistryFactory(), + node.CoreComponentsHolder.ChainParametersHandler(), ) if err != nil { return err diff --git a/node/chainSimulator/configs/configs.go b/node/chainSimulator/configs/configs.go index 22fc863c7a0..ce2cdf6b5d4 100644 --- a/node/chainSimulator/configs/configs.go +++ b/node/chainSimulator/configs/configs.go @@ -52,6 +52,7 @@ type ArgsChainSimulatorConfigs struct { ConsensusGroupSize uint32 MetaChainMinNodes uint32 MetaChainConsensusGroupSize uint32 + Hysteresis float32 InitialEpoch uint32 RoundsPerEpoch core.OptionalUint64 NumNodesWaitingListShard uint32 @@ -132,6 +133,13 @@ func CreateChainSimulatorConfigs(args ArgsChainSimulatorConfigs) (*ArgsConfigsSi return nil, err } + configs.GeneralConfig.GeneralSettings.ChainParametersByEpoch[0].ShardMinNumNodes = args.MinNodesPerShard + configs.GeneralConfig.GeneralSettings.ChainParametersByEpoch[0].MetachainMinNumNodes = args.MetaChainMinNodes + configs.GeneralConfig.GeneralSettings.ChainParametersByEpoch[0].MetachainConsensusGroupSize = args.MetaChainConsensusGroupSize + configs.GeneralConfig.GeneralSettings.ChainParametersByEpoch[0].ShardConsensusGroupSize = args.ConsensusGroupSize + configs.GeneralConfig.GeneralSettings.ChainParametersByEpoch[0].RoundDuration = args.RoundDurationInMillis + configs.GeneralConfig.GeneralSettings.ChainParametersByEpoch[0].Hysteresis = args.Hysteresis + node.ApplyArchCustomConfigs(configs) if args.AlterConfigsFunction != nil { @@ -280,14 +288,10 @@ func generateValidatorsKeyAndUpdateFiles( nodes.RoundDuration = args.RoundDurationInMillis nodes.StartTime = args.GenesisTimeStamp - nodes.ConsensusGroupSize = args.ConsensusGroupSize - nodes.MetaChainConsensusGroupSize = args.MetaChainConsensusGroupSize nodes.Hysteresis = 0 - nodes.MinNodesPerShard = args.MinNodesPerShard - nodes.MetaChainMinNodes = args.MetaChainMinNodes - nodes.InitialNodes = make([]*sharding.InitialNode, 0) + configs.NodesConfig.InitialNodes = make([]*config.InitialNodeConfig, 0) privateKeys := make([]crypto.PrivateKey, 0) publicKeys := make([]crypto.PublicKey, 0) walletIndex := 0 @@ -307,6 +311,12 @@ func generateValidatorsKeyAndUpdateFiles( Address: stakeWallets[walletIndex].Address.Bech32, }) + configs.NodesConfig.InitialNodes = append(configs.NodesConfig.InitialNodes, &config.InitialNodeConfig{ + PubKey: hex.EncodeToString(pkBytes), + Address: stakeWallets[walletIndex].Address.Bech32, + InitialRating: 5000001, + }) + walletIndex++ } @@ -326,6 +336,13 @@ func generateValidatorsKeyAndUpdateFiles( PubKey: hex.EncodeToString(pkBytes), Address: stakeWallets[walletIndex].Address.Bech32, }) + + configs.NodesConfig.InitialNodes = append(configs.NodesConfig.InitialNodes, &config.InitialNodeConfig{ + PubKey: hex.EncodeToString(pkBytes), + Address: stakeWallets[walletIndex].Address.Bech32, + InitialRating: 5000001, + }) + walletIndex++ } } diff --git a/node/chainSimulator/disabled/antiflooder.go b/node/chainSimulator/disabled/antiflooder.go index 0d4c45fd0e3..1e705b29c47 100644 --- a/node/chainSimulator/disabled/antiflooder.go +++ b/node/chainSimulator/disabled/antiflooder.go @@ -16,6 +16,10 @@ func NewAntiFlooder() *antiFlooder { return &antiFlooder{} } +// SetConsensusSizeNotifier does nothing +func (a *antiFlooder) SetConsensusSizeNotifier(_ process.ChainParametersSubscriber, _ uint32) { +} + // CanProcessMessage returns nil func (a *antiFlooder) CanProcessMessage(_ p2p.MessageP2P, _ core.PeerID) error { return nil diff --git a/node/interface.go b/node/interface.go index 23a706ed25a..236e7a131e3 100644 --- a/node/interface.go +++ b/node/interface.go @@ -2,10 +2,8 @@ package node import ( "io" - "time" "github.com/multiversx/mx-chain-core-go/core" - "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/update" vmcommon "github.com/multiversx/mx-chain-vm-common-go" ) @@ -19,18 +17,6 @@ type NetworkShardingCollector interface { IsInterfaceNil() bool } -// P2PAntifloodHandler defines the behavior of a component able to signal that the system is too busy (or flooded) processing -// p2p messages -type P2PAntifloodHandler interface { - CanProcessMessage(message p2p.MessageP2P, fromConnectedPeer core.PeerID) error - CanProcessMessagesOnTopic(peer core.PeerID, topic string, numMessages uint32, totalSize uint64, sequence []byte) error - ResetForTopic(topic string) - SetMaxMessagesForTopic(topic string, maxNum uint32) - ApplyConsensusSize(size int) - BlacklistPeer(peer core.PeerID, reason string, duration time.Duration) - IsInterfaceNil() bool -} - // HardforkTrigger defines the behavior of a hardfork trigger type HardforkTrigger interface { SetExportFactoryHandler(exportFactoryHandler update.ExportFactoryHandler) error diff --git a/node/mock/factory/coreComponentsStub.go b/node/mock/factory/coreComponentsStub.go index 24c10b94a52..7b7fb27a402 100644 --- a/node/mock/factory/coreComponentsStub.go +++ b/node/mock/factory/coreComponentsStub.go @@ -20,39 +20,41 @@ import ( // CoreComponentsMock - type CoreComponentsMock struct { - IntMarsh marshal.Marshalizer - TxMarsh marshal.Marshalizer - VmMarsh marshal.Marshalizer - Hash hashing.Hasher - TxSignHasherField hashing.Hasher - UInt64ByteSliceConv typeConverters.Uint64ByteSliceConverter - AddrPubKeyConv core.PubkeyConverter - ValPubKeyConv core.PubkeyConverter - PathHdl storage.PathManagerHandler - ChainIdCalled func() string - MinTransactionVersionCalled func() uint32 - WDTimer core.WatchdogTimer - Alarm core.TimersScheduler - NtpTimer ntp.SyncTimer - RoundChangeNotifier process.RoundNotifier - RoundHandlerField consensus.RoundHandler - EconomicsHandler process.EconomicsDataHandler - APIEconomicsHandler process.EconomicsDataHandler - RatingsConfig process.RatingsInfoHandler - RatingHandler sharding.PeerAccountListAndRatingHandler - NodesConfig sharding.GenesisNodesSetupHandler - EpochChangeNotifier process.EpochNotifier - EnableRoundsHandlerField process.EnableRoundsHandler - EpochNotifierWithConfirm factory.EpochStartNotifierWithConfirm - ChanStopProcess chan endProcess.ArgEndProcess - Shuffler nodesCoordinator.NodesShuffler - TxVersionCheckHandler process.TxVersionCheckerHandler - StartTime time.Time - NodeTypeProviderField core.NodeTypeProviderHandler - WasmVMChangeLockerInternal common.Locker - ProcessStatusHandlerInternal common.ProcessStatusHandler - HardforkTriggerPubKeyField []byte - EnableEpochsHandlerField common.EnableEpochsHandler + IntMarsh marshal.Marshalizer + TxMarsh marshal.Marshalizer + VmMarsh marshal.Marshalizer + Hash hashing.Hasher + TxSignHasherField hashing.Hasher + UInt64ByteSliceConv typeConverters.Uint64ByteSliceConverter + AddrPubKeyConv core.PubkeyConverter + ValPubKeyConv core.PubkeyConverter + PathHdl storage.PathManagerHandler + ChainIdCalled func() string + MinTransactionVersionCalled func() uint32 + WDTimer core.WatchdogTimer + Alarm core.TimersScheduler + NtpTimer ntp.SyncTimer + RoundChangeNotifier process.RoundNotifier + RoundHandlerField consensus.RoundHandler + EconomicsHandler process.EconomicsDataHandler + APIEconomicsHandler process.EconomicsDataHandler + RatingsConfig process.RatingsInfoHandler + RatingHandler sharding.PeerAccountListAndRatingHandler + NodesConfig sharding.GenesisNodesSetupHandler + EpochChangeNotifier process.EpochNotifier + EnableRoundsHandlerField process.EnableRoundsHandler + EpochNotifierWithConfirm factory.EpochStartNotifierWithConfirm + ChanStopProcess chan endProcess.ArgEndProcess + Shuffler nodesCoordinator.NodesShuffler + TxVersionCheckHandler process.TxVersionCheckerHandler + StartTime time.Time + NodeTypeProviderField core.NodeTypeProviderHandler + WasmVMChangeLockerInternal common.Locker + ProcessStatusHandlerInternal common.ProcessStatusHandler + HardforkTriggerPubKeyField []byte + EnableEpochsHandlerField common.EnableEpochsHandler + ChainParametersHandlerField process.ChainParametersHandler + ChainParametersSubscriberField process.ChainParametersSubscriber } // Create - @@ -258,6 +260,16 @@ func (ccm *CoreComponentsMock) EnableEpochsHandler() common.EnableEpochsHandler return ccm.EnableEpochsHandlerField } +// ChainParametersHandler - +func (ccm *CoreComponentsMock) ChainParametersHandler() process.ChainParametersHandler { + return ccm.ChainParametersHandlerField +} + +// ChainParametersSubscriber - +func (ccm *CoreComponentsMock) ChainParametersSubscriber() process.ChainParametersSubscriber { + return ccm.ChainParametersSubscriberField +} + // IsInterfaceNil - func (ccm *CoreComponentsMock) IsInterfaceNil() bool { return ccm == nil diff --git a/node/mock/p2pAntifloodHandlerStub.go b/node/mock/p2pAntifloodHandlerStub.go index bda3da406d5..92f7bafdc88 100644 --- a/node/mock/p2pAntifloodHandlerStub.go +++ b/node/mock/p2pAntifloodHandlerStub.go @@ -16,6 +16,7 @@ type P2PAntifloodHandlerStub struct { SetDebuggerCalled func(debugger process.AntifloodDebugger) error BlacklistPeerCalled func(peer core.PeerID, reason string, duration time.Duration) IsOriginatorEligibleForTopicCalled func(pid core.PeerID, topic string) error + SetConsensusSizeNotifierCalled func(chainParametersNotifier process.ChainParametersSubscriber, shardID uint32) } // CanProcessMessage - @@ -75,6 +76,13 @@ func (p2pahs *P2PAntifloodHandlerStub) SetMaxMessagesForTopic(_ string, _ uint32 } +// SetConsensusSizeNotifier - +func (p2pahs *P2PAntifloodHandlerStub) SetConsensusSizeNotifier(chainParametersNotifier process.ChainParametersSubscriber, shardID uint32) { + if p2pahs.SetConsensusSizeNotifierCalled != nil { + p2pahs.SetConsensusSizeNotifierCalled(chainParametersNotifier, shardID) + } +} + // SetPeerValidatorMapper - func (p2pahs *P2PAntifloodHandlerStub) SetPeerValidatorMapper(_ process.PeerValidatorMapper) error { return nil diff --git a/node/node.go b/node/node.go index 38b00841d2a..f85f02e7b01 100644 --- a/node/node.go +++ b/node/node.go @@ -71,7 +71,6 @@ type accountInfo struct { type Node struct { initialNodesPubkeys map[uint32][]string roundDuration uint64 - consensusGroupSize int genesisTime time.Time peerDenialEvaluator p2p.PeerDenialEvaluator esdtStorageHandler vmcommon.ESDTNFTStorageHandler @@ -157,11 +156,6 @@ func (n *Node) CreateShardedStores() error { return nil } -// GetConsensusGroupSize returns the configured consensus size -func (n *Node) GetConsensusGroupSize() int { - return n.consensusGroupSize -} - // GetBalance gets the balance for a specific address func (n *Node) GetBalance(address string, options api.AccountQueryOptions) (*big.Int, api.BlockInfo, error) { userAccount, blockInfo, err := n.loadUserAccountHandlerByAddress(address, options) diff --git a/node/nodeHelper.go b/node/nodeHelper.go index b1b5a27c816..2e0099cc3d6 100644 --- a/node/nodeHelper.go +++ b/node/nodeHelper.go @@ -65,11 +65,6 @@ func CreateNode( genesisTime := time.Unix(coreComponents.GenesisNodesSetup().GetStartTime(), 0) - consensusGroupSize, err := consensusComponents.ConsensusGroupSize() - if err != nil { - return nil, err - } - var nd *Node nd, err = NewNode( WithStatusCoreComponents(statusCoreComponents), @@ -85,7 +80,6 @@ func CreateNode( WithNetworkComponents(networkComponents), WithInitialNodesPubKeys(coreComponents.GenesisNodesSetup().InitialNodesPubKeys()), WithRoundDuration(coreComponents.GenesisNodesSetup().GetRoundDuration()), - WithConsensusGroupSize(consensusGroupSize), WithGenesisTime(genesisTime), WithConsensusType(config.Consensus.Type), WithBootstrapRoundIndex(bootstrapRoundIndex), diff --git a/node/nodeRunner.go b/node/nodeRunner.go index 749d70ffd2a..c98f25f0093 100644 --- a/node/nodeRunner.go +++ b/node/nodeRunner.go @@ -398,6 +398,7 @@ func (nr *nodeRunner) executeOneComponentCreationCycle( managedCoreComponents.EnableEpochsHandler(), managedDataComponents.Datapool().CurrentEpochValidatorInfo(), managedBootstrapComponents.NodesCoordinatorRegistryFactory(), + managedCoreComponents.ChainParametersHandler(), ) if err != nil { return true, err @@ -1474,7 +1475,7 @@ func (nr *nodeRunner) CreateManagedCoreComponents( ImportDbConfig: *nr.configs.ImportDbConfig, RatingsConfig: *nr.configs.RatingsConfig, EconomicsConfig: *nr.configs.EconomicsConfig, - NodesFilename: nr.configs.ConfigurationPathsHolder.Nodes, + NodesConfig: *nr.configs.NodesConfig, WorkingDirectory: nr.configs.FlagsConfig.DbDir, ChanStopNodeProcess: chanStopNodeProcess, } diff --git a/node/node_test.go b/node/node_test.go index 7c516e18a93..a94fdb01087 100644 --- a/node/node_test.go +++ b/node/node_test.go @@ -4678,7 +4678,6 @@ func TestNode_Getters(t *testing.T) { heartbeatComponents := &factoryMock.HeartbeatV2ComponentsStub{} networkComponents := getDefaultNetworkComponents() processComponents := getDefaultProcessComponents() - consensusGroupSize := 10 n, err := node.NewNode( node.WithCoreComponents(coreComponents), @@ -4690,7 +4689,6 @@ func TestNode_Getters(t *testing.T) { node.WithHeartbeatV2Components(heartbeatComponents), node.WithNetworkComponents(networkComponents), node.WithProcessComponents(processComponents), - node.WithConsensusGroupSize(consensusGroupSize), node.WithImportMode(true), ) require.Nil(t, err) @@ -4705,7 +4703,6 @@ func TestNode_Getters(t *testing.T) { assert.True(t, n.GetHeartbeatV2Components() == heartbeatComponents) assert.True(t, n.GetNetworkComponents() == networkComponents) assert.True(t, n.GetProcessComponents() == processComponents) - assert.Equal(t, consensusGroupSize, n.GetConsensusGroupSize()) assert.True(t, n.IsInImportMode()) } diff --git a/node/options.go b/node/options.go index 90385b3b8f4..f3ddc9bde10 100644 --- a/node/options.go +++ b/node/options.go @@ -210,18 +210,6 @@ func WithRoundDuration(roundDuration uint64) Option { } } -// WithConsensusGroupSize sets up the consensus group size option for the Node -func WithConsensusGroupSize(consensusGroupSize int) Option { - return func(n *Node) error { - if consensusGroupSize < 1 { - return ErrNegativeOrZeroConsensusGroupSize - } - log.Info("consensus group", "size", consensusGroupSize) - n.consensusGroupSize = consensusGroupSize - return nil - } -} - // WithGenesisTime sets up the genesis time option for the Node func WithGenesisTime(genesisTime time.Time) Option { return func(n *Node) error { diff --git a/node/options_test.go b/node/options_test.go index fa4a92ea449..1f565b58d34 100644 --- a/node/options_test.go +++ b/node/options_test.go @@ -71,32 +71,6 @@ func TestWithRoundDuration_ShouldWork(t *testing.T) { assert.Nil(t, err) } -func TestWithConsensusGroupSize_NegativeGroupSizeShouldErr(t *testing.T) { - t.Parallel() - - node, _ := NewNode() - - opt := WithConsensusGroupSize(-1) - err := opt(node) - - assert.Equal(t, 0, node.consensusGroupSize) - assert.Equal(t, ErrNegativeOrZeroConsensusGroupSize, err) -} - -func TestWithConsensusGroupSize_ShouldWork(t *testing.T) { - t.Parallel() - - node, _ := NewNode() - - groupSize := 567 - - opt := WithConsensusGroupSize(groupSize) - err := opt(node) - - assert.True(t, node.consensusGroupSize == groupSize) - assert.Nil(t, err) -} - func TestWithGenesisTime(t *testing.T) { t.Parallel() diff --git a/process/errors.go b/process/errors.go index 2f58af09f46..e2ddf248032 100644 --- a/process/errors.go +++ b/process/errors.go @@ -465,6 +465,9 @@ var ErrNilEpochNotifier = errors.New("nil EpochNotifier") // ErrNilRoundNotifier signals that the provided EpochNotifier is nil var ErrNilRoundNotifier = errors.New("nil RoundNotifier") +// ErrNilChainParametersHandler signals that the provided chain parameters handler is nil +var ErrNilChainParametersHandler = errors.New("nil chain parameters handler") + // ErrInvalidCacheRefreshIntervalInSec signals that the cacheRefreshIntervalInSec is invalid - zero or less var ErrInvalidCacheRefreshIntervalInSec = errors.New("invalid cacheRefreshIntervalInSec") @@ -1230,6 +1233,15 @@ var ErrTransferAndExecuteByUserAddressesAreNil = errors.New("transfer and execut // ErrRelayedTxV3Disabled signals that relayed tx v3 are disabled var ErrRelayedTxV3Disabled = errors.New("relayed tx v3 are disabled") +// ErrMissingConfigurationForEpochZero signals that the provided configuration doesn't include anything for epoch 0 +var ErrMissingConfigurationForEpochZero = errors.New("missing configuration for epoch 0") + +// ErrEmptyChainParametersConfiguration signals that an empty chain parameters configuration has been provided +var ErrEmptyChainParametersConfiguration = errors.New("empty chain parameters configuration") + +// ErrNoMatchingConfigForProvidedEpoch signals that there is no matching configuration for the provided epoch +var ErrNoMatchingConfigForProvidedEpoch = errors.New("no matching configuration") + // ErrGuardedRelayerNotAllowed signals that the provided relayer is guarded var ErrGuardedRelayerNotAllowed = errors.New("guarded relayer not allowed") diff --git a/process/interface.go b/process/interface.go index e5800a54796..c5e9570f8db 100644 --- a/process/interface.go +++ b/process/interface.go @@ -25,6 +25,7 @@ import ( "github.com/multiversx/mx-chain-go/common" cryptoCommon "github.com/multiversx/mx-chain-go/common/crypto" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process/block/bootstrapStorage" @@ -913,12 +914,18 @@ type TopicFloodPreventer interface { IsInterfaceNil() bool } +// ChainParametersSubscriber is the interface that can be used to subscribe for chain parameters changes +type ChainParametersSubscriber interface { + RegisterNotifyHandler(handler common.ChainParametersSubscriptionHandler) + IsInterfaceNil() bool +} + // P2PAntifloodHandler defines the behavior of a component able to signal that the system is too busy (or flooded) processing // p2p messages type P2PAntifloodHandler interface { CanProcessMessage(message p2p.MessageP2P, fromConnectedPeer core.PeerID) error CanProcessMessagesOnTopic(pid core.PeerID, topic string, numMessages uint32, totalSize uint64, sequence []byte) error - ApplyConsensusSize(size int) + SetConsensusSizeNotifier(chainParametersNotifier ChainParametersSubscriber, shardID uint32) SetDebugger(debugger AntifloodDebugger) error BlacklistPeer(peer core.PeerID, reason string, duration time.Duration) IsOriginatorEligibleForTopic(pid core.PeerID, topic string) error @@ -1047,6 +1054,36 @@ type RatingsStepHandler interface { ConsecutiveMissedBlocksPenalty() float32 } +// NodesSetupHandler returns the nodes' configuration +type NodesSetupHandler interface { + AllInitialNodes() []nodesCoordinator.GenesisNodeInfoHandler + InitialNodesPubKeys() map[uint32][]string + GetShardIDForPubKey(pubkey []byte) (uint32, error) + InitialEligibleNodesPubKeysForShard(shardId uint32) ([]string, error) + InitialNodesInfoForShard(shardId uint32) ([]nodesCoordinator.GenesisNodeInfoHandler, []nodesCoordinator.GenesisNodeInfoHandler, error) + InitialNodesInfo() (map[uint32][]nodesCoordinator.GenesisNodeInfoHandler, map[uint32][]nodesCoordinator.GenesisNodeInfoHandler) + GetStartTime() int64 + GetRoundDuration() uint64 + GetShardConsensusGroupSize() uint32 + GetMetaConsensusGroupSize() uint32 + NumberOfShards() uint32 + MinNumberOfNodes() uint32 + MinNumberOfShardNodes() uint32 + MinNumberOfMetaNodes() uint32 + GetHysteresis() float32 + GetAdaptivity() bool + MinNumberOfNodesWithHysteresis() uint32 + IsInterfaceNil() bool +} + +// ChainParametersHandler defines the actions that need to be done by a component that can handle chain parameters +type ChainParametersHandler interface { + CurrentChainParameters() config.ChainParametersByEpochConfig + AllChainParameters() []config.ChainParametersByEpochConfig + ChainParametersForEpoch(epoch uint32) (config.ChainParametersByEpochConfig, error) + IsInterfaceNil() bool +} + // ValidatorInfoSyncer defines the method needed for validatorInfoProcessing type ValidatorInfoSyncer interface { SyncMiniBlocks(headerHandler data.HeaderHandler) ([][]byte, data.BodyHandler, error) @@ -1186,11 +1223,13 @@ type CoreComponentsHolder interface { TxVersionChecker() TxVersionCheckerHandler GenesisNodesSetup() sharding.GenesisNodesSetupHandler EpochNotifier() EpochNotifier + ChainParametersSubscriber() ChainParametersSubscriber ChanStopNodeProcess() chan endProcess.ArgEndProcess NodeTypeProvider() core.NodeTypeProviderHandler ProcessStatusHandler() common.ProcessStatusHandler HardforkTriggerPubKey() []byte EnableEpochsHandler() common.EnableEpochsHandler + ChainParametersHandler() ChainParametersHandler IsInterfaceNil() bool } diff --git a/process/mock/coreComponentsMock.go b/process/mock/coreComponentsMock.go index 32455e5d54b..322066711a8 100644 --- a/process/mock/coreComponentsMock.go +++ b/process/mock/coreComponentsMock.go @@ -17,29 +17,31 @@ import ( // CoreComponentsMock - type CoreComponentsMock struct { - IntMarsh marshal.Marshalizer - TxMarsh marshal.Marshalizer - Hash hashing.Hasher - TxSignHasherField hashing.Hasher - UInt64ByteSliceConv typeConverters.Uint64ByteSliceConverter - AddrPubKeyConv core.PubkeyConverter - ValPubKeyConv core.PubkeyConverter - PathHdl storage.PathManagerHandler - ChainIdCalled func() string - MinTransactionVersionCalled func() uint32 - GenesisNodesSetupCalled func() sharding.GenesisNodesSetupHandler - TxVersionCheckField process.TxVersionCheckerHandler - EpochNotifierField process.EpochNotifier - EnableEpochsHandlerField common.EnableEpochsHandler - RoundNotifierField process.RoundNotifier - EnableRoundsHandlerField process.EnableRoundsHandler - RoundField consensus.RoundHandler - StatusField core.AppStatusHandler - ChanStopNode chan endProcess.ArgEndProcess - NodeTypeProviderField core.NodeTypeProviderHandler - EconomicsDataField process.EconomicsDataHandler - ProcessStatusHandlerField common.ProcessStatusHandler - HardforkTriggerPubKeyField []byte + IntMarsh marshal.Marshalizer + TxMarsh marshal.Marshalizer + Hash hashing.Hasher + TxSignHasherField hashing.Hasher + UInt64ByteSliceConv typeConverters.Uint64ByteSliceConverter + AddrPubKeyConv core.PubkeyConverter + ValPubKeyConv core.PubkeyConverter + PathHdl storage.PathManagerHandler + ChainIdCalled func() string + MinTransactionVersionCalled func() uint32 + GenesisNodesSetupCalled func() sharding.GenesisNodesSetupHandler + TxVersionCheckField process.TxVersionCheckerHandler + EpochNotifierField process.EpochNotifier + EnableEpochsHandlerField common.EnableEpochsHandler + RoundNotifierField process.RoundNotifier + EnableRoundsHandlerField process.EnableRoundsHandler + RoundField consensus.RoundHandler + StatusField core.AppStatusHandler + ChanStopNode chan endProcess.ArgEndProcess + NodeTypeProviderField core.NodeTypeProviderHandler + EconomicsDataField process.EconomicsDataHandler + ProcessStatusHandlerField common.ProcessStatusHandler + ChainParametersHandlerField process.ChainParametersHandler + HardforkTriggerPubKeyField []byte + ChainParametersSubscriberField process.ChainParametersSubscriber } // ChanStopNodeProcess - @@ -82,6 +84,11 @@ func (ccm *CoreComponentsMock) Uint64ByteSliceConverter() typeConverters.Uint64B return ccm.UInt64ByteSliceConv } +// ChainParametersHandler - +func (ccm *CoreComponentsMock) ChainParametersHandler() process.ChainParametersHandler { + return ccm.ChainParametersHandlerField +} + // AddressPubKeyConverter - func (ccm *CoreComponentsMock) AddressPubKeyConverter() core.PubkeyConverter { return ccm.AddrPubKeyConv @@ -175,6 +182,11 @@ func (ccm *CoreComponentsMock) HardforkTriggerPubKey() []byte { return ccm.HardforkTriggerPubKeyField } +// ChainParametersSubscriber - +func (ccm *CoreComponentsMock) ChainParametersSubscriber() process.ChainParametersSubscriber { + return ccm.ChainParametersSubscriberField +} + // IsInterfaceNil - func (ccm *CoreComponentsMock) IsInterfaceNil() bool { return ccm == nil diff --git a/process/mock/floodPreventerStub.go b/process/mock/floodPreventerStub.go index d9d1a8881c3..85367b10545 100644 --- a/process/mock/floodPreventerStub.go +++ b/process/mock/floodPreventerStub.go @@ -11,7 +11,11 @@ type FloodPreventerStub struct { // IncreaseLoad - func (fps *FloodPreventerStub) IncreaseLoad(pid core.PeerID, size uint64) error { - return fps.IncreaseLoadCalled(pid, size) + if fps.IncreaseLoadCalled != nil { + return fps.IncreaseLoadCalled(pid, size) + } + + return nil } // ApplyConsensusSize - diff --git a/process/mock/p2pAntifloodHandlerStub.go b/process/mock/p2pAntifloodHandlerStub.go index 7dd6e011474..819267b91ae 100644 --- a/process/mock/p2pAntifloodHandlerStub.go +++ b/process/mock/p2pAntifloodHandlerStub.go @@ -12,7 +12,7 @@ import ( type P2PAntifloodHandlerStub struct { CanProcessMessageCalled func(message p2p.MessageP2P, fromConnectedPeer core.PeerID) error CanProcessMessagesOnTopicCalled func(peer core.PeerID, topic string, numMessages uint32, totalSize uint64, sequence []byte) error - ApplyConsensusSizeCalled func(size int) + SetConsensusSizeNotifierCalled func(subscriber process.ChainParametersSubscriber, shardID uint32) SetDebuggerCalled func(debugger process.AntifloodDebugger) error BlacklistPeerCalled func(peer core.PeerID, reason string, duration time.Duration) IsOriginatorEligibleForTopicCalled func(pid core.PeerID, topic string) error @@ -42,10 +42,10 @@ func (p2pahs *P2PAntifloodHandlerStub) CanProcessMessagesOnTopic(peer core.PeerI return p2pahs.CanProcessMessagesOnTopicCalled(peer, topic, numMessages, totalSize, sequence) } -// ApplyConsensusSize - -func (p2pahs *P2PAntifloodHandlerStub) ApplyConsensusSize(size int) { - if p2pahs.ApplyConsensusSizeCalled != nil { - p2pahs.ApplyConsensusSizeCalled(size) +// SetConsensusSizeNotifier - +func (p2pahs *P2PAntifloodHandlerStub) SetConsensusSizeNotifier(subscriber process.ChainParametersSubscriber, shardID uint32) { + if p2pahs.SetConsensusSizeNotifierCalled != nil { + p2pahs.SetConsensusSizeNotifierCalled(subscriber, shardID) } } diff --git a/process/peer/process.go b/process/peer/process.go index 20e0b377365..7cb50db55be 100644 --- a/process/peer/process.go +++ b/process/peer/process.go @@ -1192,7 +1192,7 @@ func (vs *validatorStatistics) decreaseAll( } log.Debug("ValidatorStatistics decreasing all", "shardID", shardID, "missedRounds", missedRounds) - consensusGroupSize := vs.nodesCoordinator.ConsensusGroupSize(shardID) + consensusGroupSize := vs.nodesCoordinator.ConsensusGroupSizeForShardAndEpoch(shardID, epoch) validators, err := vs.nodesCoordinator.GetAllEligibleValidatorsPublicKeys(epoch) if err != nil { return err diff --git a/process/peer/process_test.go b/process/peer/process_test.go index d4c85a5601f..4a3bd5a212b 100644 --- a/process/peer/process_test.go +++ b/process/peer/process_test.go @@ -1822,7 +1822,7 @@ func DoComputeMissingBlocks( GetAllEligibleValidatorsPublicKeysCalled: func(_ uint32) (map[uint32][][]byte, error) { return validatorPublicKeys, nil }, - ConsensusGroupSizeCalled: func(uint32) int { + ConsensusGroupSizeCalled: func(uint32, uint32) int { return consensusGroupSize }, GetValidatorWithPublicKeyCalled: func(publicKey []byte) (nodesCoordinator.Validator, uint32, error) { diff --git a/process/rating/ratingsData.go b/process/rating/ratingsData.go index 5e0b34ce75b..daf8c60ee42 100644 --- a/process/rating/ratingsData.go +++ b/process/rating/ratingsData.go @@ -3,19 +3,28 @@ package rating import ( "fmt" "math" + "sort" + "sync" + "github.com/multiversx/mx-chain-core-go/core/check" "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/process" ) var _ process.RatingsInfoHandler = (*RatingsData)(nil) -const milisecondsInHour = 3600 * 1000 +const millisecondsInHour = 3600 * 1000 + +type ratingsStepsData struct { + enableEpoch uint32 + shardRatingsStepData process.RatingsStepHandler + metaRatingsStepData process.RatingsStepHandler +} type computeRatingStepArg struct { shardSize uint32 consensusSize uint32 - roundTimeMilis uint64 + roundTimeMillis uint64 startRating uint32 maxRating uint32 hoursToMaxRatingFromStartRating uint32 @@ -27,27 +36,35 @@ type computeRatingStepArg struct { // RatingsData will store information about ratingsComputation type RatingsData struct { - startRating uint32 - maxRating uint32 - minRating uint32 - signedBlocksThreshold float32 - metaRatingsStepData process.RatingsStepHandler - shardRatingsStepData process.RatingsStepHandler - selectionChances []process.SelectionChance + startRating uint32 + maxRating uint32 + minRating uint32 + signedBlocksThreshold float32 + currentRatingsStepData ratingsStepsData + ratingsStepsConfig []ratingsStepsData + selectionChances []process.SelectionChance + chainParametersHandler process.ChainParametersHandler + ratingsSetup config.RatingsConfig + roundDurationInMilliseconds uint64 + mutConfiguration sync.RWMutex } // RatingsDataArg contains information for the creation of the new ratingsData type RatingsDataArg struct { - Config config.RatingsConfig - ShardConsensusSize uint32 - MetaConsensusSize uint32 - ShardMinNodes uint32 - MetaMinNodes uint32 - RoundDurationMiliseconds uint64 + EpochNotifier process.EpochNotifier + Config config.RatingsConfig + ChainParametersHolder process.ChainParametersHandler + RoundDurationMilliseconds uint64 } // NewRatingsData creates a new RatingsData instance func NewRatingsData(args RatingsDataArg) (*RatingsData, error) { + if check.IfNil(args.EpochNotifier) { + return nil, process.ErrNilEpochNotifier + } + if check.IfNil(args.ChainParametersHolder) { + return nil, process.ErrNilChainParametersHandler + } ratingsConfig := args.Config err := verifyRatingsConfig(ratingsConfig) if err != nil { @@ -62,10 +79,11 @@ func NewRatingsData(args RatingsDataArg) (*RatingsData, error) { }) } + currentChainParameters := args.ChainParametersHolder.CurrentChainParameters() arg := computeRatingStepArg{ - shardSize: args.ShardMinNodes, - consensusSize: args.ShardConsensusSize, - roundTimeMilis: args.RoundDurationMiliseconds, + shardSize: currentChainParameters.ShardMinNumNodes, + consensusSize: currentChainParameters.ShardConsensusGroupSize, + roundTimeMillis: args.RoundDurationMilliseconds, startRating: ratingsConfig.General.StartRating, maxRating: ratingsConfig.General.MaxRating, hoursToMaxRatingFromStartRating: ratingsConfig.ShardChain.HoursToMaxRatingFromStartRating, @@ -80,9 +98,9 @@ func NewRatingsData(args RatingsDataArg) (*RatingsData, error) { } arg = computeRatingStepArg{ - shardSize: args.MetaMinNodes, - consensusSize: args.MetaConsensusSize, - roundTimeMilis: args.RoundDurationMiliseconds, + shardSize: currentChainParameters.MetachainMinNumNodes, + consensusSize: currentChainParameters.MetachainConsensusGroupSize, + roundTimeMillis: args.RoundDurationMilliseconds, startRating: ratingsConfig.General.StartRating, maxRating: ratingsConfig.General.MaxRating, hoursToMaxRatingFromStartRating: ratingsConfig.MetaChain.HoursToMaxRatingFromStartRating, @@ -96,17 +114,154 @@ func NewRatingsData(args RatingsDataArg) (*RatingsData, error) { return nil, err } - return &RatingsData{ - startRating: ratingsConfig.General.StartRating, - maxRating: ratingsConfig.General.MaxRating, - minRating: ratingsConfig.General.MinRating, - signedBlocksThreshold: ratingsConfig.General.SignedBlocksThreshold, - metaRatingsStepData: metaRatingStep, - shardRatingsStepData: shardRatingStep, - selectionChances: chances, + ratingsConfigValue := ratingsStepsData{ + enableEpoch: args.EpochNotifier.CurrentEpoch(), + shardRatingsStepData: shardRatingStep, + metaRatingsStepData: metaRatingStep, + } + + ratingData := &RatingsData{ + startRating: ratingsConfig.General.StartRating, + maxRating: ratingsConfig.General.MaxRating, + minRating: ratingsConfig.General.MinRating, + signedBlocksThreshold: ratingsConfig.General.SignedBlocksThreshold, + currentRatingsStepData: ratingsConfigValue, + selectionChances: chances, + chainParametersHandler: args.ChainParametersHolder, + ratingsSetup: ratingsConfig, + roundDurationInMilliseconds: args.RoundDurationMilliseconds, + } + + err = ratingData.computeRatingStepsConfig(args.ChainParametersHolder.AllChainParameters()) + if err != nil { + return nil, err + } + + args.EpochNotifier.RegisterNotifyHandler(ratingData) + + return ratingData, nil +} + +func (rd *RatingsData) computeRatingStepsConfig(chainParamsList []config.ChainParametersByEpochConfig) error { + if len(chainParamsList) == 0 { + return process.ErrEmptyChainParametersConfiguration + } + + ratingsStepsConfig := make([]ratingsStepsData, 0) + for _, chainParams := range chainParamsList { + configForEpoch, err := rd.computeRatingStepsConfigForParams(chainParams) + if err != nil { + return err + } + + ratingsStepsConfig = append(ratingsStepsConfig, configForEpoch) + } + + // sort the config values descending + sort.SliceStable(ratingsStepsConfig, func(i, j int) bool { + return ratingsStepsConfig[i].enableEpoch > ratingsStepsConfig[j].enableEpoch + }) + + earliestConfig := ratingsStepsConfig[len(ratingsStepsConfig)-1] + if earliestConfig.enableEpoch != 0 { + return process.ErrMissingConfigurationForEpochZero + } + + rd.ratingsStepsConfig = ratingsStepsConfig + + return nil +} + +func (rd *RatingsData) computeRatingStepsConfigForParams(chainParams config.ChainParametersByEpochConfig) (ratingsStepsData, error) { + shardRatingsStepsArgs := computeRatingStepArg{ + shardSize: chainParams.ShardMinNumNodes, + consensusSize: chainParams.ShardConsensusGroupSize, + roundTimeMillis: rd.roundDurationInMilliseconds, + startRating: rd.ratingsSetup.General.StartRating, + maxRating: rd.ratingsSetup.General.MaxRating, + hoursToMaxRatingFromStartRating: rd.ratingsSetup.ShardChain.HoursToMaxRatingFromStartRating, + proposerDecreaseFactor: rd.ratingsSetup.ShardChain.ProposerDecreaseFactor, + validatorDecreaseFactor: rd.ratingsSetup.ShardChain.ValidatorDecreaseFactor, + consecutiveMissedBlocksPenalty: rd.ratingsSetup.ShardChain.ConsecutiveMissedBlocksPenalty, + proposerValidatorImportance: rd.ratingsSetup.ShardChain.ProposerValidatorImportance, + } + shardRatingsStepData, err := computeRatingStep(shardRatingsStepsArgs) + if err != nil { + return ratingsStepsData{}, fmt.Errorf("%w while computing shard rating steps for epoch %d", err, chainParams.EnableEpoch) + } + + metaRatingsStepsArgs := computeRatingStepArg{ + shardSize: chainParams.MetachainMinNumNodes, + consensusSize: chainParams.MetachainConsensusGroupSize, + roundTimeMillis: rd.roundDurationInMilliseconds, + startRating: rd.ratingsSetup.General.StartRating, + maxRating: rd.ratingsSetup.General.MaxRating, + hoursToMaxRatingFromStartRating: rd.ratingsSetup.MetaChain.HoursToMaxRatingFromStartRating, + proposerDecreaseFactor: rd.ratingsSetup.MetaChain.ProposerDecreaseFactor, + validatorDecreaseFactor: rd.ratingsSetup.MetaChain.ValidatorDecreaseFactor, + consecutiveMissedBlocksPenalty: rd.ratingsSetup.MetaChain.ConsecutiveMissedBlocksPenalty, + proposerValidatorImportance: rd.ratingsSetup.MetaChain.ProposerValidatorImportance, + } + metaRatingsStepData, err := computeRatingStep(metaRatingsStepsArgs) + if err != nil { + return ratingsStepsData{}, fmt.Errorf("%w while computing metachain rating steps for epoch %d", err, chainParams.EnableEpoch) + } + + return ratingsStepsData{ + enableEpoch: chainParams.EnableEpoch, + shardRatingsStepData: shardRatingsStepData, + metaRatingsStepData: metaRatingsStepData, }, nil } +// EpochConfirmed will be called whenever a new epoch is confirmed +func (rd *RatingsData) EpochConfirmed(epoch uint32, _ uint64) { + log.Debug("RatingsData - epoch confirmed", "epoch", epoch) + + rd.mutConfiguration.Lock() + defer rd.mutConfiguration.Unlock() + + newVersion, err := rd.getMatchingVersion(epoch) + if err != nil { + log.Error("RatingsData.EpochConfirmed - cannot get matching version", "epoch", epoch, "error", err) + return + } + + if rd.currentRatingsStepData.enableEpoch == newVersion.enableEpoch { + return + } + + rd.currentRatingsStepData = newVersion + + log.Debug("updated shard ratings step data", + "epoch", epoch, + "proposer increase rating step", newVersion.shardRatingsStepData.ProposerIncreaseRatingStep(), + "proposer decrease rating step", newVersion.shardRatingsStepData.ProposerDecreaseRatingStep(), + "validator increase rating step", newVersion.shardRatingsStepData.ValidatorIncreaseRatingStep(), + "validator decrease rating step", newVersion.shardRatingsStepData.ValidatorDecreaseRatingStep(), + ) + + log.Debug("updated metachain ratings step data", + "epoch", epoch, + "proposer increase rating step", newVersion.metaRatingsStepData.ProposerIncreaseRatingStep(), + "proposer decrease rating step", newVersion.metaRatingsStepData.ProposerDecreaseRatingStep(), + "validator increase rating step", newVersion.metaRatingsStepData.ValidatorIncreaseRatingStep(), + "validator decrease rating step", newVersion.metaRatingsStepData.ValidatorDecreaseRatingStep(), + ) +} + +func (rd *RatingsData) getMatchingVersion(epoch uint32) (ratingsStepsData, error) { + // the config values are sorted in descending order, so the matching version is the first one whose enable epoch is less or equal than the provided epoch + for _, ratingsStepConfig := range rd.ratingsStepsConfig { + if ratingsStepConfig.enableEpoch <= epoch { + return ratingsStepConfig, nil + } + } + + // the code should never reach this point, since the config values are checked on the constructor + return ratingsStepsData{}, process.ErrNoMatchingConfigForProvidedEpoch +} + func verifyRatingsConfig(settings config.RatingsConfig) error { if settings.General.MinRating < 1 { return process.ErrMinRatingSmallerThanOne @@ -165,7 +320,7 @@ func verifyRatingsConfig(settings config.RatingsConfig) error { func computeRatingStep( arg computeRatingStepArg, ) (process.RatingsStepHandler, error) { - blocksProducedInHours := uint64(arg.hoursToMaxRatingFromStartRating*milisecondsInHour) / arg.roundTimeMilis + blocksProducedInHours := uint64(arg.hoursToMaxRatingFromStartRating*millisecondsInHour) / arg.roundTimeMillis ratingDifference := arg.maxRating - arg.startRating proposerProbability := float32(blocksProducedInHours) / float32(arg.shardSize) @@ -210,37 +365,48 @@ func computeRatingStep( // StartRating will return the start rating func (rd *RatingsData) StartRating() uint32 { + // no need for mutex protection since this value is only set on constructor return rd.startRating } // MaxRating will return the max rating func (rd *RatingsData) MaxRating() uint32 { + // no need for mutex protection since this value is only set on constructor return rd.maxRating } // MinRating will return the min rating func (rd *RatingsData) MinRating() uint32 { + // no need for mutex protection since this value is only set on constructor return rd.minRating } // SignedBlocksThreshold will return the signed blocks threshold func (rd *RatingsData) SignedBlocksThreshold() float32 { + // no need for mutex protection since this value is only set on constructor return rd.signedBlocksThreshold } // SelectionChances will return the array of selectionChances and thresholds func (rd *RatingsData) SelectionChances() []process.SelectionChance { + // no need for mutex protection since this value is only set on constructor return rd.selectionChances } // MetaChainRatingsStepHandler returns the RatingsStepHandler used for the Metachain func (rd *RatingsData) MetaChainRatingsStepHandler() process.RatingsStepHandler { - return rd.metaRatingsStepData + rd.mutConfiguration.RLock() + defer rd.mutConfiguration.RUnlock() + + return rd.currentRatingsStepData.metaRatingsStepData } // ShardChainRatingsStepHandler returns the RatingsStepHandler used for the ShardChains func (rd *RatingsData) ShardChainRatingsStepHandler() process.RatingsStepHandler { - return rd.shardRatingsStepData + rd.mutConfiguration.RLock() + defer rd.mutConfiguration.RUnlock() + + return rd.currentRatingsStepData.shardRatingsStepData } // IsInterfaceNil returns true if underlying object is nil diff --git a/process/rating/ratingsData_test.go b/process/rating/ratingsData_test.go index 22ccd960aeb..df1a09c5640 100644 --- a/process/rating/ratingsData_test.go +++ b/process/rating/ratingsData_test.go @@ -8,6 +8,8 @@ import ( "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/process" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" + "github.com/multiversx/mx-chain-go/testscommon/epochNotifier" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -26,21 +28,46 @@ const ( signedBlocksThreshold = 0.025 consecutiveMissedBlocksPenalty = 1.1 - shardMinNodes = 6 - shardConsensusSize = 3 - metaMinNodes = 6 - metaConsensusSize = 6 - roundDurationMiliseconds = 6000 + shardMinNodes = 6 + shardConsensusSize = 3 + metaMinNodes = 6 + metaConsensusSize = 6 + roundDurationMilliseconds = 6000 ) -func createDymmyRatingsData() RatingsDataArg { +func createDummyRatingsData() RatingsDataArg { return RatingsDataArg{ - Config: config.RatingsConfig{}, - ShardConsensusSize: shardConsensusSize, - MetaConsensusSize: metaConsensusSize, - ShardMinNodes: shardMinNodes, - MetaMinNodes: metaMinNodes, - RoundDurationMiliseconds: roundDurationMiliseconds, + Config: config.RatingsConfig{}, + ChainParametersHolder: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + RoundDuration: 4000, + Hysteresis: 0.2, + EnableEpoch: 0, + ShardConsensusGroupSize: shardConsensusSize, + ShardMinNumNodes: shardMinNodes, + MetachainConsensusGroupSize: metaConsensusSize, + MetachainMinNumNodes: metaMinNodes, + Adaptivity: false, + } + }, + AllChainParametersCalled: func() []config.ChainParametersByEpochConfig { + return []config.ChainParametersByEpochConfig{ + { + RoundDuration: 4000, + Hysteresis: 0.2, + EnableEpoch: 0, + ShardConsensusGroupSize: shardConsensusSize, + ShardMinNumNodes: shardMinNodes, + MetachainConsensusGroupSize: metaConsensusSize, + MetachainMinNumNodes: metaMinNodes, + Adaptivity: false, + }, + } + }, + }, + RoundDurationMilliseconds: roundDurationMilliseconds, + EpochNotifier: &epochNotifier.EpochNotifierStub{}, } } @@ -79,10 +106,74 @@ func createDummyRatingsConfig() config.RatingsConfig { } } +func TestNewRatingsData_NilEpochNotifier(t *testing.T) { + t.Parallel() + + ratingsDataArg := createDummyRatingsData() + ratingsDataArg.EpochNotifier = nil + + ratingsData, err := NewRatingsData(ratingsDataArg) + + assert.Nil(t, ratingsData) + assert.True(t, errors.Is(err, process.ErrNilEpochNotifier)) +} + +func TestNewRatingsData_NilChainParametersHolder(t *testing.T) { + t.Parallel() + + ratingsDataArg := createDummyRatingsData() + ratingsDataArg.ChainParametersHolder = nil + + ratingsData, err := NewRatingsData(ratingsDataArg) + + assert.Nil(t, ratingsData) + assert.True(t, errors.Is(err, process.ErrNilChainParametersHandler)) +} + +func TestNewRatingsData_MissingConfigurationForEpoch0(t *testing.T) { + t.Parallel() + + ratingsDataArg := createDummyRatingsData() + ratingsDataArg.Config = createDummyRatingsConfig() + ratingsDataArg.ChainParametersHolder = &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + RoundDuration: 4000, + Hysteresis: 0.2, + EnableEpoch: 37, + ShardConsensusGroupSize: shardConsensusSize, + ShardMinNumNodes: shardMinNodes, + MetachainConsensusGroupSize: metaConsensusSize, + MetachainMinNumNodes: metaMinNodes, + Adaptivity: false, + } + }, + AllChainParametersCalled: func() []config.ChainParametersByEpochConfig { + return []config.ChainParametersByEpochConfig{ + { + RoundDuration: 4000, + Hysteresis: 0.2, + EnableEpoch: 37, + ShardConsensusGroupSize: shardConsensusSize, + ShardMinNumNodes: shardMinNodes, + MetachainConsensusGroupSize: metaConsensusSize, + MetachainMinNumNodes: metaMinNodes, + Adaptivity: false, + }, + } + }, + } + + ratingsData, err := NewRatingsData(ratingsDataArg) + + assert.Nil(t, ratingsData) + assert.True(t, errors.Is(err, process.ErrMissingConfigurationForEpochZero)) +} + func TestRatingsData_RatingsDataMinGreaterMaxShouldErr(t *testing.T) { t.Parallel() - ratingsDataArg := createDymmyRatingsData() + ratingsDataArg := createDummyRatingsData() ratingsConfig := createDummyRatingsConfig() ratingsConfig.General.MinRating = 10 ratingsConfig.General.MaxRating = 8 @@ -97,7 +188,7 @@ func TestRatingsData_RatingsDataMinGreaterMaxShouldErr(t *testing.T) { func TestRatingsData_RatingsDataMinSmallerThanOne(t *testing.T) { t.Parallel() - ratingsDataArg := createDymmyRatingsData() + ratingsDataArg := createDummyRatingsData() ratingsConfig := createDummyRatingsConfig() ratingsConfig.General.MinRating = 0 ratingsConfig.General.MaxRating = 8 @@ -111,7 +202,7 @@ func TestRatingsData_RatingsDataMinSmallerThanOne(t *testing.T) { func TestRatingsData_RatingsStartGreaterMaxShouldErr(t *testing.T) { t.Parallel() - ratingsDataArg := createDymmyRatingsData() + ratingsDataArg := createDummyRatingsData() ratingsConfig := createDummyRatingsConfig() ratingsConfig.General.MinRating = 10 ratingsConfig.General.MaxRating = 100 @@ -126,7 +217,7 @@ func TestRatingsData_RatingsStartGreaterMaxShouldErr(t *testing.T) { func TestRatingsData_RatingsStartLowerMinShouldErr(t *testing.T) { t.Parallel() - ratingsDataArg := createDymmyRatingsData() + ratingsDataArg := createDummyRatingsData() ratingsConfig := createDummyRatingsConfig() ratingsConfig.General.MinRating = 10 ratingsConfig.General.MaxRating = 100 @@ -141,7 +232,7 @@ func TestRatingsData_RatingsStartLowerMinShouldErr(t *testing.T) { func TestRatingsData_RatingsSignedBlocksThresholdNotBetweenZeroAndOneShouldErr(t *testing.T) { t.Parallel() - ratingsDataArg := createDymmyRatingsData() + ratingsDataArg := createDummyRatingsData() ratingsConfig := createDummyRatingsConfig() ratingsConfig.General.SignedBlocksThreshold = -0.1 ratingsDataArg.Config = ratingsConfig @@ -161,7 +252,7 @@ func TestRatingsData_RatingsSignedBlocksThresholdNotBetweenZeroAndOneShouldErr(t func TestRatingsData_RatingsConsecutiveMissedBlocksPenaltyLowerThanOneShouldErr(t *testing.T) { t.Parallel() - ratingsDataArg := createDymmyRatingsData() + ratingsDataArg := createDummyRatingsData() ratingsConfig := createDummyRatingsConfig() ratingsConfig.MetaChain.ConsecutiveMissedBlocksPenalty = 0.9 ratingsDataArg.Config = ratingsConfig @@ -184,7 +275,7 @@ func TestRatingsData_RatingsConsecutiveMissedBlocksPenaltyLowerThanOneShouldErr( func TestRatingsData_HoursToMaxRatingFromStartRatingZeroErr(t *testing.T) { t.Parallel() - ratingsDataArg := createDymmyRatingsData() + ratingsDataArg := createDummyRatingsData() ratingsConfig := createDummyRatingsConfig() ratingsConfig.MetaChain.HoursToMaxRatingFromStartRating = 0 ratingsDataArg.Config = ratingsConfig @@ -197,7 +288,7 @@ func TestRatingsData_HoursToMaxRatingFromStartRatingZeroErr(t *testing.T) { func TestRatingsData_PositiveDecreaseRatingsStepsShouldErr(t *testing.T) { t.Parallel() - ratingsDataArg := createDymmyRatingsData() + ratingsDataArg := createDummyRatingsData() ratingsConfig := createDummyRatingsConfig() ratingsConfig.MetaChain.ProposerDecreaseFactor = -0.5 ratingsDataArg.Config = ratingsConfig @@ -238,7 +329,7 @@ func TestRatingsData_PositiveDecreaseRatingsStepsShouldErr(t *testing.T) { func TestRatingsData_UnderflowErr(t *testing.T) { t.Parallel() - ratingsDataArg := createDymmyRatingsData() + ratingsDataArg := createDummyRatingsData() ratingsConfig := createDummyRatingsConfig() ratingsConfig.MetaChain.ProposerDecreaseFactor = math.MinInt32 ratingsDataArg.Config = ratingsConfig @@ -248,7 +339,7 @@ func TestRatingsData_UnderflowErr(t *testing.T) { require.True(t, errors.Is(err, process.ErrOverflow)) require.True(t, strings.Contains(err.Error(), "proposerDecrease")) - ratingsDataArg = createDymmyRatingsData() + ratingsDataArg = createDummyRatingsData() ratingsConfig = createDummyRatingsConfig() ratingsConfig.MetaChain.ValidatorDecreaseFactor = math.MinInt32 ratingsDataArg.Config = ratingsConfig @@ -258,7 +349,7 @@ func TestRatingsData_UnderflowErr(t *testing.T) { require.True(t, errors.Is(err, process.ErrOverflow)) require.True(t, strings.Contains(err.Error(), "validatorDecrease")) - ratingsDataArg = createDymmyRatingsData() + ratingsDataArg = createDummyRatingsData() ratingsConfig = createDummyRatingsConfig() ratingsConfig.ShardChain.ProposerDecreaseFactor = math.MinInt32 ratingsDataArg.Config = ratingsConfig @@ -268,7 +359,7 @@ func TestRatingsData_UnderflowErr(t *testing.T) { require.True(t, errors.Is(err, process.ErrOverflow)) require.True(t, strings.Contains(err.Error(), "proposerDecrease")) - ratingsDataArg = createDymmyRatingsData() + ratingsDataArg = createDummyRatingsData() ratingsConfig = createDummyRatingsConfig() ratingsConfig.ShardChain.ValidatorDecreaseFactor = math.MinInt32 ratingsDataArg.Config = ratingsConfig @@ -279,26 +370,109 @@ func TestRatingsData_UnderflowErr(t *testing.T) { require.True(t, strings.Contains(err.Error(), "validatorDecrease")) } +func TestRatingsData_EpochConfirmed(t *testing.T) { + t.Parallel() + + chainParams := make([]config.ChainParametersByEpochConfig, 0) + for i := uint32(0); i <= 10; i += 5 { + chainParams = append(chainParams, config.ChainParametersByEpochConfig{ + RoundDuration: 4000, + Hysteresis: 0.2, + EnableEpoch: i, + ShardConsensusGroupSize: shardConsensusSize, + ShardMinNumNodes: shardMinNodes, + MetachainConsensusGroupSize: metaConsensusSize, + MetachainMinNumNodes: metaMinNodes, + Adaptivity: false, + }) + } + chainParamsHandler := &chainParameters.ChainParametersHandlerStub{ + AllChainParametersCalled: func() []config.ChainParametersByEpochConfig { + return chainParams + }, + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return chainParams[0] + }, + } + ratingsDataArg := createDummyRatingsData() + ratingsDataArg.Config = createDummyRatingsConfig() + ratingsDataArg.ChainParametersHolder = chainParamsHandler + rd, err := NewRatingsData(ratingsDataArg) + require.NoError(t, err) + require.NotNil(t, rd) + + // ensure that the configs are stored in descending order + currentConfig := rd.ratingsStepsConfig[0] + for i := 1; i < len(rd.ratingsStepsConfig); i++ { + require.Less(t, rd.ratingsStepsConfig[i].enableEpoch, currentConfig.enableEpoch) + currentConfig = rd.ratingsStepsConfig[i] + } + + require.Equal(t, uint32(0), rd.currentRatingsStepData.enableEpoch) + + rd.EpochConfirmed(4, 0) + require.Equal(t, uint32(0), rd.currentRatingsStepData.enableEpoch) + + rd.EpochConfirmed(5, 0) + require.Equal(t, uint32(5), rd.currentRatingsStepData.enableEpoch) + + rd.EpochConfirmed(9, 0) + require.Equal(t, uint32(5), rd.currentRatingsStepData.enableEpoch) + + rd.EpochConfirmed(10, 0) + require.Equal(t, uint32(10), rd.currentRatingsStepData.enableEpoch) + + rd.EpochConfirmed(11, 0) + require.Equal(t, uint32(10), rd.currentRatingsStepData.enableEpoch) + + rd.EpochConfirmed(429, 0) + require.Equal(t, uint32(10), rd.currentRatingsStepData.enableEpoch) +} + func TestRatingsData_OverflowErr(t *testing.T) { t.Parallel() - ratingsDataArg := createDymmyRatingsData() + getBaseChainParams := func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + RoundDuration: 4000, + Hysteresis: 0.2, + EnableEpoch: 0, + ShardConsensusGroupSize: 5, + ShardMinNumNodes: 7, + MetachainConsensusGroupSize: 7, + MetachainMinNumNodes: 7, + Adaptivity: false, + } + } + getChainParametersHandler := func(cfg config.ChainParametersByEpochConfig) *chainParameters.ChainParametersHandlerStub { + return &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return cfg + }, + } + } + + ratingsDataArg := createDummyRatingsData() ratingsConfig := createDummyRatingsConfig() ratingsDataArg.Config = ratingsConfig - ratingsDataArg.RoundDurationMiliseconds = 3600 * 1000 - ratingsDataArg.MetaMinNodes = math.MaxUint32 + chainParams := getBaseChainParams() + chainParams.RoundDuration = 3600 * 1000 + chainParams.MetachainMinNumNodes = math.MaxUint32 + ratingsDataArg.ChainParametersHolder = getChainParametersHandler(chainParams) ratingsData, err := NewRatingsData(ratingsDataArg) require.Nil(t, ratingsData) require.True(t, errors.Is(err, process.ErrOverflow)) require.True(t, strings.Contains(err.Error(), "proposerIncrease")) - ratingsDataArg = createDymmyRatingsData() + ratingsDataArg = createDummyRatingsData() ratingsConfig = createDummyRatingsConfig() ratingsDataArg.Config = ratingsConfig - ratingsDataArg.RoundDurationMiliseconds = 3600 * 1000 - ratingsDataArg.MetaMinNodes = math.MaxUint32 - ratingsDataArg.MetaConsensusSize = 1 + chainParams = getBaseChainParams() + chainParams.RoundDuration = 3600 * 1000 + chainParams.MetachainMinNumNodes = math.MaxUint32 + chainParams.MetachainConsensusGroupSize = 1 + ratingsDataArg.ChainParametersHolder = getChainParametersHandler(chainParams) ratingsDataArg.Config.MetaChain.ProposerValidatorImportance = float32(1) / math.MaxUint32 ratingsData, err = NewRatingsData(ratingsDataArg) @@ -306,23 +480,27 @@ func TestRatingsData_OverflowErr(t *testing.T) { require.True(t, errors.Is(err, process.ErrOverflow)) require.True(t, strings.Contains(err.Error(), "validatorIncrease")) - ratingsDataArg = createDymmyRatingsData() + ratingsDataArg = createDummyRatingsData() ratingsConfig = createDummyRatingsConfig() ratingsDataArg.Config = ratingsConfig - ratingsDataArg.RoundDurationMiliseconds = 3600 * 1000 - ratingsDataArg.ShardMinNodes = math.MaxUint32 + chainParams = getBaseChainParams() + chainParams.RoundDuration = 3600 * 1000 + chainParams.ShardMinNumNodes = math.MaxUint32 + ratingsDataArg.ChainParametersHolder = getChainParametersHandler(chainParams) ratingsData, err = NewRatingsData(ratingsDataArg) require.Nil(t, ratingsData) require.True(t, errors.Is(err, process.ErrOverflow)) require.True(t, strings.Contains(err.Error(), "proposerIncrease")) - ratingsDataArg = createDymmyRatingsData() + ratingsDataArg = createDummyRatingsData() ratingsConfig = createDummyRatingsConfig() ratingsDataArg.Config = ratingsConfig - ratingsDataArg.RoundDurationMiliseconds = 3600 * 1000 - ratingsDataArg.ShardMinNodes = math.MaxUint32 - ratingsDataArg.ShardConsensusSize = 1 + chainParams = getBaseChainParams() + chainParams.RoundDuration = 3600 * 1000 + chainParams.ShardMinNumNodes = math.MaxUint32 + chainParams.ShardConsensusGroupSize = 1 + ratingsDataArg.ChainParametersHolder = getChainParametersHandler(chainParams) ratingsDataArg.Config.ShardChain.ProposerValidatorImportance = float32(1) / math.MaxUint32 ratingsData, err = NewRatingsData(ratingsDataArg) @@ -334,7 +512,7 @@ func TestRatingsData_OverflowErr(t *testing.T) { func TestRatingsData_IncreaseLowerThanZeroErr(t *testing.T) { t.Parallel() - ratingsDataArg := createDymmyRatingsData() + ratingsDataArg := createDummyRatingsData() ratingsConfig := createDummyRatingsConfig() ratingsDataArg.Config = ratingsConfig ratingsDataArg.Config.MetaChain.HoursToMaxRatingFromStartRating = math.MaxUint32 @@ -344,7 +522,7 @@ func TestRatingsData_IncreaseLowerThanZeroErr(t *testing.T) { require.True(t, errors.Is(err, process.ErrIncreaseStepLowerThanOne)) require.True(t, strings.Contains(err.Error(), "proposerIncrease")) - ratingsDataArg = createDymmyRatingsData() + ratingsDataArg = createDummyRatingsData() ratingsConfig = createDummyRatingsConfig() ratingsDataArg.Config = ratingsConfig ratingsDataArg.Config.MetaChain.HoursToMaxRatingFromStartRating = 2 @@ -359,7 +537,7 @@ func TestRatingsData_IncreaseLowerThanZeroErr(t *testing.T) { func TestRatingsData_RatingsCorrectValues(t *testing.T) { t.Parallel() - ratingsDataArg := createDymmyRatingsData() + ratingsDataArg := createDummyRatingsData() minRating := uint32(1) maxRating := uint32(10000) startRating := uint32(4000) diff --git a/process/throttle/antiflood/disabled/antiflood.go b/process/throttle/antiflood/disabled/antiflood.go index cdae45f21c1..99467e63c9a 100644 --- a/process/throttle/antiflood/disabled/antiflood.go +++ b/process/throttle/antiflood/disabled/antiflood.go @@ -47,8 +47,8 @@ func (af *AntiFlood) CanProcessMessagesOnTopic(_ core.PeerID, _ string, _ uint32 return nil } -// ApplyConsensusSize does nothing -func (af *AntiFlood) ApplyConsensusSize(_ int) { +// SetConsensusSizeNotifier does nothing +func (af *AntiFlood) SetConsensusSizeNotifier(_ process.ChainParametersSubscriber, _ uint32) { } // SetDebugger returns nil diff --git a/process/throttle/antiflood/disabled/antiflood_test.go b/process/throttle/antiflood/disabled/antiflood_test.go index e1118894cc4..a5908cc6f07 100644 --- a/process/throttle/antiflood/disabled/antiflood_test.go +++ b/process/throttle/antiflood/disabled/antiflood_test.go @@ -22,7 +22,7 @@ func TestAntiFlood_ShouldNotPanic(t *testing.T) { daf.SetMaxMessagesForTopic("test", 10) daf.ResetForTopic("test") - daf.ApplyConsensusSize(0) + daf.SetConsensusSizeNotifier(nil, 0) _ = daf.CanProcessMessagesOnTopic(core.PeerID(fmt.Sprint(1)), "test", 1, 0, nil) _ = daf.CanProcessMessage(nil, core.PeerID(fmt.Sprint(2))) } diff --git a/process/throttle/antiflood/export_test.go b/process/throttle/antiflood/export_test.go index bd97917572c..25fbf8bae30 100644 --- a/process/throttle/antiflood/export_test.go +++ b/process/throttle/antiflood/export_test.go @@ -3,5 +3,8 @@ package antiflood import "github.com/multiversx/mx-chain-go/process" func (af *p2pAntiflood) Debugger() process.AntifloodDebugger { + af.mutDebugger.RLock() + defer af.mutDebugger.RUnlock() + return af.debugger } diff --git a/process/throttle/antiflood/p2pAntiflood.go b/process/throttle/antiflood/p2pAntiflood.go index 621a0af69a8..747aca92c84 100644 --- a/process/throttle/antiflood/p2pAntiflood.go +++ b/process/throttle/antiflood/p2pAntiflood.go @@ -7,6 +7,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/throttle/antiflood/disabled" @@ -27,6 +28,8 @@ type p2pAntiflood struct { peerValidatorMapper process.PeerValidatorMapper mapTopicsFromAll map[string]struct{} mutTopicCheck sync.RWMutex + shardID uint32 + mutShardID sync.RWMutex } // NewP2PAntiflood creates a new p2p anti flood protection mechanism built on top of a flood preventer implementation. @@ -57,6 +60,31 @@ func NewP2PAntiflood( }, nil } +// SetConsensusSizeNotifier sets the consensus size notifier +func (af *p2pAntiflood) SetConsensusSizeNotifier(chainParametersNotifier process.ChainParametersSubscriber, shardID uint32) { + af.mutShardID.Lock() + af.shardID = shardID + af.mutShardID.Unlock() + + chainParametersNotifier.RegisterNotifyHandler(af) +} + +// ChainParametersChanged will be called when new chain parameters are confirmed on the network +func (af *p2pAntiflood) ChainParametersChanged(chainParameters config.ChainParametersByEpochConfig) { + af.mutShardID.RLock() + shardID := af.shardID + af.mutShardID.RUnlock() + + size := chainParameters.ShardConsensusGroupSize + if shardID == core.MetachainShardId { + size = chainParameters.MetachainConsensusGroupSize + } + + for _, fp := range af.floodPreventers { + fp.ApplyConsensusSize(int(size)) + } +} + // CanProcessMessage signals if a p2p message can be processed or not func (af *p2pAntiflood) CanProcessMessage(message p2p.MessageP2P, fromConnectedPeer core.PeerID) error { if message == nil { @@ -210,13 +238,6 @@ func (af *p2pAntiflood) ResetForTopic(topic string) { af.topicPreventer.ResetForTopic(topic) } -// ApplyConsensusSize applies the consensus size on all contained flood preventers -func (af *p2pAntiflood) ApplyConsensusSize(size int) { - for _, fp := range af.floodPreventers { - fp.ApplyConsensusSize(size) - } -} - // SetDebugger sets the antiflood debugger func (af *p2pAntiflood) SetDebugger(debugger process.AntifloodDebugger) error { if check.IfNil(debugger) { @@ -257,6 +278,9 @@ func (af *p2pAntiflood) BlacklistPeer(peer core.PeerID, reason string, duration // Close will call the close function on all sub components func (af *p2pAntiflood) Close() error { + af.mutDebugger.RLock() + defer af.mutDebugger.RUnlock() + return af.debugger.Close() } diff --git a/process/throttle/antiflood/p2pAntiflood_test.go b/process/throttle/antiflood/p2pAntiflood_test.go index 21ea5e99a8a..97637e2b621 100644 --- a/process/throttle/antiflood/p2pAntiflood_test.go +++ b/process/throttle/antiflood/p2pAntiflood_test.go @@ -2,17 +2,21 @@ package antiflood_test import ( "errors" + "sync" "sync/atomic" "testing" "time" "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/common/chainparametersnotifier" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/p2p" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/process/mock" "github.com/multiversx/mx-chain-go/process/throttle/antiflood" "github.com/multiversx/mx-chain-go/process/throttle/antiflood/disabled" + "github.com/multiversx/mx-chain-go/testscommon/commonmocks" "github.com/multiversx/mx-chain-go/testscommon/p2pmocks" "github.com/stretchr/testify/assert" ) @@ -307,24 +311,33 @@ func TestP2pAntiflood_ResetForTopicSetMaxMessagesShouldWork(t *testing.T) { assert.Equal(t, setMaxMessagesForTopicNum, setMaxMessagesForTopicParameter2) } -func TestP2pAntiflood_ApplyConsensusSize(t *testing.T) { +func TestP2pAntiflood_SetConsensusSizeNotifier(t *testing.T) { t.Parallel() wasCalled := false expectedSize := 878264 + testShardId := uint32(5) + var actualSize int afm, _ := antiflood.NewP2PAntiflood( &mock.PeerBlackListHandlerStub{}, &mock.TopicAntiFloodStub{}, &mock.FloodPreventerStub{ ApplyConsensusSizeCalled: func(size int) { - assert.Equal(t, expectedSize, size) + actualSize = size wasCalled = true }, }, ) - afm.ApplyConsensusSize(expectedSize) + chainParamsSubscriber := chainparametersnotifier.NewChainParametersNotifier() + afm.SetConsensusSizeNotifier(chainParamsSubscriber, testShardId) + + chainParamsSubscriber.UpdateCurrentChainParameters(config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(expectedSize), + }) + assert.True(t, wasCalled) + assert.Equal(t, expectedSize, actualSize) } func TestP2pAntiflood_SetDebuggerNilDebuggerShouldErr(t *testing.T) { @@ -464,3 +477,61 @@ func TestP2pAntiflood_IsOriginatorEligibleForTopic(t *testing.T) { err = afm.IsOriginatorEligibleForTopic(core.PeerID(validatorPID), "topic") assert.Nil(t, err) } + +func TestP2pAntiflood_ConcurrentOperations(t *testing.T) { + afm, _ := antiflood.NewP2PAntiflood( + &mock.PeerBlackListHandlerStub{}, + &mock.TopicAntiFloodStub{ + IncreaseLoadCalled: func(pid core.PeerID, topic string, numMessages uint32) error { + if topic == "should error" { + return errors.New("error") + } + + return nil + }, + }, + &mock.FloodPreventerStub{}, + ) + + numOperations := 500 + wg := sync.WaitGroup{} + wg.Add(numOperations) + for i := 0; i < numOperations; i++ { + go func(idx int) { + switch idx { + case 0: + afm.SetConsensusSizeNotifier(&commonmocks.ChainParametersNotifierStub{}, 1) + case 1: + afm.ChainParametersChanged(config.ChainParametersByEpochConfig{}) + case 2: + _ = afm.Close() + case 3: + _ = afm.CanProcessMessage(&p2pmocks.P2PMessageMock{}, "peer") + case 4: + afm.BlacklistPeer("peer", "reason", time.Millisecond) + case 5: + _ = afm.CanProcessMessagesOnTopic("peer", "topic", 37, 39, []byte("sequence")) + case 6: + _ = afm.IsOriginatorEligibleForTopic("peer", "topic") + case 7: + afm.ResetForTopic("topic") + case 8: + _ = afm.SetDebugger(&disabled.AntifloodDebugger{}) + case 9: + afm.SetMaxMessagesForTopic("topic", 37) + case 10: + afm.SetTopicsForAll("topic", "topic1") + case 11: + _ = afm.Debugger() + case 12: + _ = afm.SetPeerValidatorMapper(&mock.PeerShardResolverStub{}) + case 13: + _ = afm.CanProcessMessagesOnTopic("peer", "should error", 37, 39, []byte("sequence")) + } + + wg.Done() + }(i % 14) + } + + wg.Wait() +} diff --git a/sharding/chainParametersHolder.go b/sharding/chainParametersHolder.go new file mode 100644 index 00000000000..982d41679d7 --- /dev/null +++ b/sharding/chainParametersHolder.go @@ -0,0 +1,186 @@ +package sharding + +import ( + "fmt" + "sort" + "strings" + "sync" + + "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-core-go/data" + "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/config" +) + +type chainParametersHolder struct { + currentChainParameters config.ChainParametersByEpochConfig + chainParameters []config.ChainParametersByEpochConfig + chainParametersNotifier ChainParametersNotifierHandler + mutOperations sync.RWMutex +} + +// ArgsChainParametersHolder holds the arguments needed for creating a new chainParametersHolder +type ArgsChainParametersHolder struct { + EpochStartEventNotifier EpochStartEventNotifier + ChainParameters []config.ChainParametersByEpochConfig + ChainParametersNotifier ChainParametersNotifierHandler +} + +// NewChainParametersHolder returns a new instance of chainParametersHolder +func NewChainParametersHolder(args ArgsChainParametersHolder) (*chainParametersHolder, error) { + err := validateArgs(args) + if err != nil { + return nil, err + } + + chainParameters := args.ChainParameters + // sort the config values in descending order + sort.SliceStable(chainParameters, func(i, j int) bool { + return chainParameters[i].EnableEpoch > chainParameters[j].EnableEpoch + }) + + earliestChainParams := chainParameters[len(chainParameters)-1] + if earliestChainParams.EnableEpoch != 0 { + return nil, ErrMissingConfigurationForEpochZero + } + + paramsHolder := &chainParametersHolder{ + currentChainParameters: earliestChainParams, // will be updated on the epoch notifier handlers + chainParameters: args.ChainParameters, + chainParametersNotifier: args.ChainParametersNotifier, + } + args.ChainParametersNotifier.UpdateCurrentChainParameters(earliestChainParams) + args.EpochStartEventNotifier.RegisterHandler(paramsHolder) + + logInitialConfiguration(args.ChainParameters) + + return paramsHolder, nil +} + +func logInitialConfiguration(chainParameters []config.ChainParametersByEpochConfig) { + logMessage := "initialized chainParametersHolder with the values:\n" + logLines := make([]string, 0, len(chainParameters)) + for _, params := range chainParameters { + logLines = append(logLines, fmt.Sprintf("\tenable epoch=%d, round duration=%d, hysteresis=%.2f, shard consensus group size=%d, shard min nodes=%d, meta consensus group size=%d, meta min nodes=%d, adaptivity=%v", + params.EnableEpoch, params.RoundDuration, params.Hysteresis, params.ShardConsensusGroupSize, params.ShardMinNumNodes, params.MetachainConsensusGroupSize, params.MetachainMinNumNodes, params.Adaptivity)) + } + + logMessage += strings.Join(logLines, "\n") + log.Debug(logMessage) +} + +func validateArgs(args ArgsChainParametersHolder) error { + if check.IfNil(args.EpochStartEventNotifier) { + return ErrNilEpochStartEventNotifier + } + if len(args.ChainParameters) == 0 { + return ErrMissingChainParameters + } + if check.IfNil(args.ChainParametersNotifier) { + return ErrNilChainParametersNotifier + } + return validateChainParameters(args.ChainParameters) +} + +func validateChainParameters(chainParametersConfig []config.ChainParametersByEpochConfig) error { + for idx, chainParameters := range chainParametersConfig { + if chainParameters.ShardConsensusGroupSize < 1 { + return fmt.Errorf("%w for chain parameters with index %d", ErrNegativeOrZeroConsensusGroupSize, idx) + } + if chainParameters.ShardMinNumNodes < chainParameters.ShardConsensusGroupSize { + return fmt.Errorf("%w for chain parameters with index %d", ErrMinNodesPerShardSmallerThanConsensusSize, idx) + } + if chainParameters.MetachainConsensusGroupSize < 1 { + return fmt.Errorf("%w for chain parameters with index %d", ErrNegativeOrZeroConsensusGroupSize, idx) + } + if chainParameters.MetachainMinNumNodes < chainParameters.MetachainConsensusGroupSize { + return fmt.Errorf("%w for chain parameters with index %d", ErrMinNodesPerShardSmallerThanConsensusSize, idx) + } + } + + return nil +} + +// EpochStartAction is called when a new epoch is confirmed +func (c *chainParametersHolder) EpochStartAction(header data.HeaderHandler) { + c.handleEpochChange(header.GetEpoch()) +} + +// EpochStartPrepare is called when a new epoch is observed, but not yet confirmed. No action is required on this component +func (c *chainParametersHolder) EpochStartPrepare(_ data.HeaderHandler, _ data.BodyHandler) { +} + +// NotifyOrder returns the notification order for a start of epoch event +func (c *chainParametersHolder) NotifyOrder() uint32 { + return common.ChainParametersOrder +} + +func (c *chainParametersHolder) handleEpochChange(epoch uint32) { + c.mutOperations.Lock() + defer c.mutOperations.Unlock() + + matchingVersionForNewEpoch, err := getMatchingChainParametersUnprotected(epoch, c.chainParameters) + if err != nil { + log.Error("chainParametersHolder.EpochConfirmed: cannot get matching chain parameters", "epoch", epoch, "error", err) + return + } + if matchingVersionForNewEpoch.EnableEpoch == c.currentChainParameters.EnableEpoch { + return + } + + c.currentChainParameters = matchingVersionForNewEpoch + log.Debug("updated chainParametersHolder current chain parameters", + "round duration", matchingVersionForNewEpoch.RoundDuration, + "shard consensus group size", matchingVersionForNewEpoch.ShardConsensusGroupSize, + "shard min num nodes", matchingVersionForNewEpoch.ShardMinNumNodes, + "metachain consensus group size", matchingVersionForNewEpoch.MetachainConsensusGroupSize, + "metachain min num nodes", matchingVersionForNewEpoch.MetachainMinNumNodes, + "shard consensus group size", matchingVersionForNewEpoch.ShardConsensusGroupSize, + "hysteresis", matchingVersionForNewEpoch.Hysteresis, + "adaptivity", matchingVersionForNewEpoch.Adaptivity, + ) + c.chainParametersNotifier.UpdateCurrentChainParameters(matchingVersionForNewEpoch) +} + +// CurrentChainParameters will return the chain parameters that are active at the moment of calling +func (c *chainParametersHolder) CurrentChainParameters() config.ChainParametersByEpochConfig { + c.mutOperations.RLock() + defer c.mutOperations.RUnlock() + + return c.currentChainParameters +} + +// AllChainParameters will return the entire slice of chain parameters configuration +func (c *chainParametersHolder) AllChainParameters() []config.ChainParametersByEpochConfig { + c.mutOperations.RLock() + defer c.mutOperations.RUnlock() + + chainParametersCopy := make([]config.ChainParametersByEpochConfig, len(c.chainParameters)) + copy(chainParametersCopy, c.chainParameters) + + return chainParametersCopy +} + +// ChainParametersForEpoch will return the corresponding chain parameters for the provided epoch +func (c *chainParametersHolder) ChainParametersForEpoch(epoch uint32) (config.ChainParametersByEpochConfig, error) { + c.mutOperations.RLock() + defer c.mutOperations.RUnlock() + + return getMatchingChainParametersUnprotected(epoch, c.chainParameters) +} + +func getMatchingChainParametersUnprotected(epoch uint32, configValues []config.ChainParametersByEpochConfig) (config.ChainParametersByEpochConfig, error) { + for _, chainParams := range configValues { + if chainParams.EnableEpoch <= epoch { + return chainParams, nil + } + } + + // should never reach this code, as the config values are checked on the constructor + return config.ChainParametersByEpochConfig{}, ErrNoMatchingConfigurationFound +} + +// IsInterfaceNil returns true if there is no value under the interface +func (c *chainParametersHolder) IsInterfaceNil() bool { + return c == nil +} diff --git a/sharding/chainParametersHolder_test.go b/sharding/chainParametersHolder_test.go new file mode 100644 index 00000000000..f2a9b33e64a --- /dev/null +++ b/sharding/chainParametersHolder_test.go @@ -0,0 +1,384 @@ +package sharding + +import ( + "fmt" + "sync" + "testing" + + "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-core-go/data/block" + "github.com/multiversx/mx-chain-go/config" + "github.com/multiversx/mx-chain-go/testscommon/commonmocks" + "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" + "github.com/stretchr/testify/require" +) + +func TestNewChainParametersHolder(t *testing.T) { + t.Parallel() + + getDummyArgs := func() ArgsChainParametersHolder { + return ArgsChainParametersHolder{ + EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + ChainParameters: []config.ChainParametersByEpochConfig{ + { + EnableEpoch: 0, + ShardMinNumNodes: 5, + ShardConsensusGroupSize: 5, + MetachainMinNumNodes: 7, + MetachainConsensusGroupSize: 7, + RoundDuration: 4000, + Hysteresis: 0.2, + Adaptivity: false, + }, + }, + ChainParametersNotifier: &commonmocks.ChainParametersNotifierStub{}, + } + } + + t.Run("nil epoch start event notifier", func(t *testing.T) { + t.Parallel() + + args := getDummyArgs() + args.EpochStartEventNotifier = nil + + paramsHolder, err := NewChainParametersHolder(args) + require.True(t, check.IfNil(paramsHolder)) + require.Equal(t, ErrNilEpochStartEventNotifier, err) + }) + + t.Run("empty chain parameters", func(t *testing.T) { + t.Parallel() + + args := getDummyArgs() + args.ChainParameters = nil + + paramsHolder, err := NewChainParametersHolder(args) + require.True(t, check.IfNil(paramsHolder)) + require.Equal(t, ErrMissingChainParameters, err) + }) + + t.Run("invalid shard consensus size", func(t *testing.T) { + t.Parallel() + + args := getDummyArgs() + args.ChainParameters[0].ShardConsensusGroupSize = 0 + + paramsHolder, err := NewChainParametersHolder(args) + require.True(t, check.IfNil(paramsHolder)) + require.ErrorIs(t, err, ErrNegativeOrZeroConsensusGroupSize) + }) + + t.Run("min nodes per shard smaller than consensus size", func(t *testing.T) { + t.Parallel() + + args := getDummyArgs() + args.ChainParameters[0].ShardConsensusGroupSize = 5 + args.ChainParameters[0].ShardMinNumNodes = 4 + + paramsHolder, err := NewChainParametersHolder(args) + require.True(t, check.IfNil(paramsHolder)) + require.ErrorIs(t, err, ErrMinNodesPerShardSmallerThanConsensusSize) + }) + + t.Run("invalid metachain consensus size", func(t *testing.T) { + t.Parallel() + + args := getDummyArgs() + args.ChainParameters[0].MetachainConsensusGroupSize = 0 + + paramsHolder, err := NewChainParametersHolder(args) + require.True(t, check.IfNil(paramsHolder)) + require.ErrorIs(t, err, ErrNegativeOrZeroConsensusGroupSize) + }) + + t.Run("min nodes meta smaller than consensus size", func(t *testing.T) { + t.Parallel() + + args := getDummyArgs() + args.ChainParameters[0].MetachainConsensusGroupSize = 5 + args.ChainParameters[0].MetachainMinNumNodes = 4 + + paramsHolder, err := NewChainParametersHolder(args) + require.True(t, check.IfNil(paramsHolder)) + require.ErrorIs(t, err, ErrMinNodesPerShardSmallerThanConsensusSize) + }) + + t.Run("invalid future chain parameters", func(t *testing.T) { + t.Parallel() + + args := getDummyArgs() + newChainParameters := args.ChainParameters[0] + newChainParameters.ShardConsensusGroupSize = 0 + args.ChainParameters = append(args.ChainParameters, newChainParameters) + + paramsHolder, err := NewChainParametersHolder(args) + require.True(t, check.IfNil(paramsHolder)) + require.ErrorIs(t, err, ErrNegativeOrZeroConsensusGroupSize) + require.Contains(t, err.Error(), "index 1") + }) + + t.Run("no config for epoch 0", func(t *testing.T) { + t.Parallel() + + args := getDummyArgs() + args.ChainParameters[0].EnableEpoch = 37 + paramsHolder, err := NewChainParametersHolder(args) + require.True(t, check.IfNil(paramsHolder)) + require.ErrorIs(t, err, ErrMissingConfigurationForEpochZero) + }) + + t.Run("should work and have the data ready", func(t *testing.T) { + t.Parallel() + + args := getDummyArgs() + secondChainParams := args.ChainParameters[0] + secondChainParams.EnableEpoch = 5 + thirdChainParams := args.ChainParameters[0] + thirdChainParams.EnableEpoch = 10 + args.ChainParameters = append(args.ChainParameters, secondChainParams, thirdChainParams) + + paramsHolder, err := NewChainParametersHolder(args) + require.NoError(t, err) + require.False(t, check.IfNil(paramsHolder)) + + currentValue := paramsHolder.chainParameters[0] + for i := 1; i < len(paramsHolder.chainParameters); i++ { + require.Less(t, paramsHolder.chainParameters[i].EnableEpoch, currentValue.EnableEpoch) + currentValue = paramsHolder.chainParameters[i] + } + + require.Equal(t, uint32(0), paramsHolder.currentChainParameters.EnableEpoch) + }) +} + +func TestChainParametersHolder_EpochStartActionShouldCallTheNotifier(t *testing.T) { + t.Parallel() + + receivedValues := make([]uint32, 0) + notifier := &commonmocks.ChainParametersNotifierStub{ + UpdateCurrentChainParametersCalled: func(params config.ChainParametersByEpochConfig) { + receivedValues = append(receivedValues, params.ShardConsensusGroupSize) + }, + } + paramsHolder, _ := NewChainParametersHolder(ArgsChainParametersHolder{ + ChainParameters: []config.ChainParametersByEpochConfig{ + { + EnableEpoch: 0, + ShardConsensusGroupSize: 5, + ShardMinNumNodes: 7, + MetachainConsensusGroupSize: 7, + MetachainMinNumNodes: 7, + }, + { + EnableEpoch: 5, + ShardConsensusGroupSize: 37, + ShardMinNumNodes: 38, + MetachainConsensusGroupSize: 7, + MetachainMinNumNodes: 7, + }, + }, + EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + ChainParametersNotifier: notifier, + }) + + paramsHolder.EpochStartAction(&block.MetaBlock{Epoch: 5}) + require.Equal(t, []uint32{5, 37}, receivedValues) +} + +func TestChainParametersHolder_ChainParametersForEpoch(t *testing.T) { + t.Parallel() + + t.Run("single configuration, should return it every time", func(t *testing.T) { + t.Parallel() + + params := []config.ChainParametersByEpochConfig{ + { + EnableEpoch: 0, + ShardConsensusGroupSize: 5, + ShardMinNumNodes: 7, + MetachainConsensusGroupSize: 7, + MetachainMinNumNodes: 7, + }, + } + + paramsHolder, _ := NewChainParametersHolder(ArgsChainParametersHolder{ + ChainParameters: params, + EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + ChainParametersNotifier: &commonmocks.ChainParametersNotifierStub{}, + }) + + res, _ := paramsHolder.ChainParametersForEpoch(0) + require.Equal(t, uint32(5), res.ShardConsensusGroupSize) + require.Equal(t, uint32(7), res.MetachainConsensusGroupSize) + + res, _ = paramsHolder.ChainParametersForEpoch(1) + require.Equal(t, uint32(5), res.ShardConsensusGroupSize) + require.Equal(t, uint32(7), res.MetachainConsensusGroupSize) + + res, _ = paramsHolder.ChainParametersForEpoch(3700) + require.Equal(t, uint32(5), res.ShardConsensusGroupSize) + require.Equal(t, uint32(7), res.MetachainConsensusGroupSize) + }) + + t.Run("multiple configurations, should return the corresponding one", func(t *testing.T) { + t.Parallel() + + params := []config.ChainParametersByEpochConfig{ + { + EnableEpoch: 0, + ShardConsensusGroupSize: 5, + ShardMinNumNodes: 7, + MetachainConsensusGroupSize: 7, + MetachainMinNumNodes: 7, + }, + { + EnableEpoch: 10, + ShardConsensusGroupSize: 50, + ShardMinNumNodes: 70, + MetachainConsensusGroupSize: 70, + MetachainMinNumNodes: 70, + }, + { + EnableEpoch: 100, + ShardConsensusGroupSize: 500, + ShardMinNumNodes: 700, + MetachainConsensusGroupSize: 700, + MetachainMinNumNodes: 700, + }, + } + + paramsHolder, _ := NewChainParametersHolder(ArgsChainParametersHolder{ + ChainParameters: params, + EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + ChainParametersNotifier: &commonmocks.ChainParametersNotifierStub{}, + }) + + for i := 0; i < 200; i++ { + res, _ := paramsHolder.ChainParametersForEpoch(uint32(i)) + if i < 10 { + require.Equal(t, uint32(5), res.ShardConsensusGroupSize) + require.Equal(t, uint32(7), res.MetachainConsensusGroupSize) + } else if i < 100 { + require.Equal(t, uint32(50), res.ShardConsensusGroupSize) + require.Equal(t, uint32(70), res.MetachainConsensusGroupSize) + } else { + require.Equal(t, uint32(500), res.ShardConsensusGroupSize) + require.Equal(t, uint32(700), res.MetachainConsensusGroupSize) + } + } + }) +} + +func TestChainParametersHolder_CurrentChainParameters(t *testing.T) { + t.Parallel() + + params := []config.ChainParametersByEpochConfig{ + { + EnableEpoch: 0, + ShardConsensusGroupSize: 5, + ShardMinNumNodes: 7, + MetachainConsensusGroupSize: 7, + MetachainMinNumNodes: 7, + }, + { + EnableEpoch: 10, + ShardConsensusGroupSize: 50, + ShardMinNumNodes: 70, + MetachainConsensusGroupSize: 70, + MetachainMinNumNodes: 70, + }, + } + + paramsHolder, _ := NewChainParametersHolder(ArgsChainParametersHolder{ + ChainParameters: params, + EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + ChainParametersNotifier: &commonmocks.ChainParametersNotifierStub{}, + }) + + paramsHolder.EpochStartAction(&block.MetaBlock{Epoch: 0}) + require.Equal(t, uint32(5), paramsHolder.CurrentChainParameters().ShardConsensusGroupSize) + + paramsHolder.EpochStartAction(&block.MetaBlock{Epoch: 3}) + require.Equal(t, uint32(5), paramsHolder.CurrentChainParameters().ShardConsensusGroupSize) + + paramsHolder.EpochStartAction(&block.MetaBlock{Epoch: 10}) + require.Equal(t, uint32(50), paramsHolder.CurrentChainParameters().ShardConsensusGroupSize) + + paramsHolder.EpochStartAction(&block.MetaBlock{Epoch: 999}) + require.Equal(t, uint32(50), paramsHolder.CurrentChainParameters().ShardConsensusGroupSize) +} + +func TestChainParametersHolder_AllChainParameters(t *testing.T) { + t.Parallel() + + params := []config.ChainParametersByEpochConfig{ + { + EnableEpoch: 0, + ShardConsensusGroupSize: 5, + ShardMinNumNodes: 7, + MetachainConsensusGroupSize: 7, + MetachainMinNumNodes: 7, + }, + { + EnableEpoch: 10, + ShardConsensusGroupSize: 50, + ShardMinNumNodes: 70, + MetachainConsensusGroupSize: 70, + MetachainMinNumNodes: 70, + }, + } + + paramsHolder, _ := NewChainParametersHolder(ArgsChainParametersHolder{ + ChainParameters: params, + EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + ChainParametersNotifier: &commonmocks.ChainParametersNotifierStub{}, + }) + + returnedAllChainsParameters := paramsHolder.AllChainParameters() + require.Equal(t, params, returnedAllChainsParameters) + require.NotEqual(t, fmt.Sprintf("%p", returnedAllChainsParameters), fmt.Sprintf("%p", paramsHolder.chainParameters)) +} + +func TestChainParametersHolder_ConcurrentOperations(t *testing.T) { + chainParams := make([]config.ChainParametersByEpochConfig, 0) + for i := uint32(0); i <= 100; i += 5 { + chainParams = append(chainParams, config.ChainParametersByEpochConfig{ + RoundDuration: 4000, + Hysteresis: 0.2, + EnableEpoch: i, + ShardConsensusGroupSize: i*10 + 1, + ShardMinNumNodes: i*10 + 1, + MetachainConsensusGroupSize: i*10 + 1, + MetachainMinNumNodes: i*10 + 1, + Adaptivity: false, + }) + } + + paramsHolder, _ := NewChainParametersHolder(ArgsChainParametersHolder{ + ChainParameters: chainParams, + EpochStartEventNotifier: &epochstartmock.EpochStartNotifierStub{}, + ChainParametersNotifier: &commonmocks.ChainParametersNotifierStub{}, + }) + + numOperations := 500 + wg := sync.WaitGroup{} + wg.Add(numOperations) + for i := 0; i < numOperations; i++ { + go func(idx int) { + switch idx { + case 0: + paramsHolder.EpochStartAction(&block.MetaBlock{Epoch: uint32(idx)}) + case 1: + _ = paramsHolder.CurrentChainParameters() + case 2: + _, _ = paramsHolder.ChainParametersForEpoch(uint32(idx)) + case 3: + _ = paramsHolder.AllChainParameters() + } + + wg.Done() + }(i % 4) + } + + wg.Wait() +} diff --git a/sharding/dtos.go b/sharding/dtos.go new file mode 100644 index 00000000000..33cb7c8e660 --- /dev/null +++ b/sharding/dtos.go @@ -0,0 +1,18 @@ +package sharding + +// ConsensusConfiguration holds the consensus configuration that can be used by both the shard and the metachain +type ConsensusConfiguration struct { + EnableEpoch uint32 + MinNodes uint32 + ConsensusGroupSize uint32 +} + +// NodesSetupDTO is the data transfer object used to map the nodes' configuration in regard to the genesis nodes setup +type NodesSetupDTO struct { + StartTime int64 `json:"startTime"` + RoundDuration uint64 `json:"roundDuration"` + Hysteresis float32 `json:"hysteresis"` + Adaptivity bool `json:"adaptivity"` + + InitialNodes []*InitialNode `json:"initialNodes"` +} diff --git a/sharding/errors.go b/sharding/errors.go index 8190d8ba4ec..e6c2c29984c 100644 --- a/sharding/errors.go +++ b/sharding/errors.go @@ -39,3 +39,24 @@ var ErrNilOwnPublicKey = errors.New("nil own public key") // ErrNilEndOfProcessingHandler signals that a nil end of processing handler has been provided var ErrNilEndOfProcessingHandler = errors.New("nil end of processing handler") + +// ErrNilChainParametersProvider signals that a nil chain parameters provider has been given +var ErrNilChainParametersProvider = errors.New("nil chain parameters provider") + +// ErrNilEpochStartEventNotifier signals that a nil epoch start event notifier has been provided +var ErrNilEpochStartEventNotifier = errors.New("nil epoch start event notifier") + +// ErrMissingChainParameters signals that a nil chain parameters array has been provided +var ErrMissingChainParameters = errors.New("empty chain parameters array") + +// ErrMissingConfigurationForEpochZero signals that no configuration for epoch 0 exists +var ErrMissingConfigurationForEpochZero = errors.New("missing configuration for epoch 0") + +// ErrNoMatchingConfigurationFound signals that no matching configuration is found +var ErrNoMatchingConfigurationFound = errors.New("no matching configuration found") + +// ErrNilChainParametersNotifier signals that a nil chain parameters notifier has been provided +var ErrNilChainParametersNotifier = errors.New("nil chain parameters notifier") + +// ErrInvalidChainParametersForEpoch signals that an invalid chain parameters for epoch has been provided +var ErrInvalidChainParametersForEpoch = errors.New("invalid chain parameters for epoch") diff --git a/sharding/interface.go b/sharding/interface.go index 40180ec3bb5..06191510539 100644 --- a/sharding/interface.go +++ b/sharding/interface.go @@ -1,6 +1,10 @@ package sharding -import "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" +import ( + "github.com/multiversx/mx-chain-go/config" + "github.com/multiversx/mx-chain-go/epochStart" + "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" +) // Coordinator defines what a shard state coordinator should hold type Coordinator interface { @@ -61,5 +65,26 @@ type GenesisNodesSetupHandler interface { MinNumberOfNodesWithHysteresis() uint32 MinShardHysteresisNodes() uint32 MinMetaHysteresisNodes() uint32 + ExportNodesConfig() config.NodesConfig + IsInterfaceNil() bool +} + +// EpochStartEventNotifier provides Register and Unregister functionality for the end of epoch events +type EpochStartEventNotifier interface { + RegisterHandler(handler epochStart.ActionHandler) + UnregisterHandler(handler epochStart.ActionHandler) + IsInterfaceNil() bool +} + +// ChainParametersHandler defines the actions that need to be done by a component that can handle chain parameters +type ChainParametersHandler interface { + CurrentChainParameters() config.ChainParametersByEpochConfig + ChainParametersForEpoch(epoch uint32) (config.ChainParametersByEpochConfig, error) + IsInterfaceNil() bool +} + +// ChainParametersNotifierHandler defines the actions that need to be done by a component that can handle chain parameters changes +type ChainParametersNotifierHandler interface { + UpdateCurrentChainParameters(params config.ChainParametersByEpochConfig) IsInterfaceNil() bool } diff --git a/sharding/mock/epochHandlerMock.go b/sharding/mock/epochHandlerMock.go deleted file mode 100644 index 9b78066bd3e..00000000000 --- a/sharding/mock/epochHandlerMock.go +++ /dev/null @@ -1,16 +0,0 @@ -package mock - -// EpochHandlerMock - -type EpochHandlerMock struct { - EpochValue uint32 -} - -// Epoch - -func (ehm *EpochHandlerMock) Epoch() uint32 { - return ehm.EpochValue -} - -// IsInterfaceNil - -func (ehm *EpochHandlerMock) IsInterfaceNil() bool { - return ehm == nil -} diff --git a/sharding/mock/epochHandlerStub.go b/sharding/mock/epochHandlerStub.go deleted file mode 100644 index 4470eaca56c..00000000000 --- a/sharding/mock/epochHandlerStub.go +++ /dev/null @@ -1,20 +0,0 @@ -package mock - -// EpochHandlerStub - -type EpochHandlerStub struct { - EpochCalled func() uint32 -} - -// Epoch - -func (ehs *EpochHandlerStub) Epoch() uint32 { - if ehs.EpochCalled != nil { - return ehs.EpochCalled() - } - - return uint32(0) -} - -// IsInterfaceNil - -func (ehs *EpochHandlerStub) IsInterfaceNil() bool { - return ehs == nil -} diff --git a/sharding/mock/hasherStub.go b/sharding/mock/hasherStub.go deleted file mode 100644 index f05c2fd2cc8..00000000000 --- a/sharding/mock/hasherStub.go +++ /dev/null @@ -1,28 +0,0 @@ -package mock - -// HasherStub - -type HasherStub struct { - ComputeCalled func(s string) []byte - EmptyHashCalled func() []byte - SizeCalled func() int -} - -// Compute will output the SHA's equivalent of the input string -func (hs *HasherStub) Compute(s string) []byte { - return hs.ComputeCalled(s) -} - -// EmptyHash will return the equivalent of empty string SHA's -func (hs *HasherStub) EmptyHash() []byte { - return hs.EmptyHashCalled() -} - -// Size returns the required size in bytes -func (hs *HasherStub) Size() int { - return hs.SizeCalled() -} - -// IsInterfaceNil returns true if there is no value under the interface -func (hs *HasherStub) IsInterfaceNil() bool { - return hs == nil -} diff --git a/sharding/mock/listIndexUpdaterStub.go b/sharding/mock/listIndexUpdaterStub.go deleted file mode 100644 index 31c5ae19b76..00000000000 --- a/sharding/mock/listIndexUpdaterStub.go +++ /dev/null @@ -1,20 +0,0 @@ -package mock - -// ListIndexUpdaterStub - -type ListIndexUpdaterStub struct { - UpdateListAndIndexCalled func(pubKey string, shardID uint32, list string, index uint32) error -} - -// UpdateListAndIndex - -func (lius *ListIndexUpdaterStub) UpdateListAndIndex(pubKey string, shardID uint32, list string, index uint32) error { - if lius.UpdateListAndIndexCalled != nil { - return lius.UpdateListAndIndexCalled(pubKey, shardID, list, index) - } - - return nil -} - -// IsInterfaceNil returns true if there is no value under the interface -func (lius *ListIndexUpdaterStub) IsInterfaceNil() bool { - return lius == nil -} diff --git a/sharding/mock/multipleShardsCoordinatorFake.go b/sharding/mock/multipleShardsCoordinatorFake.go deleted file mode 100644 index 89a145beca2..00000000000 --- a/sharding/mock/multipleShardsCoordinatorFake.go +++ /dev/null @@ -1,94 +0,0 @@ -package mock - -import ( - "fmt" - "math" -) - -type multipleShardsCoordinatorFake struct { - numOfShards uint32 - CurrentShard uint32 - maskHigh uint32 - maskLow uint32 -} - -// NewMultipleShardsCoordinatorFake - -func NewMultipleShardsCoordinatorFake(numOfShards uint32, currentShard uint32) *multipleShardsCoordinatorFake { - mscf := &multipleShardsCoordinatorFake{ - numOfShards: numOfShards, - CurrentShard: currentShard, - } - mscf.maskHigh, mscf.maskLow = mscf.calculateMasks() - return mscf -} - -func (mscf *multipleShardsCoordinatorFake) calculateMasks() (uint32, uint32) { - n := math.Ceil(math.Log2(float64(mscf.numOfShards))) - return (1 << uint(n)) - 1, (1 << uint(n-1)) - 1 -} - -// NumberOfShards - -func (mscf *multipleShardsCoordinatorFake) NumberOfShards() uint32 { - return mscf.numOfShards -} - -// ComputeId - -func (mscf *multipleShardsCoordinatorFake) ComputeId(address []byte) uint32 { - bytesNeed := int(mscf.numOfShards/256) + 1 - startingIndex := 0 - if len(address) > bytesNeed { - startingIndex = len(address) - bytesNeed - } - - buffNeeded := address[startingIndex:] - - addr := uint32(0) - for i := 0; i < len(buffNeeded); i++ { - addr = addr<<8 + uint32(buffNeeded[i]) - } - - shard := addr & mscf.maskHigh - if shard > mscf.numOfShards-1 { - shard = addr & mscf.maskLow - } - return shard -} - -// SelfId - -func (mscf *multipleShardsCoordinatorFake) SelfId() uint32 { - return mscf.CurrentShard -} - -// SetSelfId - -func (mscf *multipleShardsCoordinatorFake) SetSelfId(_ uint32) error { - return nil -} - -// SameShard - -func (mscf *multipleShardsCoordinatorFake) SameShard(_, _ []byte) bool { - return true -} - -// SetNoShards - -func (mscf *multipleShardsCoordinatorFake) SetNoShards(numOfShards uint32) { - mscf.numOfShards = numOfShards -} - -// CommunicationIdentifier returns the identifier between current shard ID and destination shard ID -// identifier is generated such as the first shard from identifier is always smaller than the last -func (mscf *multipleShardsCoordinatorFake) CommunicationIdentifier(destShardID uint32) string { - if destShardID == mscf.CurrentShard { - return fmt.Sprintf("_%d", mscf.CurrentShard) - } - - if destShardID < mscf.CurrentShard { - return fmt.Sprintf("_%d_%d", destShardID, mscf.CurrentShard) - } - - return fmt.Sprintf("_%d_%d", mscf.CurrentShard, destShardID) -} - -// IsInterfaceNil returns true if there is no value under the interface -func (mscf *multipleShardsCoordinatorFake) IsInterfaceNil() bool { - return mscf == nil -} diff --git a/sharding/mock/pubkeyConverterMock.go b/sharding/mock/pubkeyConverterMock.go index e81d21ff4f6..2679da82d02 100644 --- a/sharding/mock/pubkeyConverterMock.go +++ b/sharding/mock/pubkeyConverterMock.go @@ -8,7 +8,8 @@ import ( // PubkeyConverterMock - type PubkeyConverterMock struct { - len int + len int + DecodeCalled func() ([]byte, error) } // NewPubkeyConverterMock - @@ -20,6 +21,9 @@ func NewPubkeyConverterMock(addressLen int) *PubkeyConverterMock { // Decode - func (pcm *PubkeyConverterMock) Decode(humanReadable string) ([]byte, error) { + if pcm.DecodeCalled != nil { + return pcm.DecodeCalled() + } return hex.DecodeString(humanReadable) } diff --git a/sharding/mock/testdata/invalidNodesSetupMock.json b/sharding/mock/testdata/invalidNodesSetupMock.json deleted file mode 100644 index 67458949a71..00000000000 --- a/sharding/mock/testdata/invalidNodesSetupMock.json +++ /dev/null @@ -1,19 +0,0 @@ -{ - "startTime": 0, - "roundDuration": 4000, - "consensusGroupSize": 0, - "minNodesPerShard": 1, - "metaChainActive" : true, - "metaChainConsensusGroupSize" : 1, - "metaChainMinNodes" : 1, - "initialNodes": [ - { - "pubkey": "41378f754e2c7b2745208c3ed21b151d297acdc84c3aca00b9e292cf28ec2d444771070157ea7760ed83c26f4fed387d0077e00b563a95825dac2cbc349fc0025ccf774e37b0a98ad9724d30e90f8c29b4091ccb738ed9ffc0573df776ee9ea30b3c038b55e532760ea4a8f152f2a52848020e5cee1cc537f2c2323399723081", - "address": "9e95a4e46da335a96845b4316251fc1bb197e1b8136d96ecc62bf6604eca9e49" - }, - { - "pubkey": "52f3bf5c01771f601ec2137e267319ab6716ef6ff5dfddaea48b42d955f631167f2ce19296a202bb8fd174f4e94f8c85f619df85a7f9f8de0f3768e5e6d8c48187b767deccf9829be246aa331aa86d182eb8fa28ea8a3e45d357ed1647a9be020a5569d686253a6f89e9123c7f21f302e82f67d3e3cd69cf267b9910a663ef32", - "address": "7a330039e77ca06bc127319fd707cc4911a80db489a39fcfb746283a05f61836" - } - ] -} diff --git a/sharding/mock/testdata/nodesSetupMock.json b/sharding/mock/testdata/nodesSetupMock.json deleted file mode 100644 index 17cf384c5b4..00000000000 --- a/sharding/mock/testdata/nodesSetupMock.json +++ /dev/null @@ -1,31 +0,0 @@ -{ - "startTime": 0, - "roundDuration": 4000, - "consensusGroupSize": 1, - "minNodesPerShard": 1, - "metaChainActive" : true, - "metaChainConsensusGroupSize" : 1, - "metaChainMinNodes" : 1, - "initialNodes": [ - { - "pubkey": "41378f754e2c7b2745208c3ed21b151d297acdc84c3aca00b9e292cf28ec2d444771070157ea7760ed83c26f4fed387d0077e00b563a95825dac2cbc349fc0025ccf774e37b0a98ad9724d30e90f8c29b4091ccb738ed9ffc0573df776ee9ea30b3c038b55e532760ea4a8f152f2a52848020e5cee1cc537f2c2323399723081", - "address": "9e95a4e46da335a96845b4316251fc1bb197e1b8136d96ecc62bf6604eca9e49" - }, - { - "pubkey": "52f3bf5c01771f601ec2137e267319ab6716ef6ff5dfddaea48b42d955f631167f2ce19296a202bb8fd174f4e94f8c85f619df85a7f9f8de0f3768e5e6d8c48187b767deccf9829be246aa331aa86d182eb8fa28ea8a3e45d357ed1647a9be020a5569d686253a6f89e9123c7f21f302e82f67d3e3cd69cf267b9910a663ef32", - "address": "7a330039e77ca06bc127319fd707cc4911a80db489a39fcfb746283a05f61836" - }, - { - "pubkey": "5e91c426c5c8f5f805f86de1e0653e2ec33853772e583b88e9f0f201089d03d8570759c3c3ab610ce573493c33ba0adf954c8939dba5d5ef7f2be4e87145d8153fc5b4fb91cecb8d9b1f62e080743fbf69c8c3096bf07980bb82cb450ba9b902673373d5b671ea73620cc5bc4d36f7a0f5ca3684d4c8aa5c1b425ab2a8673140", - "address": "131e2e717f2d33bdf7850c12b03dfe41ea8a5e76fdd6d4f23aebe558603e746f" - }, - { - "pubkey": "73972bf46dca59fba211c58f11b530f8e9d6392c499655ce760abc6458fd9c6b54b9676ee4b95aa32f6c254c9aad2f63a6195cd65d837a4320d7b8e915ba3a7123c8f4983b201035573c0752bb54e9021eb383b40d302447b62ea7a3790c89c47f5ab81d183f414e87611a31ff635ad22e969495356d5bc44eec7917aaad4c5e", - "address": "4c9e66b605882c1099088f26659692f084e41dc0dedfaedf6a6409af21c02aac" - }, - { - "pubkey": "7391ccce066ab5674304b10220643bc64829afa626a165f1e7a6618e260fa68f8e79018ac5964f7a1b8dd419645049042e34ebe7f2772def71e6176ce9daf50a57c17ee2a7445b908fe47e8f978380fcc2654a19925bf73db2402b09dde515148081f8ca7c331fbedec689de1b7bfce6bf106e4433557c29752c12d0a009f47a", - "address": "90a66900634b206d20627fbaec432ebfbabeaf30b9e338af63191435e2e37022" - } - ] -} diff --git a/sharding/nodesCoordinator/common_test.go b/sharding/nodesCoordinator/common_test.go index 50be55fd1ae..b7902db0c7e 100644 --- a/sharding/nodesCoordinator/common_test.go +++ b/sharding/nodesCoordinator/common_test.go @@ -5,10 +5,13 @@ import ( "encoding/binary" "fmt" "math/big" + "strconv" "testing" - "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" ) func TestComputeStartIndexAndNumAppearancesForValidator(t *testing.T) { @@ -151,3 +154,105 @@ func getExpandedEligibleList(num int) []uint32 { func newValidatorMock(pubKey []byte, chances uint32, index uint32) *validator { return &validator{pubKey: pubKey, index: index, chances: chances} } + +func TestSerializableShardValidatorListToValidatorListShouldErrNilPubKey(t *testing.T) { + t.Parallel() + + listOfSerializableValidators := []*SerializableValidator{ + { + PubKey: nil, + Chances: 1, + Index: 1, + }, + } + + _, err := SerializableShardValidatorListToValidatorList(listOfSerializableValidators) + require.Equal(t, ErrNilPubKey, err) +} + +func TestSerializableShardValidatorListToValidatorListShouldWork(t *testing.T) { + t.Parallel() + + listOfSerializableValidators := []*SerializableValidator{ + { + PubKey: []byte("pubkey"), + Chances: 1, + Index: 1, + }, + } + + expectedListOfValidators := make([]Validator, 1) + v, _ := NewValidator(listOfSerializableValidators[0].PubKey, listOfSerializableValidators[0].Chances, listOfSerializableValidators[0].Index) + require.NotNil(t, v) + expectedListOfValidators[0] = v + + valReturned, err := SerializableShardValidatorListToValidatorList(listOfSerializableValidators) + + require.Nil(t, err) + require.Equal(t, expectedListOfValidators, valReturned) +} + +func TestSerializableValidatorsToValidatorsShouldWork(t *testing.T) { + t.Parallel() + + mapOfSerializableValidators := make(map[string][]*SerializableValidator, 1) + mapOfSerializableValidators["1"] = []*SerializableValidator{ + { + PubKey: []byte("pubkey"), + Chances: 1, + Index: 1, + }, + } + + expectedMapOfValidators := make(map[uint32][]Validator, 1) + + v, _ := NewValidator(mapOfSerializableValidators["1"][0].PubKey, mapOfSerializableValidators["1"][0].Chances, mapOfSerializableValidators["1"][0].Index) + expectedMapOfValidators[uint32(1)] = []Validator{v} + + require.NotNil(t, v) + + valReturned, err := SerializableValidatorsToValidators(mapOfSerializableValidators) + + require.Nil(t, err) + require.Equal(t, expectedMapOfValidators, valReturned) +} + +func TestSerializableValidatorsToValidatorsShouldErrNilPubKey(t *testing.T) { + t.Parallel() + + mapOfSerializableValidators := make(map[string][]*SerializableValidator, 1) + mapOfSerializableValidators["1"] = []*SerializableValidator{ + { + PubKey: nil, + Chances: 1, + Index: 1, + }, + } + + _, err := SerializableValidatorsToValidators(mapOfSerializableValidators) + + require.Equal(t, ErrNilPubKey, err) +} + +func TestSerializableValidatorsToValidatorsShouldErrEmptyString(t *testing.T) { + t.Parallel() + + mapOfSerializableValidators := make(map[string][]*SerializableValidator, 1) + mapOfSerializableValidators[""] = []*SerializableValidator{ + { + PubKey: []byte("pubkey"), + Chances: 1, + Index: 1, + }, + } + + expectedMapOfValidators := make(map[uint32][]Validator, 1) + + v, _ := NewValidator(mapOfSerializableValidators[""][0].PubKey, mapOfSerializableValidators[""][0].Chances, mapOfSerializableValidators[""][0].Index) + require.NotNil(t, v) + expectedMapOfValidators[uint32(1)] = []Validator{v} + + _, err := SerializableValidatorsToValidators(mapOfSerializableValidators) + + require.Equal(t, &strconv.NumError{Func: "ParseUint", Num: "", Err: strconv.ErrSyntax}, err) +} diff --git a/sharding/nodesCoordinator/dtos.go b/sharding/nodesCoordinator/dtos.go index 75c28194a6a..5bd82fd0432 100644 --- a/sharding/nodesCoordinator/dtos.go +++ b/sharding/nodesCoordinator/dtos.go @@ -1,7 +1,10 @@ package nodesCoordinator +import "github.com/multiversx/mx-chain-go/config" + // ArgsUpdateNodes holds the parameters required by the shuffler to generate a new nodes configuration type ArgsUpdateNodes struct { + ChainParameters config.ChainParametersByEpochConfig Eligible map[uint32][]Validator Waiting map[uint32][]Validator NewNodes []Validator diff --git a/sharding/nodesCoordinator/errors.go b/sharding/nodesCoordinator/errors.go index 3d063f4605e..19c1bda084f 100644 --- a/sharding/nodesCoordinator/errors.go +++ b/sharding/nodesCoordinator/errors.go @@ -40,9 +40,6 @@ var ErrNilPreviousEpochConfig = errors.New("nil previous epoch config") // ErrEpochNodesConfigDoesNotExist signals that the epoch nodes configuration is missing var ErrEpochNodesConfigDoesNotExist = errors.New("epoch nodes configuration does not exist") -// ErrInvalidConsensusGroupSize signals that the consensus size is invalid (e.g. value is negative) -var ErrInvalidConsensusGroupSize = errors.New("invalid consensus group size") - // ErrNilRandomness signals that a nil randomness source has been provided var ErrNilRandomness = errors.New("nil randomness source") @@ -123,3 +120,6 @@ var ErrReceivedAuctionValidatorsBeforeStakingV4 = errors.New("should not have re // ErrNilEpochNotifier signals that a nil EpochNotifier has been provided var ErrNilEpochNotifier = errors.New("nil epoch notifier provided") + +// ErrNilChainParametersHandler signals that a nil chain parameters handler has been provided +var ErrNilChainParametersHandler = errors.New("nil chain parameters handler") diff --git a/sharding/nodesCoordinator/hashValidatorShuffler.go b/sharding/nodesCoordinator/hashValidatorShuffler.go index 71d2b5351b3..f60bd12fadc 100644 --- a/sharding/nodesCoordinator/hashValidatorShuffler.go +++ b/sharding/nodesCoordinator/hashValidatorShuffler.go @@ -20,10 +20,6 @@ var _ NodesShuffler = (*randHashShuffler)(nil) // NodesShufflerArgs defines the arguments required to create a nodes shuffler type NodesShufflerArgs struct { - NodesShard uint32 - NodesMeta uint32 - Hysteresis float32 - Adaptivity bool ShuffleBetweenShards bool MaxNodesEnableConfig []config.MaxNodesChangeConfig EnableEpochsHandler common.EnableEpochsHandler @@ -65,16 +61,11 @@ type randHashShuffler struct { // when reinitialization of node in new shard is implemented shuffleBetweenShards bool - adaptivity bool - nodesShard uint32 - nodesMeta uint32 - shardHysteresis uint32 - metaHysteresis uint32 - activeNodesConfig config.MaxNodesChangeConfig - availableNodesConfigs []config.MaxNodesChangeConfig - mutShufflerParams sync.RWMutex - validatorDistributor ValidatorsDistributor - enableEpochsHandler common.EnableEpochsHandler + activeNodesConfig config.MaxNodesChangeConfig + availableNodesConfigs []config.MaxNodesChangeConfig + mutShufflerParams sync.RWMutex + validatorDistributor ValidatorsDistributor + enableEpochsHandler common.EnableEpochsHandler stakingV4Step2EnableEpoch uint32 flagStakingV4Step2 atomic.Flag stakingV4Step3EnableEpoch uint32 @@ -118,8 +109,6 @@ func NewHashValidatorsShuffler(args *NodesShufflerArgs) (*randHashShuffler, erro stakingV4Step3EnableEpoch: args.EnableEpochs.StakingV4Step3EnableEpoch, } - rxs.UpdateParams(args.NodesShard, args.NodesMeta, args.Hysteresis, args.Adaptivity) - if rxs.shuffleBetweenShards { rxs.validatorDistributor = &CrossShardValidatorDistributor{} } else { @@ -131,52 +120,34 @@ func NewHashValidatorsShuffler(args *NodesShufflerArgs) (*randHashShuffler, erro return rxs, nil } -// UpdateParams updates the shuffler parameters -// Should be called when new params are agreed through governance -func (rhs *randHashShuffler) UpdateParams( - nodesShard uint32, - nodesMeta uint32, - hysteresis float32, - adaptivity bool, -) { - // TODO: are there constraints we want to enforce? e.g min/max hysteresis - shardHysteresis := uint32(float32(nodesShard) * hysteresis) - metaHysteresis := uint32(float32(nodesMeta) * hysteresis) - - rhs.mutShufflerParams.Lock() - rhs.shardHysteresis = shardHysteresis - rhs.metaHysteresis = metaHysteresis - rhs.nodesShard = nodesShard - rhs.nodesMeta = nodesMeta - rhs.adaptivity = adaptivity - rhs.mutShufflerParams.Unlock() -} - // UpdateNodeLists shuffles the nodes and returns the lists with the new nodes configuration // The function needs to ensure that: -// 1. Old eligible nodes list will have up to shuffleOutThreshold percent nodes shuffled out from each shard -// 2. The leaving nodes are checked against the eligible nodes and waiting nodes and removed if present from the -// pools and leaving nodes list (if remaining nodes can still sustain the shard) -// 3. shuffledOutNodes = oldEligibleNodes + waitingListNodes - minNbNodesPerShard (for each shard) -// 4. Old waiting nodes list for each shard will be added to the remaining eligible nodes list -// 5. The new nodes are equally distributed among the existing shards into waiting lists -// 6. The shuffled out nodes are distributed among the existing shards into waiting lists. -// We may have three situations: -// a) In case (shuffled out nodes + new nodes) > (nbShards * perShardHysteresis + minNodesPerShard) then -// we need to prepare for a split event, so a higher percentage of nodes need to be directed to the shard -// that will be split. -// b) In case (shuffled out nodes + new nodes) < (nbShards * perShardHysteresis) then we can immediately -// execute the shard merge -// c) No change in the number of shards then nothing extra needs to be done +// 1. Old eligible nodes list will have up to shuffleOutThreshold percent nodes shuffled out from each shard +// 2. The leaving nodes are checked against the eligible nodes and waiting nodes and removed if present from the +// pools and leaving nodes list (if remaining nodes can still sustain the shard) +// 3. shuffledOutNodes = oldEligibleNodes + waitingListNodes - minNbNodesPerShard (for each shard) +// 4. Old waiting nodes list for each shard will be added to the remaining eligible nodes list +// 5. The new nodes are equally distributed among the existing shards into waiting lists +// 6. The shuffled out nodes are distributed among the existing shards into waiting lists. +// We may have three situations: +// a) In case (shuffled out nodes + new nodes) > (nbShards * perShardHysteresis + minNodesPerShard) then +// we need to prepare for a split event, so a higher percentage of nodes need to be directed to the shard +// that will be split. +// b) In case (shuffled out nodes + new nodes) < (nbShards * perShardHysteresis) then we can immediately +// execute the shard merge +// c) No change in the number of shards then nothing extra needs to be done func (rhs *randHashShuffler) UpdateNodeLists(args ArgsUpdateNodes) (*ResUpdateNodes, error) { - rhs.updateShufflerConfig(args.Epoch) + chainParameters := args.ChainParameters + + rhs.UpdateShufflerConfig(args.Epoch, chainParameters) eligibleAfterReshard := copyValidatorMap(args.Eligible) waitingAfterReshard := copyValidatorMap(args.Waiting) - args.AdditionalLeaving = removeDupplicates(args.UnStakeLeaving, args.AdditionalLeaving) + args.AdditionalLeaving = removeDuplicates(args.UnStakeLeaving, args.AdditionalLeaving) totalLeavingNum := len(args.AdditionalLeaving) + len(args.UnStakeLeaving) newNbShards := rhs.computeNewShards( + chainParameters, args.Eligible, args.Waiting, len(args.NewNodes), @@ -185,10 +156,10 @@ func (rhs *randHashShuffler) UpdateNodeLists(args ArgsUpdateNodes) (*ResUpdateNo ) rhs.mutShufflerParams.RLock() - canSplit := rhs.adaptivity && newNbShards > args.NbShards - canMerge := rhs.adaptivity && newNbShards < args.NbShards - nodesPerShard := rhs.nodesShard - nodesMeta := rhs.nodesMeta + canSplit := chainParameters.Adaptivity && newNbShards > args.NbShards + canMerge := chainParameters.Adaptivity && newNbShards < args.NbShards + nodesPerShard := chainParameters.ShardMinNumNodes + nodesMeta := chainParameters.MetachainMinNumNodes rhs.mutShufflerParams.RUnlock() if canSplit { @@ -219,7 +190,7 @@ func (rhs *randHashShuffler) UpdateNodeLists(args ArgsUpdateNodes) (*ResUpdateNo }) } -func removeDupplicates(unstake []Validator, additionalLeaving []Validator) []Validator { +func removeDuplicates(unstake []Validator, additionalLeaving []Validator) []Validator { additionalCopy := make([]Validator, 0, len(additionalLeaving)) additionalCopy = append(additionalCopy, additionalLeaving...) @@ -451,6 +422,7 @@ func removeLeavingNodesFromValidatorMaps( // computeNewShards determines the new number of shards based on the number of nodes in the network func (rhs *randHashShuffler) computeNewShards( + chainParameters config.ChainParametersByEpochConfig, eligible map[uint32][]Validator, waiting map[uint32][]Validator, numNewNodes int, @@ -468,10 +440,10 @@ func (rhs *randHashShuffler) computeNewShards( nodesNewEpoch := uint32(nbEligible + nbWaiting + numNewNodes - numLeavingNodes) rhs.mutShufflerParams.RLock() - maxNodesMeta := rhs.nodesMeta + rhs.metaHysteresis - maxNodesShard := rhs.nodesShard + rhs.shardHysteresis + maxNodesMeta := chainParameters.MetachainMinNumNodes + rhs.metaHysteresis(chainParameters) + maxNodesShard := chainParameters.ShardMinNumNodes + rhs.shardHysteresis(chainParameters) nodesForSplit := (nbShards+1)*maxNodesShard + maxNodesMeta - nodesForMerge := nbShards*rhs.nodesShard + rhs.nodesMeta + nodesForMerge := nbShards*chainParameters.ShardMinNumNodes + chainParameters.MetachainMinNumNodes rhs.mutShufflerParams.RUnlock() nbShardsNew := nbShards @@ -489,6 +461,14 @@ func (rhs *randHashShuffler) computeNewShards( return nbShardsNew } +func (rhs *randHashShuffler) metaHysteresis(chainParameters config.ChainParametersByEpochConfig) uint32 { + return uint32(chainParameters.Hysteresis * float32(chainParameters.MetachainMinNumNodes)) +} + +func (rhs *randHashShuffler) shardHysteresis(chainParameters config.ChainParametersByEpochConfig) uint32 { + return uint32(chainParameters.Hysteresis * float32(chainParameters.ShardMinNumNodes)) +} + // shuffleOutNodes shuffles the list of eligible validators in each shard and returns the map of shuffled out // validators func shuffleOutNodes( @@ -828,11 +808,11 @@ func sortKeys(nodes map[uint32][]Validator) []uint32 { return keys } -// updateShufflerConfig updates the shuffler config according to the current epoch. -func (rhs *randHashShuffler) updateShufflerConfig(epoch uint32) { +// UpdateShufflerConfig updates the shuffler config according to the current epoch. +func (rhs *randHashShuffler) UpdateShufflerConfig(epoch uint32, chainParameters config.ChainParametersByEpochConfig) { rhs.mutShufflerParams.Lock() defer rhs.mutShufflerParams.Unlock() - rhs.activeNodesConfig.NodesToShufflePerShard = rhs.nodesShard + rhs.activeNodesConfig.NodesToShufflePerShard = chainParameters.ShardMinNumNodes for _, maxNodesConfig := range rhs.availableNodesConfigs { if epoch >= maxNodesConfig.EpochEnable { rhs.activeNodesConfig = maxNodesConfig diff --git a/sharding/nodesCoordinator/hashValidatorShuffler_test.go b/sharding/nodesCoordinator/hashValidatorShuffler_test.go index 788ec3f9b59..089441bc287 100644 --- a/sharding/nodesCoordinator/hashValidatorShuffler_test.go +++ b/sharding/nodesCoordinator/hashValidatorShuffler_test.go @@ -16,6 +16,7 @@ import ( "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/sharding/mock" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -28,6 +29,52 @@ const ( waitingPerShard = 30 ) +type testChainParametersCreator struct { + numNodesShards uint32 + numNodesMeta uint32 + hysteresis float32 + adaptivity bool +} + +func (t testChainParametersCreator) build() ChainParametersHandler { + return &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + EnableEpoch: 0, + Hysteresis: t.hysteresis, + ShardMinNumNodes: t.numNodesShards, + MetachainMinNumNodes: t.numNodesMeta, + ShardConsensusGroupSize: t.numNodesShards, + MetachainConsensusGroupSize: t.numNodesMeta, + Adaptivity: t.adaptivity, + } + }, + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + EnableEpoch: 0, + Hysteresis: t.hysteresis, + ShardMinNumNodes: t.numNodesShards, + MetachainMinNumNodes: t.numNodesMeta, + ShardConsensusGroupSize: t.numNodesShards, + MetachainConsensusGroupSize: t.numNodesMeta, + Adaptivity: t.adaptivity, + }, nil + }, + } +} + +func getTestChainParameters() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + EnableEpoch: 0, + Hysteresis: hysteresis, + ShardConsensusGroupSize: eligiblePerShard, + ShardMinNumNodes: eligiblePerShard, + MetachainConsensusGroupSize: eligiblePerShard, + MetachainMinNumNodes: eligiblePerShard, + Adaptivity: false, + } +} + func generateRandomByteArray(size int) []byte { r := make([]byte, size) _, _ = rand.Read(r) @@ -188,10 +235,6 @@ func testShuffledOut( func createHashShufflerInter() (*randHashShuffler, error) { shufflerArgs := &NodesShufflerArgs{ - NodesShard: eligiblePerShard, - NodesMeta: eligiblePerShard, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: true, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, EnableEpochs: config.EnableEpochs{ @@ -207,10 +250,6 @@ func createHashShufflerInter() (*randHashShuffler, error) { func createHashShufflerIntraShards() (*randHashShuffler, error) { shufflerArgs := &NodesShufflerArgs{ - NodesShard: eligiblePerShard, - NodesMeta: eligiblePerShard, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, EnableEpochs: config.EnableEpochs{ StakingV4Step2EnableEpoch: 443, @@ -589,8 +628,8 @@ func Test_removeValidatorsFromListRemoveFromLastMaxGreater(t *testing.T) { func Test_removeValidatorsFromListRandomValidatorsMaxSmaller(t *testing.T) { t.Parallel() - nbValidatotrsToRemove := 10 - maxToRemove := nbValidatotrsToRemove - 3 + nbValidatorsToRemove := 10 + maxToRemove := nbValidatorsToRemove - 3 validators := generateValidatorList(30) validatorsCopy := make([]Validator, len(validators)) validatorsToRemove := make([]Validator, 0) @@ -599,7 +638,7 @@ func Test_removeValidatorsFromListRandomValidatorsMaxSmaller(t *testing.T) { sort.Sort(validatorList(validators)) - validatorsToRemove = append(validatorsToRemove, validators[:nbValidatotrsToRemove]...) + validatorsToRemove = append(validatorsToRemove, validators[:nbValidatorsToRemove]...) v, removed := removeValidatorsFromList(validators, validatorsToRemove, maxToRemove) testRemoveValidators(t, validatorsCopy, validatorsToRemove, v, removed, maxToRemove) @@ -608,8 +647,8 @@ func Test_removeValidatorsFromListRandomValidatorsMaxSmaller(t *testing.T) { func Test_removeValidatorsFromListRandomValidatorsMaxGreater(t *testing.T) { t.Parallel() - nbValidatotrsToRemove := 10 - maxToRemove := nbValidatotrsToRemove + 3 + nbValidatorsToRemove := 10 + maxToRemove := nbValidatorsToRemove + 3 validators := generateValidatorList(30) validatorsCopy := make([]Validator, len(validators)) validatorsToRemove := make([]Validator, 0) @@ -618,13 +657,13 @@ func Test_removeValidatorsFromListRandomValidatorsMaxGreater(t *testing.T) { sort.Sort(validatorList(validators)) - validatorsToRemove = append(validatorsToRemove, validators[:nbValidatotrsToRemove]...) + validatorsToRemove = append(validatorsToRemove, validators[:nbValidatorsToRemove]...) v, removed := removeValidatorsFromList(validators, validatorsToRemove, maxToRemove) testRemoveValidators(t, validatorsCopy, validatorsToRemove, v, removed, maxToRemove) } -func Test_removeDupplicates_NoDupplicates(t *testing.T) { +func Test_removeDuplicates_NoDuplicates(t *testing.T) { t.Parallel() firstList := generateValidatorList(30) @@ -636,13 +675,13 @@ func Test_removeDupplicates_NoDupplicates(t *testing.T) { secondListCopy := make([]Validator, len(secondList)) copy(secondListCopy, secondList) - secondListAfterRemove := removeDupplicates(firstList, secondList) + secondListAfterRemove := removeDuplicates(firstList, secondList) assert.Equal(t, firstListCopy, firstList) assert.Equal(t, secondListCopy, secondListAfterRemove) } -func Test_removeDupplicates_SomeDupplicates(t *testing.T) { +func Test_removeDuplicates_SomeDuplicates(t *testing.T) { t.Parallel() firstList := generateValidatorList(30) @@ -656,14 +695,14 @@ func Test_removeDupplicates_SomeDupplicates(t *testing.T) { secondListCopy := make([]Validator, len(secondList)) copy(secondListCopy, secondList) - secondListAfterRemove := removeDupplicates(firstList, secondList) + secondListAfterRemove := removeDuplicates(firstList, secondList) assert.Equal(t, firstListCopy, firstList) assert.Equal(t, len(secondListCopy)-len(validatorsFromFirstList), len(secondListAfterRemove)) assert.Equal(t, secondListCopy[:20], secondListAfterRemove) } -func Test_removeDupplicates_AllDupplicates(t *testing.T) { +func Test_removeDuplicates_AllDuplicates(t *testing.T) { t.Parallel() firstList := generateValidatorList(30) @@ -675,7 +714,7 @@ func Test_removeDupplicates_AllDupplicates(t *testing.T) { secondListCopy := make([]Validator, len(secondList)) copy(secondListCopy, secondList) - secondListAfterRemove := removeDupplicates(firstList, secondList) + secondListAfterRemove := removeDuplicates(firstList, secondList) assert.Equal(t, firstListCopy, firstList) assert.Equal(t, len(secondListCopy)-len(firstListCopy), len(secondListAfterRemove)) @@ -1078,10 +1117,6 @@ func TestNewHashValidatorsShuffler(t *testing.T) { t.Parallel() shufflerArgs := &NodesShufflerArgs{ - NodesShard: eligiblePerShard, - NodesMeta: eligiblePerShard, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -1098,10 +1133,12 @@ func TestRandHashShuffler_computeNewShardsNotChanging(t *testing.T) { shuffler, err := createHashShufflerInter() require.Nil(t, err) - eligible := generateValidatorMap(int(shuffler.nodesShard), currentNbShards) + testChainParams := getTestChainParameters() + + eligible := generateValidatorMap(int(getTestChainParameters().ShardMinNumNodes), currentNbShards) nbShards := currentNbShards + 1 // account for meta - maxNodesNoSplit := (nbShards + 1) * (shuffler.nodesShard + shuffler.shardHysteresis) - nbWaitingPerShard := int(maxNodesNoSplit/nbShards - shuffler.nodesShard) + maxNodesNoSplit := (nbShards + 1) * (testChainParams.ShardMinNumNodes + shuffler.shardHysteresis(testChainParams)) + nbWaitingPerShard := int(maxNodesNoSplit/nbShards - testChainParams.ShardMinNumNodes) waiting := generateValidatorMap(nbWaitingPerShard, currentNbShards) newNodes := generateValidatorList(0) leavingUnstake := generateValidatorList(0) @@ -1110,7 +1147,7 @@ func TestRandHashShuffler_computeNewShardsNotChanging(t *testing.T) { numNewNodes := len(newNodes) numLeaving := len(leavingUnstake) + len(leavingRating) - newNbShards := shuffler.computeNewShards(eligible, waiting, numNewNodes, numLeaving, currentNbShards) + newNbShards := shuffler.computeNewShards(testChainParams, eligible, waiting, numNewNodes, numLeaving, currentNbShards) assert.Equal(t, currentNbShards, newNbShards) } @@ -1121,10 +1158,11 @@ func TestRandHashShuffler_computeNewShardsWithSplit(t *testing.T) { shuffler, err := createHashShufflerInter() require.Nil(t, err) - eligible := generateValidatorMap(int(shuffler.nodesShard), currentNbShards) + testChainParams := getTestChainParameters() + eligible := generateValidatorMap(int(testChainParams.ShardMinNumNodes), currentNbShards) nbShards := currentNbShards + 1 // account for meta - maxNodesNoSplit := (nbShards + 1) * (shuffler.nodesShard + shuffler.shardHysteresis) - nbWaitingPerShard := int(maxNodesNoSplit/nbShards-shuffler.nodesShard) + 1 + maxNodesNoSplit := (nbShards + 1) * (testChainParams.ShardMinNumNodes + shuffler.shardHysteresis(testChainParams)) + nbWaitingPerShard := int(maxNodesNoSplit/nbShards-testChainParams.ShardMinNumNodes) + 1 waiting := generateValidatorMap(nbWaitingPerShard, currentNbShards) newNodes := generateValidatorList(0) leavingUnstake := generateValidatorList(0) @@ -1133,7 +1171,7 @@ func TestRandHashShuffler_computeNewShardsWithSplit(t *testing.T) { numNewNodes := len(newNodes) numLeaving := len(leavingUnstake) + len(leavingRating) - newNbShards := shuffler.computeNewShards(eligible, waiting, numNewNodes, numLeaving, currentNbShards) + newNbShards := shuffler.computeNewShards(testChainParams, eligible, waiting, numNewNodes, numLeaving, currentNbShards) assert.Equal(t, currentNbShards+1, newNbShards) } @@ -1144,7 +1182,7 @@ func TestRandHashShuffler_computeNewShardsWithMerge(t *testing.T) { shuffler, err := createHashShufflerInter() require.Nil(t, err) - eligible := generateValidatorMap(int(shuffler.nodesShard), currentNbShards) + eligible := generateValidatorMap(int(getTestChainParameters().ShardMinNumNodes), currentNbShards) nbWaitingPerShard := 0 waiting := generateValidatorMap(nbWaitingPerShard, currentNbShards) newNodes := generateValidatorList(0) @@ -1154,47 +1192,17 @@ func TestRandHashShuffler_computeNewShardsWithMerge(t *testing.T) { numNewNodes := len(newNodes) numLeaving := len(leavingUnstake) + len(leavingRating) - newNbShards := shuffler.computeNewShards(eligible, waiting, numNewNodes, numLeaving, currentNbShards) + newNbShards := shuffler.computeNewShards(getTestChainParameters(), eligible, waiting, numNewNodes, numLeaving, currentNbShards) assert.Equal(t, currentNbShards-1, newNbShards) } -func TestRandHashShuffler_UpdateParams(t *testing.T) { - t.Parallel() - - shuffler, err := createHashShufflerInter() - require.Nil(t, err) - - shuffler2 := &randHashShuffler{ - nodesShard: 200, - nodesMeta: 200, - shardHysteresis: 0, - metaHysteresis: 0, - adaptivity: true, - shuffleBetweenShards: true, - validatorDistributor: &CrossShardValidatorDistributor{}, - availableNodesConfigs: nil, - stakingV4Step2EnableEpoch: 443, - stakingV4Step3EnableEpoch: 444, - enableEpochsHandler: &mock.EnableEpochsHandlerMock{}, - } - - shuffler.UpdateParams( - shuffler2.nodesShard, - shuffler2.nodesMeta, - 0, - shuffler2.adaptivity, - ) - - assert.Equal(t, shuffler2, shuffler) -} - func TestRandHashShuffler_UpdateNodeListsNoReSharding(t *testing.T) { t.Parallel() shuffler, err := createHashShufflerInter() require.Nil(t, err) - eligiblePerShard := int(shuffler.nodesShard) + eligiblePerShard := int(getTestChainParameters().ShardMinNumNodes) waitingPerShard := 30 nbShards := uint32(3) randomness := generateRandomByteArray(32) @@ -1215,6 +1223,12 @@ func TestRandHashShuffler_UpdateNodeListsNoReSharding(t *testing.T) { Rand: randomness, NbShards: nbShards, } + args.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() resUpdateNodeList, err := shuffler.UpdateNodeLists(args) require.Nil(t, err) @@ -1234,10 +1248,6 @@ func TestRandHashShuffler_UpdateNodeListsWithUnstakeLeavingRemovesFromEligible(t eligibleMeta := 10 shufflerArgs := &NodesShufflerArgs{ - NodesShard: uint32(eligiblePerShard), - NodesMeta: uint32(eligibleMeta), - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -1256,6 +1266,13 @@ func TestRandHashShuffler_UpdateNodeListsWithUnstakeLeavingRemovesFromEligible(t args.Eligible[core.MetachainShardId][1], } + args.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligibleMeta), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() + resUpdateNodeList, err := shuffler.UpdateNodeLists(args) require.Nil(t, err) @@ -1291,10 +1308,6 @@ func testUpdateNodesAndCheckNumLeaving(t *testing.T, beforeFix bool) { numNodesToShuffle := 80 shufflerArgs := &NodesShufflerArgs{ - NodesShard: uint32(eligiblePerShard), - NodesMeta: uint32(eligibleMeta), - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: []config.MaxNodesChangeConfig{ { @@ -1315,6 +1328,13 @@ func testUpdateNodesAndCheckNumLeaving(t *testing.T, beforeFix bool) { args.UnStakeLeaving = append(args.UnStakeLeaving, args.Waiting[0][i]) } + args.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligibleMeta), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() + resUpdateNodeList, err := shuffler.UpdateNodeLists(args) require.Nil(t, err) @@ -1342,10 +1362,6 @@ func TestRandHashShuffler_UpdateNodeListsAndCheckWaitingList(t *testing.T) { numNodesToShuffle := 80 shufflerArgs := &NodesShufflerArgs{ - NodesShard: uint32(eligiblePerShard), - NodesMeta: uint32(eligibleMeta), - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: []config.MaxNodesChangeConfig{ { @@ -1372,6 +1388,13 @@ func TestRandHashShuffler_UpdateNodeListsAndCheckWaitingList(t *testing.T) { args.UnStakeLeaving = append(args.UnStakeLeaving, args.Waiting[0][i]) } + args.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligibleMeta), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() + resUpdateNodeList, err := shuffler.UpdateNodeLists(args) require.Nil(t, err) @@ -1400,10 +1423,6 @@ func TestRandHashShuffler_UpdateNodeListsWithUnstakeLeavingRemovesFromWaiting(t eligibleMeta := 10 shufflerArgs := &NodesShufflerArgs{ - NodesShard: uint32(eligiblePerShard), - NodesMeta: uint32(eligibleMeta), - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -1422,6 +1441,13 @@ func TestRandHashShuffler_UpdateNodeListsWithUnstakeLeavingRemovesFromWaiting(t args.Waiting[core.MetachainShardId][1], } + args.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligibleMeta), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() + resUpdateNodeList, err := shuffler.UpdateNodeLists(args) require.Nil(t, err) @@ -1443,10 +1469,6 @@ func TestRandHashShuffler_UpdateNodeListsWithNonExistentUnstakeLeavingDoesNotRem t.Parallel() shufflerArgs := &NodesShufflerArgs{ - NodesShard: 10, - NodesMeta: 10, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -1454,7 +1476,7 @@ func TestRandHashShuffler_UpdateNodeListsWithNonExistentUnstakeLeavingDoesNotRem shuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - eligiblePerShard := int(shuffler.nodesShard) + eligiblePerShard := 10 waitingPerShard := 2 nbShards := uint32(0) @@ -1469,6 +1491,12 @@ func TestRandHashShuffler_UpdateNodeListsWithNonExistentUnstakeLeavingDoesNotRem }, } + args.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(10), + numNodesMeta: uint32(10), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() resUpdateNodeList, err := shuffler.UpdateNodeLists(args) require.Nil(t, err) @@ -1495,10 +1523,6 @@ func TestRandHashShuffler_UpdateNodeListsWithRangeOnMaps(t *testing.T) { for _, shuffle := range shuffleBetweenShards { shufflerArgs := &NodesShufflerArgs{ - NodesShard: uint32(eligiblePerShard), - NodesMeta: uint32(eligiblePerShard), - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffle, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -1534,6 +1558,13 @@ func TestRandHashShuffler_UpdateNodeListsWithRangeOnMaps(t *testing.T) { args.UnStakeLeaving = leavingValidators + args.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() + resUpdateNodeListInitial, err := shuffler.UpdateNodeLists(args) require.Nil(t, err) @@ -1573,6 +1604,12 @@ func TestRandHashShuffler_UpdateNodeListsNoReShardingIntraShardShuffling(t *test Rand: randomness, NbShards: nbShards, } + args.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() resUpdateNodeList, err := shuffler.UpdateNodeLists(args) require.Nil(t, err) @@ -1937,6 +1974,12 @@ func TestRandHashShuffler_UpdateNodeLists_WithUnstakeLeaving(t *testing.T) { Rand: generateRandomByteArray(32), NbShards: nbShards, } + arg.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() result, err := shuffler.UpdateNodeLists(arg) require.Nil(t, err) @@ -1986,6 +2029,12 @@ func TestRandHashShuffler_UpdateNodeLists_WithUnstakeLeaving_EnoughRemaining(t * Rand: generateRandomByteArray(32), NbShards: nbShards, } + arg.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() result, err := shuffler.UpdateNodeLists(arg) assert.NotNil(t, result) @@ -2018,6 +2067,12 @@ func TestRandHashShuffler_UpdateNodeLists_WithUnstakeLeaving_NotEnoughRemaining( Rand: generateRandomByteArray(32), NbShards: nbShards, } + arg.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() _, err = shuffler.UpdateNodeLists(arg) assert.True(t, errors.Is(err, ErrSmallShardEligibleListSize)) @@ -2039,6 +2094,12 @@ func TestRandHashShuffler_UpdateNodeLists_WithUnstakeLeaving_NotEnoughRemaining( Rand: generateRandomByteArray(32), NbShards: uint32(len(eligibleMap)), } + arg.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() _, err = shuffler.UpdateNodeLists(arg) assert.True(t, errors.Is(err, ErrSmallShardEligibleListSize)) @@ -2086,6 +2147,12 @@ func TestRandHashShuffler_UpdateNodeLists_WithAdditionalLeaving(t *testing.T) { Rand: generateRandomByteArray(32), NbShards: nbShards, } + arg.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() result, err := shuffler.UpdateNodeLists(arg) require.Nil(t, err) @@ -2110,7 +2177,7 @@ func TestRandHashShuffler_UpdateNodeLists_WithAdditionalLeaving(t *testing.T) { } } -func TestRandHashShuffler_UpdateNodeLists_WithUnstakeAndAdditionalLeaving_NoDupplicates(t *testing.T) { +func TestRandHashShuffler_UpdateNodeLists_WithUnstakeAndAdditionalLeaving_NoDuplicates(t *testing.T) { t.Parallel() eligiblePerShard := 100 @@ -2162,6 +2229,12 @@ func TestRandHashShuffler_UpdateNodeLists_WithUnstakeAndAdditionalLeaving_NoDupp Rand: generateRandomByteArray(32), NbShards: nbShards, } + arg.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() result, err := shuffler.UpdateNodeLists(arg) require.Nil(t, err) @@ -2185,7 +2258,7 @@ func TestRandHashShuffler_UpdateNodeLists_WithUnstakeAndAdditionalLeaving_NoDupp ) } } -func TestRandHashShuffler_UpdateNodeLists_WithAdditionalLeaving_WithDupplicates(t *testing.T) { +func TestRandHashShuffler_UpdateNodeLists_WithAdditionalLeaving_WithDuplicates(t *testing.T) { t.Parallel() eligiblePerShard := 100 @@ -2244,6 +2317,12 @@ func TestRandHashShuffler_UpdateNodeLists_WithAdditionalLeaving_WithDupplicates( Rand: generateRandomByteArray(32), NbShards: nbShards, } + arg.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() result, err := shuffler.UpdateNodeLists(arg) require.Nil(t, err) @@ -2316,10 +2395,6 @@ func TestRandHashShuffler_UpdateNodeLists_All(t *testing.T) { unstakeLeavingList, additionalLeavingList := prepareListsFromMaps(unstakeLeaving, additionalLeaving) shufflerArgs := &NodesShufflerArgs{ - NodesShard: uint32(eligiblePerShard), - NodesMeta: uint32(eligiblePerShard), - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, EnableEpochs: config.EnableEpochs{ StakingV4Step2EnableEpoch: 443, @@ -2339,6 +2414,12 @@ func TestRandHashShuffler_UpdateNodeLists_All(t *testing.T) { Rand: generateRandomByteArray(32), NbShards: nbShards, } + arg.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() result, err := shuffler.UpdateNodeLists(arg) require.Nil(t, err) @@ -2423,10 +2504,6 @@ func TestRandHashShuffler_UpdateNodeLists_WithNewNodes_NoWaiting(t *testing.T) { } shufflerArgs := &NodesShufflerArgs{ - NodesShard: uint32(eligiblePerShard), - NodesMeta: uint32(eligiblePerShard), - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -2435,6 +2512,12 @@ func TestRandHashShuffler_UpdateNodeLists_WithNewNodes_NoWaiting(t *testing.T) { shuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) + args.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() resUpdateNodeList, err := shuffler.UpdateNodeLists(args) require.Nil(t, err) @@ -2485,10 +2568,6 @@ func TestRandHashShuffler_UpdateNodeLists_WithNewNodes_NilOrEmptyWaiting(t *test } shufflerArgs := &NodesShufflerArgs{ - NodesShard: uint32(eligiblePerShard), - NodesMeta: uint32(eligiblePerShard), - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -2497,6 +2576,13 @@ func TestRandHashShuffler_UpdateNodeLists_WithNewNodes_NilOrEmptyWaiting(t *test shuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) + args.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() + resUpdateNodeList, err := shuffler.UpdateNodeLists(args) require.Nil(t, err) require.Equal(t, int(nbShards+1), len(resUpdateNodeList.Waiting)) @@ -2511,6 +2597,13 @@ func TestRandHashShuffler_UpdateNodeLists_WithNewNodes_NilOrEmptyWaiting(t *test NbShards: nbShards, } + args.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() + resUpdateNodeList, err = shuffler.UpdateNodeLists(args) require.Nil(t, err) require.Equal(t, int(nbShards+1), len(resUpdateNodeList.Waiting)) @@ -2541,6 +2634,12 @@ func TestRandHashShuffler_UpdateNodeLists_WithNewNodes_WithWaiting(t *testing.T) Rand: randomness, NbShards: uint32(nbShards), } + args.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() shuffler, err := createHashShufflerIntraShards() require.Nil(t, err) @@ -2586,6 +2685,7 @@ func TestRandHashShuffler_UpdateNodeLists_WithStakingV4(t *testing.T) { Auction: auctionList, NbShards: nbShards, Epoch: stakingV4Epoch, + ChainParameters: getTestChainParameters(), } shuffler, _ := createHashShufflerIntraShards() @@ -2668,10 +2768,6 @@ func TestRandHashShuffler_UpdateNodeLists_WithNewNodes_WithWaiting_WithLeaving(t } shufflerArgs := &NodesShufflerArgs{ - NodesShard: uint32(numEligiblePerShard), - NodesMeta: uint32(numEligiblePerShard), - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, EnableEpochs: config.EnableEpochs{ StakingV4Step2EnableEpoch: 443, @@ -2682,6 +2778,12 @@ func TestRandHashShuffler_UpdateNodeLists_WithNewNodes_WithWaiting_WithLeaving(t shuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) + args.ChainParameters = testChainParametersCreator{ + numNodesShards: uint32(numEligiblePerShard), + numNodesMeta: uint32(numEligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() resUpdateNodeList, err := shuffler.UpdateNodeLists(args) require.Nil(t, err) @@ -2723,7 +2825,7 @@ func verifyResultsIntraShardShuffling( initialNumWaiting := len(waiting) numToRemove := initialNumWaiting - additionalLeaving = removeDupplicates(unstakeLeaving, additionalLeaving) + additionalLeaving = removeDuplicates(unstakeLeaving, additionalLeaving) computedNewWaiting, removedFromWaiting := removeValidatorsFromList(waiting, unstakeLeaving, numToRemove) removedNodes = append(removedNodes, removedFromWaiting...) @@ -2899,10 +3001,6 @@ func TestRandHashShuffler_sortConfigs(t *testing.T) { require.NotEqual(t, orderedConfigs, shuffledConfigs) shufflerArgs := &NodesShufflerArgs{ - NodesShard: eligiblePerShard, - NodesMeta: eligiblePerShard, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: shuffledConfigs, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -2918,10 +3016,6 @@ func TestRandHashShuffler_UpdateShufflerConfig(t *testing.T) { orderedConfigs := getDummyShufflerConfigs() shufflerArgs := &NodesShufflerArgs{ - NodesShard: eligiblePerShard, - NodesMeta: eligiblePerShard, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: orderedConfigs, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -2934,7 +3028,13 @@ func TestRandHashShuffler_UpdateShufflerConfig(t *testing.T) { if epoch == orderedConfigs[(i+1)%len(orderedConfigs)].EpochEnable { i++ } - shuffler.updateShufflerConfig(epoch) + chainParams := testChainParametersCreator{ + numNodesShards: uint32(eligiblePerShard), + numNodesMeta: uint32(eligiblePerShard), + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build().CurrentChainParameters() + shuffler.UpdateShufflerConfig(epoch, chainParams) require.Equal(t, orderedConfigs[i], shuffler.activeNodesConfig) } } diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator.go index 4898f018010..ca95be3a522 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator.go @@ -75,8 +75,7 @@ type epochNodesConfig struct { type indexHashedNodesCoordinator struct { shardIDAsObserver uint32 currentEpoch uint32 - shardConsensusGroupSize int - metaConsensusGroupSize int + chainParametersHandler ChainParametersHandler numTotalEligible uint64 selfPubKey []byte savedStateKey []byte @@ -140,8 +139,7 @@ func NewIndexHashedNodesCoordinator(arguments ArgNodesCoordinator) (*indexHashed nodesConfig: nodesConfig, currentEpoch: arguments.Epoch, savedStateKey: savedKey, - shardConsensusGroupSize: arguments.ShardConsensusGroupSize, - metaConsensusGroupSize: arguments.MetaConsensusGroupSize, + chainParametersHandler: arguments.ChainParametersHandler, consensusGroupCacher: arguments.ConsensusGroupCache, shardIDAsObserver: arguments.ShardIDAsObserver, shuffledOutHandler: arguments.ShuffledOutHandler, @@ -195,8 +193,8 @@ func NewIndexHashedNodesCoordinator(arguments ArgNodesCoordinator) (*indexHashed } func checkArguments(arguments ArgNodesCoordinator) error { - if arguments.ShardConsensusGroupSize < 1 || arguments.MetaConsensusGroupSize < 1 { - return ErrInvalidConsensusGroupSize + if check.IfNil(arguments.ChainParametersHandler) { + return ErrNilChainParametersHandler } if arguments.NbShards < 1 { return ErrInvalidNumberOfShards @@ -278,21 +276,25 @@ func (ihnc *indexHashedNodesCoordinator) setNodesPerShards( return ErrNilInputNodesMap } + currentChainParameters, err := ihnc.chainParametersHandler.ChainParametersForEpoch(epoch) + if err != nil { + return err + } + nodesList := eligible[core.MetachainShardId] - if len(nodesList) < ihnc.metaConsensusGroupSize { + if len(nodesList) < int(currentChainParameters.MetachainConsensusGroupSize) { return ErrSmallMetachainEligibleListSize } numTotalEligible := uint64(len(nodesList)) for shardId := uint32(0); shardId < uint32(len(eligible)-1); shardId++ { nbNodesShard := len(eligible[shardId]) - if nbNodesShard < ihnc.shardConsensusGroupSize { + if nbNodesShard < int(currentChainParameters.ShardConsensusGroupSize) { return ErrSmallShardEligibleListSize } numTotalEligible += uint64(nbNodesShard) } - var err error var isCurrentNodeValidator bool // nbShards holds number of shards without meta nodesConfig.nbShards = uint32(len(eligible) - 1) @@ -381,7 +383,7 @@ func (ihnc *indexHashedNodesCoordinator) ComputeConsensusGroup( return validators, nil } - consensusSize := ihnc.ConsensusGroupSize(shardID) + consensusSize := ihnc.ConsensusGroupSizeForShardAndEpoch(shardID, epoch) randomness = []byte(fmt.Sprintf("%d-%s", round, randomness)) log.Debug("computeValidatorsGroup", @@ -663,7 +665,14 @@ func (ihnc *indexHashedNodesCoordinator) EpochStartPrepare(metaHdr data.HeaderHa unStakeLeavingList := ihnc.createSortedListFromMap(newNodesConfig.leavingMap) additionalLeavingList := ihnc.createSortedListFromMap(additionalLeavingMap) + chainParamsForEpoch, err := ihnc.chainParametersHandler.ChainParametersForEpoch(newEpoch) + if err != nil { + log.Warn("indexHashedNodesCoordinator.EpochStartPrepare: could not compute chain params for epoch. "+ + "Will use the current chain parameters", "epoch", newEpoch, "error", err) + chainParamsForEpoch = ihnc.chainParametersHandler.CurrentChainParameters() + } shufflerArgs := ArgsUpdateNodes{ + ChainParameters: chainParamsForEpoch, Eligible: newNodesConfig.eligibleMap, Waiting: newNodesConfig.waitingMap, NewNodes: newNodesConfig.newList, @@ -1134,15 +1143,23 @@ func (ihnc *indexHashedNodesCoordinator) computeShardForSelfPublicKey(nodesConfi return selfShard, false } -// ConsensusGroupSize returns the consensus group size for a specific shard -func (ihnc *indexHashedNodesCoordinator) ConsensusGroupSize( +// ConsensusGroupSizeForShardAndEpoch returns the consensus group size for a specific shard in a given epoch +func (ihnc *indexHashedNodesCoordinator) ConsensusGroupSizeForShardAndEpoch( shardID uint32, + epoch uint32, ) int { + currentChainParameters, err := ihnc.chainParametersHandler.ChainParametersForEpoch(epoch) + if err != nil { + log.Warn("indexHashedNodesCoordinator.ConsensusGroupSizeForShardAndEpoch: could not compute chain params for epoch. "+ + "Will use the current chain parameters", "epoch", epoch, "error", err) + currentChainParameters = ihnc.chainParametersHandler.CurrentChainParameters() + } + if shardID == core.MetachainShardId { - return ihnc.metaConsensusGroupSize + return int(currentChainParameters.MetachainConsensusGroupSize) } - return ihnc.shardConsensusGroupSize + return int(currentChainParameters.ShardConsensusGroupSize) } // GetNumTotalEligible returns the number of total eligible accross all shards from current setup diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinatorLite.go b/sharding/nodesCoordinator/indexHashedNodesCoordinatorLite.go index b5b87781a73..03d697b6689 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinatorLite.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinatorLite.go @@ -19,7 +19,13 @@ func (ihnc *indexHashedNodesCoordinator) SetNodesConfigFromValidatorsInfo(epoch unStakeLeavingList := ihnc.createSortedListFromMap(newNodesConfig.leavingMap) additionalLeavingList := ihnc.createSortedListFromMap(additionalLeavingMap) + chainParameters, err := ihnc.chainParametersHandler.ChainParametersForEpoch(epoch) + if err != nil { + log.Warn("indexHashedNodesCoordinator.SetNodesConfigFromValidatorsInfo: could not compute chain params for epoch. "+ + "Will use the current chain parameters", "epoch", epoch, "error", err) + } shufflerArgs := ArgsUpdateNodes{ + ChainParameters: chainParameters, Eligible: newNodesConfig.eligibleMap, Waiting: newNodesConfig.waitingMap, NewNodes: newNodesConfig.newList, diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinatorLite_test.go b/sharding/nodesCoordinator/indexHashedNodesCoordinatorLite_test.go index e880d564ca2..b54fed7860d 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinatorLite_test.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinatorLite_test.go @@ -84,10 +84,13 @@ func TestIndexHashedNodesCoordinator_SetNodesConfigFromValidatorsInfo(t *testing t.Parallel() arguments := createArguments() - + arguments.ChainParametersHandler = testChainParametersCreator{ + numNodesShards: 3, + numNodesMeta: 3, + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build() shufflerArgs := &NodesShufflerArgs{ - NodesShard: 3, - NodesMeta: 3, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, } nodeShuffler, _ := NewHashValidatorsShuffler(shufflerArgs) @@ -108,10 +111,14 @@ func TestIndexHashedNodesCoordinator_SetNodesConfigFromValidatorsInfoMultipleEpo t.Parallel() arguments := createArguments() + arguments.ChainParametersHandler = testChainParametersCreator{ + numNodesShards: 3, + numNodesMeta: 3, + hysteresis: hysteresis, + adaptivity: adaptivity, + }.build() shufflerArgs := &NodesShufflerArgs{ - NodesShard: 3, - NodesMeta: 3, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, } nodeShuffler, _ := NewHashValidatorsShuffler(shufflerArgs) diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go b/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go index a80006cceae..2a879d125d2 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinatorWithRater_test.go @@ -17,8 +17,11 @@ import ( "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/sharding/mock" "github.com/multiversx/mx-chain-go/state" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" + "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" "github.com/multiversx/mx-chain-go/testscommon/genericMocks" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/nodeTypeProviderMock" @@ -65,10 +68,6 @@ func TestIndexHashedGroupSelectorWithRater_OkValShouldWork(t *testing.T) { eligibleMap := createDummyNodesMap(3, 1, "waiting") waitingMap := make(map[uint32][]Validator) shufflerArgs := &NodesShufflerArgs{ - NodesShard: 3, - NodesMeta: 3, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -76,12 +75,18 @@ func TestIndexHashedGroupSelectorWithRater_OkValShouldWork(t *testing.T) { nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 2, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 2, + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -165,22 +170,24 @@ func BenchmarkIndexHashedGroupSelectorWithRater_ComputeValidatorsGroup63of400(b eligibleMap[core.MetachainShardId] = listMeta shufflerArgs := &NodesShufflerArgs{ - NodesShard: 400, - NodesMeta: 1, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, } nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: consensusGroupSize, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(consensusGroupSize), + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -241,10 +248,6 @@ func Test_ComputeValidatorsGroup63of400(t *testing.T) { eligibleMap[0] = list eligibleMap[core.MetachainShardId] = listMeta shufflerArgs := &NodesShufflerArgs{ - NodesShard: shardSize, - NodesMeta: 1, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -252,12 +255,18 @@ func Test_ComputeValidatorsGroup63of400(t *testing.T) { nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: consensusGroupSize, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(consensusGroupSize), + MetachainConsensusGroupSize: 1, + } + }, + }, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, EpochStartNotifier: epochStartSubscriber, @@ -315,10 +324,6 @@ func TestIndexHashedGroupSelectorWithRater_GetValidatorWithPublicKeyShouldReturn eligibleMap[0] = list eligibleMap[core.MetachainShardId] = list sufflerArgs := &NodesShufflerArgs{ - NodesShard: 1, - NodesMeta: 1, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -326,12 +331,18 @@ func TestIndexHashedGroupSelectorWithRater_GetValidatorWithPublicKeyShouldReturn nodeShuffler, err := NewHashValidatorsShuffler(sufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 1, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -371,10 +382,6 @@ func TestIndexHashedGroupSelectorWithRater_GetValidatorWithPublicKeyShouldReturn eligibleMap[core.MetachainShardId] = list shufflerArgs := &NodesShufflerArgs{ - NodesShard: 1, - NodesMeta: 1, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -382,12 +389,18 @@ func TestIndexHashedGroupSelectorWithRater_GetValidatorWithPublicKeyShouldReturn nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 1, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -437,10 +450,6 @@ func TestIndexHashedGroupSelectorWithRater_GetValidatorWithPublicKeyShouldWork(t waitingMap := make(map[uint32][]Validator) shufflerArgs := &NodesShufflerArgs{ - NodesShard: 3, - NodesMeta: 3, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -448,7 +457,7 @@ func TestIndexHashedGroupSelectorWithRater_GetValidatorWithPublicKeyShouldWork(t nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap[core.MetachainShardId] = listMeta @@ -456,8 +465,14 @@ func TestIndexHashedGroupSelectorWithRater_GetValidatorWithPublicKeyShouldWork(t eligibleMap[1] = listShard1 arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 1, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -524,17 +539,13 @@ func TestIndexHashedGroupSelectorWithRater_GetAllEligibleValidatorsPublicKeys(t waitingMap := make(map[uint32][]Validator) shufflerArgs := &NodesShufflerArgs{ - NodesShard: 3, - NodesMeta: 3, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, } nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap[core.MetachainShardId] = listMeta @@ -542,8 +553,14 @@ func TestIndexHashedGroupSelectorWithRater_GetAllEligibleValidatorsPublicKeys(t eligibleMap[shardOneId] = listShard1 arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 1, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -836,10 +853,6 @@ func BenchmarkIndexHashedWithRaterGroupSelector_ComputeValidatorsGroup21of400(b eligibleMap[0] = list eligibleMap[core.MetachainShardId] = listMeta shufflerArgs := &NodesShufflerArgs{ - NodesShard: 400, - NodesMeta: 1, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -847,12 +860,18 @@ func BenchmarkIndexHashedWithRaterGroupSelector_ComputeValidatorsGroup21of400(b nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: consensusGroupSize, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(consensusGroupSize), + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, diff --git a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go index 32cc2ca8326..7f516e7cd6e 100644 --- a/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go +++ b/sharding/nodesCoordinator/indexHashedNodesCoordinator_test.go @@ -24,12 +24,15 @@ import ( "github.com/stretchr/testify/require" "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/dataRetriever/dataPool" "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/sharding/mock" "github.com/multiversx/mx-chain-go/state" "github.com/multiversx/mx-chain-go/storage/cache" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" "github.com/multiversx/mx-chain-go/testscommon/enableEpochsHandlerMock" + "github.com/multiversx/mx-chain-go/testscommon/epochstartmock" "github.com/multiversx/mx-chain-go/testscommon/genericMocks" "github.com/multiversx/mx-chain-go/testscommon/hashingMocks" "github.com/multiversx/mx-chain-go/testscommon/nodeTypeProviderMock" @@ -98,35 +101,45 @@ func createArguments() ArgNodesCoordinator { eligibleMap := createDummyNodesMap(10, nbShards, "eligible") waitingMap := createDummyNodesMap(3, nbShards, "waiting") shufflerArgs := &NodesShufflerArgs{ - NodesShard: 10, - NodesMeta: 10, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, } nodeShuffler, _ := NewHashValidatorsShuffler(shufflerArgs) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 1, - MetaConsensusGroupSize: 1, - Marshalizer: &mock.MarshalizerMock{}, - Hasher: &hashingMocks.HasherMock{}, - Shuffler: nodeShuffler, - EpochStartNotifier: epochStartSubscriber, - BootStorer: bootStorer, - NbShards: nbShards, - EligibleNodes: eligibleMap, - WaitingNodes: waitingMap, - SelfPublicKey: []byte("test"), - ConsensusGroupCache: &mock.NodesCoordinatorCacheMock{}, - ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, - IsFullArchive: false, - ChanStopNode: make(chan endProcess.ArgEndProcess), - NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + ShardMinNumNodes: 10, + MetachainConsensusGroupSize: 1, + MetachainMinNumNodes: 10, + }, nil + }, + }, + Marshalizer: &mock.MarshalizerMock{}, + Hasher: &hashingMocks.HasherMock{}, + Shuffler: nodeShuffler, + EpochStartNotifier: epochStartSubscriber, + BootStorer: bootStorer, + NbShards: nbShards, + EligibleNodes: eligibleMap, + WaitingNodes: waitingMap, + SelfPublicKey: []byte("test"), + ConsensusGroupCache: &mock.NodesCoordinatorCacheMock{}, + ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, + IsFullArchive: false, + ChanStopNode: make(chan endProcess.ArgEndProcess), + NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{ IsRefactorPeersMiniBlocksFlagEnabledField: true, }, @@ -159,14 +172,14 @@ func TestNewIndexHashedNodesCoordinator_NilHasherShouldErr(t *testing.T) { require.Nil(t, ihnc) } -func TestNewIndexHashedNodesCoordinator_InvalidConsensusGroupSizeShouldErr(t *testing.T) { +func TestNewIndexHashedNodesCoordinator_NilChainParametersHandleShouldErr(t *testing.T) { t.Parallel() arguments := createArguments() - arguments.ShardConsensusGroupSize = 0 + arguments.ChainParametersHandler = nil ihnc, err := NewIndexHashedNodesCoordinator(arguments) - require.Equal(t, ErrInvalidConsensusGroupSize, err) + require.Equal(t, ErrNilChainParametersHandler, err) require.Nil(t, ihnc) } @@ -277,10 +290,6 @@ func TestIndexHashedNodesCoordinator_OkValShouldWork(t *testing.T) { waitingMap := createDummyNodesMap(3, 3, "waiting") shufflerArgs := &NodesShufflerArgs{ - NodesShard: 10, - NodesMeta: 10, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -288,12 +297,18 @@ func TestIndexHashedNodesCoordinator_OkValShouldWork(t *testing.T) { nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 2, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 2, + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -322,27 +337,12 @@ func TestIndexHashedNodesCoordinator_OkValShouldWork(t *testing.T) { // ------- ComputeValidatorsGroup -func TestIndexHashedNodesCoordinator_NewCoordinatorGroup0SizeShouldErr(t *testing.T) { - t.Parallel() - - arguments := createArguments() - arguments.MetaConsensusGroupSize = 0 - ihnc, err := NewIndexHashedNodesCoordinator(arguments) - - require.Equal(t, ErrInvalidConsensusGroupSize, err) - require.Nil(t, ihnc) -} - func TestIndexHashedNodesCoordinator_NewCoordinatorTooFewNodesShouldErr(t *testing.T) { t.Parallel() eligibleMap := createDummyNodesMap(5, 3, "eligible") waitingMap := createDummyNodesMap(3, 3, "waiting") shufflerArgs := &NodesShufflerArgs{ - NodesShard: 10, - NodesMeta: 10, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -350,12 +350,24 @@ func TestIndexHashedNodesCoordinator_NewCoordinatorTooFewNodesShouldErr(t *testi nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 10, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 10, + MetachainConsensusGroupSize: 1, + } + }, + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 10, + MetachainConsensusGroupSize: 1, + }, nil + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -415,10 +427,6 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup1ValidatorShouldRetur nodesMap[0] = list nodesMap[core.MetachainShardId] = tmp[core.MetachainShardId] shufflerArgs := &NodesShufflerArgs{ - NodesShard: 10, - NodesMeta: 10, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -426,12 +434,24 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup1ValidatorShouldRetur nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 1, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + }, nil + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -463,10 +483,6 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup400of400For10locksNoM waitingMap := make(map[uint32][]Validator) eligibleMap := createDummyNodesMap(nodesPerShard, 1, "eligible") shufflerArgs := &NodesShufflerArgs{ - NodesShard: nodesPerShard, - NodesMeta: nodesPerShard, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -474,7 +490,7 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup400of400For10locksNoM nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() getCounter := int32(0) @@ -492,8 +508,20 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup400of400For10locksNoM } arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: consensusGroupSize, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(consensusGroupSize), + MetachainConsensusGroupSize: 1, + } + }, + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(consensusGroupSize), + MetachainConsensusGroupSize: 1, + }, nil + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -540,10 +568,6 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup400of400For10BlocksMe waitingMap := make(map[uint32][]Validator) eligibleMap := createDummyNodesMap(nodesPerShard, 1, "eligible") shufflerArgs := &NodesShufflerArgs{ - NodesShard: nodesPerShard, - NodesMeta: nodesPerShard, - Hysteresis: 0, - Adaptivity: false, ShuffleBetweenShards: false, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -551,7 +575,7 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup400of400For10BlocksMe nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() getCounter := 0 @@ -582,8 +606,20 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup400of400For10BlocksMe } arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: consensusGroupSize, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(consensusGroupSize), + MetachainConsensusGroupSize: 1, + } + }, + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(consensusGroupSize), + MetachainConsensusGroupSize: 1, + }, nil + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -641,10 +677,6 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup63of400TestEqualSameP eligibleMap := createDummyNodesMap(nodesPerShard, 1, "eligible") shufflerArgs := &NodesShufflerArgs{ - NodesShard: nodesPerShard, - NodesMeta: nodesPerShard, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -652,12 +684,18 @@ func TestIndexHashedNodesCoordinator_ComputeValidatorsGroup63of400TestEqualSameP nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: consensusGroupSize, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(consensusGroupSize), + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -705,10 +743,6 @@ func BenchmarkIndexHashedGroupSelector_ComputeValidatorsGroup21of400(b *testing. waitingMap := make(map[uint32][]Validator) eligibleMap := createDummyNodesMap(nodesPerShard, 1, "eligible") shufflerArgs := &NodesShufflerArgs{ - NodesShard: nodesPerShard, - NodesMeta: nodesPerShard, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -716,12 +750,18 @@ func BenchmarkIndexHashedGroupSelector_ComputeValidatorsGroup21of400(b *testing. nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: consensusGroupSize, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(consensusGroupSize), + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -779,10 +819,6 @@ func BenchmarkIndexHashedNodesCoordinator_CopyMaps(b *testing.B) { func runBenchmark(consensusGroupCache Cacher, consensusGroupSize int, nodesMap map[uint32][]Validator, b *testing.B) { waitingMap := make(map[uint32][]Validator) shufflerArgs := &NodesShufflerArgs{ - NodesShard: 10, - NodesMeta: 10, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -790,12 +826,18 @@ func runBenchmark(consensusGroupCache Cacher, consensusGroupSize int, nodesMap m nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: consensusGroupSize, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(consensusGroupSize), + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, EpochStartNotifier: epochStartSubscriber, @@ -830,10 +872,6 @@ func runBenchmark(consensusGroupCache Cacher, consensusGroupSize int, nodesMap m func computeMemoryRequirements(consensusGroupCache Cacher, consensusGroupSize int, nodesMap map[uint32][]Validator, b *testing.B) { waitingMap := make(map[uint32][]Validator) shufflerArgs := &NodesShufflerArgs{ - NodesShard: 10, - NodesMeta: 10, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -841,12 +879,18 @@ func computeMemoryRequirements(consensusGroupCache Cacher, consensusGroupSize in nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(b, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: consensusGroupSize, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(consensusGroupSize), + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, EpochStartNotifier: epochStartSubscriber, @@ -971,10 +1015,6 @@ func TestIndexHashedNodesCoordinator_GetValidatorWithPublicKeyShouldWork(t *test eligibleMap[0] = listShard0 eligibleMap[1] = listShard1 shufflerArgs := &NodesShufflerArgs{ - NodesShard: 10, - NodesMeta: 10, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -982,12 +1022,18 @@ func TestIndexHashedNodesCoordinator_GetValidatorWithPublicKeyShouldWork(t *test nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 1, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -1056,10 +1102,6 @@ func TestIndexHashedGroupSelector_GetAllEligibleValidatorsPublicKeys(t *testing. eligibleMap[shardZeroId] = listShard0 eligibleMap[shardOneId] = listShard1 shufflerArgs := &NodesShufflerArgs{ - NodesShard: 10, - NodesMeta: 10, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -1067,12 +1109,18 @@ func TestIndexHashedGroupSelector_GetAllEligibleValidatorsPublicKeys(t *testing. nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 1, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -1133,10 +1181,6 @@ func TestIndexHashedGroupSelector_GetAllWaitingValidatorsPublicKeys(t *testing.T waitingMap[shardOneId] = listShard1 shufflerArgs := &NodesShufflerArgs{ - NodesShard: 10, - NodesMeta: 10, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -1144,7 +1188,7 @@ func TestIndexHashedGroupSelector_GetAllWaitingValidatorsPublicKeys(t *testing.T nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap := make(map[uint32][]Validator) @@ -1152,8 +1196,14 @@ func TestIndexHashedGroupSelector_GetAllWaitingValidatorsPublicKeys(t *testing.T eligibleMap[shardZeroId] = []Validator{&validator{}} arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 1, - MetaConsensusGroupSize: 1, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + }, Marshalizer: &mock.MarshalizerMock{}, Hasher: &hashingMocks.HasherMock{}, Shuffler: nodeShuffler, @@ -1529,10 +1579,6 @@ func TestIndexHashedNodesCoordinator_EpochStart_EligibleSortedAscendingByIndex(t eligibleMap[core.MetachainShardId] = list shufflerArgs := &NodesShufflerArgs{ - NodesShard: 2, - NodesMeta: 2, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -1540,25 +1586,39 @@ func TestIndexHashedNodesCoordinator_EpochStart_EligibleSortedAscendingByIndex(t nodeShuffler, err := NewHashValidatorsShuffler(shufflerArgs) require.Nil(t, err) - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 1, - MetaConsensusGroupSize: 1, - Marshalizer: &mock.MarshalizerMock{}, - Hasher: &hashingMocks.HasherMock{}, - Shuffler: nodeShuffler, - EpochStartNotifier: epochStartSubscriber, - BootStorer: bootStorer, - NbShards: nbShards, - EligibleNodes: eligibleMap, - WaitingNodes: map[uint32][]Validator{}, - SelfPublicKey: []byte("test"), - ConsensusGroupCache: &mock.NodesCoordinatorCacheMock{}, - ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, - ChanStopNode: make(chan endProcess.ArgEndProcess), - NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + ChainParametersForEpochCalled: func(_ uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + ShardMinNumNodes: 2, + MetachainMinNumNodes: 2, + }, nil + }, + }, + Marshalizer: &mock.MarshalizerMock{}, + Hasher: &hashingMocks.HasherMock{}, + Shuffler: nodeShuffler, + EpochStartNotifier: epochStartSubscriber, + BootStorer: bootStorer, + NbShards: nbShards, + EligibleNodes: eligibleMap, + WaitingNodes: map[uint32][]Validator{}, + SelfPublicKey: []byte("test"), + ConsensusGroupCache: &mock.NodesCoordinatorCacheMock{}, + ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, + ChanStopNode: make(chan endProcess.ArgEndProcess), + NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{ IsRefactorPeersMiniBlocksFlagEnabledField: true, }, @@ -1850,15 +1910,43 @@ func TestIndexHashedNodesCoordinator_GetConsensusWhitelistedNodesAfterRevertToEp func TestIndexHashedNodesCoordinator_ConsensusGroupSize(t *testing.T) { t.Parallel() + testEpoch := uint32(37) + shardConsensusGroupSize, metaConsensusGroupSize := 1, 1 arguments := createArguments() + arguments.Epoch = testEpoch - 1 + numTimesChainParametersForEpochWasCalled := 0 + arguments.ChainParametersHandler = &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: uint32(shardConsensusGroupSize), + MetachainConsensusGroupSize: uint32(metaConsensusGroupSize), + } + }, + ChainParametersForEpochCalled: func(epoch uint32) (config.ChainParametersByEpochConfig, error) { + if numTimesChainParametersForEpochWasCalled == 0 { + require.Equal(t, testEpoch-1, epoch) + } else { + require.Equal(t, testEpoch, epoch) + } + numTimesChainParametersForEpochWasCalled++ + + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + }, nil + }, + } ihnc, err := NewIndexHashedNodesCoordinator(arguments) require.Nil(t, err) - consensusSizeShard := ihnc.ConsensusGroupSize(0) - consensusSizeMeta := ihnc.ConsensusGroupSize(core.MetachainShardId) + consensusSizeShard := ihnc.ConsensusGroupSizeForShardAndEpoch(0, testEpoch) + consensusSizeMeta := ihnc.ConsensusGroupSizeForShardAndEpoch(core.MetachainShardId, testEpoch) + + require.Equal(t, shardConsensusGroupSize, consensusSizeShard) + require.Equal(t, metaConsensusGroupSize, consensusSizeMeta) - require.Equal(t, arguments.ShardConsensusGroupSize, consensusSizeShard) - require.Equal(t, arguments.MetaConsensusGroupSize, consensusSizeMeta) + // consensus group size from chain parameters should have been called once from the constructor, once for shard and once for meta + require.Equal(t, 3, numTimesChainParametersForEpochWasCalled) } func TestIndexHashedNodesCoordinator_GetNumTotalEligible(t *testing.T) { @@ -2516,14 +2604,10 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) t.Run("missing nodes config for current epoch should error ", func(t *testing.T) { t.Parallel() - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() shufflerArgs := &NodesShufflerArgs{ - NodesShard: 10, - NodesMeta: 10, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -2532,15 +2616,21 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) require.Nil(t, err) arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 1, - MetaConsensusGroupSize: 1, - Marshalizer: &mock.MarshalizerMock{}, - Hasher: &hashingMocks.HasherMock{}, - Shuffler: nodeShuffler, - EpochStartNotifier: epochStartSubscriber, - BootStorer: bootStorer, - ShardIDAsObserver: 0, - NbShards: 2, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + }, + Marshalizer: &mock.MarshalizerMock{}, + Hasher: &hashingMocks.HasherMock{}, + Shuffler: nodeShuffler, + EpochStartNotifier: epochStartSubscriber, + BootStorer: bootStorer, + ShardIDAsObserver: 0, + NbShards: 2, EligibleNodes: map[uint32][]Validator{ core.MetachainShardId: {newValidatorMock([]byte("pk"), 1, 0)}, }, @@ -2584,7 +2674,7 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) waitingMap[core.MetachainShardId] = listMeta waitingMap[shardZeroId] = listShard0 - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap := make(map[uint32][]Validator) @@ -2592,10 +2682,6 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) eligibleMap[shardZeroId] = []Validator{&validator{}} shufflerArgs := &NodesShufflerArgs{ - NodesShard: 10, - NodesMeta: 10, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -2604,24 +2690,30 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) require.Nil(t, err) arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 1, - MetaConsensusGroupSize: 1, - Marshalizer: &mock.MarshalizerMock{}, - Hasher: &hashingMocks.HasherMock{}, - Shuffler: nodeShuffler, - EpochStartNotifier: epochStartSubscriber, - BootStorer: bootStorer, - ShardIDAsObserver: shardZeroId, - NbShards: 2, - EligibleNodes: eligibleMap, - WaitingNodes: waitingMap, - SelfPublicKey: []byte("key"), - ConsensusGroupCache: &mock.NodesCoordinatorCacheMock{}, - ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, - ChanStopNode: make(chan endProcess.ArgEndProcess), - NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, - EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, - ValidatorInfoCacher: &vic.ValidatorInfoCacherStub{}, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + }, + Marshalizer: &mock.MarshalizerMock{}, + Hasher: &hashingMocks.HasherMock{}, + Shuffler: nodeShuffler, + EpochStartNotifier: epochStartSubscriber, + BootStorer: bootStorer, + ShardIDAsObserver: shardZeroId, + NbShards: 2, + EligibleNodes: eligibleMap, + WaitingNodes: waitingMap, + SelfPublicKey: []byte("key"), + ConsensusGroupCache: &mock.NodesCoordinatorCacheMock{}, + ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, + ChanStopNode: make(chan endProcess.ArgEndProcess), + NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, + EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, + ValidatorInfoCacher: &vic.ValidatorInfoCacherStub{}, GenesisNodesSetupHandler: &mock.NodesSetupMock{ MinShardHysteresisNodesCalled: func() uint32 { return 0 @@ -2669,7 +2761,7 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) waitingMap[core.MetachainShardId] = listMeta waitingMap[shardZeroId] = listShard0 - epochStartSubscriber := &mock.EpochStartNotifierStub{} + epochStartSubscriber := &epochstartmock.EpochStartNotifierStub{} bootStorer := genericMocks.NewStorerMock() eligibleMap := make(map[uint32][]Validator) @@ -2677,10 +2769,6 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) eligibleMap[shardZeroId] = []Validator{&validator{}} shufflerArgs := &NodesShufflerArgs{ - NodesShard: 10, - NodesMeta: 10, - Hysteresis: hysteresis, - Adaptivity: adaptivity, ShuffleBetweenShards: shuffleBetweenShards, MaxNodesEnableConfig: nil, EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, @@ -2689,24 +2777,30 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) require.Nil(t, err) arguments := ArgNodesCoordinator{ - ShardConsensusGroupSize: 1, - MetaConsensusGroupSize: 1, - Marshalizer: &mock.MarshalizerMock{}, - Hasher: &hashingMocks.HasherMock{}, - Shuffler: nodeShuffler, - EpochStartNotifier: epochStartSubscriber, - BootStorer: bootStorer, - ShardIDAsObserver: shardZeroId, - NbShards: 2, - EligibleNodes: eligibleMap, - WaitingNodes: waitingMap, - SelfPublicKey: []byte("key"), - ConsensusGroupCache: &mock.NodesCoordinatorCacheMock{}, - ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, - ChanStopNode: make(chan endProcess.ArgEndProcess), - NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, - EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, - ValidatorInfoCacher: &vic.ValidatorInfoCacherStub{}, + ChainParametersHandler: &chainParameters.ChainParametersHandlerStub{ + CurrentChainParametersCalled: func() config.ChainParametersByEpochConfig { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: 1, + MetachainConsensusGroupSize: 1, + } + }, + }, + Marshalizer: &mock.MarshalizerMock{}, + Hasher: &hashingMocks.HasherMock{}, + Shuffler: nodeShuffler, + EpochStartNotifier: epochStartSubscriber, + BootStorer: bootStorer, + ShardIDAsObserver: shardZeroId, + NbShards: 2, + EligibleNodes: eligibleMap, + WaitingNodes: waitingMap, + SelfPublicKey: []byte("key"), + ConsensusGroupCache: &mock.NodesCoordinatorCacheMock{}, + ShuffledOutHandler: &mock.ShuffledOutHandlerStub{}, + ChanStopNode: make(chan endProcess.ArgEndProcess), + NodeTypeProvider: &nodeTypeProviderMock.NodeTypeProviderStub{}, + EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, + ValidatorInfoCacher: &vic.ValidatorInfoCacherStub{}, GenesisNodesSetupHandler: &mock.NodesSetupMock{ MinShardHysteresisNodesCalled: func() uint32 { return 2 @@ -2761,3 +2855,126 @@ func TestIndexHashedGroupSelector_GetWaitingEpochsLeftForPublicKey(t *testing.T) require.Equal(t, uint32(3), epochsLeft) }) } + +func TestNodesCoordinator_CustomConsensusGroupSize(t *testing.T) { + arguments := createArguments() + numShards := uint32(2) + nodesPerShard := uint32(3) + eligibleMap := createDummyNodesMap(nodesPerShard, numShards, "eligible") + waitingMap := createDummyNodesMap(0, numShards, "waiting") + arguments.EligibleNodes = eligibleMap + arguments.WaitingNodes = waitingMap + arguments.ValidatorInfoCacher = dataPool.NewCurrentEpochValidatorInfoPool() + + consensusParams := []struct { + enableEpoch uint32 + shardCnsSize uint32 + metaCnsSize uint32 + shardMinNodes uint32 + metaMinNodes uint32 + }{ + { + enableEpoch: 9, + shardCnsSize: 3, + shardMinNodes: 3, + metaCnsSize: 3, + metaMinNodes: 3, + }, + { + enableEpoch: 6, + shardCnsSize: 3, + shardMinNodes: 3, + metaCnsSize: 2, + metaMinNodes: 2, + }, + { + enableEpoch: 3, + shardCnsSize: 3, + shardMinNodes: 3, + metaCnsSize: 3, + metaMinNodes: 3, + }, + { + enableEpoch: 0, + shardCnsSize: 2, + shardMinNodes: 2, + metaCnsSize: 3, + metaMinNodes: 3, + }, + } + arguments.ChainParametersHandler = &chainParameters.ChainParametersHandlerStub{ + ChainParametersForEpochCalled: func(epoch uint32) (config.ChainParametersByEpochConfig, error) { + for _, cfg := range consensusParams { + if epoch >= cfg.enableEpoch { + return config.ChainParametersByEpochConfig{ + ShardConsensusGroupSize: cfg.shardCnsSize, + ShardMinNumNodes: cfg.shardMinNodes, + MetachainConsensusGroupSize: cfg.metaCnsSize, + MetachainMinNumNodes: cfg.metaMinNodes, + }, nil + } + } + + return config.ChainParametersByEpochConfig{}, errors.New("wrong test setup") + }, + } + + shufflerArgs := &NodesShufflerArgs{ + ShuffleBetweenShards: shuffleBetweenShards, + EnableEpochsHandler: &mock.EnableEpochsHandlerMock{}, + MaxNodesEnableConfig: []config.MaxNodesChangeConfig{ + {EpochEnable: 0, MaxNumNodes: nodesPerShard * (numShards + 1), NodesToShufflePerShard: 2}, + {EpochEnable: 3, MaxNumNodes: nodesPerShard * (numShards + 1), NodesToShufflePerShard: 3}, + }, + } + arguments.Shuffler, _ = NewHashValidatorsShuffler(shufflerArgs) + + ihnc, _ := NewIndexHashedNodesCoordinator(arguments) + require.NotNil(t, ihnc) + + numEpochsToCheck := uint32(100) + checksCounter := 0 + for ep := uint32(0); ep < numEpochsToCheck; ep++ { + for _, cfg := range consensusParams { + if ep >= cfg.enableEpoch { + changeEpochAndTestNewConsensusSizes(&consensusSizeChangeTestArgs{ + t: t, + ihnc: ihnc, + epoch: ep, + expectedShardMinNodes: cfg.shardMinNodes, + expectedMetaMinNodes: cfg.metaMinNodes, + }) + checksCounter++ + break + } + } + } + require.Equal(t, numEpochsToCheck, uint32(checksCounter)) +} + +type consensusSizeChangeTestArgs struct { + t *testing.T + ihnc *indexHashedNodesCoordinator + epoch uint32 + expectedShardMinNodes uint32 + expectedMetaMinNodes uint32 +} + +func changeEpochAndTestNewConsensusSizes(args *consensusSizeChangeTestArgs) { + header := &block.MetaBlock{ + PrevRandSeed: []byte("rand seed"), + EpochStart: block.EpochStart{LastFinalizedHeaders: []block.EpochStartShardData{{}}}, + } + + header.Epoch = args.epoch + epochForPrevConfig := uint32(0) + if args.epoch > 0 { + epochForPrevConfig = args.epoch - 1 + } + args.ihnc.nodesConfig[args.epoch] = args.ihnc.nodesConfig[epochForPrevConfig] + body := createBlockBodyFromNodesCoordinator(args.ihnc, args.epoch, args.ihnc.validatorInfoCacher) + args.ihnc.EpochStartPrepare(header, body) + args.ihnc.EpochStartAction(header) + require.Len(args.t, args.ihnc.nodesConfig[args.epoch].eligibleMap[0], int(args.expectedShardMinNodes)) + require.Len(args.t, args.ihnc.nodesConfig[args.epoch].eligibleMap[common.MetachainShardId], int(args.expectedMetaMinNodes)) +} diff --git a/sharding/nodesCoordinator/interface.go b/sharding/nodesCoordinator/interface.go index 5e2d5564a5c..aa1d386fc02 100644 --- a/sharding/nodesCoordinator/interface.go +++ b/sharding/nodesCoordinator/interface.go @@ -3,10 +3,10 @@ package nodesCoordinator import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/data" - vmcommon "github.com/multiversx/mx-chain-vm-common-go" - + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/epochStart" "github.com/multiversx/mx-chain-go/state" + vmcommon "github.com/multiversx/mx-chain-vm-common-go" ) // Validator defines a node that can be allocated to a shard for participation in a consensus group as validator @@ -29,7 +29,7 @@ type NodesCoordinator interface { ShardIdForEpoch(epoch uint32) (uint32, error) ShuffleOutForEpoch(_ uint32) GetConsensusWhitelistedNodes(epoch uint32) (map[string]struct{}, error) - ConsensusGroupSize(uint32) int + ConsensusGroupSizeForShardAndEpoch(uint32, uint32) int GetNumTotalEligible() uint64 GetWaitingEpochsLeftForPublicKey(publicKey []byte) (uint32, error) IsInterfaceNil() bool @@ -56,7 +56,6 @@ type PublicKeysSelector interface { // NodesShuffler provides shuffling functionality for nodes type NodesShuffler interface { - UpdateParams(numNodesShard uint32, numNodesMeta uint32, hysteresis float32, adaptivity bool) UpdateNodeLists(args ArgsUpdateNodes) (*ResUpdateNodes, error) IsInterfaceNil() bool } @@ -178,3 +177,11 @@ type EpochNotifier interface { CheckEpoch(header data.HeaderHandler) IsInterfaceNil() bool } + +// ChainParametersHandler defines the actions that need to be done by a component that can handle chain parameters +type ChainParametersHandler interface { + CurrentChainParameters() config.ChainParametersByEpochConfig + AllChainParameters() []config.ChainParametersByEpochConfig + ChainParametersForEpoch(epoch uint32) (config.ChainParametersByEpochConfig, error) + IsInterfaceNil() bool +} diff --git a/sharding/nodesCoordinator/shardingArgs.go b/sharding/nodesCoordinator/shardingArgs.go index 67c542952d7..d863adf3e22 100644 --- a/sharding/nodesCoordinator/shardingArgs.go +++ b/sharding/nodesCoordinator/shardingArgs.go @@ -11,27 +11,26 @@ import ( // ArgNodesCoordinator holds all dependencies required by the nodes coordinator in order to create new instances type ArgNodesCoordinator struct { - ShardConsensusGroupSize int - MetaConsensusGroupSize int - Marshalizer marshal.Marshalizer - Hasher hashing.Hasher - Shuffler NodesShuffler - EpochStartNotifier EpochStartEventNotifier - BootStorer storage.Storer - ShardIDAsObserver uint32 - NbShards uint32 - EligibleNodes map[uint32][]Validator - WaitingNodes map[uint32][]Validator - SelfPublicKey []byte - Epoch uint32 - StartEpoch uint32 - ConsensusGroupCache Cacher - ShuffledOutHandler ShuffledOutHandler - ChanStopNode chan endProcess.ArgEndProcess - NodeTypeProvider NodeTypeProviderHandler - IsFullArchive bool - EnableEpochsHandler common.EnableEpochsHandler - ValidatorInfoCacher epochStart.ValidatorInfoCacher - GenesisNodesSetupHandler GenesisNodesSetupHandler + ChainParametersHandler ChainParametersHandler + Marshalizer marshal.Marshalizer + Hasher hashing.Hasher + Shuffler NodesShuffler + EpochStartNotifier EpochStartEventNotifier + BootStorer storage.Storer + ShardIDAsObserver uint32 + NbShards uint32 + EligibleNodes map[uint32][]Validator + WaitingNodes map[uint32][]Validator + SelfPublicKey []byte + Epoch uint32 + StartEpoch uint32 + ConsensusGroupCache Cacher + ShuffledOutHandler ShuffledOutHandler + ChanStopNode chan endProcess.ArgEndProcess + NodeTypeProvider NodeTypeProviderHandler + IsFullArchive bool + EnableEpochsHandler common.EnableEpochsHandler + ValidatorInfoCacher epochStart.ValidatorInfoCacher + GenesisNodesSetupHandler GenesisNodesSetupHandler NodesCoordinatorRegistryFactory NodesCoordinatorRegistryFactory } diff --git a/sharding/nodesSetup.go b/sharding/nodesSetup.go index 7a1a94691c6..26e8bee3351 100644 --- a/sharding/nodesSetup.go +++ b/sharding/nodesSetup.go @@ -6,6 +6,7 @@ import ( "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-core-go/core/check" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" ) @@ -58,20 +59,11 @@ func (ni *nodeInfo) IsInterfaceNil() bool { // NodesSetup hold data for decoded data from json file type NodesSetup struct { - StartTime int64 `json:"startTime"` - RoundDuration uint64 `json:"roundDuration"` - ConsensusGroupSize uint32 `json:"consensusGroupSize"` - MinNodesPerShard uint32 `json:"minNodesPerShard"` - - MetaChainConsensusGroupSize uint32 `json:"metaChainConsensusGroupSize"` - MetaChainMinNodes uint32 `json:"metaChainMinNodes"` - Hysteresis float32 `json:"hysteresis"` - Adaptivity bool `json:"adaptivity"` - - InitialNodes []*InitialNode `json:"initialNodes"` + NodesSetupDTO + genesisChainParameters config.ChainParametersByEpochConfig genesisMaxNumShards uint32 - nrOfShards uint32 + numberOfShards uint32 nrOfNodes uint32 nrOfMetaChainNodes uint32 eligible map[uint32][]nodesCoordinator.GenesisNodeInfoHandler @@ -82,31 +74,54 @@ type NodesSetup struct { // NewNodesSetup creates a new decoded nodes structure from json config file func NewNodesSetup( - nodesFilePath string, + nodesConfig config.NodesConfig, + chainParametersProvider ChainParametersHandler, addressPubkeyConverter core.PubkeyConverter, validatorPubkeyConverter core.PubkeyConverter, genesisMaxNumShards uint32, ) (*NodesSetup, error) { - if check.IfNil(addressPubkeyConverter) { return nil, fmt.Errorf("%w for addressPubkeyConverter", ErrNilPubkeyConverter) } if check.IfNil(validatorPubkeyConverter) { return nil, fmt.Errorf("%w for validatorPubkeyConverter", ErrNilPubkeyConverter) } + if check.IfNil(chainParametersProvider) { + return nil, ErrNilChainParametersProvider + } if genesisMaxNumShards < 1 { return nil, fmt.Errorf("%w for genesisMaxNumShards", ErrInvalidMaximumNumberOfShards) } + genesisParams, err := chainParametersProvider.ChainParametersForEpoch(0) + if err != nil { + return nil, fmt.Errorf("NewNodesSetup: %w while fetching parameters for epoch 0", err) + } + nodes := &NodesSetup{ addressPubkeyConverter: addressPubkeyConverter, validatorPubkeyConverter: validatorPubkeyConverter, genesisMaxNumShards: genesisMaxNumShards, + genesisChainParameters: genesisParams, } - err := core.LoadJsonFile(nodes, nodesFilePath) - if err != nil { - return nil, err + initialNodes := make([]*InitialNode, 0, len(nodesConfig.InitialNodes)) + for _, item := range nodesConfig.InitialNodes { + initialNodes = append(initialNodes, &InitialNode{ + PubKey: item.PubKey, + Address: item.Address, + InitialRating: item.InitialRating, + nodeInfo: nodeInfo{}, + }) + } + + genesisChainParameters := nodes.genesisChainParameters + nodes.NodesSetupDTO = NodesSetupDTO{ + StartTime: nodesConfig.StartTime, + RoundDuration: genesisChainParameters.RoundDuration, + Hysteresis: genesisChainParameters.Hysteresis, + Adaptivity: genesisChainParameters.Adaptivity, + InitialNodes: initialNodes, } err = nodes.processConfig() @@ -160,34 +175,31 @@ func (ns *NodesSetup) processConfig() error { ns.nrOfNodes++ } - if ns.ConsensusGroupSize < 1 { + if ns.genesisChainParameters.ShardConsensusGroupSize < 1 { return ErrNegativeOrZeroConsensusGroupSize } - if ns.MinNodesPerShard < ns.ConsensusGroupSize { + if ns.genesisChainParameters.ShardMinNumNodes < ns.genesisChainParameters.ShardConsensusGroupSize { return ErrMinNodesPerShardSmallerThanConsensusSize } - if ns.nrOfNodes < ns.MinNodesPerShard { + if ns.nrOfNodes < ns.genesisChainParameters.ShardMinNumNodes { return ErrNodesSizeSmallerThanMinNoOfNodes } - - if ns.MetaChainConsensusGroupSize < 1 { + if ns.genesisChainParameters.MetachainMinNumNodes < 1 { return ErrNegativeOrZeroConsensusGroupSize } - if ns.MetaChainMinNodes < ns.MetaChainConsensusGroupSize { + if ns.genesisChainParameters.MetachainMinNumNodes < ns.genesisChainParameters.MetachainConsensusGroupSize { return ErrMinNodesPerShardSmallerThanConsensusSize } - - totalMinNodes := ns.MetaChainMinNodes + ns.MinNodesPerShard + totalMinNodes := ns.genesisChainParameters.MetachainMinNumNodes + ns.genesisChainParameters.ShardMinNumNodes if ns.nrOfNodes < totalMinNodes { return ErrNodesSizeSmallerThanMinNoOfNodes } - return nil } func (ns *NodesSetup) processMetaChainAssigment() { ns.nrOfMetaChainNodes = 0 - for id := uint32(0); id < ns.MetaChainMinNodes; id++ { + for id := uint32(0); id < ns.genesisChainParameters.MetachainMinNumNodes; id++ { if ns.InitialNodes[id].pubKey != nil { ns.InitialNodes[id].assignedShard = core.MetachainShardId ns.InitialNodes[id].eligible = true @@ -195,13 +207,13 @@ func (ns *NodesSetup) processMetaChainAssigment() { } } - hystMeta := uint32(float32(ns.MetaChainMinNodes) * ns.Hysteresis) - hystShard := uint32(float32(ns.MinNodesPerShard) * ns.Hysteresis) + hystMeta := uint32(float32(ns.genesisChainParameters.MetachainMinNumNodes) * ns.Hysteresis) + hystShard := uint32(float32(ns.genesisChainParameters.ShardMinNumNodes) * ns.Hysteresis) - ns.nrOfShards = (ns.nrOfNodes - ns.nrOfMetaChainNodes - hystMeta) / (ns.MinNodesPerShard + hystShard) + ns.numberOfShards = (ns.nrOfNodes - ns.nrOfMetaChainNodes - hystMeta) / (ns.genesisChainParameters.ShardMinNumNodes + hystShard) - if ns.nrOfShards > ns.genesisMaxNumShards { - ns.nrOfShards = ns.genesisMaxNumShards + if ns.numberOfShards > ns.genesisMaxNumShards { + ns.numberOfShards = ns.genesisMaxNumShards } } @@ -209,8 +221,8 @@ func (ns *NodesSetup) processShardAssignment() { // initial implementation - as there is no other info than public key, we allocate first nodes in FIFO order to shards currentShard := uint32(0) countSetNodes := ns.nrOfMetaChainNodes - for ; currentShard < ns.nrOfShards; currentShard++ { - for id := countSetNodes; id < ns.nrOfMetaChainNodes+(currentShard+1)*ns.MinNodesPerShard; id++ { + for ; currentShard < ns.numberOfShards; currentShard++ { + for id := countSetNodes; id < ns.nrOfMetaChainNodes+(currentShard+1)*ns.genesisChainParameters.ShardMinNumNodes; id++ { // consider only nodes with valid public key if ns.InitialNodes[id].pubKey != nil { ns.InitialNodes[id].assignedShard = currentShard @@ -223,8 +235,8 @@ func (ns *NodesSetup) processShardAssignment() { // allocate the rest to waiting lists currentShard = 0 for i := countSetNodes; i < ns.nrOfNodes; i++ { - currentShard = (currentShard + 1) % (ns.nrOfShards + 1) - if currentShard == ns.nrOfShards { + currentShard = (currentShard + 1) % (ns.numberOfShards + 1) + if currentShard == ns.numberOfShards { currentShard = core.MetachainShardId } @@ -236,7 +248,7 @@ func (ns *NodesSetup) processShardAssignment() { } func (ns *NodesSetup) createInitialNodesInfo() { - nrOfShardAndMeta := ns.nrOfShards + 1 + nrOfShardAndMeta := ns.numberOfShards + 1 ns.eligible = make(map[uint32][]nodesCoordinator.GenesisNodeInfoHandler, nrOfShardAndMeta) ns.waiting = make(map[uint32][]nodesCoordinator.GenesisNodeInfoHandler, nrOfShardAndMeta) @@ -320,22 +332,22 @@ func (ns *NodesSetup) InitialNodesInfoForShard(shardId uint32) ([]nodesCoordinat // NumberOfShards returns the calculated number of shards func (ns *NodesSetup) NumberOfShards() uint32 { - return ns.nrOfShards + return ns.numberOfShards } // MinNumberOfNodes returns the minimum number of nodes func (ns *NodesSetup) MinNumberOfNodes() uint32 { - return ns.nrOfShards*ns.MinNodesPerShard + ns.MetaChainMinNodes + return ns.numberOfShards*ns.genesisChainParameters.ShardMinNumNodes + ns.genesisChainParameters.MetachainMinNumNodes } // MinShardHysteresisNodes returns the minimum number of hysteresis nodes per shard func (ns *NodesSetup) MinShardHysteresisNodes() uint32 { - return uint32(float32(ns.MinNodesPerShard) * ns.Hysteresis) + return uint32(float32(ns.genesisChainParameters.ShardMinNumNodes) * ns.Hysteresis) } // MinMetaHysteresisNodes returns the minimum number of hysteresis nodes in metachain func (ns *NodesSetup) MinMetaHysteresisNodes() uint32 { - return uint32(float32(ns.MetaChainMinNodes) * ns.Hysteresis) + return uint32(float32(ns.genesisChainParameters.MetachainMinNumNodes) * ns.Hysteresis) } // MinNumberOfNodesWithHysteresis returns the minimum number of nodes with hysteresis @@ -344,17 +356,17 @@ func (ns *NodesSetup) MinNumberOfNodesWithHysteresis() uint32 { hystNodesShard := ns.MinShardHysteresisNodes() minNumberOfNodes := ns.MinNumberOfNodes() - return minNumberOfNodes + hystNodesMeta + ns.nrOfShards*hystNodesShard + return minNumberOfNodes + hystNodesMeta + ns.numberOfShards*hystNodesShard } // MinNumberOfShardNodes returns the minimum number of nodes per shard func (ns *NodesSetup) MinNumberOfShardNodes() uint32 { - return ns.MinNodesPerShard + return ns.genesisChainParameters.ShardMinNumNodes } // MinNumberOfMetaNodes returns the minimum number of nodes in metachain func (ns *NodesSetup) MinNumberOfMetaNodes() uint32 { - return ns.MetaChainMinNodes + return ns.genesisChainParameters.MetachainMinNumNodes } // GetHysteresis returns the hysteresis value @@ -389,12 +401,30 @@ func (ns *NodesSetup) GetRoundDuration() uint64 { // GetShardConsensusGroupSize returns the shard consensus group size func (ns *NodesSetup) GetShardConsensusGroupSize() uint32 { - return ns.ConsensusGroupSize + return ns.genesisChainParameters.ShardConsensusGroupSize } // GetMetaConsensusGroupSize returns the metachain consensus group size func (ns *NodesSetup) GetMetaConsensusGroupSize() uint32 { - return ns.MetaChainConsensusGroupSize + return ns.genesisChainParameters.MetachainConsensusGroupSize +} + +// ExportNodesConfig will create and return the nodes' configuration +func (ns *NodesSetup) ExportNodesConfig() config.NodesConfig { + initialNodes := ns.InitialNodes + initialNodesToExport := make([]*config.InitialNodeConfig, 0, len(initialNodes)) + for _, item := range initialNodes { + initialNodesToExport = append(initialNodesToExport, &config.InitialNodeConfig{ + PubKey: item.PubKey, + Address: item.Address, + InitialRating: item.InitialRating, + }) + } + + return config.NodesConfig{ + StartTime: ns.StartTime, + InitialNodes: initialNodesToExport, + } } // IsInterfaceNil returns true if underlying object is nil diff --git a/sharding/nodesSetup_test.go b/sharding/nodesSetup_test.go index ca8d3ce479b..f5e6de19a8b 100644 --- a/sharding/nodesSetup_test.go +++ b/sharding/nodesSetup_test.go @@ -5,8 +5,11 @@ import ( "testing" "github.com/multiversx/mx-chain-core-go/core" + "github.com/stretchr/testify/require" + + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/sharding/mock" - "github.com/stretchr/testify/assert" + "github.com/multiversx/mx-chain-go/testscommon/chainParameters" ) var ( @@ -29,10 +32,10 @@ var ( } ) -func createAndAssignNodes(ns NodesSetup, noOfInitialNodes int) *NodesSetup { - ns.InitialNodes = make([]*InitialNode, noOfInitialNodes) +func createAndAssignNodes(ns *NodesSetup, numInitialNodes int) *NodesSetup { + ns.InitialNodes = make([]*InitialNode, numInitialNodes) - for i := 0; i < noOfInitialNodes; i++ { + for i := 0; i < numInitialNodes; i++ { lookupIndex := i % len(pubKeys) ns.InitialNodes[i] = &InitialNode{} ns.InitialNodes[i].PubKey = pubKeys[lookupIndex] @@ -48,178 +51,85 @@ func createAndAssignNodes(ns NodesSetup, noOfInitialNodes int) *NodesSetup { ns.processShardAssignment() ns.createInitialNodesInfo() - return &ns -} - -func createNodesSetupOneShardOneNodeWithOneMeta() *NodesSetup { - ns := &NodesSetup{ - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - genesisMaxNumShards: 100, - } - ns.ConsensusGroupSize = 1 - ns.MinNodesPerShard = 1 - ns.MetaChainConsensusGroupSize = 1 - ns.MetaChainMinNodes = 1 - ns.InitialNodes = []*InitialNode{ - { - PubKey: pubKeys[0], - Address: address[0], - }, - { - PubKey: pubKeys[1], - Address: address[1], - }, - } - - err := ns.processConfig() - if err != nil { - return nil - } - - ns.processMetaChainAssigment() - ns.processShardAssignment() - ns.createInitialNodesInfo() - - return ns -} - -func initNodesConfig(ns *NodesSetup, noOfInitialNodes int) bool { - ns.InitialNodes = make([]*InitialNode, noOfInitialNodes) - - for i := 0; i < noOfInitialNodes; i++ { - ns.InitialNodes[i] = &InitialNode{} - ns.InitialNodes[i].PubKey = pubKeys[i] - ns.InitialNodes[i].Address = address[i] - } - - err := ns.processConfig() - if err != nil { - return false - } - - ns.processMetaChainAssigment() - ns.processShardAssignment() - ns.createInitialNodesInfo() - return true -} - -func createNodesSetupTwoShardTwoNodesWithOneMeta() *NodesSetup { - noOfInitialNodes := 6 - ns := &NodesSetup{ - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - genesisMaxNumShards: 100, - } - ns.ConsensusGroupSize = 1 - ns.MinNodesPerShard = 2 - ns.MetaChainConsensusGroupSize = 1 - ns.MetaChainMinNodes = 2 - ok := initNodesConfig(ns, noOfInitialNodes) - if !ok { - return nil - } - return ns } -func createNodesSetupTwoShard5NodesWithMeta() *NodesSetup { - noOfInitialNodes := 5 - ns := &NodesSetup{ - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - genesisMaxNumShards: 100, - } - ns.ConsensusGroupSize = 1 - ns.MinNodesPerShard = 2 - ns.MetaChainConsensusGroupSize = 1 - ns.MetaChainMinNodes = 1 - ok := initNodesConfig(ns, noOfInitialNodes) - if !ok { - return nil - } - - return ns +type argsTestNodesSetup struct { + shardConsensusSize uint32 + shardMinNodes uint32 + metaConsensusSize uint32 + metaMinNodes uint32 + numInitialNodes uint32 + genesisMaxShards uint32 } -func createNodesSetupTwoShard6NodesMeta() *NodesSetup { - noOfInitialNodes := 6 - ns := &NodesSetup{ - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - genesisMaxNumShards: 100, - } - ns.ConsensusGroupSize = 1 - ns.MinNodesPerShard = 2 - ns.MetaChainMinNodes = 2 - ns.MetaChainConsensusGroupSize = 2 - ok := initNodesConfig(ns, noOfInitialNodes) - if !ok { - return nil +func createTestNodesSetup(args argsTestNodesSetup) (*NodesSetup, error) { + initialNodes := make([]*config.InitialNodeConfig, 0) + for i := 0; uint32(i) < args.numInitialNodes; i++ { + lookupIndex := i % len(pubKeys) + initialNodes = append(initialNodes, &config.InitialNodeConfig{ + PubKey: pubKeys[lookupIndex], + Address: address[lookupIndex], + }) } - - return ns -} - -func TestNodesSetup_NewNodesSetupWrongFile(t *testing.T) { - t.Parallel() - ns, err := NewNodesSetup( - "", - mock.NewPubkeyConverterMock(32), - mock.NewPubkeyConverterMock(96), - 100, - ) - - assert.Nil(t, ns) - assert.NotNil(t, err) -} - -func TestNodesSetup_NewNodesSetupWrongDataInFile(t *testing.T) { - t.Parallel() - - ns, err := NewNodesSetup( - "mock/testdata/invalidNodesSetupMock.json", + config.NodesConfig{ + StartTime: 0, + InitialNodes: initialNodes, + }, + &chainParameters.ChainParametersHandlerStub{ + ChainParametersForEpochCalled: func(epoch uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + EnableEpoch: 0, + ShardMinNumNodes: args.shardMinNodes, + ShardConsensusGroupSize: args.shardConsensusSize, + MetachainMinNumNodes: args.metaMinNodes, + MetachainConsensusGroupSize: args.metaConsensusSize, + }, nil + }, + }, mock.NewPubkeyConverterMock(32), mock.NewPubkeyConverterMock(96), - 100, + args.genesisMaxShards, ) - assert.Nil(t, ns) - assert.Equal(t, ErrNegativeOrZeroConsensusGroupSize, err) + return ns, err } -func TestNodesSetup_NewNodesShouldWork(t *testing.T) { - t.Parallel() +func createTestNodesSetupWithSpecificMockedComponents(args argsTestNodesSetup, + initialNodes []*config.InitialNodeConfig, + addressPubkeyConverter core.PubkeyConverter, + validatorPubkeyConverter core.PubkeyConverter) (*NodesSetup, error) { ns, err := NewNodesSetup( - "mock/testdata/nodesSetupMock.json", - mock.NewPubkeyConverterMock(32), - mock.NewPubkeyConverterMock(96), - 100, + config.NodesConfig{ + StartTime: 0, + InitialNodes: initialNodes, + }, + &chainParameters.ChainParametersHandlerStub{ + ChainParametersForEpochCalled: func(epoch uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{ + EnableEpoch: 0, + ShardMinNumNodes: args.shardMinNodes, + ShardConsensusGroupSize: args.shardConsensusSize, + MetachainMinNumNodes: args.metaMinNodes, + MetachainConsensusGroupSize: args.metaConsensusSize, + }, nil + }, + }, + addressPubkeyConverter, + validatorPubkeyConverter, + args.genesisMaxShards, ) - assert.NotNil(t, ns) - assert.Nil(t, err) - assert.Equal(t, 5, len(ns.InitialNodes)) -} - -func TestNodesSetup_InitialNodesPubKeysFromNil(t *testing.T) { - t.Parallel() - - ns := NodesSetup{} - eligible, waiting := ns.InitialNodesInfo() - - assert.NotNil(t, ns) - assert.Nil(t, eligible) - assert.Nil(t, waiting) + return ns, err } func TestNodesSetup_ProcessConfigNodesWithIncompleteDataShouldErr(t *testing.T) { t.Parallel() noOfInitialNodes := 2 - ns := NodesSetup{ + ns := &NodesSetup{ addressPubkeyConverter: mock.NewPubkeyConverterMock(32), validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), } @@ -234,496 +144,930 @@ func TestNodesSetup_ProcessConfigNodesWithIncompleteDataShouldErr(t *testing.T) err := ns.processConfig() - assert.NotNil(t, ns) - assert.Equal(t, ErrCouldNotParsePubKey, err) + require.NotNil(t, ns) + require.Equal(t, ErrCouldNotParsePubKey, err) } -func TestNodesSetup_ProcessConfigInvalidConsensusGroupSizeShouldErr(t *testing.T) { +func TestNodesSetup_ProcessConfigNodesShouldErrCouldNotParsePubKeyForString(t *testing.T) { t.Parallel() - noOfInitialNodes := 2 - ns := NodesSetup{ - ConsensusGroupSize: 0, - MinNodesPerShard: 0, - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), + mockedNodes := make([]*config.InitialNodeConfig, 2) + mockedNodes[0] = &config.InitialNodeConfig{ + PubKey: pubKeys[0], + Address: address[0], } - ns.InitialNodes = make([]*InitialNode, noOfInitialNodes) + mockedNodes[1] = &config.InitialNodeConfig{ + PubKey: pubKeys[1], + Address: address[1], + } - for i := 0; i < noOfInitialNodes; i++ { - ns.InitialNodes[i] = &InitialNode{} - ns.InitialNodes[i].PubKey = pubKeys[i] - ns.InitialNodes[i].Address = address[i] + addressPubkeyConverterMocked := mock.NewPubkeyConverterMock(32) + validatorPubkeyConverterMocked := &mock.PubkeyConverterMock{ + DecodeCalled: func() ([]byte, error) { + return nil, ErrCouldNotParsePubKey + }, } - err := ns.processConfig() + _, err := createTestNodesSetupWithSpecificMockedComponents(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 2, + genesisMaxShards: 3, + }, + mockedNodes, + addressPubkeyConverterMocked, + validatorPubkeyConverterMocked, + ) - assert.NotNil(t, ns) - assert.Equal(t, ErrNegativeOrZeroConsensusGroupSize, err) + require.ErrorIs(t, err, ErrCouldNotParsePubKey) } -func TestNodesSetup_ProcessConfigInvalidMetaConsensusGroupSizeShouldErr(t *testing.T) { +func TestNodesSetup_ProcessConfigNodesShouldErrCouldNotParseAddressForString(t *testing.T) { t.Parallel() - noOfInitialNodes := 2 - ns := NodesSetup{ - ConsensusGroupSize: 1, - MinNodesPerShard: 1, - MetaChainConsensusGroupSize: 0, - MetaChainMinNodes: 0, - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), + mockedNodes := make([]*config.InitialNodeConfig, 2) + mockedNodes[0] = &config.InitialNodeConfig{ + PubKey: pubKeys[0], + Address: address[0], } - ns.InitialNodes = make([]*InitialNode, noOfInitialNodes) - - for i := 0; i < noOfInitialNodes; i++ { - ns.InitialNodes[i] = &InitialNode{} - ns.InitialNodes[i].PubKey = pubKeys[i] - ns.InitialNodes[i].Address = address[i] + mockedNodes[1] = &config.InitialNodeConfig{ + PubKey: pubKeys[1], + Address: address[1], } - err := ns.processConfig() + addressPubkeyConverterMocked := &mock.PubkeyConverterMock{ + DecodeCalled: func() ([]byte, error) { + return nil, ErrCouldNotParseAddress + }, + } + validatorPubkeyConverterMocked := mock.NewPubkeyConverterMock(96) + + _, err := createTestNodesSetupWithSpecificMockedComponents(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 2, + genesisMaxShards: 3, + }, + mockedNodes, + addressPubkeyConverterMocked, + validatorPubkeyConverterMocked, + ) - assert.NotNil(t, ns) - assert.Equal(t, ErrNegativeOrZeroConsensusGroupSize, err) + require.ErrorIs(t, err, ErrCouldNotParseAddress) } -func TestNodesSetup_ProcessConfigInvalidConsensusGroupSizeLargerThanNumOfNodesShouldErr(t *testing.T) { +func TestNodesSetup_ProcessConfigNodesWithEmptyDataShouldErrCouldNotParseAddress(t *testing.T) { t.Parallel() - noOfInitialNodes := 2 - ns := NodesSetup{ - ConsensusGroupSize: 2, - MinNodesPerShard: 0, - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), + mockedNodes := make([]*config.InitialNodeConfig, 2) + mockedNodes[0] = &config.InitialNodeConfig{ + PubKey: pubKeys[0], + Address: address[0], } - ns.InitialNodes = make([]*InitialNode, noOfInitialNodes) - - for i := 0; i < noOfInitialNodes; i++ { - ns.InitialNodes[i] = &InitialNode{} - ns.InitialNodes[i].PubKey = pubKeys[i] - ns.InitialNodes[i].Address = address[i] + mockedNodes[1] = &config.InitialNodeConfig{ + PubKey: pubKeys[1], + Address: "", } - err := ns.processConfig() + addressPubkeyConverterMocked := mock.NewPubkeyConverterMock(32) + validatorPubkeyConverterMocked := mock.NewPubkeyConverterMock(96) + + _, err := createTestNodesSetupWithSpecificMockedComponents(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 2, + genesisMaxShards: 3, + }, + mockedNodes, + addressPubkeyConverterMocked, + validatorPubkeyConverterMocked, + ) - assert.NotNil(t, ns) - assert.Equal(t, ErrMinNodesPerShardSmallerThanConsensusSize, err) + require.ErrorIs(t, err, ErrCouldNotParseAddress) } -func TestNodesSetup_ProcessConfigInvalidMetaConsensusGroupSizeLargerThanNumOfNodesShouldErr(t *testing.T) { +func TestNodesSetup_ProcessConfigInvalidConsensusGroupSizeShouldErr(t *testing.T) { t.Parallel() - noOfInitialNodes := 2 - ns := NodesSetup{ - ConsensusGroupSize: 1, - MinNodesPerShard: 1, - MetaChainConsensusGroupSize: 1, - MetaChainMinNodes: 0, - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - } - - ns.InitialNodes = make([]*InitialNode, 2) - - for i := 0; i < noOfInitialNodes; i++ { - ns.InitialNodes[i] = &InitialNode{} - ns.InitialNodes[i].PubKey = pubKeys[i] - ns.InitialNodes[i].Address = address[i] - } - - err := ns.processConfig() - - assert.NotNil(t, ns) - assert.Equal(t, ErrMinNodesPerShardSmallerThanConsensusSize, err) + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 0, + shardMinNodes: 0, + metaConsensusSize: 0, + metaMinNodes: 0, + numInitialNodes: 0, + genesisMaxShards: 3, + }) + require.Equal(t, ErrNegativeOrZeroConsensusGroupSize, err) + require.Nil(t, ns) } -func TestNodesSetup_ProcessConfigInvalidMinNodesPerShardShouldErr(t *testing.T) { +func TestNodesSetup_ProcessConfigInvalidMetaConsensusGroupSizeShouldErr(t *testing.T) { t.Parallel() - noOfInitialNodes := 2 - ns := NodesSetup{ - ConsensusGroupSize: 2, - MinNodesPerShard: 0, - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - } - - ns.InitialNodes = make([]*InitialNode, noOfInitialNodes) - - for i := 0; i < noOfInitialNodes; i++ { - ns.InitialNodes[i] = &InitialNode{} - ns.InitialNodes[i].PubKey = pubKeys[i] - ns.InitialNodes[i].Address = address[i] - } - - err := ns.processConfig() - - assert.NotNil(t, ns) - assert.Equal(t, ErrMinNodesPerShardSmallerThanConsensusSize, err) + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 0, + metaMinNodes: 0, + numInitialNodes: 1, + genesisMaxShards: 3, + }) + require.Equal(t, ErrNegativeOrZeroConsensusGroupSize, err) + require.Nil(t, ns) } -func TestNodesSetup_ProcessConfigInvalidMetaMinNodesPerShardShouldErr(t *testing.T) { +func TestNodesSetup_ProcessConfigInvalidConsensusGroupSizeLargerThanNumOfNodesShouldErr(t *testing.T) { t.Parallel() - noOfInitialNodes := 1 - ns := NodesSetup{ - ConsensusGroupSize: 1, - MinNodesPerShard: 1, - MetaChainConsensusGroupSize: 1, - MetaChainMinNodes: 0, - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - } - - ns.InitialNodes = make([]*InitialNode, noOfInitialNodes) - - for i := 0; i < noOfInitialNodes; i++ { - ns.InitialNodes[i] = &InitialNode{} - ns.InitialNodes[i].PubKey = pubKeys[i] - ns.InitialNodes[i].Address = address[i] - } - - err := ns.processConfig() - - assert.NotNil(t, ns) - assert.Equal(t, ErrMinNodesPerShardSmallerThanConsensusSize, err) + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 2, + shardMinNodes: 0, + metaConsensusSize: 0, + metaMinNodes: 0, + numInitialNodes: 2, + genesisMaxShards: 3, + }) + require.Equal(t, ErrMinNodesPerShardSmallerThanConsensusSize, err) + require.Nil(t, ns) } -func TestNodesSetup_ProcessConfigInvalidNumOfNodesSmallerThanMinNodesPerShardShouldErr(t *testing.T) { +func TestNodesSetup_ProcessConfigInvalidMetaConsensusGroupSizeLargerThanNumOfNodesShouldErr(t *testing.T) { t.Parallel() - noOfInitialNodes := 2 - ns := NodesSetup{ - ConsensusGroupSize: 2, - MinNodesPerShard: 3, - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - } - - ns.InitialNodes = make([]*InitialNode, noOfInitialNodes) - - for i := 0; i < noOfInitialNodes; i++ { - ns.InitialNodes[i] = &InitialNode{} - ns.InitialNodes[i].PubKey = pubKeys[i] - ns.InitialNodes[i].Address = address[i] - } - - err := ns.processConfig() - - assert.NotNil(t, ns) - assert.Equal(t, ErrNodesSizeSmallerThanMinNoOfNodes, err) + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 2, + metaMinNodes: 1, + numInitialNodes: 2, + genesisMaxShards: 3, + }) + require.Equal(t, ErrMinNodesPerShardSmallerThanConsensusSize, err) + require.Nil(t, ns) } -func TestNodesSetup_ProcessConfigInvalidMetaNumOfNodesSmallerThanMinNodesPerShardShouldErr(t *testing.T) { +func TestNodesSetup_ProcessConfigInvalidNumOfNodesSmallerThanMinNodesPerShardShouldErr(t *testing.T) { t.Parallel() - noOfInitialNodes := 3 - ns := NodesSetup{ - ConsensusGroupSize: 1, - MinNodesPerShard: 1, - MetaChainConsensusGroupSize: 2, - MetaChainMinNodes: 3, - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - } - - ns.InitialNodes = make([]*InitialNode, noOfInitialNodes) - - for i := 0; i < noOfInitialNodes; i++ { - ns.InitialNodes[i] = &InitialNode{} - ns.InitialNodes[i].PubKey = pubKeys[i] - ns.InitialNodes[i].Address = address[i] - } - - err := ns.processConfig() - - assert.NotNil(t, ns) - assert.Equal(t, ErrNodesSizeSmallerThanMinNoOfNodes, err) + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 2, + shardMinNodes: 3, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 2, + genesisMaxShards: 3, + }) + require.Nil(t, ns) + require.Equal(t, ErrNodesSizeSmallerThanMinNoOfNodes, err) } -func TestNodesSetup_InitialNodesPubKeysForShardNil(t *testing.T) { +func TestNodesSetup_ProcessConfigInvalidNumOfNodesSmallerThanTotalMinNodesShouldErr(t *testing.T) { t.Parallel() - ns := NodesSetup{ - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - } - eligible, waiting, err := ns.InitialNodesInfoForShard(0) - - assert.NotNil(t, ns) - assert.Nil(t, eligible) - assert.Nil(t, waiting) - assert.NotNil(t, err) + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 2, + shardMinNodes: 3, + metaConsensusSize: 1, + metaMinNodes: 3, + numInitialNodes: 5, + genesisMaxShards: 3, + }) + require.Nil(t, ns) + require.Equal(t, ErrNodesSizeSmallerThanMinNoOfNodes, err) } func TestNodesSetup_InitialNodesPubKeysWithHysteresis(t *testing.T) { t.Parallel() - ns := &NodesSetup{ - ConsensusGroupSize: 63, - MinNodesPerShard: 400, - MetaChainConsensusGroupSize: 400, - MetaChainMinNodes: 400, - Hysteresis: 0.2, - Adaptivity: false, - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - genesisMaxNumShards: 100, - } - - ns = createAndAssignNodes(*ns, 3000) - - assert.Equal(t, 6, len(ns.eligible)) + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 63, + shardMinNodes: 400, + metaConsensusSize: 400, + metaMinNodes: 400, + numInitialNodes: 3000, + genesisMaxShards: 100, + }) + ns.Hysteresis = 0.2 + ns.Adaptivity = false + require.NoError(t, err) + + ns = createAndAssignNodes(ns, 3000) + require.Equal(t, 6, len(ns.eligible)) for shard, shardNodes := range ns.eligible { - assert.Equal(t, 400, len(shardNodes)) - assert.Equal(t, 100, len(ns.waiting[shard])) + require.Equal(t, 400, len(shardNodes)) + require.Equal(t, 100, len(ns.waiting[shard])) } - ns = createAndAssignNodes(*ns, 3570) - assert.Equal(t, 7, len(ns.eligible)) + ns = createAndAssignNodes(ns, 3570) + require.Equal(t, 7, len(ns.eligible)) for shard, shardNodes := range ns.eligible { - assert.Equal(t, 400, len(shardNodes)) - assert.Equal(t, 110, len(ns.waiting[shard])) + require.Equal(t, 400, len(shardNodes)) + require.Equal(t, 110, len(ns.waiting[shard])) } - ns = createAndAssignNodes(*ns, 2400) - assert.Equal(t, 5, len(ns.eligible)) + ns = createAndAssignNodes(ns, 2400) + require.Equal(t, 5, len(ns.eligible)) for shard, shardNodes := range ns.eligible { - assert.Equal(t, 400, len(shardNodes)) - assert.Equal(t, 80, len(ns.waiting[shard])) + require.Equal(t, 400, len(shardNodes)) + require.Equal(t, 80, len(ns.waiting[shard])) } } func TestNodesSetup_InitialNodesPubKeysForShardWrongShard(t *testing.T) { t.Parallel() - ns := createNodesSetupOneShardOneNodeWithOneMeta() + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 2, + genesisMaxShards: 3, + }) + require.NoError(t, err) eligible, waiting, err := ns.InitialNodesInfoForShard(1) - assert.NotNil(t, ns) - assert.Nil(t, eligible) - assert.Nil(t, waiting) - assert.NotNil(t, err) + require.NotNil(t, ns) + require.Nil(t, eligible) + require.Nil(t, waiting) + require.NotNil(t, err) } func TestNodesSetup_InitialNodesPubKeysForShardGood(t *testing.T) { t.Parallel() - ns := createNodesSetupTwoShardTwoNodesWithOneMeta() + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 2, + metaConsensusSize: 1, + metaMinNodes: 2, + numInitialNodes: 6, + genesisMaxShards: 3, + }) + require.NoError(t, err) + eligible, waiting, err := ns.InitialNodesInfoForShard(1) - assert.NotNil(t, ns) - assert.Equal(t, 2, len(eligible)) - assert.Equal(t, 0, len(waiting)) - assert.Nil(t, err) + require.NotNil(t, ns) + require.Equal(t, 2, len(eligible)) + require.Equal(t, 0, len(waiting)) + require.Nil(t, err) } func TestNodesSetup_InitialNodesPubKeysForShardGoodMeta(t *testing.T) { t.Parallel() - ns := createNodesSetupTwoShard6NodesMeta() + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 2, + metaConsensusSize: 2, + metaMinNodes: 2, + numInitialNodes: 6, + genesisMaxShards: 3, + }) + require.NoError(t, err) metaId := core.MetachainShardId eligible, waiting, err := ns.InitialNodesInfoForShard(metaId) - assert.NotNil(t, ns) - assert.Equal(t, 2, len(eligible)) - assert.Equal(t, 0, len(waiting)) - assert.Nil(t, err) + require.NotNil(t, ns) + require.Equal(t, 2, len(eligible)) + require.Equal(t, 0, len(waiting)) + require.Nil(t, err) } func TestNodesSetup_PublicKeyNotGood(t *testing.T) { t.Parallel() - ns := createNodesSetupTwoShard6NodesMeta() + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 5, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 6, + genesisMaxShards: 3, + }) + require.NoError(t, err) - _, err := ns.GetShardIDForPubKey([]byte(pubKeys[0])) + _, err = ns.GetShardIDForPubKey([]byte(pubKeys[0])) - assert.NotNil(t, ns) - assert.NotNil(t, err) + require.NotNil(t, ns) + require.NotNil(t, err) } func TestNodesSetup_PublicKeyGood(t *testing.T) { t.Parallel() - ns := createNodesSetupTwoShard5NodesWithMeta() + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 5, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 6, + genesisMaxShards: 3, + }) + require.NoError(t, err) + publicKey, _ := hex.DecodeString(pubKeys[2]) selfId, err := ns.GetShardIDForPubKey(publicKey) - assert.NotNil(t, ns) - assert.Nil(t, err) - assert.Equal(t, uint32(0), selfId) + require.NotNil(t, ns) + require.Nil(t, err) + require.Equal(t, uint32(0), selfId) } func TestNodesSetup_ShardPublicKeyGoodMeta(t *testing.T) { t.Parallel() - ns := createNodesSetupTwoShard6NodesMeta() + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 5, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 6, + genesisMaxShards: 3, + }) + require.NoError(t, err) publicKey, _ := hex.DecodeString(pubKeys[2]) selfId, err := ns.GetShardIDForPubKey(publicKey) - assert.NotNil(t, ns) - assert.Nil(t, err) - assert.Equal(t, uint32(0), selfId) + require.NotNil(t, ns) + require.Nil(t, err) + require.Equal(t, uint32(0), selfId) } func TestNodesSetup_MetaPublicKeyGoodMeta(t *testing.T) { t.Parallel() - ns := createNodesSetupTwoShard6NodesMeta() + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 5, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 6, + genesisMaxShards: 3, + }) + require.NoError(t, err) metaId := core.MetachainShardId publicKey, _ := hex.DecodeString(pubKeys[0]) selfId, err := ns.GetShardIDForPubKey(publicKey) - assert.NotNil(t, ns) - assert.Nil(t, err) - assert.Equal(t, metaId, selfId) + require.NotNil(t, ns) + require.Nil(t, err) + require.Equal(t, metaId, selfId) } func TestNodesSetup_MinNumberOfNodes(t *testing.T) { t.Parallel() - ns := &NodesSetup{ - ConsensusGroupSize: 63, - MinNodesPerShard: 400, - MetaChainConsensusGroupSize: 400, - MetaChainMinNodes: 400, - Hysteresis: 0.2, - Adaptivity: false, - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - genesisMaxNumShards: 100, - } - - ns = createAndAssignNodes(*ns, 2169) - assert.Equal(t, 4, len(ns.eligible)) + + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 63, + shardMinNodes: 400, + metaConsensusSize: 400, + metaMinNodes: 400, + numInitialNodes: 2169, + genesisMaxShards: 3, + }) + ns.Hysteresis = 0.2 + ns.Adaptivity = false + require.NoError(t, err) + + ns = createAndAssignNodes(ns, 2169) + require.Equal(t, 4, len(ns.eligible)) for shard, shardNodes := range ns.eligible { - assert.Equal(t, 400, len(shardNodes)) - assert.LessOrEqual(t, len(ns.waiting[shard]), 143) - assert.GreaterOrEqual(t, len(ns.waiting[shard]), 142) + require.Equal(t, 400, len(shardNodes)) + require.LessOrEqual(t, len(ns.waiting[shard]), 143) + require.GreaterOrEqual(t, len(ns.waiting[shard]), 142) } minNumNodes := ns.MinNumberOfNodes() - assert.Equal(t, uint32(1600), minNumNodes) + require.Equal(t, uint32(1600), minNumNodes) minHysteresisNodesShard := ns.MinShardHysteresisNodes() - assert.Equal(t, uint32(80), minHysteresisNodesShard) + require.Equal(t, uint32(80), minHysteresisNodesShard) minHysteresisNodesMeta := ns.MinMetaHysteresisNodes() - assert.Equal(t, uint32(80), minHysteresisNodesMeta) + require.Equal(t, uint32(80), minHysteresisNodesMeta) } func TestNewNodesSetup_InvalidMaxNumShardsShouldErr(t *testing.T) { t.Parallel() ns, err := NewNodesSetup( - "", + config.NodesConfig{}, + &chainParameters.ChainParametersHandlerStub{}, mock.NewPubkeyConverterMock(32), mock.NewPubkeyConverterMock(96), 0, ) - assert.Nil(t, ns) - assert.NotNil(t, err) - assert.Contains(t, err.Error(), ErrInvalidMaximumNumberOfShards.Error()) + require.Nil(t, ns) + require.NotNil(t, err) + require.Contains(t, err.Error(), ErrInvalidMaximumNumberOfShards.Error()) } -func TestNodesSetup_IfNodesWithinMaxShardLimitEquivalentDistribution(t *testing.T) { +func TestNewNodesSetup_ErrNilPubkeyConverterForAddressPubkeyConverter(t *testing.T) { t.Parallel() - ns := &NodesSetup{ - ConsensusGroupSize: 63, - MinNodesPerShard: 400, - MetaChainConsensusGroupSize: 400, - MetaChainMinNodes: 400, - Hysteresis: 0.2, - Adaptivity: false, - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - genesisMaxNumShards: 100, - } + _, err := NewNodesSetup( + config.NodesConfig{}, + &chainParameters.ChainParametersHandlerStub{}, + nil, + mock.NewPubkeyConverterMock(96), + 3, + ) + + require.ErrorIs(t, err, ErrNilPubkeyConverter) +} + +func TestNewNodesSetup_ErrNilPubkeyConverterForValidatorPubkeyConverter(t *testing.T) { + t.Parallel() + + _, err := NewNodesSetup( + config.NodesConfig{}, + &chainParameters.ChainParametersHandlerStub{}, + mock.NewPubkeyConverterMock(32), + nil, + 3, + ) + + require.ErrorIs(t, err, ErrNilPubkeyConverter) +} + +func TestNewNodesSetup_ErrNilChainParametersProvider(t *testing.T) { + t.Parallel() + + _, err := NewNodesSetup( + config.NodesConfig{}, + nil, + mock.NewPubkeyConverterMock(32), + mock.NewPubkeyConverterMock(96), + 3, + ) - ns = createAndAssignNodes(*ns, 2169) + require.Equal(t, err, ErrNilChainParametersProvider) +} + +func TestNewNodesSetup_ErrChainParametersForEpoch(t *testing.T) { + t.Parallel() + + _, err := NewNodesSetup( + config.NodesConfig{}, + &chainParameters.ChainParametersHandlerStub{ + ChainParametersForEpochCalled: func(epoch uint32) (config.ChainParametersByEpochConfig, error) { + return config.ChainParametersByEpochConfig{}, ErrInvalidChainParametersForEpoch + }, + }, + mock.NewPubkeyConverterMock(32), + mock.NewPubkeyConverterMock(96), + 3, + ) + + require.ErrorIs(t, err, ErrInvalidChainParametersForEpoch) +} + +func TestNodesSetup_IfNodesWithinMaxShardLimitEquivalentDistribution(t *testing.T) { + t.Parallel() + + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 64, + shardMinNodes: 400, + metaConsensusSize: 400, + metaMinNodes: 400, + numInitialNodes: 2169, + genesisMaxShards: 3, + }) + ns.Hysteresis = 0.2 + ns.Adaptivity = false + require.NoError(t, err) ns2 := &(*ns) //nolint ns2.genesisMaxNumShards = 3 - ns2 = createAndAssignNodes(*ns2, 2169) + ns2 = createAndAssignNodes(ns2, 2169) - assert.Equal(t, 4, len(ns.eligible)) - assert.Equal(t, 4, len(ns2.eligible)) + require.Equal(t, 4, len(ns.eligible)) + require.Equal(t, 4, len(ns2.eligible)) for shard, shardNodes := range ns.eligible { - assert.Equal(t, len(shardNodes), len(ns2.eligible[shard])) - assert.Equal(t, len(ns.waiting[shard]), len(ns2.waiting[shard])) - assert.GreaterOrEqual(t, len(ns.waiting[shard]), 142) - assert.Equal(t, len(ns.waiting[shard]), len(ns2.waiting[shard])) + require.Equal(t, len(shardNodes), len(ns2.eligible[shard])) + require.Equal(t, len(ns.waiting[shard]), len(ns2.waiting[shard])) + require.GreaterOrEqual(t, len(ns.waiting[shard]), 142) + require.Equal(t, len(ns.waiting[shard]), len(ns2.waiting[shard])) for i, node := range shardNodes { - assert.Equal(t, node, ns2.eligible[shard][i]) + require.Equal(t, node, ns2.eligible[shard][i]) } for i, node := range ns.waiting[shard] { - assert.Equal(t, node, ns2.waiting[shard][i]) + require.Equal(t, node, ns2.waiting[shard][i]) } } minNumNodes := ns.MinNumberOfNodes() - assert.Equal(t, minNumNodes, ns2.MinNumberOfNodes()) + require.Equal(t, minNumNodes, ns2.MinNumberOfNodes()) minHysteresisNodesShard := ns.MinShardHysteresisNodes() - assert.Equal(t, minHysteresisNodesShard, ns2.MinShardHysteresisNodes()) + require.Equal(t, minHysteresisNodesShard, ns2.MinShardHysteresisNodes()) minHysteresisNodesMeta := ns.MinMetaHysteresisNodes() - assert.Equal(t, minHysteresisNodesMeta, ns2.MinMetaHysteresisNodes()) + require.Equal(t, minHysteresisNodesMeta, ns2.MinMetaHysteresisNodes()) } func TestNodesSetup_NodesAboveMaxShardLimit(t *testing.T) { t.Parallel() - ns := &NodesSetup{ - ConsensusGroupSize: 63, - MinNodesPerShard: 400, - MetaChainConsensusGroupSize: 400, - MetaChainMinNodes: 400, - Hysteresis: 0.2, - Adaptivity: false, - addressPubkeyConverter: mock.NewPubkeyConverterMock(32), - validatorPubkeyConverter: mock.NewPubkeyConverterMock(96), - genesisMaxNumShards: 3, - } - - ns = createAndAssignNodes(*ns, 3200) - - assert.Equal(t, 4, len(ns.eligible)) + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 63, + shardMinNodes: 400, + metaConsensusSize: 400, + metaMinNodes: 400, + numInitialNodes: 3200, + genesisMaxShards: 3, + }) + ns.Hysteresis = 0.2 + ns.Adaptivity = false + require.NoError(t, err) + + require.Equal(t, 4, len(ns.eligible)) for shard, shardNodes := range ns.eligible { - assert.Equal(t, 400, len(shardNodes)) - assert.Equal(t, len(ns.waiting[shard]), 400) + require.Equal(t, 400, len(shardNodes)) + require.Equal(t, len(ns.waiting[shard]), 400) } minNumNodes := ns.MinNumberOfNodes() - assert.Equal(t, uint32(1600), minNumNodes) + require.Equal(t, uint32(1600), minNumNodes) minHysteresisNodesShard := ns.MinShardHysteresisNodes() - assert.Equal(t, uint32(80), minHysteresisNodesShard) + require.Equal(t, uint32(80), minHysteresisNodesShard) minHysteresisNodesMeta := ns.MinMetaHysteresisNodes() - assert.Equal(t, uint32(80), minHysteresisNodesMeta) + require.Equal(t, uint32(80), minHysteresisNodesMeta) - ns = createAndAssignNodes(*ns, 3600) + ns = createAndAssignNodes(ns, 3600) for shard, shardNodes := range ns.eligible { - assert.Equal(t, 400, len(shardNodes)) - assert.Equal(t, len(ns.waiting[shard]), 500) + require.Equal(t, 400, len(shardNodes)) + require.Equal(t, len(ns.waiting[shard]), 500) } minNumNodes = ns.MinNumberOfNodes() - assert.Equal(t, uint32(1600), minNumNodes) + require.Equal(t, uint32(1600), minNumNodes) minHysteresisNodesShard = ns.MinShardHysteresisNodes() - assert.Equal(t, uint32(80), minHysteresisNodesShard) + require.Equal(t, uint32(80), minHysteresisNodesShard) minHysteresisNodesMeta = ns.MinMetaHysteresisNodes() - assert.Equal(t, uint32(80), minHysteresisNodesMeta) + require.Equal(t, uint32(80), minHysteresisNodesMeta) +} + +func TestNodesSetup_AllInitialNodesShouldWork(t *testing.T) { + t.Parallel() + + noOfInitialNodes := 2 + + var listOfInitialNodes = [2]InitialNode{ + { + PubKey: pubKeys[0], + Address: address[0], + }, + { + PubKey: pubKeys[1], + Address: address[1], + }, + } + + var expectedConvertedPubKeys = make([][]byte, 2) + pubKeyConverter := mock.NewPubkeyConverterMock(96) + + for i, nod := range listOfInitialNodes { + convertedValue, err := pubKeyConverter.Decode(nod.PubKey) + require.Nil(t, err) + require.NotNil(t, convertedValue) + expectedConvertedPubKeys[i] = convertedValue + } + + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 2, + genesisMaxShards: 1, + }) + + require.Nil(t, err) + ns.Hysteresis = 0.2 + ns.Adaptivity = false + + ns = createAndAssignNodes(ns, noOfInitialNodes) + + allInitialNodes := ns.AllInitialNodes() + + for i, expectedConvertedKey := range expectedConvertedPubKeys { + require.Equal(t, expectedConvertedKey, allInitialNodes[i].PubKeyBytes()) + } + +} + +func TestNodesSetup_InitialNodesInfoShouldWork(t *testing.T) { + t.Parallel() + + noOfInitialNodes := 3 + + var listOfInitialNodes = [3]InitialNode{ + { + PubKey: pubKeys[0], + Address: address[0], + }, + { + PubKey: pubKeys[1], + Address: address[1], + }, + { + PubKey: pubKeys[2], + Address: address[2], + }, + } + + var listOfExpectedConvertedPubKeysEligibleNodes = make([][]byte, 2) + pubKeyConverter := mock.NewPubkeyConverterMock(96) + + for i := 0; i < 2; i++ { + convertedValue, err := pubKeyConverter.Decode(listOfInitialNodes[i].PubKey) + require.Nil(t, err) + require.NotNil(t, convertedValue) + listOfExpectedConvertedPubKeysEligibleNodes[i] = convertedValue + } + + var listOfExpectedConvertedPubKeysWaitingNode = make([][]byte, 1) + listOfExpectedConvertedPubKeysWaitingNode[0], _ = pubKeyConverter.Decode(listOfInitialNodes[2].PubKey) + + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 3, + genesisMaxShards: 1, + }) + require.Nil(t, err) + ns.Hysteresis = 0.2 + ns.Adaptivity = false + + ns = createAndAssignNodes(ns, noOfInitialNodes) + + allEligibleNodes, allWaitingNodes := ns.InitialNodesInfo() + + require.Equal(t, listOfExpectedConvertedPubKeysEligibleNodes[0], allEligibleNodes[(core.MetachainShardId)][0].PubKeyBytes()) + require.Equal(t, listOfExpectedConvertedPubKeysEligibleNodes[1], allEligibleNodes[0][0].PubKeyBytes()) + require.Equal(t, listOfExpectedConvertedPubKeysWaitingNode[0], allWaitingNodes[(core.MetachainShardId)][0].PubKeyBytes()) + +} + +func TestNodesSetup_InitialNodesPubKeysShouldWork(t *testing.T) { + t.Parallel() + + noOfInitialNodes := 3 + + var listOfInitialNodes = [3]InitialNode{ + { + PubKey: pubKeys[0], + Address: address[0], + }, + { + PubKey: pubKeys[1], + Address: address[1], + }, + { + PubKey: pubKeys[2], + Address: address[2], + }, + } + + var listOfExpectedConvertedPubKeysEligibleNodes = make([]string, 2) + pubKeyConverter := mock.NewPubkeyConverterMock(96) + + for i := 0; i < 2; i++ { + convertedValue, err := pubKeyConverter.Decode(listOfInitialNodes[i].PubKey) + require.Nil(t, err) + require.NotNil(t, convertedValue) + listOfExpectedConvertedPubKeysEligibleNodes[i] = string(convertedValue) + } + + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 3, + genesisMaxShards: 1, + }) + require.Nil(t, err) + ns.Hysteresis = 0.2 + ns.Adaptivity = false + + ns = createAndAssignNodes(ns, noOfInitialNodes) + + allEligibleNodes := ns.InitialNodesPubKeys() + + require.Equal(t, listOfExpectedConvertedPubKeysEligibleNodes[0], allEligibleNodes[(core.MetachainShardId)][0]) + require.Equal(t, listOfExpectedConvertedPubKeysEligibleNodes[1], allEligibleNodes[0][0]) + +} + +func TestNodesSetup_InitialEligibleNodesPubKeysForShardShouldErrShardIdOutOfRange(t *testing.T) { + t.Parallel() + + noOfInitialNodes := 3 + + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 3, + genesisMaxShards: 1, + }) + require.Nil(t, err) + ns.Hysteresis = 0.2 + ns.Adaptivity = false + + ns = createAndAssignNodes(ns, noOfInitialNodes) + + returnedPubKeys, err := ns.InitialEligibleNodesPubKeysForShard(1) + require.Nil(t, returnedPubKeys) + require.Equal(t, ErrShardIdOutOfRange, err) + +} + +func TestNodesSetup_InitialEligibleNodesPubKeysForShardShouldWork(t *testing.T) { + t.Parallel() + + noOfInitialNodes := 3 + + var listOfInitialNodes = [3]InitialNode{ + { + PubKey: pubKeys[0], + Address: address[0], + }, + { + PubKey: pubKeys[1], + Address: address[1], + }, + { + PubKey: pubKeys[2], + Address: address[2], + }, + } + + var listOfExpectedPubKeysEligibleNodes = make([]string, 2) + pubKeyConverter := mock.NewPubkeyConverterMock(96) + + for i := 0; i < 2; i++ { + convertedValue, err := pubKeyConverter.Decode(listOfInitialNodes[i].PubKey) + require.Nil(t, err) + require.NotNil(t, convertedValue) + listOfExpectedPubKeysEligibleNodes[i] = string(convertedValue) + } + + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 3, + genesisMaxShards: 1, + }) + require.Nil(t, err) + ns.Hysteresis = 0.2 + ns.Adaptivity = false + + ns = createAndAssignNodes(ns, noOfInitialNodes) + + allEligibleNodes, err := ns.InitialEligibleNodesPubKeysForShard(0) + + require.Nil(t, err) + require.Equal(t, listOfExpectedPubKeysEligibleNodes[1], allEligibleNodes[0]) +} + +func TestNodesSetup_NumberOfShardsShouldWork(t *testing.T) { + t.Parallel() + + noOfInitialNodes := 3 + + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 3, + genesisMaxShards: 1, + }) + require.Nil(t, err) + require.NotNil(t, ns) + + ns.Hysteresis = 0.2 + ns.Adaptivity = false + + ns = createAndAssignNodes(ns, noOfInitialNodes) + + require.NotNil(t, ns) + + valReturned := ns.NumberOfShards() + require.Equal(t, uint32(1), valReturned) + + valReturned = ns.MinNumberOfNodesWithHysteresis() + require.Equal(t, uint32(2), valReturned) + + valReturned = ns.MinNumberOfShardNodes() + require.Equal(t, uint32(1), valReturned) + + valReturned = ns.MinNumberOfShardNodes() + require.Equal(t, uint32(1), valReturned) + + shardConsensusGroupSize := ns.GetShardConsensusGroupSize() + require.Equal(t, uint32(1), shardConsensusGroupSize) + + metaConsensusGroupSize := ns.GetMetaConsensusGroupSize() + require.Equal(t, uint32(1), metaConsensusGroupSize) + + ns.Hysteresis = 0.5 + hysteresis := ns.GetHysteresis() + require.Equal(t, float32(0.5), hysteresis) + + ns.Adaptivity = true + adaptivity := ns.GetAdaptivity() + require.True(t, adaptivity) + + ns.StartTime = 2 + startTime := ns.GetStartTime() + require.Equal(t, int64(2), startTime) + + ns.RoundDuration = 2 + roundDuration := ns.GetRoundDuration() + require.Equal(t, uint64(2), roundDuration) + +} + +func TestNodesSetup_ExportNodesConfigShouldWork(t *testing.T) { + t.Parallel() + + noOfInitialNodes := 3 + + ns, err := createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 3, + genesisMaxShards: 1, + }) + require.Nil(t, err) + + ns.Hysteresis = 0.2 + ns.Adaptivity = false + ns.StartTime = 10 + + ns = createAndAssignNodes(ns, noOfInitialNodes) + configNodes := ns.ExportNodesConfig() + + require.Equal(t, int64(10), configNodes.StartTime) + + var expectedNodesConfigs = make([]config.InitialNodeConfig, len(configNodes.InitialNodes)) + var actualNodesConfigs = make([]config.InitialNodeConfig, len(configNodes.InitialNodes)) + + for i, nodeConfig := range configNodes.InitialNodes { + expectedNodesConfigs[i] = config.InitialNodeConfig{PubKey: pubKeys[i], Address: address[i], InitialRating: 0} + actualNodesConfigs[i] = config.InitialNodeConfig{PubKey: nodeConfig.PubKey, Address: nodeConfig.Address, InitialRating: nodeConfig.InitialRating} + + } + + for i := range configNodes.InitialNodes { + require.Equal(t, expectedNodesConfigs[i], actualNodesConfigs[i]) + } + +} + +func TestNodesSetup_IsInterfaceNil(t *testing.T) { + t.Parallel() + + ns, _ := NewNodesSetup(config.NodesConfig{}, nil, nil, nil, 0) + require.True(t, ns.IsInterfaceNil()) + + ns, _ = createTestNodesSetup(argsTestNodesSetup{ + shardConsensusSize: 1, + shardMinNodes: 1, + metaConsensusSize: 1, + metaMinNodes: 1, + numInitialNodes: 3, + genesisMaxShards: 1, + }) + require.False(t, ns.IsInterfaceNil()) } diff --git a/sharding/oneShardCoordinator_test.go b/sharding/oneShardCoordinator_test.go new file mode 100644 index 00000000000..c2c5d68edfe --- /dev/null +++ b/sharding/oneShardCoordinator_test.go @@ -0,0 +1,33 @@ +package sharding + +import ( + "testing" + + "github.com/multiversx/mx-chain-core-go/core" + "github.com/stretchr/testify/require" +) + +func TestOneShardCoordinator_NumberOfShardsShouldWork(t *testing.T) { + t.Parallel() + + oneShardCoordinator := OneShardCoordinator{} + + returnedVal := oneShardCoordinator.NumberOfShards() + require.Equal(t, uint32(1), returnedVal) + + returnedVal = oneShardCoordinator.ComputeId([]byte{}) + require.Equal(t, uint32(0), returnedVal) + + returnedVal = oneShardCoordinator.SelfId() + require.Equal(t, uint32(0), returnedVal) + + isShameShard := oneShardCoordinator.SameShard(nil, nil) + require.True(t, isShameShard) + + communicationID := oneShardCoordinator.CommunicationIdentifier(0) + require.Equal(t, core.CommunicationIdentifierBetweenShards(0, 0), communicationID) + + isInterfaceNil := oneShardCoordinator.IsInterfaceNil() + require.False(t, isInterfaceNil) + +} diff --git a/testscommon/chainParameters/chainParametersHolderMock.go b/testscommon/chainParameters/chainParametersHolderMock.go new file mode 100644 index 00000000000..50721aa5716 --- /dev/null +++ b/testscommon/chainParameters/chainParametersHolderMock.go @@ -0,0 +1,42 @@ +package chainParameters + +import ( + "github.com/multiversx/mx-chain-go/config" +) + +var testChainParams = config.ChainParametersByEpochConfig{ + RoundDuration: 6000, + Hysteresis: 0, + EnableEpoch: 0, + ShardConsensusGroupSize: 1, + ShardMinNumNodes: 1, + MetachainConsensusGroupSize: 1, + MetachainMinNumNodes: 1, + Adaptivity: false, +} + +// ChainParametersHolderMock - +type ChainParametersHolderMock struct { +} + +// CurrentChainParameters - +func (c *ChainParametersHolderMock) CurrentChainParameters() config.ChainParametersByEpochConfig { + return testChainParams +} + +// AllChainParameters - +func (c *ChainParametersHolderMock) AllChainParameters() []config.ChainParametersByEpochConfig { + return []config.ChainParametersByEpochConfig{ + testChainParams, + } +} + +// ChainParametersForEpoch - +func (c *ChainParametersHolderMock) ChainParametersForEpoch(_ uint32) (config.ChainParametersByEpochConfig, error) { + return testChainParams, nil +} + +// IsInterfaceNil - +func (c *ChainParametersHolderMock) IsInterfaceNil() bool { + return c == nil +} diff --git a/testscommon/chainParameters/chainParametersHolderStub.go b/testscommon/chainParameters/chainParametersHolderStub.go new file mode 100644 index 00000000000..6d12bb3fa46 --- /dev/null +++ b/testscommon/chainParameters/chainParametersHolderStub.go @@ -0,0 +1,42 @@ +package chainParameters + +import "github.com/multiversx/mx-chain-go/config" + +// ChainParametersHandlerStub - +type ChainParametersHandlerStub struct { + CurrentChainParametersCalled func() config.ChainParametersByEpochConfig + AllChainParametersCalled func() []config.ChainParametersByEpochConfig + ChainParametersForEpochCalled func(epoch uint32) (config.ChainParametersByEpochConfig, error) +} + +// CurrentChainParameters - +func (stub *ChainParametersHandlerStub) CurrentChainParameters() config.ChainParametersByEpochConfig { + if stub.CurrentChainParametersCalled != nil { + return stub.CurrentChainParametersCalled() + } + + return config.ChainParametersByEpochConfig{} +} + +// AllChainParameters - +func (stub *ChainParametersHandlerStub) AllChainParameters() []config.ChainParametersByEpochConfig { + if stub.AllChainParametersCalled != nil { + return stub.AllChainParametersCalled() + } + + return nil +} + +// ChainParametersForEpoch - +func (stub *ChainParametersHandlerStub) ChainParametersForEpoch(epoch uint32) (config.ChainParametersByEpochConfig, error) { + if stub.ChainParametersForEpochCalled != nil { + return stub.ChainParametersForEpochCalled(epoch) + } + + return config.ChainParametersByEpochConfig{}, nil +} + +// IsInterfaceNil - +func (stub *ChainParametersHandlerStub) IsInterfaceNil() bool { + return stub == nil +} diff --git a/testscommon/commonmocks/chainParametersNotifierStub.go b/testscommon/commonmocks/chainParametersNotifierStub.go new file mode 100644 index 00000000000..94971a354b5 --- /dev/null +++ b/testscommon/commonmocks/chainParametersNotifierStub.go @@ -0,0 +1,38 @@ +package commonmocks + +import ( + "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/config" +) + +// ChainParametersNotifierStub - +type ChainParametersNotifierStub struct { + ChainParametersChangedCalled func(chainParameters config.ChainParametersByEpochConfig) + UpdateCurrentChainParametersCalled func(params config.ChainParametersByEpochConfig) + RegisterNotifyHandlerCalled func(handler common.ChainParametersSubscriptionHandler) +} + +// ChainParametersChanged - +func (c *ChainParametersNotifierStub) ChainParametersChanged(chainParameters config.ChainParametersByEpochConfig) { + if c.ChainParametersChangedCalled != nil { + c.ChainParametersChangedCalled(chainParameters) + } +} + +// UpdateCurrentChainParameters - +func (c *ChainParametersNotifierStub) UpdateCurrentChainParameters(params config.ChainParametersByEpochConfig) { + if c.UpdateCurrentChainParametersCalled != nil { + c.UpdateCurrentChainParametersCalled(params) + } +} + +// RegisterNotifyHandler - +func (c *ChainParametersNotifierStub) RegisterNotifyHandler(handler common.ChainParametersSubscriptionHandler) { + if c.RegisterNotifyHandlerCalled != nil { + c.RegisterNotifyHandlerCalled(handler) + } +} + +func (c *ChainParametersNotifierStub) IsInterfaceNil() bool { + return c == nil +} diff --git a/testscommon/components/components.go b/testscommon/components/components.go index 01828396e90..0c96f669709 100644 --- a/testscommon/components/components.go +++ b/testscommon/components/components.go @@ -77,9 +77,25 @@ func GetCoreArgs() coreComp.CoreComponentsFactoryArgs { ConfigPathsHolder: config.ConfigurationPathsHolder{ GasScheduleDirectoryName: "../../cmd/node/config/gasSchedules", }, - RatingsConfig: CreateDummyRatingsConfig(), - EconomicsConfig: CreateDummyEconomicsConfig(), - NodesFilename: "../mock/testdata/nodesSetupMock.json", + RatingsConfig: CreateDummyRatingsConfig(), + EconomicsConfig: CreateDummyEconomicsConfig(), + NodesConfig: config.NodesConfig{ + StartTime: 0, + InitialNodes: []*config.InitialNodeConfig{ + { + PubKey: "227a5a5ec0c58171b7f4ee9ecc304ea7b176fb626741a25c967add76d6cd361d6995929f9b60a96237381091cefb1b061225e5bb930b40494a5ac9d7524fd67dfe478e5ccd80f17b093cff5722025761fb0217c39dbd5ae45e01eb5a3113be93", + Address: "erd1ulhw20j7jvgfgak5p05kv667k5k9f320sgef5ayxkt9784ql0zssrzyhjp", + }, + { + PubKey: "ef9522d654bc08ebf2725468f41a693aa7f3cf1cb93922cff1c8c81fba78274016010916f4a7e5b0855c430a724a2d0b3acd1fe8e61e37273a17d58faa8c0d3ef6b883a33ec648950469a1e9757b978d9ae662a019068a401cff56eea059fd08", + Address: "erd17c4fs6mz2aa2hcvva2jfxdsrdknu4220496jmswer9njznt22eds0rxlr4", + }, + { + PubKey: "e91ab494cedd4da346f47aaa1a3e792bea24fb9f6cc40d3546bc4ca36749b8bfb0164e40dbad2195a76ee0fd7fb7da075ecbf1b35a2ac20638d53ea5520644f8c16952225c48304bb202867e2d71d396bff5a5971f345bcfe32c7b6b0ca34c84", + Address: "erd10d2gufxesrp8g409tzxljlaefhs0rsgjle3l7nq38de59txxt8csj54cd3", + }, + }, + }, WorkingDirectory: "home", ChanStopNodeProcess: make(chan endProcess.ArgEndProcess), EpochConfig: config.EpochConfig{ diff --git a/testscommon/components/configs.go b/testscommon/components/configs.go index f86a5ae59cc..ffac8034a9b 100644 --- a/testscommon/components/configs.go +++ b/testscommon/components/configs.go @@ -156,6 +156,18 @@ func GetGeneralConfig() config.Config { MinTransactionVersion: 1, GenesisMaxNumberOfShards: 3, SetGuardianEpochsDelay: 20, + ChainParametersByEpoch: []config.ChainParametersByEpochConfig{ + { + EnableEpoch: 0, + RoundDuration: 4000, + ShardConsensusGroupSize: 1, + ShardMinNumNodes: 1, + MetachainConsensusGroupSize: 1, + MetachainMinNumNodes: 1, + Hysteresis: 0, + Adaptivity: false, + }, + }, }, Marshalizer: config.MarshalizerConfig{ Type: TestMarshalizer, diff --git a/sharding/mock/epochStartNotifierStub.go b/testscommon/epochstartmock/epochStartNotifierStub.go similarity index 98% rename from sharding/mock/epochStartNotifierStub.go rename to testscommon/epochstartmock/epochStartNotifierStub.go index 53406c12920..d8a7bdceea3 100644 --- a/sharding/mock/epochStartNotifierStub.go +++ b/testscommon/epochstartmock/epochStartNotifierStub.go @@ -1,4 +1,4 @@ -package mock +package epochstartmock import ( "github.com/multiversx/mx-chain-core-go/data" diff --git a/testscommon/factory/coreComponentsHolderStub.go b/testscommon/factory/coreComponentsHolderStub.go index d26a12c33e2..23d9fdf6e5c 100644 --- a/testscommon/factory/coreComponentsHolderStub.go +++ b/testscommon/factory/coreComponentsHolderStub.go @@ -55,6 +55,8 @@ type CoreComponentsHolderStub struct { HardforkTriggerPubKeyCalled func() []byte EnableEpochsHandlerCalled func() common.EnableEpochsHandler RoundNotifierCalled func() process.RoundNotifier + ChainParametersSubscriberCalled func() process.ChainParametersSubscriber + ChainParametersHandlerCalled func() process.ChainParametersHandler } // NewCoreComponentsHolderStubFromRealComponent - @@ -95,6 +97,8 @@ func NewCoreComponentsHolderStubFromRealComponent(coreComponents factory.CoreCom HardforkTriggerPubKeyCalled: coreComponents.HardforkTriggerPubKey, EnableEpochsHandlerCalled: coreComponents.EnableEpochsHandler, RoundNotifierCalled: coreComponents.RoundNotifier, + ChainParametersHandlerCalled: coreComponents.ChainParametersHandler, + ChainParametersSubscriberCalled: coreComponents.ChainParametersSubscriber, } } @@ -378,6 +382,22 @@ func (stub *CoreComponentsHolderStub) RoundNotifier() process.RoundNotifier { return nil } +// ChainParametersSubscriber - +func (stub *CoreComponentsHolderStub) ChainParametersSubscriber() process.ChainParametersSubscriber { + if stub.ChainParametersSubscriberCalled != nil { + return stub.ChainParametersSubscriberCalled() + } + return nil +} + +// ChainParametersHandler - +func (stub *CoreComponentsHolderStub) ChainParametersHandler() process.ChainParametersHandler { + if stub.ChainParametersHandlerCalled != nil { + return stub.ChainParametersHandlerCalled() + } + return nil +} + // IsInterfaceNil - func (stub *CoreComponentsHolderStub) IsInterfaceNil() bool { return stub == nil diff --git a/testscommon/generalConfig.go b/testscommon/generalConfig.go index 1eea96a2bdb..515c64518b4 100644 --- a/testscommon/generalConfig.go +++ b/testscommon/generalConfig.go @@ -57,6 +57,18 @@ func GetGeneralConfig() config.Config { SyncProcessTimeInMillis: 6000, SetGuardianEpochsDelay: 20, StatusPollingIntervalSec: 10, + ChainParametersByEpoch: []config.ChainParametersByEpochConfig{ + { + EnableEpoch: 0, + RoundDuration: 6000, + ShardConsensusGroupSize: 1, + ShardMinNumNodes: 1, + MetachainConsensusGroupSize: 1, + MetachainMinNumNodes: 1, + Hysteresis: 0, + Adaptivity: false, + }, + }, }, EpochStartConfig: config.EpochStartConfig{ MinRoundsBetweenEpochs: 5, diff --git a/testscommon/genesisMocks/nodesSetupStub.go b/testscommon/genesisMocks/nodesSetupStub.go index ebe1cfe778a..e06a881dbf2 100644 --- a/testscommon/genesisMocks/nodesSetupStub.go +++ b/testscommon/genesisMocks/nodesSetupStub.go @@ -1,6 +1,7 @@ package genesisMocks import ( + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" ) @@ -27,6 +28,7 @@ type NodesSetupStub struct { MinMetaHysteresisNodesCalled func() uint32 GetChainIdCalled func() string GetMinTransactionVersionCalled func() uint32 + ExportNodesConfigCalled func() config.NodesConfig } // InitialNodesPubKeys - @@ -203,6 +205,15 @@ func (n *NodesSetupStub) MinMetaHysteresisNodes() uint32 { return 1 } +// ExportNodesConfig - +func (n *NodesSetupStub) ExportNodesConfig() config.NodesConfig { + if n.ExportNodesConfigCalled != nil { + return n.ExportNodesConfigCalled() + } + + return config.NodesConfig{} +} + // IsInterfaceNil - func (n *NodesSetupStub) IsInterfaceNil() bool { return n == nil diff --git a/testscommon/realConfigsHandling.go b/testscommon/realConfigsHandling.go index e58b36923f8..c59661b7234 100644 --- a/testscommon/realConfigsHandling.go +++ b/testscommon/realConfigsHandling.go @@ -6,6 +6,7 @@ import ( "path" "strings" + "github.com/multiversx/mx-chain-core-go/core" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/config" ) @@ -83,6 +84,12 @@ func CreateTestConfigs(tempDir string, originalConfigsPath string) (*config.Conf return nil, err } + var nodesSetup config.NodesConfig + err = core.LoadJsonFile(&nodesSetup, path.Join(newConfigsPath, "nodesSetup.json")) + if err != nil { + return nil, err + } + // make the node pass the network wait constraints mainP2PConfig.Node.MinNumPeersToWaitForOnBootstrap = 0 mainP2PConfig.Node.ThresholdMinConnectedPeers = 0 @@ -114,6 +121,7 @@ func CreateTestConfigs(tempDir string, originalConfigsPath string) (*config.Conf }, EpochConfig: epochConfig, RoundConfig: roundConfig, + NodesConfig: &nodesSetup, }, nil } diff --git a/testscommon/shardingMocks/nodesCoordinatorMock.go b/testscommon/shardingMocks/nodesCoordinatorMock.go index 9f1b872e2ab..0343546364f 100644 --- a/testscommon/shardingMocks/nodesCoordinatorMock.go +++ b/testscommon/shardingMocks/nodesCoordinatorMock.go @@ -12,24 +12,25 @@ import ( // NodesCoordinatorMock defines the behaviour of a struct able to do validator group selection type NodesCoordinatorMock struct { - Validators map[uint32][]nodesCoordinator.Validator - ShardConsensusSize uint32 - MetaConsensusSize uint32 - ShardId uint32 - NbShards uint32 - GetSelectedPublicKeysCalled func(selection []byte, shardId uint32, epoch uint32) (publicKeys []string, err error) - GetValidatorsPublicKeysCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) - GetValidatorsRewardsAddressesCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) - SetNodesPerShardsCalled func(nodes map[uint32][]nodesCoordinator.Validator, epoch uint32) error - ComputeValidatorsGroupCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) - GetValidatorWithPublicKeyCalled func(publicKey []byte) (validator nodesCoordinator.Validator, shardId uint32, err error) - GetAllEligibleValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) - GetAllWaitingValidatorsPublicKeysCalled func() (map[uint32][][]byte, error) - ConsensusGroupSizeCalled func(uint32) int - GetValidatorsIndexesCalled func(publicKeys []string, epoch uint32) ([]uint64, error) - GetAllShuffledOutValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) + Validators map[uint32][]nodesCoordinator.Validator + ShardConsensusSize uint32 + MetaConsensusSize uint32 + ShardId uint32 + NbShards uint32 + GetSelectedPublicKeysCalled func(selection []byte, shardId uint32, epoch uint32) (publicKeys []string, err error) + GetValidatorsPublicKeysCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) + GetValidatorsRewardsAddressesCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) ([]string, error) + SetNodesPerShardsCalled func(nodes map[uint32][]nodesCoordinator.Validator, epoch uint32) error + ComputeValidatorsGroupCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) + GetValidatorWithPublicKeyCalled func(publicKey []byte) (validator nodesCoordinator.Validator, shardId uint32, err error) + GetAllEligibleValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) + GetAllWaitingValidatorsPublicKeysCalled func() (map[uint32][][]byte, error) + ConsensusGroupSizeCalled func(uint32, uint32) int + GetValidatorsIndexesCalled func(publicKeys []string, epoch uint32) ([]uint64, error) + GetConsensusWhitelistedNodesCalled func(epoch uint32) (map[string]struct{}, error) + GetAllShuffledOutValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) GetShuffledOutToAuctionValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) - GetNumTotalEligibleCalled func() uint64 + GetNumTotalEligibleCalled func() uint64 } // NewNodesCoordinatorMock - @@ -230,10 +231,10 @@ func (ncm *NodesCoordinatorMock) ComputeConsensusGroup( return validatorsGroup, nil } -// ConsensusGroupSize - -func (ncm *NodesCoordinatorMock) ConsensusGroupSize(shardId uint32) int { +// ConsensusGroupSizeForShardAndEpoch - +func (ncm *NodesCoordinatorMock) ConsensusGroupSizeForShardAndEpoch(shardId uint32, epoch uint32) int { if ncm.ConsensusGroupSizeCalled != nil { - return ncm.ConsensusGroupSizeCalled(shardId) + return ncm.ConsensusGroupSizeCalled(shardId, epoch) } return 1 } @@ -285,9 +286,10 @@ func (ncm *NodesCoordinatorMock) ShuffleOutForEpoch(_ uint32) { } // GetConsensusWhitelistedNodes return the whitelisted nodes allowed to send consensus messages, for each of the shards -func (ncm *NodesCoordinatorMock) GetConsensusWhitelistedNodes( - _ uint32, -) (map[string]struct{}, error) { +func (ncm *NodesCoordinatorMock) GetConsensusWhitelistedNodes(epoch uint32) (map[string]struct{}, error) { + if ncm.GetConsensusWhitelistedNodesCalled != nil { + return ncm.GetConsensusWhitelistedNodesCalled(epoch) + } return make(map[string]struct{}), nil } diff --git a/testscommon/shardingMocks/nodesCoordinatorStub.go b/testscommon/shardingMocks/nodesCoordinatorStub.go index a142f0509ed..4694676a9b0 100644 --- a/testscommon/shardingMocks/nodesCoordinatorStub.go +++ b/testscommon/shardingMocks/nodesCoordinatorStub.go @@ -15,7 +15,7 @@ type NodesCoordinatorStub struct { GetAllValidatorsPublicKeysCalled func() (map[uint32][][]byte, error) GetAllWaitingValidatorsPublicKeysCalled func(_ uint32) (map[uint32][][]byte, error) GetAllEligibleValidatorsPublicKeysCalled func(epoch uint32) (map[uint32][][]byte, error) - ConsensusGroupSizeCalled func(shardID uint32) int + ConsensusGroupSizeCalled func(shardID uint32, epoch uint32) int ComputeConsensusGroupCalled func(randomness []byte, round uint64, shardId uint32, epoch uint32) (validatorsGroup []nodesCoordinator.Validator, err error) EpochStartPrepareCalled func(metaHdr data.HeaderHandler, body data.BodyHandler) GetConsensusWhitelistedNodesCalled func(epoch uint32) (map[string]struct{}, error) @@ -126,10 +126,10 @@ func (ncm *NodesCoordinatorStub) ComputeConsensusGroup( return list, nil } -// ConsensusGroupSize - -func (ncm *NodesCoordinatorStub) ConsensusGroupSize(shardID uint32) int { +// ConsensusGroupSizeForShardAndEpoch - +func (ncm *NodesCoordinatorStub) ConsensusGroupSizeForShardAndEpoch(shardID uint32, epoch uint32) int { if ncm.ConsensusGroupSizeCalled != nil { - return ncm.ConsensusGroupSizeCalled(shardID) + return ncm.ConsensusGroupSizeCalled(shardID, epoch) } return 1 } diff --git a/testscommon/shardingMocks/shufflerMock.go b/testscommon/shardingMocks/shufflerMock.go index 82015b638a3..a96b5ea500c 100644 --- a/testscommon/shardingMocks/shufflerMock.go +++ b/testscommon/shardingMocks/shufflerMock.go @@ -8,16 +8,6 @@ import ( type NodeShufflerMock struct { } -// UpdateParams - -func (nsm *NodeShufflerMock) UpdateParams( - _ uint32, - _ uint32, - _ float32, - _ bool, -) { - -} - // UpdateNodeLists - func (nsm *NodeShufflerMock) UpdateNodeLists(args nodesCoordinator.ArgsUpdateNodes) (*nodesCoordinator.ResUpdateNodes, error) { return &nodesCoordinator.ResUpdateNodes{ diff --git a/update/genesis/export.go b/update/genesis/export.go index ba4e678a0f8..c87e977f487 100644 --- a/update/genesis/export.go +++ b/update/genesis/export.go @@ -17,6 +17,7 @@ import ( "github.com/multiversx/mx-chain-core-go/marshal" "github.com/multiversx/mx-chain-go/common" "github.com/multiversx/mx-chain-go/common/errChan" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/sharding" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" @@ -444,7 +445,7 @@ func (se *stateExport) exportValidatorInfo(key string, validatorInfo *state.Shar func (se *stateExport) exportNodesSetupJson(validators state.ShardValidatorsInfoMapHandler) error { acceptedListsForExport := []common.PeerType{common.EligibleList, common.WaitingList, common.JailedList} - initialNodes := make([]*sharding.InitialNode, 0) + initialNodes := make([]*config.InitialNodeConfig, 0) for _, validator := range validators.GetAllValidatorsInfo() { if shouldExportValidator(validator, acceptedListsForExport) { @@ -459,7 +460,7 @@ func (se *stateExport) exportNodesSetupJson(validators state.ShardValidatorsInfo return nil } - initialNodes = append(initialNodes, &sharding.InitialNode{ + initialNodes = append(initialNodes, &config.InitialNodeConfig{ PubKey: pubKey, Address: rewardAddress, InitialRating: validator.GetRating(), @@ -471,20 +472,10 @@ func (se *stateExport) exportNodesSetupJson(validators state.ShardValidatorsInfo return strings.Compare(initialNodes[i].PubKey, initialNodes[j].PubKey) < 0 }) - genesisNodesSetupHandler := se.genesisNodesSetupHandler - nodesSetup := &sharding.NodesSetup{ - StartTime: genesisNodesSetupHandler.GetStartTime(), - RoundDuration: genesisNodesSetupHandler.GetRoundDuration(), - ConsensusGroupSize: genesisNodesSetupHandler.GetShardConsensusGroupSize(), - MinNodesPerShard: genesisNodesSetupHandler.MinNumberOfShardNodes(), - MetaChainConsensusGroupSize: genesisNodesSetupHandler.GetMetaConsensusGroupSize(), - MetaChainMinNodes: genesisNodesSetupHandler.MinNumberOfMetaNodes(), - Hysteresis: genesisNodesSetupHandler.GetHysteresis(), - Adaptivity: genesisNodesSetupHandler.GetAdaptivity(), - InitialNodes: initialNodes, - } + exportedNodesConfig := se.genesisNodesSetupHandler.ExportNodesConfig() + exportedNodesConfig.InitialNodes = initialNodes - nodesSetupBytes, err := json.MarshalIndent(nodesSetup, "", " ") + nodesSetupBytes, err := json.MarshalIndent(exportedNodesConfig, "", " ") if err != nil { return err } diff --git a/update/interface.go b/update/interface.go index 6487b71438e..2959b16c3d7 100644 --- a/update/interface.go +++ b/update/interface.go @@ -8,6 +8,7 @@ import ( "github.com/multiversx/mx-chain-core-go/data" "github.com/multiversx/mx-chain-core-go/data/block" "github.com/multiversx/mx-chain-go/common" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/process" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" "github.com/multiversx/mx-chain-go/state" @@ -254,6 +255,7 @@ type GenesisNodesSetupHandler interface { GetAdaptivity() bool NumberOfShards() uint32 MinNumberOfNodes() uint32 + ExportNodesConfig() config.NodesConfig IsInterfaceNil() bool } diff --git a/update/mock/nodesSetupHandlerStub.go b/update/mock/nodesSetupHandlerStub.go index 499e14187c2..9c6b64e99a7 100644 --- a/update/mock/nodesSetupHandlerStub.go +++ b/update/mock/nodesSetupHandlerStub.go @@ -3,6 +3,7 @@ package mock import ( "time" + "github.com/multiversx/mx-chain-go/config" "github.com/multiversx/mx-chain-go/sharding/nodesCoordinator" ) @@ -10,6 +11,7 @@ import ( type GenesisNodesSetupHandlerStub struct { InitialNodesInfoForShardCalled func(shardId uint32) ([]nodesCoordinator.GenesisNodeInfoHandler, []nodesCoordinator.GenesisNodeInfoHandler, error) InitialNodesInfoCalled func() (map[uint32][]nodesCoordinator.GenesisNodeInfoHandler, map[uint32][]nodesCoordinator.GenesisNodeInfoHandler) + ExportNodesConfigCalled func() config.NodesConfig GetStartTimeCalled func() int64 GetRoundDurationCalled func() uint64 GetChainIdCalled func() string @@ -150,6 +152,15 @@ func (g *GenesisNodesSetupHandlerStub) MinNumberOfNodes() uint32 { return 1 } +// ExportNodesConfig - +func (g *GenesisNodesSetupHandlerStub) ExportNodesConfig() config.NodesConfig { + if g.ExportNodesConfigCalled != nil { + return g.ExportNodesConfigCalled() + } + + return config.NodesConfig{} +} + // IsInterfaceNil - func (g *GenesisNodesSetupHandlerStub) IsInterfaceNil() bool { return g == nil